提交 4dc56714 编写于 作者: A Alexey Milovidov

Merge branch 'master' into system_columns_race_condition

......@@ -43,32 +43,55 @@ struct RadixSortMallocAllocator
};
/** A transformation that transforms the bit representation of a key into an unsigned integer number,
* that the order relation over the keys will match the order relation over the obtained unsigned numbers.
* For floats this conversion does the following:
* if the signed bit is set, it flips all other bits.
* In this case, NaN-s are bigger than all normal numbers.
*/
template <typename KeyBits>
struct RadixSortIdentityTransform
struct RadixSortFloatTransform
{
static constexpr bool transform_is_simple = true;
/// Is it worth writing the result in memory, or is it better to do calculation every time again?
static constexpr bool transform_is_simple = false;
static KeyBits forward(KeyBits x) { return x; }
static KeyBits backward(KeyBits x) { return x; }
};
static KeyBits forward(KeyBits x)
{
return x ^ ((-(x >> (sizeof(KeyBits) * 8 - 1))) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)));
}
static KeyBits backward(KeyBits x)
{
return x ^ (((x >> (sizeof(KeyBits) * 8 - 1)) - 1) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)));
}
};
template <typename TElement>
struct RadixSortUIntTraits
struct RadixSortFloatTraits
{
using Element = TElement;
using Key = Element;
using CountType = uint32_t;
using KeyBits = Key;
using Element = TElement; /// The type of the element. It can be a structure with a key and some other payload. Or just a key.
using Key = Element; /// The key to sort by.
using CountType = uint32_t; /// Type for calculating histograms. In the case of a known small number of elements, it can be less than size_t.
static constexpr size_t PART_SIZE_BITS = 8;
/// The type to which the key is transformed to do bit operations. This UInt is the same size as the key.
using KeyBits = std::conditional_t<sizeof(Key) == 8, uint64_t, uint32_t>;
using Transform = RadixSortIdentityTransform<KeyBits>;
static constexpr size_t PART_SIZE_BITS = 8; /// With what pieces of the key, in bits, to do one pass - reshuffle of the array.
/// Converting a key into KeyBits is such that the order relation over the key corresponds to the order relation over KeyBits.
using Transform = RadixSortFloatTransform<KeyBits>;
/// An object with the functions allocate and deallocate.
/// Can be used, for example, to allocate memory for a temporary array on the stack.
/// To do this, the allocator itself is created on the stack.
using Allocator = RadixSortMallocAllocator;
/// The function to get the key from an array element.
static Key & extractKey(Element & elem) { return elem; }
/// Used when fallback to comparison based sorting is needed.
/// TODO: Correct handling of NaNs, NULLs, etc
static bool less(Key x, Key y)
{
return x < y;
......@@ -77,25 +100,27 @@ struct RadixSortUIntTraits
template <typename KeyBits>
struct RadixSortSignedTransform
struct RadixSortIdentityTransform
{
static constexpr bool transform_is_simple = true;
static KeyBits forward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); }
static KeyBits backward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); }
static KeyBits forward(KeyBits x) { return x; }
static KeyBits backward(KeyBits x) { return x; }
};
template <typename TElement>
struct RadixSortIntTraits
struct RadixSortUIntTraits
{
using Element = TElement;
using Key = Element;
using CountType = uint32_t;
using KeyBits = std::make_unsigned_t<Key>;
using KeyBits = Key;
static constexpr size_t PART_SIZE_BITS = 8;
using Transform = RadixSortSignedTransform<KeyBits>;
using Transform = RadixSortIdentityTransform<KeyBits>;
using Allocator = RadixSortMallocAllocator;
static Key & extractKey(Element & elem) { return elem; }
......@@ -107,54 +132,31 @@ struct RadixSortIntTraits
};
/** A transformation that transforms the bit representation of a key into an unsigned integer number,
* that the order relation over the keys will match the order relation over the obtained unsigned numbers.
* For floats this conversion does the following:
* if the signed bit is set, it flips all other bits.
* In this case, NaN-s are bigger than all normal numbers.
*/
template <typename KeyBits>
struct RadixSortFloatTransform
struct RadixSortSignedTransform
{
/// Is it worth writing the result in memory, or is it better to do calculation every time again?
static constexpr bool transform_is_simple = false;
static KeyBits forward(KeyBits x)
{
return x ^ ((-(x >> (sizeof(KeyBits) * 8 - 1))) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)));
}
static constexpr bool transform_is_simple = true;
static KeyBits backward(KeyBits x)
{
return x ^ (((x >> (sizeof(KeyBits) * 8 - 1)) - 1) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)));
}
static KeyBits forward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); }
static KeyBits backward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); }
};
template <typename TElement>
struct RadixSortFloatTraits
struct RadixSortIntTraits
{
using Element = TElement; /// The type of the element. It can be a structure with a key and some other payload. Or just a key.
using Key = Element; /// The key to sort by.
using CountType = uint32_t; /// Type for calculating histograms. In the case of a known small number of elements, it can be less than size_t.
/// The type to which the key is transformed to do bit operations. This UInt is the same size as the key.
using KeyBits = std::conditional_t<sizeof(Key) == 8, uint64_t, uint32_t>;
static constexpr size_t PART_SIZE_BITS = 8; /// With what pieces of the key, in bits, to do one pass - reshuffle of the array.
using Element = TElement;
using Key = Element;
using CountType = uint32_t;
using KeyBits = std::make_unsigned_t<Key>;
/// Converting a key into KeyBits is such that the order relation over the key corresponds to the order relation over KeyBits.
using Transform = RadixSortFloatTransform<KeyBits>;
static constexpr size_t PART_SIZE_BITS = 8;
/// An object with the functions allocate and deallocate.
/// Can be used, for example, to allocate memory for a temporary array on the stack.
/// To do this, the allocator itself is created on the stack.
using Transform = RadixSortSignedTransform<KeyBits>;
using Allocator = RadixSortMallocAllocator;
/// The function to get the key from an array element.
static Key & extractKey(Element & elem) { return elem; }
// TODO: Correct handling of NaNs, NULLs, etc
static bool less(Key x, Key y)
{
return x < y;
......@@ -220,9 +222,9 @@ private:
* Puts elements to buckets based on PASS-th digit, then recursively calls insertion sort or itself on the buckets
*/
template <size_t PASS>
static inline void radixSortMSDInternal(Element *arr, size_t size, size_t limit)
static inline void radixSortMSDInternal(Element * arr, size_t size, size_t limit)
{
Element *last_list[HISTOGRAM_SIZE + 1];
Element * last_list[HISTOGRAM_SIZE + 1];
Element ** last = last_list + 1;
size_t count[HISTOGRAM_SIZE] = {0};
......@@ -295,7 +297,7 @@ private:
// A helper to choose sorting algorithm based on array length
template <size_t PASS>
static inline void radixSortMSDInternalHelper(Element *arr, size_t size, size_t limit)
static inline void radixSortMSDInternalHelper(Element * arr, size_t size, size_t limit)
{
if (size <= INSERTION_SORT_THRESHOLD)
insertionSortInternal(arr, size);
......@@ -304,8 +306,8 @@ private:
}
public:
// Least significant digit radix sort (stable)
static void executeLSD(Element *arr, size_t size)
/// Least significant digit radix sort (stable)
static void executeLSD(Element * arr, size_t size)
{
/// If the array is smaller than 256, then it is better to use another algorithm.
......@@ -400,7 +402,7 @@ public:
* CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
* SOFTWARE.
*/
static void executeMSD(Element *arr, size_t size, size_t limit)
static void executeMSD(Element * arr, size_t size, size_t limit)
{
limit = std::min(limit, size);
radixSortMSDInternalHelper<NUM_PASSES - 1>(arr, size, limit);
......
......@@ -28,8 +28,8 @@ void registerFunctionsCoding(FunctionFactory & factory)
factory.registerFunction<FunctionBitmaskToArray>();
factory.registerFunction<FunctionToIPv4>();
factory.registerFunction<FunctionToIPv6>();
factory.registerFunction<FunctionIPv6CIDRtoIPv6Range>();
factory.registerFunction<FunctionIPv4CIDRtoIPv4Range>();
factory.registerFunction<FunctionIPv6CIDRToRange>();
factory.registerFunction<FunctionIPv4CIDRToRange>();
}
}
......@@ -1453,50 +1453,37 @@ public:
};
constexpr size_t ip_range_tuple_size = 2;
class FunctionIPv6CIDRtoIPv6Range : public IFunction
class FunctionIPv6CIDRToRange : public IFunction
{
static constexpr size_t bits_in_uint8 = 8;
public:
template <bool lower_range>
static void setCIDRMask(const UInt8 * __restrict src, UInt8 * __restrict dst, UInt8 bits_to_keep)
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)
{
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;
UInt8 mask[16]{};
constexpr UInt8 byte_reference = lower_range ? 0 : std::numeric_limits<UInt8>::max();
UInt8 bytes_to_keep = bits_to_keep / 8;
UInt8 bits_to_keep_in_last_byte = bits_to_keep % 8;
/// 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);
for (size_t i = 0; i < bits_to_keep / 8; ++i)
mask[i] = 0xFFU;
dst[offset] = static_cast<UInt8>(src_byte_shift | cidr_mask_byte_shift);
}
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];
}
}
static constexpr auto name = "IPv6CIDRtoIPv6Range";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv6CIDRtoIPv6Range>(); }
public:
static constexpr auto name = "IPv6CIDRToRange";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv6CIDRToRange>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool isInjective(const Block &) override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
......@@ -1508,117 +1495,97 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypePtr & second_argument = arguments[1];
if (!isNumber(second_argument))
if (!isUInt8(second_argument))
throw Exception{"Illegal type " + second_argument->getName()
+ " of second argument of function " + getName()
+ ", expected numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
decltype(arguments) return_type = {
DataTypeFactory::instance().get("IPv6"),
DataTypeFactory::instance().get("IPv6")
};
return std::make_shared<DataTypeTuple>(return_type);
DataTypePtr element = DataTypeFactory::instance().get("IPv6");
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
}
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 ColumnPtr & column_ip = col_type_name_ip.column;
const auto col_ip_in = checkAndGetColumn<ColumnFixedString>(column_ip.get());
const auto & col_type_name_cidr = block.getByPosition(arguments[1]);
const ColumnPtr & column_cidr = col_type_name_cidr.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)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
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() +
" of column " + col_ip_in->getName() +
" argument of function " + getName() +
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto & vec_in = col_ip_in->getChars();
const auto size = vec_in.size();
const auto & col_type_name_cidr = block.getByPosition(arguments[1]);
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());
auto col_res_lower_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
auto col_res_upper_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
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);
ColumnString::Chars & vec_res_lower_range = col_res_lower_range->getChars();
vec_res_lower_range.resize(size);
const auto & vec_in = col_ip_in->getChars();
ColumnString::Chars & vec_res_upper_range = col_res_upper_range->getChars();
vec_res_upper_range.resize(size);
auto col_res_lower_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
auto col_res_upper_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
for (size_t offset = 0; offset < col_ip_in->size(); ++offset)
{
const size_t offset_ipv6 = offset * IPV6_BINARY_LENGTH;
UInt8 cidr = col_const_cidr_in
? col_const_cidr_in->getValue<UInt8>()
: col_cidr_in->getData()[offset];
ColumnString::Chars & vec_res_lower_range = col_res_lower_range->getChars();
vec_res_lower_range.resize(input_rows_count * IPV6_BINARY_LENGTH);
setCIDRMask<true>(&vec_in[offset_ipv6], &vec_res_lower_range[offset_ipv6], cidr);
setCIDRMask<false>(&vec_in[offset_ipv6], &vec_res_upper_range[offset_ipv6], cidr);
}
ColumnString::Chars & vec_res_upper_range = col_res_upper_range->getChars();
vec_res_upper_range.resize(input_rows_count * IPV6_BINARY_LENGTH);
tuple_columns[0] = std::move(col_res_lower_range);
tuple_columns[1] = std::move(col_res_upper_range);
for (size_t offset = 0; offset < input_rows_count; ++offset)
{
const size_t offset_ipv6 = offset * IPV6_BINARY_LENGTH;
UInt8 cidr = col_const_cidr_in
? col_const_cidr_in->getValue<UInt8>()
: col_cidr_in->getData()[offset];
block.getByPosition(result).column = ColumnTuple::create(tuple_columns);
applyCIDRMask(&vec_in[offset_ipv6], &vec_res_lower_range[offset_ipv6], &vec_res_upper_range[offset_ipv6], cidr);
}
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);
block.getByPosition(result).column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
}
};
class FunctionIPv4CIDRtoIPv4Range : public IFunction
{
static constexpr size_t bits_in_uint32 = 32;
public:
template <bool lower_range>
static UInt32 setCIDRMask(UInt32 src, UInt8 bits_to_keep)
class FunctionIPv4CIDRToRange : public IFunction
{
private:
static inline std::pair<UInt32, UInt32> applyCIDRMask(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)
return src;
UInt32 mask = UInt32(-1) << (8 * sizeof(UInt32) - bits_to_keep);
UInt32 lower = src & mask;
UInt32 upper = lower | ~mask;
const UInt8 shifts_bits = bits_in_uint32 - bits_to_keep;
/** 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);
return { lower, upper };
}
static constexpr auto name = "IPv4CIDRtoIPv4Range";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv4CIDRtoIPv4Range>(); }
public:
static constexpr auto name = "IPv4CIDRToRange";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv4CIDRToRange>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool isInjective(const Block &) override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
......@@ -1630,24 +1597,28 @@ public:
const DataTypePtr & second_argument = arguments[1];
if (!isNumber(second_argument))
if (!isUInt8(second_argument))
throw Exception{"Illegal type " + second_argument->getName()
+ " of second argument of function " + getName()
+ ", expected numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
decltype(arguments) return_type = { DataTypeFactory::instance().get("IPv4"), DataTypeFactory::instance().get("IPv4") };
return std::make_shared<DataTypeTuple>(return_type);
DataTypePtr element = DataTypeFactory::instance().get("IPv4");
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
}
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 ColumnPtr & column_ip = col_type_name_ip.column;
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 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_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
if (col_ip_in && (col_const_cidr_in || col_cidr_in))
{
const auto size = col_ip_in->size();
const auto & vec_in = col_ip_in->getData();
Columns tuple_columns(ip_range_tuple_size);
auto col_res_lower_range = ColumnUInt32::create();
auto col_res_upper_range = ColumnUInt32::create();
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);
auto & vec_res_lower_range = col_res_lower_range->getData();
vec_res_lower_range.resize(size);
const auto & vec_in = col_ip_in->getData();
auto & vec_res_upper_range = col_res_upper_range->getData();
vec_res_upper_range.resize(size);
auto col_res_lower_range = ColumnUInt32::create();
auto col_res_upper_range = ColumnUInt32::create();
for (size_t i = 0; i < vec_in.size(); ++i)
{
UInt8 cidr = col_const_cidr_in
? col_const_cidr_in->getValue<UInt8>()
: col_cidr_in->getData()[i];
auto & vec_res_lower_range = col_res_lower_range->getData();
vec_res_lower_range.resize(input_rows_count);
vec_res_lower_range[i] = setCIDRMask<true>(vec_in[i], cidr);
vec_res_upper_range[i] = setCIDRMask<false>(vec_in[i], cidr);
}
auto & vec_res_upper_range = col_res_upper_range->getData();
vec_res_upper_range.resize(input_rows_count);
tuple_columns[0] = std::move(col_res_lower_range);
tuple_columns[1] = std::move(col_res_upper_range);
for (size_t i = 0; i < input_rows_count; ++i)
{
UInt8 cidr = col_const_cidr_in
? col_const_cidr_in->getValue<UInt8>()
: col_cidr_in->getData()[i];
block.getByPosition(result).column = ColumnTuple::create(tuple_columns);
std::tie(vec_res_lower_range[i], vec_res_upper_range[i]) = applyCIDRMask(vec_in[i], cidr);
}
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);
block.getByPosition(result).column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
}
};
}
......@@ -383,7 +383,7 @@ ColumnsDescription InterpreterCreateQuery::setColumns(
else if (!create.as_table.empty())
{
columns = as_storage->getColumns();
indices = as_storage->getIndicesDescription();
indices = as_storage->getIndices();
}
else if (create.select)
{
......
......@@ -15,7 +15,7 @@ void IStorage::alter(const AlterCommands & params, const String & database_name,
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
auto new_columns = getColumns();
auto new_indices = getIndicesDescription();
auto new_indices = getIndices();
params.apply(new_columns);
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
setColumns(std::move(new_columns));
......
......@@ -31,7 +31,7 @@ void ITableDeclaration::setColumns(ColumnsDescription columns_)
columns = std::move(columns_);
}
void ITableDeclaration::setIndicesDescription(IndicesDescription indices_)
void ITableDeclaration::setIndices(IndicesDescription indices_)
{
indices = std::move(indices_);
}
......
......@@ -16,8 +16,8 @@ public:
virtual const ColumnsDescription & getColumns() const { return columns; }
virtual void setColumns(ColumnsDescription columns_);
virtual const IndicesDescription & getIndicesDescription() const { return indices; }
virtual void setIndicesDescription(IndicesDescription indices_);
virtual const IndicesDescription & getIndices() const { return indices; }
virtual void setIndices(IndicesDescription indices_);
/// NOTE: These methods should include virtual columns, but should NOT include ALIAS columns
/// (they are treated separately).
......
......@@ -406,7 +406,7 @@ void MergeTreeData::setPrimaryKeyIndicesAndColumns(
if (!only_check)
{
setColumns(new_columns);
setColumns(std::move(new_columns));
order_by_ast = new_order_by_ast;
sorting_key_columns = std::move(new_sorting_key_columns);
......@@ -420,7 +420,7 @@ void MergeTreeData::setPrimaryKeyIndicesAndColumns(
primary_key_sample = std::move(new_primary_key_sample);
primary_key_data_types = std::move(new_primary_key_data_types);
setIndicesDescription(indices_description);
setIndices(indices_description);
skip_indices = std::move(new_indices);
primary_key_and_skip_indices_expr = new_indices_with_primary_key_expr;
......@@ -1177,13 +1177,13 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
{
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
auto new_columns = getColumns();
auto new_indices = getIndicesDescription();
auto new_indices = getIndices();
ASTPtr new_order_by_ast = order_by_ast;
ASTPtr new_primary_key_ast = primary_key_ast;
ASTPtr new_ttl_table_ast = ttl_table_ast;
commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
if (getIndicesDescription().empty() && !new_indices.empty() &&
if (getIndices().empty() && !new_indices.empty() &&
!context.getSettingsRef().allow_experimental_data_skipping_indices)
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
......@@ -1274,7 +1274,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
NameToNameMap unused_map;
bool unused_bool;
createConvertExpression(nullptr, getColumns().getAllPhysical(), new_columns.getAllPhysical(),
getIndicesDescription().indices, new_indices.indices, unused_expression, unused_map, unused_bool);
getIndices().indices, new_indices.indices, unused_expression, unused_map, unused_bool);
}
void MergeTreeData::createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, const NamesAndTypesList & new_columns,
......@@ -1445,7 +1445,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
AlterDataPartTransactionPtr transaction(new AlterDataPartTransaction(part)); /// Blocks changes to the part.
bool force_update_metadata;
createConvertExpression(part, part->columns, new_columns,
getIndicesDescription().indices, new_indices,
getIndices().indices, new_indices,
expression, transaction->rename_map, force_update_metadata);
size_t num_files_to_modify = transaction->rename_map.size();
......@@ -1597,7 +1597,7 @@ void MergeTreeData::removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr
LOG_INFO(log, "Removing empty columns: " << log_message.str() << " from part " << data_part->name);
if (auto transaction = alterDataPart(data_part, new_columns, getIndicesDescription().indices, false))
if (auto transaction = alterDataPart(data_part, new_columns, getIndices().indices, false))
transaction->commit();
empty_columns.clear();
}
......@@ -2133,6 +2133,18 @@ size_t MergeTreeData::getTotalActiveSizeInBytes() const
}
size_t MergeTreeData::getPartsCount() const
{
auto lock = lockParts();
size_t res = 0;
for (const auto & part [[maybe_unused]] : getDataPartsStateRange(DataPartState::Committed))
++res;
return res;
}
size_t MergeTreeData::getMaxPartsCountForPartition() const
{
auto lock = lockParts();
......@@ -2165,7 +2177,7 @@ std::optional<Int64> MergeTreeData::getMinPartDataVersion() const
auto lock = lockParts();
std::optional<Int64> result;
for (const DataPartPtr & part : getDataPartsStateRange(DataPartState::Committed))
for (const auto & part : getDataPartsStateRange(DataPartState::Committed))
{
if (!result || *result > part->info.getDataVersion())
result = part->info.getDataVersion();
......@@ -2177,18 +2189,25 @@ std::optional<Int64> MergeTreeData::getMinPartDataVersion() const
void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event *until) const
{
const size_t parts_count = getMaxPartsCountForPartition();
if (parts_count < settings.parts_to_delay_insert)
const size_t parts_count_in_total = getPartsCount();
if (parts_count_in_total >= settings.max_parts_in_total)
{
ProfileEvents::increment(ProfileEvents::RejectedInserts);
throw Exception("Too many parts (" + toString(parts_count_in_total) + ") in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in <merge_tree> element in config.xml or with per-table setting.", ErrorCodes::TOO_MANY_PARTS);
}
const size_t parts_count_in_partition = getMaxPartsCountForPartition();
if (parts_count_in_partition < settings.parts_to_delay_insert)
return;
if (parts_count >= settings.parts_to_throw_insert)
if (parts_count_in_partition >= settings.parts_to_throw_insert)
{
ProfileEvents::increment(ProfileEvents::RejectedInserts);
throw Exception("Too many parts (" + toString(parts_count) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS);
throw Exception("Too many parts (" + toString(parts_count_in_partition) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS);
}
const size_t max_k = settings.parts_to_throw_insert - settings.parts_to_delay_insert; /// always > 0
const size_t k = 1 + parts_count - settings.parts_to_delay_insert; /// from 1 to max_k
const size_t k = 1 + parts_count_in_partition - settings.parts_to_delay_insert; /// from 1 to max_k
const double delay_milliseconds = ::pow(settings.max_delay_to_insert * 1000, static_cast<double>(k) / max_k);
ProfileEvents::increment(ProfileEvents::DelayedInserts);
......@@ -2197,7 +2216,7 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event *until) const
CurrentMetrics::Increment metric_increment(CurrentMetrics::DelayedInserts);
LOG_INFO(log, "Delaying inserting block by "
<< std::fixed << std::setprecision(4) << delay_milliseconds << " ms. because there are " << parts_count << " parts");
<< std::fixed << std::setprecision(4) << delay_milliseconds << " ms. because there are " << parts_count_in_partition << " parts");
if (until)
until->tryWait(delay_milliseconds);
......@@ -2207,12 +2226,19 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event *until) const
void MergeTreeData::throwInsertIfNeeded() const
{
const size_t parts_count = getMaxPartsCountForPartition();
const size_t parts_count_in_total = getPartsCount();
if (parts_count_in_total >= settings.max_parts_in_total)
{
ProfileEvents::increment(ProfileEvents::RejectedInserts);
throw Exception("Too many parts (" + toString(parts_count_in_total) + ") in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in <merge_tree> element in config.xml or with per-table setting.", ErrorCodes::TOO_MANY_PARTS);
}
const size_t parts_count_in_partition = getMaxPartsCountForPartition();
if (parts_count >= settings.parts_to_throw_insert)
if (parts_count_in_partition >= settings.parts_to_throw_insert)
{
ProfileEvents::increment(ProfileEvents::RejectedInserts);
throw Exception("Too many parts (" + toString(parts_count) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS);
throw Exception("Too many parts (" + toString(parts_count_in_partition) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS);
}
}
......
......@@ -422,6 +422,7 @@ public:
/// Total size of active parts in bytes.
size_t getTotalActiveSizeInBytes() const;
size_t getPartsCount() const;
size_t getMaxPartsCountForPartition() const;
/// Get min value of part->info.getDataVersion() for all active parts.
......
......@@ -36,9 +36,10 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.") \
\
/** Inserts settings. */ \
M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts, artificially slow down insert into table.") \
M(SettingUInt64, parts_to_throw_insert, 300, "If more than this number active parts, throw 'Too many parts ...' exception.") \
M(SettingUInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts.") \
M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.") \
M(SettingUInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.") \
M(SettingUInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.") \
M(SettingUInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.") \
\
/** Replication settings. */ \
M(SettingUInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).") \
......
......@@ -143,7 +143,7 @@ void ReplicatedMergeTreeAlterThread::run()
parts = storage.data.getDataParts();
const auto columns_for_parts = storage.getColumns().getAllPhysical();
const auto indices_for_parts = storage.getIndicesDescription();
const auto indices_for_parts = storage.getIndices();
for (const MergeTreeData::DataPartPtr & part : parts)
{
......
......@@ -45,7 +45,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast));
skip_indices = data.getIndicesDescription().toString();
skip_indices = data.getIndices().toString();
index_granularity_bytes = data.index_granularity_info.index_granularity_bytes;
ttl_table = formattedAST(data.ttl_table_ast);
}
......
......@@ -700,7 +700,7 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_
optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context);
auto new_columns = getColumns();
auto new_indices = getIndicesDescription();
auto new_indices = getIndices();
params.apply(new_columns);
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
setColumns(std::move(new_columns));
......
......@@ -344,7 +344,7 @@ void StorageDistributed::alter(
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
auto new_columns = getColumns();
auto new_indices = getIndicesDescription();
auto new_indices = getIndices();
params.apply(new_columns);
context.getDatabase(database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
setColumns(std::move(new_columns));
......
......@@ -402,7 +402,7 @@ void StorageMerge::alter(
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
auto new_columns = getColumns();
auto new_indices = getIndicesDescription();
auto new_indices = getIndices();
params.apply(new_columns);
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
setColumns(new_columns);
......
......@@ -204,7 +204,7 @@ void StorageMergeTree::alter(
{
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
auto new_columns = getColumns();
auto new_indices = getIndicesDescription();
auto new_indices = getIndices();
params.apply(new_columns);
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
setColumns(std::move(new_columns));
......@@ -219,7 +219,7 @@ void StorageMergeTree::alter(
data.checkAlter(params, context);
auto new_columns = data.getColumns();
auto new_indices = data.getIndicesDescription();
auto new_indices = data.getIndices();
ASTPtr new_order_by_ast = data.order_by_ast;
ASTPtr new_primary_key_ast = data.primary_key_ast;
ASTPtr new_ttl_table_ast = data.ttl_table_ast;
......@@ -800,7 +800,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi
alter_command.column_name = get<String>(column_name);
auto new_columns = getColumns();
auto new_indices = getIndicesDescription();
auto new_indices = getIndices();
ASTPtr ignored_order_by_ast;
ASTPtr ignored_primary_key_ast;
ASTPtr ignored_ttl_table_ast;
......
......@@ -43,8 +43,8 @@ public:
const ColumnsDescription & getColumns() const override { return data.getColumns(); }
void setColumns(ColumnsDescription columns_) override { return data.setColumns(std::move(columns_)); }
virtual const IndicesDescription & getIndicesDescription() const override { return data.getIndicesDescription(); }
virtual void setIndicesDescription(IndicesDescription indices_) override { data.setIndicesDescription(std::move(indices_)); }
virtual const IndicesDescription & getIndices() const override { return data.getIndices(); }
virtual void setIndices(IndicesDescription indices_) override { data.setIndices(std::move(indices_)); }
NameAndTypePair getColumn(const String & column_name) const override { return data.getColumn(column_name); }
bool hasColumn(const String & column_name) const override { return data.hasColumn(column_name); }
......
......@@ -37,7 +37,7 @@ void StorageNull::alter(
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
ColumnsDescription new_columns = getColumns();
IndicesDescription new_indices = getIndicesDescription();
IndicesDescription new_indices = getIndices();
params.apply(new_columns);
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
setColumns(std::move(new_columns));
......
......@@ -426,7 +426,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
{
ASTPtr new_primary_key_ast = data.primary_key_ast;
ASTPtr new_order_by_ast = data.order_by_ast;
auto new_indices = data.getIndicesDescription();
auto new_indices = data.getIndices();
ASTPtr new_ttl_table_ast = data.ttl_table_ast;
IDatabase::ASTModifier storage_modifier;
if (!metadata_diff.empty())
......@@ -1511,7 +1511,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
alter_command.column_name = entry.column_name;
auto new_columns = getColumns();
auto new_indices = getIndicesDescription();
auto new_indices = getIndices();
ASTPtr ignored_order_by_ast;
ASTPtr ignored_primary_key_ast;
ASTPtr ignored_ttl_table_ast;
......@@ -3117,7 +3117,7 @@ void StorageReplicatedMergeTree::alter(
data.checkAlter(params, query_context);
ColumnsDescription new_columns = data.getColumns();
IndicesDescription new_indices = data.getIndicesDescription();
IndicesDescription new_indices = data.getIndices();
ASTPtr new_order_by_ast = data.order_by_ast;
ASTPtr new_primary_key_ast = data.primary_key_ast;
ASTPtr new_ttl_table_ast = data.ttl_table_ast;
......@@ -3135,7 +3135,7 @@ void StorageReplicatedMergeTree::alter(
new_metadata.ttl_table = serializeAST(*new_ttl_table_ast);
String new_indices_str = new_indices.toString();
if (new_indices_str != data.getIndicesDescription().toString())
if (new_indices_str != data.getIndices().toString())
new_metadata.skip_indices = new_indices_str;
String new_metadata_str = new_metadata.toString();
......
<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>
<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);
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;
SELECT IPv4CIDRtoIPv4Range(toIPv4('192.168.5.2'), 0);
SELEcT IPv4CIDRtoIPv4Range(toIPv4('255.255.255.255'), 8);
SELECT IPv4CIDRtoIPv4Range(toIPv4('192.168.5.2'), 32);
SELECT IPv4CIDRtoIPv4Range(toIPv4('0.0.0.0'), 8);
SELECT IPv4CIDRtoIPv4Range(toIPv4('255.0.0.0'), 4);
SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 0);
SELEcT IPv4CIDRToRange(toIPv4('255.255.255.255'), 8);
SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 32);
SELECT IPv4CIDRToRange(toIPv4('0.0.0.0'), 8);
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);
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;
SELECT IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0);
SELECT IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 128);
SELECT IPv6CIDRtoIPv6Range(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'), 64);
SELECT IPv6CIDRtoIPv6Range(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('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0);
SELECT IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 128);
SELECT IPv6CIDRToRange(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'), 64);
SELECT IPv6CIDRToRange(IPv6StringToNum('0000:0000:0000:0000:0000:0000:0000:0000'), 8);
SELECT IPv6CIDRToRange(IPv6StringToNum('ffff:0000:0000:0000:0000:0000:0000:0000'), 4);
drop table if exists max_parts_in_total;
create table max_parts_in_total (x UInt64) ENGINE = MergeTree PARTITION BY x ORDER BY x SETTINGS max_parts_in_total = 10;
INSERT INTO max_parts_in_total SELECT number FROM numbers(10);
SELECT 1;
INSERT INTO max_parts_in_total SELECT 123; -- { serverError 252 }
drop table max_parts_in_total;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册