From 2dc1721262c9f483917750aaa6139ff7409e02dc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 11:53:00 +0000 Subject: [PATCH 01/78] Refactorings for consistency --- src/Functions/array/arrayDistance.cpp | 54 +++++++-------- src/Functions/array/arrayDotProduct.cpp | 91 +++++++++++++------------ 2 files changed, 73 insertions(+), 72 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 71564f6fa93..6b72c99d829 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -379,17 +379,17 @@ public: } -#define SUPPORTED_TYPES(action) \ - action(UInt8) \ - action(UInt16) \ - action(UInt32) \ - action(UInt64) \ - action(Int8) \ - action(Int16) \ - action(Int32) \ - action(Int64) \ - action(Float32) \ - action(Float64) +#define SUPPORTED_TYPES(ACTION) \ + ACTION(UInt8) \ + ACTION(UInt16) \ + ACTION(UInt32) \ + ACTION(UInt64) \ + ACTION(Int8) \ + ACTION(Int16) \ + ACTION(Int32) \ + ACTION(Int64) \ + ACTION(Float32) \ + ACTION(Float64) private: @@ -398,12 +398,11 @@ private: { DataTypePtr type_x = typeid_cast(arguments[0].type.get())->getNestedType(); - /// Dynamic disaptch based on the 1st argument type switch (type_x->getTypeId()) { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithFirstType(arguments, input_rows_count); \ + return executeWithResultTypeAndLeftType(arguments, input_rows_count); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -419,17 +418,16 @@ private: } } - template - ColumnPtr executeWithFirstType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const + template + ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { DataTypePtr type_y = typeid_cast(arguments[1].type.get())->getNestedType(); - /// Dynamic disaptch based on the 2nd argument type switch (type_y->getTypeId()) { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithTypes(arguments[0].column, arguments[1].column, input_rows_count, arguments); \ + return executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column, input_rows_count, arguments); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -445,16 +443,16 @@ private: } } - template - ColumnPtr executeWithTypes(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count, const ColumnsWithTypeAndName & arguments) const + template + ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count, const ColumnsWithTypeAndName & arguments) const { if (typeid_cast(col_x.get())) { - return executeWithTypesFirstArgConst(col_x, col_y, input_rows_count, arguments); + return executeWithLeftArgConst(col_x, col_y, input_rows_count, arguments); } else if (typeid_cast(col_y.get())) { - return executeWithTypesFirstArgConst(col_y, col_x, input_rows_count, arguments); + return executeWithLeftArgConst(col_y, col_x, input_rows_count, arguments); } col_x = col_x->convertToFullColumnIfConst(); @@ -463,8 +461,8 @@ private: const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); - const auto & data_x = typeid_cast &>(array_x.getData()).getData(); - const auto & data_y = typeid_cast &>(array_y.getData()).getData(); + const auto & data_x = typeid_cast &>(array_x.getData()).getData(); + const auto & data_y = typeid_cast &>(array_y.getData()).getData(); const auto & offsets_x = array_x.getOffsets(); const auto & offsets_y = array_y.getOffsets(); @@ -521,8 +519,8 @@ private: } /// Special case when the 1st parameter is Const - template - ColumnPtr executeWithTypesFirstArgConst(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count, const ColumnsWithTypeAndName & arguments) const + template + ColumnPtr executeWithLeftArgConst(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count, const ColumnsWithTypeAndName & arguments) const { col_x = assert_cast(col_x.get())->getDataColumnPtr(); col_y = col_y->convertToFullColumnIfConst(); @@ -530,8 +528,8 @@ private: const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); - const auto & data_x = typeid_cast &>(array_x.getData()).getData(); - const auto & data_y = typeid_cast &>(array_y.getData()).getData(); + const auto & data_x = typeid_cast &>(array_x.getData()).getData(); + const auto & data_y = typeid_cast &>(array_y.getData()).getData(); const auto & offsets_x = array_x.getOffsets(); const auto & offsets_y = array_y.getOffsets(); @@ -574,7 +572,7 @@ private: /// - the two most common metrics L2 and cosine distance, /// - the most powerful SIMD instruction set (AVX-512F). #if USE_MULTITARGET_CODE - if constexpr (std::is_same_v && std::is_same_v) /// ResultType is Float32 or Float64 + if constexpr (std::is_same_v && std::is_same_v) /// ResultType is Float32 or Float64 { if constexpr (std::is_same_v || std::is_same_v) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 6c615a058c3..548c79c567f 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -163,26 +163,29 @@ public: return Kernel::getReturnType(nested_types[0], nested_types[1]); } +#define SUPPORTED_TYPES(ACTION) \ + ACTION(UInt8) \ + ACTION(UInt16) \ + ACTION(UInt32) \ + ACTION(UInt64) \ + ACTION(Int8) \ + ACTION(Int16) \ + ACTION(Int32) \ + ACTION(Int64) \ + ACTION(Float32) \ + ACTION(Float64) + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override { switch (result_type->getTypeId()) { - #define SUPPORTED_TYPE(type) \ + #define ON_TYPE(type) \ case TypeIndex::type: \ return executeWithResultType(arguments); \ break; - SUPPORTED_TYPE(UInt8) - SUPPORTED_TYPE(UInt16) - SUPPORTED_TYPE(UInt32) - SUPPORTED_TYPE(UInt64) - SUPPORTED_TYPE(Int8) - SUPPORTED_TYPE(Int16) - SUPPORTED_TYPE(Int32) - SUPPORTED_TYPE(Int64) - SUPPORTED_TYPE(Float32) - SUPPORTED_TYPE(Float64) - #undef SUPPORTED_TYPE + SUPPORTED_TYPES(ON_TYPE) + #undef ON_TYPE default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type {}", result_type->getName()); @@ -194,16 +197,16 @@ private: ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if (!((res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)) - || (res = executeWithResultTypeAndLeft(arguments)))) + if (!((res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)) + || (res = executeWithResultTypeAndLeftType(arguments)))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); @@ -211,43 +214,43 @@ private: } template - ColumnPtr executeWithResultTypeAndLeft(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if ( (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments)) - || (res = executeWithResultTypeAndLeftAndRight(arguments))) + if ( (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) + || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column))) return res; return nullptr; } template - ColumnPtr executeWithResultTypeAndLeftAndRight(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y) const { - ColumnPtr col_left = arguments[0].column->convertToFullColumnIfConst(); - ColumnPtr col_right = arguments[1].column->convertToFullColumnIfConst(); - if (!col_left || !col_right) + col_x = col_x->convertToFullColumnIfConst(); + col_y = col_y->convertToFullColumnIfConst(); + if (!col_x || !col_y) return nullptr; - const ColumnArray * col_arr_left = checkAndGetColumn(col_left.get()); - const ColumnArray * cokl_arr_right = checkAndGetColumn(col_right.get()); - if (!col_arr_left || !cokl_arr_right) + const ColumnArray * array_x = checkAndGetColumn(col_x.get()); + const ColumnArray * array_y = checkAndGetColumn(col_y.get()); + if (!array_x || !array_y) return nullptr; - const ColumnVector * col_arr_nested_left = checkAndGetColumn>(col_arr_left->getData()); - const ColumnVector * col_arr_nested_right = checkAndGetColumn>(cokl_arr_right->getData()); + const ColumnVector * col_arr_nested_left = checkAndGetColumn>(array_x->getData()); + const ColumnVector * col_arr_nested_right = checkAndGetColumn>(array_y->getData()); if (!col_arr_nested_left || !col_arr_nested_right) return nullptr; - if (!col_arr_left->hasEqualOffsets(*cokl_arr_right)) + if (!array_x->hasEqualOffsets(*array_y)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); auto col_res = ColumnVector::create(); @@ -255,7 +258,7 @@ private: vector( col_arr_nested_left->getData(), col_arr_nested_right->getData(), - col_arr_left->getOffsets(), + array_x->getOffsets(), col_res->getData()); return col_res; From c3909743ed95adf5efd16e69f353aab8af73978b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 12:36:00 +0000 Subject: [PATCH 02/78] Remove repeated unnecessary unpacking of const columns --- src/Functions/array/arrayDotProduct.cpp | 88 +++++++++++++------------ 1 file changed, 46 insertions(+), 42 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 548c79c567f..c27170cd35b 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int LOGICAL_ERROR; } @@ -196,40 +195,51 @@ private: template ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const { - ColumnPtr res; - if (!((res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)) - || (res = executeWithResultTypeAndLeftType(arguments)))) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); + DataTypePtr type_x = typeid_cast(arguments[0].type.get())->getNestedType(); - return res; + switch (type_x->getTypeId()) + { +#define ON_TYPE(type) \ + case TypeIndex::type: \ + return executeWithResultTypeAndLeftType(arguments); \ + break; + + SUPPORTED_TYPES(ON_TYPE) +#undef ON_TYPE + + default: + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Arguments of function {} has nested type {}. " + "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + getName(), + type_x->getName()); + } } template ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments) const { - ColumnPtr res; - if ( (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column)) - || (res = executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column))) - return res; + DataTypePtr type_y = typeid_cast(arguments[1].type.get())->getNestedType(); - return nullptr; + switch (type_y->getTypeId()) + { + #define ON_TYPE(type) \ + case TypeIndex::type: \ + return executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column); \ + break; + + SUPPORTED_TYPES(ON_TYPE) + #undef ON_TYPE + + default: + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Arguments of function {} has nested type {}. " + "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + getName(), + type_y->getName()); + } } template @@ -237,28 +247,22 @@ private: { col_x = col_x->convertToFullColumnIfConst(); col_y = col_y->convertToFullColumnIfConst(); - if (!col_x || !col_y) - return nullptr; - const ColumnArray * array_x = checkAndGetColumn(col_x.get()); - const ColumnArray * array_y = checkAndGetColumn(col_y.get()); - if (!array_x || !array_y) - return nullptr; + const auto & array_x = *assert_cast(col_x.get()); + const auto & array_y = *assert_cast(col_y.get()); - const ColumnVector * col_arr_nested_left = checkAndGetColumn>(array_x->getData()); - const ColumnVector * col_arr_nested_right = checkAndGetColumn>(array_y->getData()); - if (!col_arr_nested_left || !col_arr_nested_right) - return nullptr; + const auto & data_x = typeid_cast &>(array_x.getData()).getData(); + const auto & data_y = typeid_cast &>(array_y.getData()).getData(); - if (!array_x->hasEqualOffsets(*array_y)) + if (!array_x.hasEqualOffsets(array_y)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); auto col_res = ColumnVector::create(); vector( - col_arr_nested_left->getData(), - col_arr_nested_right->getData(), - array_x->getOffsets(), + data_x, + data_y, + array_x.getOffsets(), col_res->getData()); return col_res; From a69bcc29f5bb0bdaca1757673dac1574c97b1e2f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 14:53:11 +0000 Subject: [PATCH 03/78] Refactor a sanity check --- src/Functions/array/arrayDistance.cpp | 20 +++---------------- src/Functions/array/arrayDotProduct.cpp | 8 +++++--- .../queries/0_stateless/02708_dotProduct.sql | 2 +- 3 files changed, 9 insertions(+), 21 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 6b72c99d829..c9e6e97749f 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -18,11 +18,11 @@ namespace DB { namespace ErrorCodes { + extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int LOGICAL_ERROR; extern const int SIZES_OF_ARRAYS_DONT_MATCH; - extern const int ARGUMENT_OUT_OF_BOUND; } struct L1Distance @@ -465,22 +465,9 @@ private: const auto & data_y = typeid_cast &>(array_y.getData()).getData(); const auto & offsets_x = array_x.getOffsets(); - const auto & offsets_y = array_y.getOffsets(); - /// Check that arrays in both columns are the sames size - for (size_t row = 0; row < offsets_x.size(); ++row) - { - if (offsets_x[row] != offsets_y[row]) [[unlikely]] - { - ColumnArray::Offset prev_offset = row > 0 ? offsets_x[row] : 0; - throw Exception( - ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "Arguments of function {} have different array sizes: {} and {}", - getName(), - offsets_x[row] - prev_offset, - offsets_y[row] - prev_offset); - } - } + if (!array_x.hasEqualOffsets(array_y)) + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); const typename Kernel::ConstParams kernel_params = initConstParams(arguments); @@ -534,7 +521,6 @@ private: const auto & offsets_x = array_x.getOffsets(); const auto & offsets_y = array_y.getOffsets(); - /// Check that arrays in both columns are the sames size ColumnArray::Offset prev_offset = 0; for (size_t row : collections::range(0, offsets_y.size())) { diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index c27170cd35b..3abd1a6c6db 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -18,9 +18,9 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int LOGICAL_ERROR; + extern const int SIZES_OF_ARRAYS_DONT_MATCH; } @@ -254,15 +254,17 @@ private: const auto & data_x = typeid_cast &>(array_x.getData()).getData(); const auto & data_y = typeid_cast &>(array_y.getData()).getData(); + const auto & offsets_x = array_x.getOffsets(); + if (!array_x.hasEqualOffsets(array_y)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); auto col_res = ColumnVector::create(); vector( data_x, data_y, - array_x.getOffsets(), + offsets_x, col_res->getData()); return col_res; diff --git a/tests/queries/0_stateless/02708_dotProduct.sql b/tests/queries/0_stateless/02708_dotProduct.sql index 6ad615664e8..2035e23cf1d 100644 --- a/tests/queries/0_stateless/02708_dotProduct.sql +++ b/tests/queries/0_stateless/02708_dotProduct.sql @@ -4,7 +4,7 @@ SELECT arrayDotProduct([1, 2]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATC SELECT arrayDotProduct([1, 2], 'abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayDotProduct('abc', [1, 2]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayDotProduct([1, 2], ['abc', 'def']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayDotProduct([1, 2], [3, 4, 5]); -- { serverError BAD_ARGUMENTS } +SELECT arrayDotProduct([1, 2], [3, 4, 5]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT dotProduct([1, 2], (3, 4, 5)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT '-- Tests'; From 245ea0c186614a5a13a7e0bed79f94bc6ad46d87 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 15:26:11 +0000 Subject: [PATCH 04/78] Implement const/non-const shortcut --- src/Functions/array/arrayDistance.cpp | 3 +- src/Functions/array/arrayDotProduct.cpp | 110 ++++++++++++++---- tests/performance/dotProduct.xml | 1 + .../0_stateless/02708_dotProduct.reference | 7 ++ .../queries/0_stateless/02708_dotProduct.sql | 9 ++ 5 files changed, 106 insertions(+), 24 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index c9e6e97749f..03f0bc7b286 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -477,6 +477,7 @@ private: /// Do the actual computation ColumnArray::Offset prev = 0; size_t row = 0; + for (auto off : offsets_x) { /// Process chunks in vectorized manner @@ -522,7 +523,7 @@ private: const auto & offsets_y = array_y.getOffsets(); ColumnArray::Offset prev_offset = 0; - for (size_t row : collections::range(0, offsets_y.size())) + for (size_t row = 0; row < offsets_y.size(); ++row) { if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]] { diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 3abd1a6c6db..f9a6687e028 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -245,6 +245,15 @@ private: template ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y) const { + if (typeid_cast(col_x.get())) + { + return executeWithLeftArgConst(col_x, col_y); + } + else if (typeid_cast(col_y.get())) + { + return executeWithLeftArgConst(col_y, col_x); + } + col_x = col_x->convertToFullColumnIfConst(); col_y = col_y->convertToFullColumnIfConst(); @@ -260,30 +269,83 @@ private: throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); auto col_res = ColumnVector::create(); + auto & result = col_res->getData(); - vector( - data_x, - data_y, - offsets_x, - col_res->getData()); - - return col_res; - } - - template - static void vector( - const PaddedPODArray & left, - const PaddedPODArray & right, - const ColumnArray::Offsets & offsets, - PaddedPODArray & result) - { - size_t size = offsets.size(); + size_t size = offsets_x.size(); result.resize(size); ColumnArray::Offset current_offset = 0; for (size_t row = 0; row < size; ++row) { - size_t array_size = offsets[row] - current_offset; + size_t array_size = offsets_x[row] - current_offset; + + size_t i = 0; + + /// Process chunks in vectorized manner + static constexpr size_t VEC_SIZE = 4; + typename Kernel::template State states[VEC_SIZE]; + for (; i + VEC_SIZE < array_size; i += VEC_SIZE) + { + for (size_t j = 0; j < VEC_SIZE; ++j) + Kernel::template accumulate(states[j], static_cast(data_x[current_offset + i + j]), static_cast(data_y[current_offset + i + j])); + } + + typename Kernel::template State state; + for (const auto & other_state : states) + Kernel::template combine(state, other_state); + + /// Process the tail + for (; i < array_size; ++i) + Kernel::template accumulate(state, static_cast(data_x[current_offset + i]), static_cast(data_y[current_offset + i])); + + result[row] = Kernel::template finalize(state); + + current_offset = offsets_x[row]; + } + + return col_res; + } + + template + ColumnPtr executeWithLeftArgConst(ColumnPtr col_x, ColumnPtr col_y) const + { + col_x = assert_cast(col_x.get())->getDataColumnPtr(); + col_y = col_y->convertToFullColumnIfConst(); + + const auto & array_x = *assert_cast(col_x.get()); + const auto & array_y = *assert_cast(col_y.get()); + + const auto & data_x = typeid_cast &>(array_x.getData()).getData(); + const auto & data_y = typeid_cast &>(array_y.getData()).getData(); + + const auto & offsets_x = array_x.getOffsets(); + const auto & offsets_y = array_y.getOffsets(); + + ColumnArray::Offset prev_offset = 0; + for (size_t row = 0; row < offsets_y.size(); ++row) + { + if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]] + { + throw Exception( + ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, + "Arguments of function {} have different array sizes: {} and {}", + getName(), + offsets_x[0], + offsets_y[row] - prev_offset); + } + prev_offset = offsets_y[row]; + } + + auto col_res = ColumnVector::create(); + auto & result = col_res->getData(); + + size_t size = offsets_y.size(); + result.resize(size); + + ColumnArray::Offset current_offset = 0; + for (size_t row = 0; row < size; ++row) + { + size_t array_size = offsets_x[0]; typename Kernel::template State state; size_t i = 0; @@ -292,13 +354,14 @@ private: /// To avoid combinatorial explosion of SIMD kernels, focus on /// - the two most common input/output types (Float32 x Float32) --> Float32 and (Float64 x Float64) --> Float64 instead of 10 x /// 10 input types x 8 output types, + /// - const/non-const inputs instead of non-const/non-const inputs /// - the most powerful SIMD instruction set (AVX-512F). #if USE_MULTITARGET_CODE if constexpr ((std::is_same_v || std::is_same_v) && std::is_same_v && std::is_same_v) { if (isArchSupported(TargetArch::AVX512F)) - Kernel::template accumulateCombine(&left[current_offset], &right[current_offset], array_size, i, state); + Kernel::template accumulateCombine(&data_x[0], &data_y[current_offset], array_size, i, state); } #else /// Process chunks in vectorized manner @@ -307,7 +370,7 @@ private: for (; i + VEC_SIZE < array_size; i += VEC_SIZE) { for (size_t j = 0; j < VEC_SIZE; ++j) - Kernel::template accumulate(states[j], static_cast(left[i + j]), static_cast(right[i + j])); + Kernel::template accumulate(states[j], static_cast(data_x[i + j]), static_cast(data_y[current_offset + i + j])); } for (const auto & other_state : states) @@ -316,13 +379,14 @@ private: /// Process the tail for (; i < array_size; ++i) - Kernel::template accumulate(state, static_cast(left[i]), static_cast(right[i])); + Kernel::template accumulate(state, static_cast(data_x[i]), static_cast(data_y[current_offset + i])); - /// ResultType res = Kernel::template finalize(state); result[row] = Kernel::template finalize(state); - current_offset = offsets[row]; + current_offset = offsets_y[row]; } + + return col_res; } }; diff --git a/tests/performance/dotProduct.xml b/tests/performance/dotProduct.xml index 6e056964ebb..a0ab7beea9c 100644 --- a/tests/performance/dotProduct.xml +++ b/tests/performance/dotProduct.xml @@ -56,6 +56,7 @@ SELECT sum(dp) FROM (SELECT dotProduct(v, v) AS dp FROM vecs_{element_type}) + WITH (SELECT v FROM vecs_{element_type} limit 1) AS a SELECT sum(dp) FROM (SELECT dotProduct(a, v) AS dp FROM vecs_{element_type}) DROP TABLE vecs_{element_type} diff --git a/tests/queries/0_stateless/02708_dotProduct.reference b/tests/queries/0_stateless/02708_dotProduct.reference index 5cc9a9f0502..593071a3521 100644 --- a/tests/queries/0_stateless/02708_dotProduct.reference +++ b/tests/queries/0_stateless/02708_dotProduct.reference @@ -32,3 +32,10 @@ 32 32 32 +-- Tests that trigger special paths +0 61 +1 186 +0 62 +1 187 +0 62 +1 187 diff --git a/tests/queries/0_stateless/02708_dotProduct.sql b/tests/queries/0_stateless/02708_dotProduct.sql index 2035e23cf1d..ac94ecc28d3 100644 --- a/tests/queries/0_stateless/02708_dotProduct.sql +++ b/tests/queries/0_stateless/02708_dotProduct.sql @@ -45,3 +45,12 @@ SELECT '-- Aliases'; SELECT scalarProduct([1, 2, 3], [4, 5, 6]); SELECT scalarProduct((1, 2, 3), (4, 5, 6)); SELECT arrayDotProduct([1, 2, 3], [4, 5, 6]); -- actually no alias but the internal function for arrays + +SELECT '-- Tests that trigger special paths'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(id UInt64, vec Array(Float32)) ENGINE = MergeTree ORDER BY id; +INSERT INTO tab VALUES (0, [0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0]) (1, [5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]); +SELECT id, arrayDotProduct(vec, vec) FROM tab ORDER BY id; -- non-const / non-const +SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float32), vec) FROM tab ORDER BY id; -- const / non-const +SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float64), vec) FROM tab ORDER BY id; -- const / non-const +DROP TABLE tab; From 87f3c957c7dc3d16d9967723e30215e12a0b5dc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Mar 2024 21:16:22 +0100 Subject: [PATCH 05/78] Blind experiment --- base/base/itoa.h | 30 +++++++++++++++++++++++------- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/base/base/itoa.h b/base/base/itoa.h index 513070c99d9..c450090d82f 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -349,16 +349,32 @@ namespace convert template -static inline char * writeUIntText(T x, char * p) +static inline char * writeUIntText(T _x, char * p) { - static_assert(is_unsigned_v); +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wbit-int-extension" + int len = digits10(_x); + static_assert(std::is_same_v || std::is_same_v); + using T_ = std::conditional_t, unsigned __int128, unsigned _BitInt(256)>; +#pragma clang diagnostic pop - int len = digits10(x); - auto * pp = p + len; - while (x >= 100) + T_ x; + T_ hundred(100ULL); + if constexpr (std::is_same_v) { - const auto i = x % 100; - x /= 100; + x = (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); + } + else + { + x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) + + (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); + } + + auto * pp = p + len; + while (x >= hundred) + { + const auto i = x % hundred; + x /= hundred; pp -= 2; outTwoDigits(pp, i); } From c192d0b12532060d14934e60164df7ce771d9399 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 6 Mar 2024 22:29:41 +0100 Subject: [PATCH 06/78] impl --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 83 +++++++++++++------ src/Storages/MergeTree/IMergeTreeDataPart.h | 8 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartCompact.h | 12 +-- .../MergeTree/MergeTreeDataPartWide.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.h | 4 +- .../MergeTree/MergeTreeIndexGranularity.cpp | 5 +- .../MergeTree/MergeTreeIndexGranularity.h | 2 + .../MergeTreeIndexGranularityInfo.cpp | 14 +++- .../MergeTree/MergeTreeIndexGranularityInfo.h | 4 + 10 files changed, 97 insertions(+), 41 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index a9bdceacef0..3922d5018c6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -2,38 +2,41 @@ #include #include +#include #include -#include #include +#include +#include #include -#include +#include +#include +#include +#include #include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include @@ -1966,7 +1969,39 @@ void IMergeTreeDataPart::checkConsistencyBase() const } } -void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) const +void IMergeTreeDataPart::checkConsistency(bool require_part_metadata) const +{ + try + { + checkConsistencyBase(); + doCheckConsistency(require_part_metadata); + } + catch (Exception & e) + { + const auto part_state = fmt::format( + "state: {}, is_unexpected_local_part: {}, is_frozen: {}, is_duplicate: {}", + stateString(), + is_unexpected_local_part, + is_frozen, + is_duplicate, + is_temp); + + const auto debug_info = fmt::format( + "columns: {}, getMarkSizeInBytes: {}, getMarksCount: {}, index_granularity_info: [{}], index_granularity: [{}], " + "part_state: [{}]", + columns.toString(), + index_granularity_info.getMarkSizeInBytes(columns.size()), + index_granularity.getMarksCount(), + index_granularity_info.describe(), + index_granularity.describe(), + part_state); + + e.addMessage(debug_info); + e.rethrow(); + } +} + +void IMergeTreeDataPart::doCheckConsistency(bool /* require_part_metadata */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'checkConsistency' is not implemented for part with type {}", getType().toString()); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 91c559d30c8..209c2d9a256 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -498,7 +498,7 @@ public: void writeChecksums(const MergeTreeDataPartChecksums & checksums_, const WriteSettings & settings); /// Checks the consistency of this data part. - virtual void checkConsistency(bool require_part_metadata) const; + void checkConsistency(bool require_part_metadata) const; /// Checks the consistency of this data part, and check the consistency of its projections (if any) as well. void checkConsistencyWithProjections(bool require_part_metadata) const; @@ -586,8 +586,6 @@ protected: void removeIfNeeded(); - void checkConsistencyBase() const; - /// Fill each_columns_size and total_size with sizes from columns files on /// disk using columns and checksums. virtual void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0; @@ -610,6 +608,8 @@ protected: void initializeIndexGranularityInfo(); + virtual void doCheckConsistency(bool require_part_metadata) const; + private: String mutable_name; mutable MergeTreeDataPartState state{MergeTreeDataPartState::Temporary}; @@ -697,6 +697,8 @@ private: void incrementStateMetric(MergeTreeDataPartState state) const; void decrementStateMetric(MergeTreeDataPartState state) const; + void checkConsistencyBase() const; + /// This ugly flag is needed for debug assertions only mutable bool part_is_probably_removed_from_disk = false; }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 0ecd7abe183..5d4b602b5b8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -157,9 +157,8 @@ std::optional MergeTreeDataPartCompact::getColumnModificationTime(const return getDataPartStorage().getFileLastModified(DATA_FILE_NAME_WITH_EXTENSION).epochTime(); } -void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata) const +void MergeTreeDataPartCompact::doCheckConsistency(bool require_part_metadata) const { - checkConsistencyBase(); String mrk_file_name = DATA_FILE_NAME + getMarksFileExtension(); if (!checksums.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 35a358b3720..f897bcb0bfd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -67,14 +67,14 @@ protected: MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_, size_t columns_count, const IDataPartStorage & data_part_storage_); -private: - void checkConsistency(bool require_part_metadata) const override; + void doCheckConsistency(bool require_part_metadata) const override; - /// Loads marks index granularity into memory - void loadIndexGranularity() override; + private: + /// Loads marks index granularity into memory + void loadIndexGranularity() override; - /// Compact parts doesn't support per column size, only total size - void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + /// Compact parts doesn't support per column size, only total size + void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index dc6c1f0019d..0111f1e7b40 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -182,9 +182,8 @@ MergeTreeDataPartWide::~MergeTreeDataPartWide() removeIfNeeded(); } -void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const +void MergeTreeDataPartWide::doCheckConsistency(bool require_part_metadata) const { - checkConsistencyBase(); std::string marks_file_extension = index_granularity_info.mark_type.getFileExtension(); if (!checksums.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 14147c4ad56..508ea16d2d4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -62,9 +62,9 @@ protected: MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, const IDataPartStorage & data_part_storage_, const std::string & any_column_file_name); -private: - void checkConsistency(bool require_part_metadata) const override; + void doCheckConsistency(bool require_part_metadata) const override; +private: /// Loads marks index granularity into memory void loadIndexGranularity() override; diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index 5fdd0555777..2a45ab1d927 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -121,5 +121,8 @@ void MergeTreeIndexGranularity::resizeWithFixedGranularity(size_t size, size_t f } } - +std::string MergeTreeIndexGranularity::describe() const +{ + return fmt::format("initialized: {}, marks_rows_partial_sums: [{}]", initialized, fmt::join(marks_rows_partial_sums, ", ")); +} } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.h b/src/Storages/MergeTree/MergeTreeIndexGranularity.h index f5677995ae0..d67762f7293 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.h @@ -95,6 +95,8 @@ public: /// Add `size` of marks with `fixed_granularity` rows void resizeWithFixedGranularity(size_t size, size_t fixed_granularity); + + std::string describe() const; }; } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index da89d52a9ff..1ff72a4e36d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -88,6 +88,10 @@ std::string MarkType::getFileExtension() const } } +std::string MarkType::describe() const +{ + return fmt::format("adaptive: {}, compressed: {}, part_type: {}", adaptive, compressed, part_type); +} std::optional MergeTreeIndexGranularityInfo::getMarksTypeFromFilesystem(const IDataPartStorage & data_part_storage) { @@ -132,10 +136,18 @@ size_t MergeTreeIndexGranularityInfo::getMarkSizeInBytes(size_t columns_num) con throw Exception(ErrorCodes::UNKNOWN_PART_TYPE, "Unknown part type"); } +std::string MergeTreeIndexGranularityInfo::describe() const +{ + return fmt::format( + "mark_type: [{}], index_granularity_bytes: {}, fixed_index_granularity: {}", + mark_type.describe(), + index_granularity_bytes, + fixed_index_granularity); +} + size_t getAdaptiveMrkSizeCompact(size_t columns_num) { /// Each mark contains number of rows in granule and two offsets for every column. return sizeof(UInt64) * (columns_num * 2 + 1); } - } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h index af008866919..85006c3ffde 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h @@ -24,6 +24,8 @@ struct MarkType static bool isMarkFileExtension(std::string_view extension); std::string getFileExtension() const; + std::string describe() const; + bool adaptive = false; bool compressed = false; MergeTreeDataPartType::Value part_type = MergeTreeDataPartType::Unknown; @@ -58,6 +60,8 @@ public: size_t getMarkSizeInBytes(size_t columns_num = 1) const; static std::optional getMarksTypeFromFilesystem(const IDataPartStorage & data_part_storage); + + std::string describe() const; }; constexpr inline auto getNonAdaptiveMrkSizeWide() { return sizeof(UInt64) * 2; } From 2b52583e06056e19df97216f41b81102bca8bd9d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 6 Mar 2024 23:01:42 +0100 Subject: [PATCH 07/78] fix style --- src/Storages/MergeTree/MergeTreeDataPartCompact.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index f897bcb0bfd..8bbec2808d7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -69,7 +69,7 @@ protected: void doCheckConsistency(bool require_part_metadata) const override; - private: +private: /// Loads marks index granularity into memory void loadIndexGranularity() override; From f0a8d8843de5dffae2e1d4476fb119ad34059340 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Mar 2024 00:10:06 +0100 Subject: [PATCH 08/78] Not x86_64 are lagging behind in features --- base/base/itoa.h | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/base/base/itoa.h b/base/base/itoa.h index c450090d82f..a36eecaf1e5 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -351,12 +351,20 @@ namespace convert template static inline char * writeUIntText(T _x, char * p) { -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wbit-int-extension" int len = digits10(_x); static_assert(std::is_same_v || std::is_same_v); - using T_ = std::conditional_t, unsigned __int128, unsigned _BitInt(256)>; + using T_ = std::conditional_t< + std::is_same_v, + unsigned __int128, +#if defined(__x86_64__) +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wbit-int-extension" + unsigned _BitInt(256) #pragma clang diagnostic pop +#else + T +#endif + >; T_ x; T_ hundred(100ULL); @@ -366,8 +374,12 @@ static inline char * writeUIntText(T _x, char * p) } else { +#if defined(__x86_64__) x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) + (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); +#else + x = _x; +#endif } auto * pp = p + len; From 66dea5111298abd4301df55b5615d158105fe78f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Mar 2024 12:40:48 +0100 Subject: [PATCH 09/78] fix clang-tidy --- src/Functions/array/arrayDotProduct.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index f9a6687e028..8b7c85e05dd 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -322,18 +322,18 @@ private: const auto & offsets_y = array_y.getOffsets(); ColumnArray::Offset prev_offset = 0; - for (size_t row = 0; row < offsets_y.size(); ++row) + for (auto offset_y : offsets_y) { - if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]] + if (offsets_x[0] != offset_y - prev_offset) [[unlikely]] { throw Exception( ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Arguments of function {} have different array sizes: {} and {}", getName(), offsets_x[0], - offsets_y[row] - prev_offset); + offset_y - prev_offset); } - prev_offset = offsets_y[row]; + prev_offset = offset_y; } auto col_res = ColumnVector::create(); From cd9d9018e0db8139e48cb722e9e9685d2a212c8b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Mar 2024 17:15:42 +0100 Subject: [PATCH 10/78] Be able to iterate --- base/base/CMakeLists.txt | 1 + base/base/itoa.cpp | 503 +++++++++++++++++++++++++++++++++++ base/base/itoa.h | 498 +++------------------------------- src/Functions/CMakeLists.txt | 1 + 4 files changed, 540 insertions(+), 463 deletions(-) create mode 100644 base/base/itoa.cpp diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 548ba01d86a..55d046767b8 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -19,6 +19,7 @@ set (SRCS getPageSize.cpp getThreadId.cpp int8_to_string.cpp + itoa.cpp JSON.cpp mremap.cpp phdr_cache.cpp diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp new file mode 100644 index 00000000000..9fefc9f0f07 --- /dev/null +++ b/base/base/itoa.cpp @@ -0,0 +1,503 @@ +// Based on https://github.com/amdn/itoa and combined with our optimizations +// +//=== itoa.h - Fast integer to ascii conversion --*- C++ -*-// +// +// The MIT License (MIT) +// Copyright (c) 2016 Arturo Martin-de-Nicolas +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included +// in all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +// SOFTWARE. +//===----------------------------------------------------------------------===// + +#include +#include +#include +#include +#include +#include +#include + + +template +int digits10(T x) +{ + if (x < T(10ULL)) + return 1; + if (x < T(100ULL)) + return 2; + if constexpr (sizeof(T) == 1) + return 3; + else + { + if (x < T(1000ULL)) + return 3; + + if (x < T(1000000000000ULL)) + { + if (x < T(100000000ULL)) + { + if (x < T(1000000ULL)) + { + if (x < T(10000ULL)) + return 4; + else + return 5 + (x >= T(100000ULL)); + } + + return 7 + (x >= T(10000000ULL)); + } + + if (x < T(10000000000ULL)) + return 9 + (x >= T(1000000000ULL)); + + return 11 + (x >= T(100000000000ULL)); + } + + return 12 + digits10(x / T(1000000000000ULL)); + } +} + + +namespace +{ + +template +static constexpr T pow10(size_t x) +{ + return x ? 10 * pow10(x - 1) : 1; +} + +// Division by a power of 10 is implemented using a multiplicative inverse. +// This strength reduction is also done by optimizing compilers, but +// presently the fastest results are produced by using the values +// for the multiplication and the shift as given by the algorithm +// described by Agner Fog in "Optimizing Subroutines in Assembly Language" +// +// http://www.agner.org/optimize/optimizing_assembly.pdf +// +// "Integer division by a constant (all processors) +// A floating point number can be divided by a constant by multiplying +// with the reciprocal. If we want to do the same with integers, we have +// to scale the reciprocal by 2n and then shift the product to the right +// by n. There are various algorithms for finding a suitable value of n +// and compensating for rounding errors. The algorithm described below +// was invented by Terje Mathisen, Norway, and not published elsewhere." + +/// Division by constant is performed by: +/// 1. Adding 1 if needed; +/// 2. Multiplying by another constant; +/// 3. Shifting right by another constant. +template +struct Division +{ + static constexpr bool add{add_}; + static constexpr UInt multiplier{multiplier_}; + static constexpr unsigned shift{shift_}; +}; + +/// Select a type with appropriate number of bytes from the list of types. +/// First parameter is the number of bytes requested. Then goes a list of types with 1, 2, 4, ... number of bytes. +/// Example: SelectType<4, uint8_t, uint16_t, uint32_t, uint64_t> will select uint32_t. +template +struct SelectType +{ + using Result = typename SelectType::Result; +}; + +template +struct SelectType<1, T, Ts...> +{ + using Result = T; +}; + + +/// Division by 10^N where N is the size of the type. +template +using DivisionBy10PowN = typename SelectType< + N, + Division, /// divide by 10 + Division, /// divide by 100 + Division, /// divide by 10000 + Division /// divide by 100000000 + >::Result; + +template +using UnsignedOfSize = typename SelectType::Result; + +/// Holds the result of dividing an unsigned N-byte variable by 10^N resulting in +template +struct QuotientAndRemainder +{ + UnsignedOfSize quotient; // quotient with fewer than 2*N decimal digits + UnsignedOfSize remainder; // remainder with at most N decimal digits +}; + +template +QuotientAndRemainder static inline split(UnsignedOfSize value) +{ + constexpr DivisionBy10PowN division; + + UnsignedOfSize quotient = (division.multiplier * (UnsignedOfSize<2 * N>(value) + division.add)) >> division.shift; + UnsignedOfSize remainder = static_cast>(value - quotient * pow10>(N)); + + return {quotient, remainder}; +} + + +static inline char * outDigit(char * p, uint8_t value) +{ + *p = '0' + value; + ++p; + return p; +} + +// Using a lookup table to convert binary numbers from 0 to 99 +// into ascii characters as described by Andrei Alexandrescu in +// https://www.facebook.com/notes/facebook-engineering/three-optimization-tips-for-c/10151361643253920/ + +static const char digits[201] = "00010203040506070809" + "10111213141516171819" + "20212223242526272829" + "30313233343536373839" + "40414243444546474849" + "50515253545556575859" + "60616263646566676869" + "70717273747576777879" + "80818283848586878889" + "90919293949596979899"; + +static inline char * outTwoDigits(char * p, uint8_t value) +{ + memcpy(p, &digits[value * 2], 2); + p += 2; + return p; +} + + +namespace convert +{ +template +static char * head(char * p, UInt u); +template +static char * tail(char * p, UInt u); + +//===----------------------------------------------------------===// +// head: find most significant digit, skip leading zeros +//===----------------------------------------------------------===// + +// "x" contains quotient and remainder after division by 10^N +// quotient is less than 10^N +template +static inline char * head(char * p, QuotientAndRemainder x) +{ + p = head(p, UnsignedOfSize(x.quotient)); + p = tail(p, x.remainder); + return p; +} + +// "u" is less than 10^2*N +template +static inline char * head(char * p, UInt u) +{ + return u < pow10>(N) ? head(p, UnsignedOfSize(u)) : head(p, split(u)); +} + +// recursion base case, selected when "u" is one byte +template <> +inline char * head, 1>(char * p, UnsignedOfSize<1> u) +{ + return u < 10 ? outDigit(p, u) : outTwoDigits(p, u); +} + +//===----------------------------------------------------------===// +// tail: produce all digits including leading zeros +//===----------------------------------------------------------===// + +// recursive step, "u" is less than 10^2*N +template +static inline char * tail(char * p, UInt u) +{ + QuotientAndRemainder x = split(u); + p = tail(p, UnsignedOfSize(x.quotient)); + p = tail(p, x.remainder); + return p; +} + +// recursion base case, selected when "u" is one byte +template <> +inline char * tail, 1>(char * p, UnsignedOfSize<1> u) +{ + return outTwoDigits(p, u); +} + +//===----------------------------------------------------------===// +// large values are >= 10^2*N +// where x contains quotient and remainder after division by 10^N +//===----------------------------------------------------------===// + +template +static inline char * large(char * p, QuotientAndRemainder x) +{ + QuotientAndRemainder y = split(x.quotient); + p = head(p, UnsignedOfSize(y.quotient)); + p = tail(p, y.remainder); + p = tail(p, x.remainder); + return p; +} + +//===----------------------------------------------------------===// +// handle values of "u" that might be >= 10^2*N +// where N is the size of "u" in bytes +//===----------------------------------------------------------===// + +template +static inline char * uitoa(char * p, UInt u) +{ + if (u < pow10>(N)) + return head(p, UnsignedOfSize(u)); + QuotientAndRemainder x = split(u); + + return u < pow10>(2 * N) ? head(p, x) : large(p, x); +} + +// selected when "u" is one byte +template <> +inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) +{ + if (u < 10) + return outDigit(p, u); + else if (u < 100) + return outTwoDigits(p, u); + else + { + p = outDigit(p, u / 100); + p = outTwoDigits(p, u % 100); + return p; + } +} + +//===----------------------------------------------------------===// +// handle unsigned and signed integral operands +//===----------------------------------------------------------===// + +// itoa: handle unsigned integral operands (selected by SFINAE) +template && std::is_integral_v> * = nullptr> +static inline char * itoa(U u, char * p) +{ + return convert::uitoa(p, u); +} + +// itoa: handle signed integral operands (selected by SFINAE) +template && std::is_integral_v> * = nullptr> +static inline char * itoa(I i, char * p) +{ + // Need "mask" to be filled with a copy of the sign bit. + // If "i" is a negative value, then the result of "operator >>" + // is implementation-defined, though usually it is an arithmetic + // right shift that replicates the sign bit. + // Use a conditional expression to be portable, + // a good optimizing compiler generates an arithmetic right shift + // and avoids the conditional branch. + UnsignedOfSize mask = i < 0 ? ~UnsignedOfSize(0) : 0; + // Now get the absolute value of "i" and cast to unsigned type UnsignedOfSize. + // Cannot use std::abs() because the result is undefined + // in 2's complement systems for the most-negative value. + // Want to avoid conditional branch for performance reasons since + // CPU branch prediction will be ineffective when negative values + // occur randomly. + // Let "u" be "i" cast to unsigned type UnsignedOfSize. + // Subtract "u" from 2*u if "i" is positive or 0 if "i" is negative. + // This yields the absolute value with the desired type without + // using a conditional branch and without invoking undefined or + // implementation defined behavior: + UnsignedOfSize u = ((2 * UnsignedOfSize(i)) & ~mask) - UnsignedOfSize(i); + // Unconditionally store a minus sign when producing digits + // in a forward direction and increment the pointer only if + // the value is in fact negative. + // This avoids a conditional branch and is safe because we will + // always produce at least one digit and it will overwrite the + // minus sign when the value is not negative. + *p = '-'; + p += (mask & 1); + p = convert::uitoa(p, u); + return p; +} +} + + +template +static NO_INLINE char * writeUIntText(T _x, char * p) +{ + static_assert(std::is_same_v || std::is_same_v); + using T_ = std::conditional_t< + std::is_same_v, + unsigned __int128, +#if defined(__x86_64__) +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wbit-int-extension" + unsigned _BitInt(256) +# pragma clang diagnostic pop +#else + T +#endif + >; + + T_ x; + T_ hundred(100ULL); + if constexpr (std::is_same_v) + { + x = (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); + } + else + { +#if defined(__x86_64__) + x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) + + (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); +#else + x = _x; +#endif + } + + int len = digits10(x); + auto * pp = p + len; + while (x >= hundred) + { + const auto i = x % hundred; + x /= hundred; + pp -= 2; + outTwoDigits(pp, i); + } + if (x < 10) + *p = '0' + x; + else + outTwoDigits(p, x); + return p + len; +} + +static ALWAYS_INLINE inline char * writeLeadingMinus(char * pos) +{ + *pos = '-'; + return pos + 1; +} + +template +static ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) +{ + static_assert(std::is_same_v || std::is_same_v); + + using UnsignedT = make_unsigned_t; + static constexpr T min_int = UnsignedT(1) << (sizeof(T) * 8 - 1); + + if (unlikely(x == min_int)) + { + if constexpr (std::is_same_v) + { + const char * res = "-170141183460469231731687303715884105728"; + memcpy(pos, res, strlen(res)); + return pos + strlen(res); + } + else if constexpr (std::is_same_v) + { + const char * res = "-57896044618658097711785492504343953926634992332820282019728792003956564819968"; + memcpy(pos, res, strlen(res)); + return pos + strlen(res); + } + } + + if (x < 0) + { + x = -x; + pos = writeLeadingMinus(pos); + } + return writeUIntText(UnsignedT(x), pos); +} +} + +template +char * itoa(T i, char * p) +{ + return convert::itoa(i, p); +} + +template <> +char * itoa(UInt8 i, char * p) +{ + return convert::itoa(uint8_t(i), p); +} + +template <> +char * itoa(Int8 i, char * p) +{ + return convert::itoa(int8_t(i), p); +} + +template <> +char * itoa(UInt128 i, char * p) +{ + return writeUIntText(i, p); +} + +template <> +char * itoa(Int128 i, char * p) +{ + return writeSIntText(i, p); +} + +template <> +char * itoa(UInt256 i, char * p) +{ + return writeUIntText(i, p); +} + +template <> +char * itoa(Int256 i, char * p) +{ + return writeSIntText(i, p); +} + +#define FOR_MISSING_INTEGER_TYPES(M) \ + M(int8_t) \ + M(uint8_t) \ + M(UInt16) \ + M(UInt32) \ + M(UInt64) \ + M(Int16) \ + M(Int32) \ + M(Int64) + +#define INSTANTIATION(T) template char * itoa(T i, char * p); +FOR_MISSING_INTEGER_TYPES(INSTANTIATION) + +#undef FOR_MISSING_INTEGER_TYPES +#undef INSTANTIATION + + +#define DIGITS_INTEGER_TYPES(M) \ + M(uint8_t) \ + M(UInt8) \ + M(UInt16) \ + M(UInt32) \ + M(UInt64) \ + M(UInt128) \ + M(UInt256) + +#define INSTANTIATION(T) template int digits10(T x); +DIGITS_INTEGER_TYPES(INSTANTIATION) + +#undef DIGITS_INTEGER_TYPES +#undef INSTANTIATION diff --git a/base/base/itoa.h b/base/base/itoa.h index a36eecaf1e5..71603cdeb88 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -1,474 +1,46 @@ #pragma once -// Based on https://github.com/amdn/itoa and combined with our optimizations -// -//=== itoa.h - Fast integer to ascii conversion --*- C++ -*-// -// -// The MIT License (MIT) -// Copyright (c) 2016 Arturo Martin-de-Nicolas -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included -// in all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -// SOFTWARE. -//===----------------------------------------------------------------------===// - -#include -#include -#include -#include #include +template char * itoa(T i, char * p); -template -inline int digits10(T x) -{ - if (x < 10ULL) - return 1; - if (x < 100ULL) - return 2; - if (x < 1000ULL) - return 3; +template <> char * itoa(UInt8 i, char * p); +template <> char * itoa(Int8 i, char * p); +template <> char * itoa(UInt128 i, char * p); +template <> char * itoa(Int128 i, char * p); +template <> char * itoa(UInt256 i, char * p); +template <> char * itoa(Int256 i, char * p); - if (x < 1000000000000ULL) - { - if (x < 100000000ULL) - { - if (x < 1000000ULL) - { - if (x < 10000ULL) - return 4; - else - return 5 + (x >= 100000ULL); - } +#define FOR_MISSING_INTEGER_TYPES(M) \ + M(int8_t) \ + M(uint8_t) \ + M(UInt16) \ + M(UInt32) \ + M(UInt64) \ + M(Int16) \ + M(Int32) \ + M(Int64) - return 7 + (x >= 10000000ULL); - } +#define INSTANTIATION(T) \ + extern template char * itoa(T i, char * p); +FOR_MISSING_INTEGER_TYPES(INSTANTIATION) - if (x < 10000000000ULL) - return 9 + (x >= 1000000000ULL); - - return 11 + (x >= 100000000000ULL); - } - - return 12 + digits10(x / 1000000000000ULL); -} +#undef FOR_MISSING_INTEGER_TYPES +#undef INSTANTIATION -namespace impl -{ +template int digits10(T x); -template -static constexpr T pow10(size_t x) -{ - return x ? 10 * pow10(x - 1) : 1; -} - -// Division by a power of 10 is implemented using a multiplicative inverse. -// This strength reduction is also done by optimizing compilers, but -// presently the fastest results are produced by using the values -// for the multiplication and the shift as given by the algorithm -// described by Agner Fog in "Optimizing Subroutines in Assembly Language" -// -// http://www.agner.org/optimize/optimizing_assembly.pdf -// -// "Integer division by a constant (all processors) -// A floating point number can be divided by a constant by multiplying -// with the reciprocal. If we want to do the same with integers, we have -// to scale the reciprocal by 2n and then shift the product to the right -// by n. There are various algorithms for finding a suitable value of n -// and compensating for rounding errors. The algorithm described below -// was invented by Terje Mathisen, Norway, and not published elsewhere." - -/// Division by constant is performed by: -/// 1. Adding 1 if needed; -/// 2. Multiplying by another constant; -/// 3. Shifting right by another constant. -template -struct Division -{ - static constexpr bool add{add_}; - static constexpr UInt multiplier{multiplier_}; - static constexpr unsigned shift{shift_}; -}; - -/// Select a type with appropriate number of bytes from the list of types. -/// First parameter is the number of bytes requested. Then goes a list of types with 1, 2, 4, ... number of bytes. -/// Example: SelectType<4, uint8_t, uint16_t, uint32_t, uint64_t> will select uint32_t. -template -struct SelectType -{ - using Result = typename SelectType::Result; -}; - -template -struct SelectType<1, T, Ts...> -{ - using Result = T; -}; - - -/// Division by 10^N where N is the size of the type. -template -using DivisionBy10PowN = typename SelectType -< - N, - Division, /// divide by 10 - Division, /// divide by 100 - Division, /// divide by 10000 - Division /// divide by 100000000 ->::Result; - -template -using UnsignedOfSize = typename SelectType -< - N, - uint8_t, - uint16_t, - uint32_t, - uint64_t, - __uint128_t ->::Result; - -/// Holds the result of dividing an unsigned N-byte variable by 10^N resulting in -template -struct QuotientAndRemainder -{ - UnsignedOfSize quotient; // quotient with fewer than 2*N decimal digits - UnsignedOfSize remainder; // remainder with at most N decimal digits -}; - -template -QuotientAndRemainder static inline split(UnsignedOfSize value) -{ - constexpr DivisionBy10PowN division; - - UnsignedOfSize quotient = (division.multiplier * (UnsignedOfSize<2 * N>(value) + division.add)) >> division.shift; - UnsignedOfSize remainder = static_cast>(value - quotient * pow10>(N)); - - return {quotient, remainder}; -} - - -static inline char * outDigit(char * p, uint8_t value) -{ - *p = '0' + value; - ++p; - return p; -} - -// Using a lookup table to convert binary numbers from 0 to 99 -// into ascii characters as described by Andrei Alexandrescu in -// https://www.facebook.com/notes/facebook-engineering/three-optimization-tips-for-c/10151361643253920/ - -static const char digits[201] = "00010203040506070809" - "10111213141516171819" - "20212223242526272829" - "30313233343536373839" - "40414243444546474849" - "50515253545556575859" - "60616263646566676869" - "70717273747576777879" - "80818283848586878889" - "90919293949596979899"; - -static inline char * outTwoDigits(char * p, uint8_t value) -{ - memcpy(p, &digits[value * 2], 2); - p += 2; - return p; -} - - -namespace convert -{ - template static char * head(char * p, UInt u); - template static char * tail(char * p, UInt u); - - //===----------------------------------------------------------===// - // head: find most significant digit, skip leading zeros - //===----------------------------------------------------------===// - - // "x" contains quotient and remainder after division by 10^N - // quotient is less than 10^N - template - static inline char * head(char * p, QuotientAndRemainder x) - { - p = head(p, UnsignedOfSize(x.quotient)); - p = tail(p, x.remainder); - return p; - } - - // "u" is less than 10^2*N - template - static inline char * head(char * p, UInt u) - { - return u < pow10>(N) - ? head(p, UnsignedOfSize(u)) - : head(p, split(u)); - } - - // recursion base case, selected when "u" is one byte - template <> - inline char * head, 1>(char * p, UnsignedOfSize<1> u) - { - return u < 10 - ? outDigit(p, u) - : outTwoDigits(p, u); - } - - //===----------------------------------------------------------===// - // tail: produce all digits including leading zeros - //===----------------------------------------------------------===// - - // recursive step, "u" is less than 10^2*N - template - static inline char * tail(char * p, UInt u) - { - QuotientAndRemainder x = split(u); - p = tail(p, UnsignedOfSize(x.quotient)); - p = tail(p, x.remainder); - return p; - } - - // recursion base case, selected when "u" is one byte - template <> - inline char * tail, 1>(char * p, UnsignedOfSize<1> u) - { - return outTwoDigits(p, u); - } - - //===----------------------------------------------------------===// - // large values are >= 10^2*N - // where x contains quotient and remainder after division by 10^N - //===----------------------------------------------------------===// - - template - static inline char * large(char * p, QuotientAndRemainder x) - { - QuotientAndRemainder y = split(x.quotient); - p = head(p, UnsignedOfSize(y.quotient)); - p = tail(p, y.remainder); - p = tail(p, x.remainder); - return p; - } - - //===----------------------------------------------------------===// - // handle values of "u" that might be >= 10^2*N - // where N is the size of "u" in bytes - //===----------------------------------------------------------===// - - template - static inline char * uitoa(char * p, UInt u) - { - if (u < pow10>(N)) - return head(p, UnsignedOfSize(u)); - QuotientAndRemainder x = split(u); - - return u < pow10>(2 * N) - ? head(p, x) - : large(p, x); - } - - // selected when "u" is one byte - template <> - inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) - { - if (u < 10) - return outDigit(p, u); - else if (u < 100) - return outTwoDigits(p, u); - else - { - p = outDigit(p, u / 100); - p = outTwoDigits(p, u % 100); - return p; - } - } - - //===----------------------------------------------------------===// - // handle unsigned and signed integral operands - //===----------------------------------------------------------===// - - // itoa: handle unsigned integral operands (selected by SFINAE) - template && std::is_integral_v> * = nullptr> - static inline char * itoa(U u, char * p) - { - return convert::uitoa(p, u); - } - - // itoa: handle signed integral operands (selected by SFINAE) - template && std::is_integral_v> * = nullptr> - static inline char * itoa(I i, char * p) - { - // Need "mask" to be filled with a copy of the sign bit. - // If "i" is a negative value, then the result of "operator >>" - // is implementation-defined, though usually it is an arithmetic - // right shift that replicates the sign bit. - // Use a conditional expression to be portable, - // a good optimizing compiler generates an arithmetic right shift - // and avoids the conditional branch. - UnsignedOfSize mask = i < 0 ? ~UnsignedOfSize(0) : 0; - // Now get the absolute value of "i" and cast to unsigned type UnsignedOfSize. - // Cannot use std::abs() because the result is undefined - // in 2's complement systems for the most-negative value. - // Want to avoid conditional branch for performance reasons since - // CPU branch prediction will be ineffective when negative values - // occur randomly. - // Let "u" be "i" cast to unsigned type UnsignedOfSize. - // Subtract "u" from 2*u if "i" is positive or 0 if "i" is negative. - // This yields the absolute value with the desired type without - // using a conditional branch and without invoking undefined or - // implementation defined behavior: - UnsignedOfSize u = ((2 * UnsignedOfSize(i)) & ~mask) - UnsignedOfSize(i); - // Unconditionally store a minus sign when producing digits - // in a forward direction and increment the pointer only if - // the value is in fact negative. - // This avoids a conditional branch and is safe because we will - // always produce at least one digit and it will overwrite the - // minus sign when the value is not negative. - *p = '-'; - p += (mask & 1); - p = convert::uitoa(p, u); - return p; - } -} - - -template -static inline char * writeUIntText(T _x, char * p) -{ - int len = digits10(_x); - static_assert(std::is_same_v || std::is_same_v); - using T_ = std::conditional_t< - std::is_same_v, - unsigned __int128, -#if defined(__x86_64__) -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wbit-int-extension" - unsigned _BitInt(256) -#pragma clang diagnostic pop -#else - T -#endif - >; - - T_ x; - T_ hundred(100ULL); - if constexpr (std::is_same_v) - { - x = (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); - } - else - { -#if defined(__x86_64__) - x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) + - (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); -#else - x = _x; -#endif - } - - auto * pp = p + len; - while (x >= hundred) - { - const auto i = x % hundred; - x /= hundred; - pp -= 2; - outTwoDigits(pp, i); - } - if (x < 10) - *p = '0' + x; - else - outTwoDigits(p, x); - return p + len; -} - -static inline char * writeLeadingMinus(char * pos) -{ - *pos = '-'; - return pos + 1; -} - -template -static inline char * writeSIntText(T x, char * pos) -{ - static_assert(std::is_same_v || std::is_same_v); - - using UnsignedT = make_unsigned_t; - static constexpr T min_int = UnsignedT(1) << (sizeof(T) * 8 - 1); - - if (unlikely(x == min_int)) - { - if constexpr (std::is_same_v) - { - const char * res = "-170141183460469231731687303715884105728"; - memcpy(pos, res, strlen(res)); - return pos + strlen(res); - } - else if constexpr (std::is_same_v) - { - const char * res = "-57896044618658097711785492504343953926634992332820282019728792003956564819968"; - memcpy(pos, res, strlen(res)); - return pos + strlen(res); - } - } - - if (x < 0) - { - x = -x; - pos = writeLeadingMinus(pos); - } - return writeUIntText(UnsignedT(x), pos); -} - -} - -template -char * itoa(I i, char * p) -{ - return impl::convert::itoa(i, p); -} - -template <> -inline char * itoa(char8_t i, char * p) -{ - return impl::convert::itoa(uint8_t(i), p); -} - -template <> -inline char * itoa(UInt128 i, char * p) -{ - return impl::writeUIntText(i, p); -} - -template <> -inline char * itoa(Int128 i, char * p) -{ - return impl::writeSIntText(i, p); -} - -template <> -inline char * itoa(UInt256 i, char * p) -{ - return impl::writeUIntText(i, p); -} - -template <> -inline char * itoa(Int256 i, char * p) -{ - return impl::writeSIntText(i, p); -} +#define DIGITS_INTEGER_TYPES(M) \ + M(uint8_t) \ + M(UInt8) \ + M(UInt16) \ + M(UInt32) \ + M(UInt64) \ + M(UInt128) \ + M(UInt256) +#define INSTANTIATION(T) \ + extern template int digits10(T x); +DIGITS_INTEGER_TYPES(INSTANTIATION) +#undef DIGITS_INTEGER_TYPES +#undef INSTANTIATION diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index ac3e3671ae0..dea369a508a 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -37,6 +37,7 @@ list (APPEND PUBLIC_LIBS clickhouse_dictionaries_embedded clickhouse_parsers ch_contrib::consistent_hashing + common dbms ch_contrib::metrohash ch_contrib::murmurhash From 38cbc2c6c40541cc916bc591cd68b7eef70b1162 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Mar 2024 18:57:16 +0100 Subject: [PATCH 11/78] Restore digits --- base/base/itoa.cpp | 43 +++++++++++++++++++------------------------ 1 file changed, 19 insertions(+), 24 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 9fefc9f0f07..ef844ff68a8 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -34,42 +34,37 @@ template -int digits10(T x) +inline int digits10(T x) { - if (x < T(10ULL)) + if (x < 10ULL) return 1; - if (x < T(100ULL)) + if (x < 100ULL) return 2; - if constexpr (sizeof(T) == 1) + if (x < 1000ULL) return 3; - else + + if (x < 1000000000000ULL) { - if (x < T(1000ULL)) - return 3; - - if (x < T(1000000000000ULL)) + if (x < 100000000ULL) { - if (x < T(100000000ULL)) + if (x < 1000000ULL) { - if (x < T(1000000ULL)) - { - if (x < T(10000ULL)) - return 4; - else - return 5 + (x >= T(100000ULL)); - } - - return 7 + (x >= T(10000000ULL)); + if (x < 10000ULL) + return 4; + else + return 5 + (x >= 100000ULL); } - if (x < T(10000000000ULL)) - return 9 + (x >= T(1000000000ULL)); - - return 11 + (x >= T(100000000000ULL)); + return 7 + (x >= 10000000ULL); } - return 12 + digits10(x / T(1000000000000ULL)); + if (x < 10000000000ULL) + return 9 + (x >= 1000000000ULL); + + return 11 + (x >= 100000000000ULL); } + + return 12 + digits10(x / 1000000000000ULL); } From 444595ac576438c9d0a259debf776187ddd3fcce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Mar 2024 20:00:07 +0100 Subject: [PATCH 12/78] More speed please Mr. compiler --- base/base/itoa.cpp | 165 +++++++++++++++++++++++++++++++-------------- 1 file changed, 114 insertions(+), 51 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index ef844ff68a8..08912edf3ea 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -72,7 +72,7 @@ namespace { template -static constexpr T pow10(size_t x) +ALWAYS_INLINE inline constexpr T pow10(size_t x) { return x ? 10 * pow10(x - 1) : 1; } @@ -143,7 +143,7 @@ struct QuotientAndRemainder }; template -QuotientAndRemainder static inline split(UnsignedOfSize value) +QuotientAndRemainder inline split(UnsignedOfSize value) { constexpr DivisionBy10PowN division; @@ -154,7 +154,7 @@ QuotientAndRemainder static inline split(UnsignedOfSize value) } -static inline char * outDigit(char * p, uint8_t value) +ALWAYS_INLINE inline char * outDigit(char * p, uint8_t value) { *p = '0' + value; ++p; @@ -176,7 +176,7 @@ static const char digits[201] = "00010203040506070809" "80818283848586878889" "90919293949596979899"; -static inline char * outTwoDigits(char * p, uint8_t value) +ALWAYS_INLINE inline char * outTwoDigits(char * p, uint8_t value) { memcpy(p, &digits[value * 2], 2); p += 2; @@ -187,9 +187,9 @@ static inline char * outTwoDigits(char * p, uint8_t value) namespace convert { template -static char * head(char * p, UInt u); +char * head(char * p, UInt u); template -static char * tail(char * p, UInt u); +char * tail(char * p, UInt u); //===----------------------------------------------------------===// // head: find most significant digit, skip leading zeros @@ -198,7 +198,7 @@ static char * tail(char * p, UInt u); // "x" contains quotient and remainder after division by 10^N // quotient is less than 10^N template -static inline char * head(char * p, QuotientAndRemainder x) +ALWAYS_INLINE inline char * head(char * p, QuotientAndRemainder x) { p = head(p, UnsignedOfSize(x.quotient)); p = tail(p, x.remainder); @@ -207,14 +207,14 @@ static inline char * head(char * p, QuotientAndRemainder x) // "u" is less than 10^2*N template -static inline char * head(char * p, UInt u) +ALWAYS_INLINE inline char * head(char * p, UInt u) { return u < pow10>(N) ? head(p, UnsignedOfSize(u)) : head(p, split(u)); } // recursion base case, selected when "u" is one byte template <> -inline char * head, 1>(char * p, UnsignedOfSize<1> u) +ALWAYS_INLINE inline char * head, 1>(char * p, UnsignedOfSize<1> u) { return u < 10 ? outDigit(p, u) : outTwoDigits(p, u); } @@ -225,7 +225,7 @@ inline char * head, 1>(char * p, UnsignedOfSize<1> u) // recursive step, "u" is less than 10^2*N template -static inline char * tail(char * p, UInt u) +ALWAYS_INLINE inline char * tail(char * p, UInt u) { QuotientAndRemainder x = split(u); p = tail(p, UnsignedOfSize(x.quotient)); @@ -235,7 +235,7 @@ static inline char * tail(char * p, UInt u) // recursion base case, selected when "u" is one byte template <> -inline char * tail, 1>(char * p, UnsignedOfSize<1> u) +ALWAYS_INLINE inline char * tail, 1>(char * p, UnsignedOfSize<1> u) { return outTwoDigits(p, u); } @@ -244,9 +244,8 @@ inline char * tail, 1>(char * p, UnsignedOfSize<1> u) // large values are >= 10^2*N // where x contains quotient and remainder after division by 10^N //===----------------------------------------------------------===// - template -static inline char * large(char * p, QuotientAndRemainder x) +ALWAYS_INLINE inline char * large(char * p, QuotientAndRemainder x) { QuotientAndRemainder y = split(x.quotient); p = head(p, UnsignedOfSize(y.quotient)); @@ -259,9 +258,8 @@ static inline char * large(char * p, QuotientAndRemainder x) // handle values of "u" that might be >= 10^2*N // where N is the size of "u" in bytes //===----------------------------------------------------------===// - template -static inline char * uitoa(char * p, UInt u) +ALWAYS_INLINE inline char * uitoa(char * p, UInt u) { if (u < pow10>(N)) return head(p, UnsignedOfSize(u)); @@ -272,7 +270,7 @@ static inline char * uitoa(char * p, UInt u) // selected when "u" is one byte template <> -inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) +ALWAYS_INLINE inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) { if (u < 10) return outDigit(p, u); @@ -292,14 +290,14 @@ inline char * uitoa, 1>(char * p, UnsignedOfSize<1> u) // itoa: handle unsigned integral operands (selected by SFINAE) template && std::is_integral_v> * = nullptr> -static inline char * itoa(U u, char * p) +ALWAYS_INLINE inline char * itoa(U u, char * p) { return convert::uitoa(p, u); } // itoa: handle signed integral operands (selected by SFINAE) template && std::is_integral_v> * = nullptr> -static inline char * itoa(I i, char * p) +ALWAYS_INLINE inline char * itoa(I i, char * p) { // Need "mask" to be filled with a copy of the sign bit. // If "i" is a negative value, then the result of "operator >>" @@ -335,63 +333,128 @@ static inline char * itoa(I i, char * p) } -template -static NO_INLINE char * writeUIntText(T _x, char * p) +const uint64_t max_multiple_of_hundred_that_fits_in_64_bits = 1'00'00'00'00'00'00'00'00'00ull; +constexpr int max_multiple_of_hundred_blocks = 9; +static_assert(max_multiple_of_hundred_that_fits_in_64_bits % 100 == 0); + +ALWAYS_INLINE inline char * writeUIntText(UInt128 _x, char * p) { - static_assert(std::is_same_v || std::is_same_v); - using T_ = std::conditional_t< - std::is_same_v, - unsigned __int128, + /// If we the highest 8 byte item is empty, we can print only the lowest item as i64 + if (_x.items[UInt128::_impl::little(1)] == 0) + return convert::itoa(_x.items[UInt128::_impl::little(0)], p); + + /// Doing operations using __int128 is faster, as we already rely on this feature + using T = unsigned __int128; + T x = (T(_x.items[UInt128::_impl::little(1)]) << 64) + T(_x.items[UInt128::_impl::little(0)]); + + /// We are going to accumulate blocks of 2 digits to print until the number is small enough to be printed as u64 + /// To do this we could do: x / 100, x % 100 + /// But this is too many iterations with long integers, so instead we can divide by a much longer integer + /// max_multiple_of_hundred_that_fits_in_64_bits and then get the blocks out of this (as u64) + static const T large_divisor = max_multiple_of_hundred_that_fits_in_64_bits; + static const T largest_uint64 = std::numeric_limits::max(); + uint8_t two_values[20] = {0}; // 39 Max characters / 2 + + int current_block = 0; + while (x > largest_uint64) + { + uint64_t remainder = uint64_t(x % large_divisor); + x /= large_divisor; + + int pos = current_block; + while (remainder) + { + two_values[pos] = uint8_t(remainder % 100); + pos++; + remainder /= 100; + } + current_block += max_multiple_of_hundred_blocks; + } + + char * highest_part_print = convert::itoa(uint64_t(x), p); + for (int i = 0; i < current_block; i++) + { + outTwoDigits(highest_part_print, two_values[current_block - 1 - i]); + highest_part_print += 2; + } + + return highest_part_print; +} + +ALWAYS_INLINE inline char * writeUIntText(UInt256 _x, char * p) +{ + /// If possible, treat it as a smaller integer as they are much faster to print + if (_x.items[UInt256::_impl::little(3)] == 0 && _x.items[UInt256::_impl::little(2)] == 0) + return writeUIntText(UInt128{_x.items[UInt256::_impl::little(0)], _x.items[UInt256::_impl::little(1)]}, p); + + /// If available (x86) we transform from our custom class to _BitInt(256) which has better support in the compiler + /// and produces better code + using T = #if defined(__x86_64__) # pragma clang diagnostic push # pragma clang diagnostic ignored "-Wbit-int-extension" unsigned _BitInt(256) # pragma clang diagnostic pop #else - T + UInt256 #endif - >; + ; - T_ x; - T_ hundred(100ULL); - if constexpr (std::is_same_v) - { - x = (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); - } - else - { #if defined(__x86_64__) - x = (T_(_x.items[T::_impl::little(3)]) << 192) + (T_(_x.items[T::_impl::little(2)]) << 128) - + (T_(_x.items[T::_impl::little(1)]) << 64) + T_(_x.items[T::_impl::little(0)]); + T x = (T(_x.items[UInt256::_impl::little(3)]) << 192) + (T(_x.items[UInt256::_impl::little(2)]) << 128) + + (T(_x.items[UInt256::_impl::little(1)]) << 64) + T(_x.items[UInt256::_impl::little(0)]); #else - x = _x; + T x = _x; #endif + + /// Similar to writeUIntText(UInt128) only that in this case we will stop as soon as we reach the largest u128 + /// and switch to that function + uint8_t two_values[39] = {0}; // 78 Max characters / 2 + int current_pos = 0; + + static const T large_divisor = max_multiple_of_hundred_that_fits_in_64_bits; + static const T largest_uint128 = T(std::numeric_limits::max()) << 64 | T(std::numeric_limits::max()); + + while (x > largest_uint128) + { + uint64_t remainder = uint64_t(x % large_divisor); + x /= large_divisor; + + int pos = current_pos; + while (remainder) + { + two_values[pos] = uint8_t(remainder % 100); + pos++; + remainder /= 100; + } + current_pos += max_multiple_of_hundred_blocks; } - int len = digits10(x); - auto * pp = p + len; - while (x >= hundred) +#if defined(__x86_64__) + UInt128 pending{uint64_t(x), uint64_t(x >> 64)}; +#else + UInt128 pending{x.items[UInt256::_impl::little(0)], x.items[UInt256::_impl::little(1)]}; +#endif + + char * highest_part_print = writeUIntText(pending, p); + for (int i = 0; i < current_pos; i++) { - const auto i = x % hundred; - x /= hundred; - pp -= 2; - outTwoDigits(pp, i); + outTwoDigits(highest_part_print, two_values[current_pos - 1 - i]); + highest_part_print += 2; } - if (x < 10) - *p = '0' + x; - else - outTwoDigits(p, x); - return p + len; + + return highest_part_print; } -static ALWAYS_INLINE inline char * writeLeadingMinus(char * pos) + +ALWAYS_INLINE inline char * writeLeadingMinus(char * pos) { *pos = '-'; return pos + 1; } template -static ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) +ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) { static_assert(std::is_same_v || std::is_same_v); From d52027c5a09f0f9619bc2f5df639f1a042b2c084 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 12:00:34 +0100 Subject: [PATCH 13/78] Style and comments --- base/base/itoa.cpp | 47 +++++++++++++++++++++++----------------------- 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 08912edf3ea..4475ae416b9 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -165,16 +165,16 @@ ALWAYS_INLINE inline char * outDigit(char * p, uint8_t value) // into ascii characters as described by Andrei Alexandrescu in // https://www.facebook.com/notes/facebook-engineering/three-optimization-tips-for-c/10151361643253920/ -static const char digits[201] = "00010203040506070809" - "10111213141516171819" - "20212223242526272829" - "30313233343536373839" - "40414243444546474849" - "50515253545556575859" - "60616263646566676869" - "70717273747576777879" - "80818283848586878889" - "90919293949596979899"; +const char digits[201] = "00010203040506070809" + "10111213141516171819" + "20212223242526272829" + "30313233343536373839" + "40414243444546474849" + "50515253545556575859" + "60616263646566676869" + "70717273747576777879" + "80818283848586878889" + "90919293949596979899"; ALWAYS_INLINE inline char * outTwoDigits(char * p, uint8_t value) { @@ -334,23 +334,24 @@ ALWAYS_INLINE inline char * itoa(I i, char * p) const uint64_t max_multiple_of_hundred_that_fits_in_64_bits = 1'00'00'00'00'00'00'00'00'00ull; -constexpr int max_multiple_of_hundred_blocks = 9; +const int max_multiple_of_hundred_blocks = 9; static_assert(max_multiple_of_hundred_that_fits_in_64_bits % 100 == 0); ALWAYS_INLINE inline char * writeUIntText(UInt128 _x, char * p) { - /// If we the highest 8 byte item is empty, we can print only the lowest item as i64 + /// If we the highest 64bit item is empty, we can print just the lowest item as u64 if (_x.items[UInt128::_impl::little(1)] == 0) return convert::itoa(_x.items[UInt128::_impl::little(0)], p); - /// Doing operations using __int128 is faster, as we already rely on this feature + /// Doing operations using __int128 is faster and we already rely on this feature using T = unsigned __int128; T x = (T(_x.items[UInt128::_impl::little(1)]) << 64) + T(_x.items[UInt128::_impl::little(0)]); /// We are going to accumulate blocks of 2 digits to print until the number is small enough to be printed as u64 /// To do this we could do: x / 100, x % 100 - /// But this is too many iterations with long integers, so instead we can divide by a much longer integer - /// max_multiple_of_hundred_that_fits_in_64_bits and then get the blocks out of this (as u64) + /// But these would mean doing many iterations with long integers, so instead we divide by a much longer integer + /// multiple of 100 (100^9) and then get the blocks out of it (as u64) + /// Once we reach u64::max we can stop and use the fast method to print that in the front static const T large_divisor = max_multiple_of_hundred_that_fits_in_64_bits; static const T largest_uint64 = std::numeric_limits::max(); uint8_t two_values[20] = {0}; // 39 Max characters / 2 @@ -358,15 +359,15 @@ ALWAYS_INLINE inline char * writeUIntText(UInt128 _x, char * p) int current_block = 0; while (x > largest_uint64) { - uint64_t remainder = uint64_t(x % large_divisor); + uint64_t u64_remainder = uint64_t(x % large_divisor); x /= large_divisor; int pos = current_block; - while (remainder) + while (u64_remainder) { - two_values[pos] = uint8_t(remainder % 100); + two_values[pos] = uint8_t(u64_remainder % 100); pos++; - remainder /= 100; + u64_remainder /= 100; } current_block += max_multiple_of_hundred_blocks; } @@ -417,15 +418,15 @@ ALWAYS_INLINE inline char * writeUIntText(UInt256 _x, char * p) while (x > largest_uint128) { - uint64_t remainder = uint64_t(x % large_divisor); + uint64_t u64_remainder = uint64_t(x % large_divisor); x /= large_divisor; int pos = current_pos; - while (remainder) + while (u64_remainder) { - two_values[pos] = uint8_t(remainder % 100); + two_values[pos] = uint8_t(u64_remainder % 100); pos++; - remainder /= 100; + u64_remainder /= 100; } current_pos += max_multiple_of_hundred_blocks; } From 2f3c103367de4fab57602e3fc1821608df718c77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 12:02:53 +0100 Subject: [PATCH 14/78] OSX quirks --- base/base/itoa.cpp | 4 ++++ base/base/itoa.h | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 4475ae416b9..d877f15b563 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -542,6 +542,10 @@ char * itoa(Int256 i, char * p) #define INSTANTIATION(T) template char * itoa(T i, char * p); FOR_MISSING_INTEGER_TYPES(INSTANTIATION) +#if defined(OS_DARWIN) +INSTANTIATION(size_t) +#endif + #undef FOR_MISSING_INTEGER_TYPES #undef INSTANTIATION diff --git a/base/base/itoa.h b/base/base/itoa.h index 71603cdeb88..98a570b12fa 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -25,6 +25,10 @@ template <> char * itoa(Int256 i, char * p); extern template char * itoa(T i, char * p); FOR_MISSING_INTEGER_TYPES(INSTANTIATION) +#if defined(OS_DARWIN) +INSTANTIATION(size_t) +#endif + #undef FOR_MISSING_INTEGER_TYPES #undef INSTANTIATION From e7cc49212a15ca8bcf87950225e066d04c8823e4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Mar 2024 12:28:14 +0100 Subject: [PATCH 15/78] fix tidy build --- src/Functions/array/arrayDistance.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 03f0bc7b286..0045075ddef 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -523,18 +523,18 @@ private: const auto & offsets_y = array_y.getOffsets(); ColumnArray::Offset prev_offset = 0; - for (size_t row = 0; row < offsets_y.size(); ++row) + for (auto offset_y : offsets_y) { - if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]] + if (offsets_x[0] != offset_y - prev_offset) [[unlikely]] { throw Exception( ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Arguments of function {} have different array sizes: {} and {}", getName(), offsets_x[0], - offsets_y[row] - prev_offset); + offset_y - prev_offset); } - prev_offset = offsets_y[row]; + prev_offset = offset_y; } const typename Kernel::ConstParams kernel_params = initConstParams(arguments); From 07ce390609238318a7ab115b3d7020f03150ce76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 13:59:24 +0100 Subject: [PATCH 16/78] No public templates, just happiness --- base/base/itoa.cpp | 44 +++++++++++++++++++++----------------------- base/base/itoa.h | 35 ++++++++++++++--------------------- 2 files changed, 35 insertions(+), 44 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index d877f15b563..3c4f0bb048d 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -32,9 +32,11 @@ #include #include +namespace +{ template -inline int digits10(T x) +int digits10T(T x) { if (x < 10ULL) return 1; @@ -64,13 +66,9 @@ inline int digits10(T x) return 11 + (x >= 100000000000ULL); } - return 12 + digits10(x / 1000000000000ULL); + return 12 + digits10T(x / 1000000000000ULL); } - -namespace -{ - template ALWAYS_INLINE inline constexpr T pow10(size_t x) { @@ -487,67 +485,62 @@ ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) } } -template -char * itoa(T i, char * p) -{ - return convert::itoa(i, p); -} -template <> char * itoa(UInt8 i, char * p) { return convert::itoa(uint8_t(i), p); } -template <> char * itoa(Int8 i, char * p) { return convert::itoa(int8_t(i), p); } -template <> char * itoa(UInt128 i, char * p) { return writeUIntText(i, p); } -template <> char * itoa(Int128 i, char * p) { return writeSIntText(i, p); } -template <> char * itoa(UInt256 i, char * p) { return writeUIntText(i, p); } -template <> char * itoa(Int256 i, char * p) { return writeSIntText(i, p); } +#define DEFAULT_ITOA(T) \ + char * itoa(T i, char * p) \ + { \ + return convert::itoa(i, p); \ + } + #define FOR_MISSING_INTEGER_TYPES(M) \ - M(int8_t) \ M(uint8_t) \ M(UInt16) \ M(UInt32) \ M(UInt64) \ + M(int8_t) \ M(Int16) \ M(Int32) \ M(Int64) -#define INSTANTIATION(T) template char * itoa(T i, char * p); -FOR_MISSING_INTEGER_TYPES(INSTANTIATION) +FOR_MISSING_INTEGER_TYPES(DEFAULT_ITOA) #if defined(OS_DARWIN) -INSTANTIATION(size_t) +DEFAULT_ITOA(unsigned long) +DEFAULT_ITOA(long) #endif #undef FOR_MISSING_INTEGER_TYPES -#undef INSTANTIATION +#undef DEFAULT_ITOA #define DIGITS_INTEGER_TYPES(M) \ @@ -559,7 +552,12 @@ INSTANTIATION(size_t) M(UInt128) \ M(UInt256) -#define INSTANTIATION(T) template int digits10(T x); +#define INSTANTIATION(T) \ + int digits10(T x) \ + { \ + return digits10T(x); \ + } + DIGITS_INTEGER_TYPES(INSTANTIATION) #undef DIGITS_INTEGER_TYPES diff --git a/base/base/itoa.h b/base/base/itoa.h index 98a570b12fa..9a89fa739dd 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -2,39 +2,33 @@ #include -template char * itoa(T i, char * p); - -template <> char * itoa(UInt8 i, char * p); -template <> char * itoa(Int8 i, char * p); -template <> char * itoa(UInt128 i, char * p); -template <> char * itoa(Int128 i, char * p); -template <> char * itoa(UInt256 i, char * p); -template <> char * itoa(Int256 i, char * p); - -#define FOR_MISSING_INTEGER_TYPES(M) \ - M(int8_t) \ +#define FOR_INTEGER_TYPES(M) \ M(uint8_t) \ + M(UInt8) \ M(UInt16) \ M(UInt32) \ M(UInt64) \ + M(UInt128) \ + M(UInt256) \ + M(int8_t) \ + M(Int8) \ M(Int16) \ M(Int32) \ - M(Int64) + M(Int64) \ + M(Int128) \ + M(Int256) -#define INSTANTIATION(T) \ - extern template char * itoa(T i, char * p); -FOR_MISSING_INTEGER_TYPES(INSTANTIATION) +#define INSTANTIATION(T) char * itoa(T i, char * p); +FOR_INTEGER_TYPES(INSTANTIATION) #if defined(OS_DARWIN) -INSTANTIATION(size_t) +INSTANTIATION(unsigned long) +INSTANTIATION(long) #endif #undef FOR_MISSING_INTEGER_TYPES #undef INSTANTIATION - -template int digits10(T x); - #define DIGITS_INTEGER_TYPES(M) \ M(uint8_t) \ M(UInt8) \ @@ -43,8 +37,7 @@ template int digits10(T x); M(UInt64) \ M(UInt128) \ M(UInt256) -#define INSTANTIATION(T) \ - extern template int digits10(T x); +#define INSTANTIATION(T) int digits10(T x); DIGITS_INTEGER_TYPES(INSTANTIATION) #undef DIGITS_INTEGER_TYPES #undef INSTANTIATION From 4b964979c2c831a2ddeee5a7b0c10066f520ab11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 14:22:33 +0100 Subject: [PATCH 17/78] Digits doesn't belong with itoa anymore --- base/base/itoa.cpp | 55 ----------------------------------- base/base/itoa.h | 13 --------- src/Functions/countDigits.cpp | 34 ++++++++++++++++++++++ 3 files changed, 34 insertions(+), 68 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 3c4f0bb048d..608258c6b56 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -35,40 +35,6 @@ namespace { -template -int digits10T(T x) -{ - if (x < 10ULL) - return 1; - if (x < 100ULL) - return 2; - if (x < 1000ULL) - return 3; - - if (x < 1000000000000ULL) - { - if (x < 100000000ULL) - { - if (x < 1000000ULL) - { - if (x < 10000ULL) - return 4; - else - return 5 + (x >= 100000ULL); - } - - return 7 + (x >= 10000000ULL); - } - - if (x < 10000000000ULL) - return 9 + (x >= 1000000000ULL); - - return 11 + (x >= 100000000000ULL); - } - - return 12 + digits10T(x / 1000000000000ULL); -} - template ALWAYS_INLINE inline constexpr T pow10(size_t x) { @@ -541,24 +507,3 @@ DEFAULT_ITOA(long) #undef FOR_MISSING_INTEGER_TYPES #undef DEFAULT_ITOA - - -#define DIGITS_INTEGER_TYPES(M) \ - M(uint8_t) \ - M(UInt8) \ - M(UInt16) \ - M(UInt32) \ - M(UInt64) \ - M(UInt128) \ - M(UInt256) - -#define INSTANTIATION(T) \ - int digits10(T x) \ - { \ - return digits10T(x); \ - } - -DIGITS_INTEGER_TYPES(INSTANTIATION) - -#undef DIGITS_INTEGER_TYPES -#undef INSTANTIATION diff --git a/base/base/itoa.h b/base/base/itoa.h index 9a89fa739dd..e69ce0ef17d 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -28,16 +28,3 @@ INSTANTIATION(long) #undef FOR_MISSING_INTEGER_TYPES #undef INSTANTIATION - -#define DIGITS_INTEGER_TYPES(M) \ - M(uint8_t) \ - M(UInt8) \ - M(UInt16) \ - M(UInt32) \ - M(UInt64) \ - M(UInt128) \ - M(UInt256) -#define INSTANTIATION(T) int digits10(T x); -DIGITS_INTEGER_TYPES(INSTANTIATION) -#undef DIGITS_INTEGER_TYPES -#undef INSTANTIATION diff --git a/src/Functions/countDigits.cpp b/src/Functions/countDigits.cpp index 2ca8d944b0a..f2712b5b301 100644 --- a/src/Functions/countDigits.cpp +++ b/src/Functions/countDigits.cpp @@ -20,6 +20,40 @@ namespace ErrorCodes namespace { +template +int digits10(T x) +{ + if (x < 10ULL) + return 1; + if (x < 100ULL) + return 2; + if (x < 1000ULL) + return 3; + + if (x < 1000000000000ULL) + { + if (x < 100000000ULL) + { + if (x < 1000000ULL) + { + if (x < 10000ULL) + return 4; + else + return 5 + (x >= 100000ULL); + } + + return 7 + (x >= 10000000ULL); + } + + if (x < 10000000000ULL) + return 9 + (x >= 1000000000ULL); + + return 11 + (x >= 100000000000ULL); + } + + return 12 + digits10(x / 1000000000000ULL); +} + /// Returns number of decimal digits you need to represent the value. /// For Decimal values takes in account their scales: calculates result over underlying int type which is (value * scale). /// countDigits(42) = 2, countDigits(42.000) = 5, countDigits(0.04200) = 4. From 4f27dd8f9c887cec6273dc7960b165ee2537ad26 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 8 Mar 2024 14:27:50 +0100 Subject: [PATCH 18/78] upd test --- .../0_stateless/02864_restore_table_with_broken_part.reference | 2 +- .../queries/0_stateless/02864_restore_table_with_broken_part.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.reference b/tests/queries/0_stateless/02864_restore_table_with_broken_part.reference index 9a8dcda81df..9247a7d6ab6 100644 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.reference +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.reference @@ -1,4 +1,4 @@ -data.bin doesn't exist: while restoring part all_2_2_0 +OK RESTORED 1 3 diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh index cf99c7e9284..d3252b29eb7 100755 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh @@ -26,7 +26,7 @@ ln -s "$SRC_BACKUP_DIR/$SRC_BACKUP_FILENAME" "$BACKUPS_DIR/$BACKUP_FILENAME" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" # First try to restore with the setting `restore_broken_parts_as_detached` set to false. -$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME" 2>&1 | grep -o -m 1 "data.bin doesn't exist: while restoring part all_2_2_0" +$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME" 2>&1 | tr -d \\n | grep "data.bin doesn't exist" | grep "while restoring part all_2_2_0" > /dev/null && echo "OK" || echo "FAILED" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" From 45a1f0f8ed33fb09fee92137a391dfd25ba63f05 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 8 Mar 2024 14:02:34 +0000 Subject: [PATCH 19/78] More consistency and extended tests --- src/Functions/array/arrayDistance.cpp | 8 ++-- src/Functions/array/arrayDotProduct.cpp | 43 ++++++++----------- .../0_stateless/02708_dotProduct.reference | 12 ++++++ .../queries/0_stateless/02708_dotProduct.sql | 16 +++++-- 4 files changed, 47 insertions(+), 32 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 0045075ddef..8b591e37ff6 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -471,10 +471,9 @@ private: const typename Kernel::ConstParams kernel_params = initConstParams(arguments); - auto result = ColumnVector::create(input_rows_count); - auto & result_data = result->getData(); + auto col_res = ColumnVector::create(input_rows_count); + auto & result_data = col_res->getData(); - /// Do the actual computation ColumnArray::Offset prev = 0; size_t row = 0; @@ -503,7 +502,7 @@ private: result_data[row] = Kernel::finalize(state, kernel_params); row++; } - return result; + return col_res; } /// Special case when the 1st parameter is Const @@ -542,7 +541,6 @@ private: auto result = ColumnVector::create(input_rows_count); auto & result_data = result->getData(); - /// Do the actual computation size_t prev = 0; size_t row = 0; diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 8b7c85e05dd..97dc9653bab 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -140,6 +140,7 @@ public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } size_t getNumberOfArguments() const override { return 2; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -174,13 +175,13 @@ public: ACTION(Float32) \ ACTION(Float64) - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { switch (result_type->getTypeId()) { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithResultType(arguments); \ + return executeWithResultType(arguments, input_rows_count); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -193,7 +194,7 @@ public: private: template - ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { DataTypePtr type_x = typeid_cast(arguments[0].type.get())->getNestedType(); @@ -201,7 +202,7 @@ private: { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithResultTypeAndLeftType(arguments); \ + return executeWithResultTypeAndLeftType(arguments, input_rows_count); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -218,7 +219,7 @@ private: } template - ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments) const + ColumnPtr executeWithResultTypeAndLeftType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { DataTypePtr type_y = typeid_cast(arguments[1].type.get())->getNestedType(); @@ -226,7 +227,7 @@ private: { #define ON_TYPE(type) \ case TypeIndex::type: \ - return executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column); \ + return executeWithResultTypeAndLeftTypeAndRightType(arguments[0].column, arguments[1].column, input_rows_count); \ break; SUPPORTED_TYPES(ON_TYPE) @@ -243,15 +244,15 @@ private: } template - ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y) const + ColumnPtr executeWithResultTypeAndLeftTypeAndRightType(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count) const { if (typeid_cast(col_x.get())) { - return executeWithLeftArgConst(col_x, col_y); + return executeWithLeftArgConst(col_x, col_y, input_rows_count); } else if (typeid_cast(col_y.get())) { - return executeWithLeftArgConst(col_y, col_x); + return executeWithLeftArgConst(col_y, col_x, input_rows_count); } col_x = col_x->convertToFullColumnIfConst(); @@ -268,16 +269,13 @@ private: if (!array_x.hasEqualOffsets(array_y)) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); - auto col_res = ColumnVector::create(); - auto & result = col_res->getData(); - - size_t size = offsets_x.size(); - result.resize(size); + auto col_res = ColumnVector::create(input_rows_count); + auto & result_data = col_res->getData(); ColumnArray::Offset current_offset = 0; - for (size_t row = 0; row < size; ++row) + for (size_t row = 0; row < input_rows_count; ++row) { - size_t array_size = offsets_x[row] - current_offset; + const size_t array_size = offsets_x[row] - current_offset; size_t i = 0; @@ -298,7 +296,7 @@ private: for (; i < array_size; ++i) Kernel::template accumulate(state, static_cast(data_x[current_offset + i]), static_cast(data_y[current_offset + i])); - result[row] = Kernel::template finalize(state); + result_data[row] = Kernel::template finalize(state); current_offset = offsets_x[row]; } @@ -307,7 +305,7 @@ private: } template - ColumnPtr executeWithLeftArgConst(ColumnPtr col_x, ColumnPtr col_y) const + ColumnPtr executeWithLeftArgConst(ColumnPtr col_x, ColumnPtr col_y, size_t input_rows_count) const { col_x = assert_cast(col_x.get())->getDataColumnPtr(); col_y = col_y->convertToFullColumnIfConst(); @@ -336,16 +334,13 @@ private: prev_offset = offset_y; } - auto col_res = ColumnVector::create(); + auto col_res = ColumnVector::create(input_rows_count); auto & result = col_res->getData(); - size_t size = offsets_y.size(); - result.resize(size); - ColumnArray::Offset current_offset = 0; - for (size_t row = 0; row < size; ++row) + for (size_t row = 0; row < input_rows_count; ++row) { - size_t array_size = offsets_x[0]; + const size_t array_size = offsets_x[0]; typename Kernel::template State state; size_t i = 0; diff --git a/tests/queries/0_stateless/02708_dotProduct.reference b/tests/queries/0_stateless/02708_dotProduct.reference index 593071a3521..93a67e4c0be 100644 --- a/tests/queries/0_stateless/02708_dotProduct.reference +++ b/tests/queries/0_stateless/02708_dotProduct.reference @@ -11,6 +11,8 @@ [-1,-2,-3] [4,5,6] -32 Int64 [1,2,3] [4,5,6] 32 Float32 [1,2,3] [4,5,6] 32 Float64 +[] [] 0 Float32 +[] [] 0 UInt16 -- Tuple (1,2,3) (4,5,6) 32 UInt64 (1,2,3) (4,5,6) 32 UInt64 @@ -24,6 +26,8 @@ (1,2,3) (4,5,6) 32 Float64 -- Non-const argument [1,2,3] [4,5,6] 32 UInt16 +[] [] 0 Float32 +[] [] 0 UInt16 -- Array with mixed element arguments types (result type is the supertype) [1,2,3] [4,5,6] 32 Float32 -- Tuple with mixed element arguments types @@ -33,8 +37,16 @@ 32 32 -- Tests that trigger special paths + -- non-const / non-const 0 61 1 186 +0 61 +1 186 +0 61 +1 186 + -- const / non-const +0 62 +1 187 0 62 1 187 0 62 diff --git a/tests/queries/0_stateless/02708_dotProduct.sql b/tests/queries/0_stateless/02708_dotProduct.sql index ac94ecc28d3..05c66777dff 100644 --- a/tests/queries/0_stateless/02708_dotProduct.sql +++ b/tests/queries/0_stateless/02708_dotProduct.sql @@ -19,6 +19,9 @@ SELECT [-1, -2, -3]::Array(Int32) AS x, [4, 5, 6]::Array(Int32) AS y, dotProduct SELECT [-1, -2, -3]::Array(Int64) AS x, [4, 5, 6]::Array(Int64) AS y, dotProduct(x, y) AS res, toTypeName(res); SELECT [1, 2, 3]::Array(Float32) AS x, [4, 5, 6]::Array(Float32) AS y, dotProduct(x, y) AS res, toTypeName(res); SELECT [1, 2, 3]::Array(Float64) AS x, [4, 5, 6]::Array(Float64) AS y, dotProduct(x, y) AS res, toTypeName(res); +-- empty arrays +SELECT []::Array(Float32) AS x, []::Array(Float32) AS y, dotProduct(x, y) AS res, toTypeName(res); +SELECT []::Array(UInt8) AS x, []::Array(UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res); SELECT ' -- Tuple'; SELECT (1::UInt8, 2::UInt8, 3::UInt8) AS x, (4::UInt8, 5::UInt8, 6::UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res); @@ -34,6 +37,8 @@ SELECT (1::Float64, 2::Float64, 3::Float64) AS x, (4::Float64, 5::Float64, 6::Fl SELECT '-- Non-const argument'; SELECT materialize([1::UInt8, 2::UInt8, 3::UInt8]) AS x, [4::UInt8, 5::UInt8, 6::UInt8] AS y, dotProduct(x, y) AS res, toTypeName(res); +SELECT materialize([]::Array(Float32)) AS x, []::Array(Float32) AS y, dotProduct(x, y) AS res, toTypeName(res); +SELECT materialize([]::Array(UInt8)) AS x, []::Array(UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res); SELECT ' -- Array with mixed element arguments types (result type is the supertype)'; SELECT [1::UInt16, 2::UInt8, 3::Float32] AS x, [4::Int16, 5::Float32, 6::UInt8] AS y, dotProduct(x, y) AS res, toTypeName(res); @@ -50,7 +55,12 @@ SELECT '-- Tests that trigger special paths'; DROP TABLE IF EXISTS tab; CREATE TABLE tab(id UInt64, vec Array(Float32)) ENGINE = MergeTree ORDER BY id; INSERT INTO tab VALUES (0, [0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0, 3.0, 0.0, 1.0, 2.0]) (1, [5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]); -SELECT id, arrayDotProduct(vec, vec) FROM tab ORDER BY id; -- non-const / non-const -SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float32), vec) FROM tab ORDER BY id; -- const / non-const -SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float64), vec) FROM tab ORDER BY id; -- const / non-const +SELECT ' -- non-const / non-const'; +SELECT id, arrayDotProduct(vec, vec) FROM tab ORDER BY id; +SELECT id, arrayDotProduct(vec::Array(Float64), vec::Array(Float64)) FROM tab ORDER BY id; +SELECT id, arrayDotProduct(vec::Array(UInt32), vec::Array(UInt32)) FROM tab ORDER BY id; +SELECT ' -- const / non-const'; +SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float32), vec) FROM tab ORDER BY id; +SELECT id, arrayDotProduct([5.0, 2.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0, 3.0, 5.0, 1.0, 2.0]::Array(Float64), vec) FROM tab ORDER BY id; +SELECT id, arrayDotProduct([5, 2, 2, 3, 5, 1, 2, 3, 5, 1, 2, 3, 5, 1, 2, 3, 5, 1, 2]::Array(UInt32), vec) FROM tab ORDER BY id; DROP TABLE tab; From 45efa69189784ce65bffd0d84462dcb30c1e6bf4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 15:17:59 +0100 Subject: [PATCH 20/78] Add perf tests --- tests/performance/bigint_formatting.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) create mode 100644 tests/performance/bigint_formatting.xml diff --git a/tests/performance/bigint_formatting.xml b/tests/performance/bigint_formatting.xml new file mode 100644 index 00000000000..c3454f91d1d --- /dev/null +++ b/tests/performance/bigint_formatting.xml @@ -0,0 +1,13 @@ + + + CREATE TABLE bigint ( u128 UInt128, i128 Int128, u256 UInt256, i256 Int256) ENGINE = Memory + AS + SELECT * FROM generateRandom('u128 UInt128, i128 Int128, u256 UInt256, i256 Int256', 42) LIMIT 50000; + + DROP TABLE IF EXISTS bigint + + SELECT * FROM bigint WHERE NOT ignore(toString(u128)) SETTINGS max_threads = 1 + SELECT * FROM bigint WHERE NOT ignore(toString(i128)) SETTINGS max_threads = 1 + SELECT * FROM bigint WHERE NOT ignore(toString(u256)) SETTINGS max_threads = 1 + SELECT * FROM bigint WHERE NOT ignore(toString(i256)) SETTINGS max_threads = 1 + From 68360aa522169b1c0955837e93687c6d3a124912 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 8 Mar 2024 18:37:03 +0100 Subject: [PATCH 21/78] Clang format --- base/base/itoa.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 608258c6b56..9bd2fcd1837 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -34,7 +34,6 @@ namespace { - template ALWAYS_INLINE inline constexpr T pow10(size_t x) { @@ -117,7 +116,6 @@ QuotientAndRemainder inline split(UnsignedOfSize value) return {quotient, remainder}; } - ALWAYS_INLINE inline char * outDigit(char * p, uint8_t value) { *p = '0' + value; @@ -147,7 +145,6 @@ ALWAYS_INLINE inline char * outTwoDigits(char * p, uint8_t value) return p; } - namespace convert { template @@ -296,7 +293,6 @@ ALWAYS_INLINE inline char * itoa(I i, char * p) } } - const uint64_t max_multiple_of_hundred_that_fits_in_64_bits = 1'00'00'00'00'00'00'00'00'00ull; const int max_multiple_of_hundred_blocks = 9; static_assert(max_multiple_of_hundred_that_fits_in_64_bits % 100 == 0); @@ -411,7 +407,6 @@ ALWAYS_INLINE inline char * writeUIntText(UInt256 _x, char * p) return highest_part_print; } - ALWAYS_INLINE inline char * writeLeadingMinus(char * pos) { *pos = '-'; @@ -451,7 +446,6 @@ ALWAYS_INLINE inline char * writeSIntText(T x, char * pos) } } - char * itoa(UInt8 i, char * p) { return convert::itoa(uint8_t(i), p); From 23b55ecbe8521692acab507408fc70665aa16f1b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Mar 2024 22:02:22 +0100 Subject: [PATCH 22/78] beautify exception --- src/Functions/array/arrayDotProduct.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 97dc9653bab..3f37e6f609f 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -212,7 +212,7 @@ private: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), type_x->getName()); } From 530efbe8102023f857ec29b1f9c7f2090973f736 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Mar 2024 13:08:50 +0000 Subject: [PATCH 23/78] Beautify exception, pt. II --- src/Functions/array/arrayDistance.cpp | 6 +++--- src/Functions/array/arrayDotProduct.cpp | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 8b591e37ff6..69e5e3712dd 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -357,7 +357,7 @@ public: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), common_type->getName()); } @@ -412,7 +412,7 @@ private: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), type_x->getName()); } @@ -437,7 +437,7 @@ private: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), type_y->getName()); } diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 3f37e6f609f..a9547ca90bb 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -237,7 +237,7 @@ private: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + "Supported types: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", getName(), type_y->getName()); } From bcd705517a510feffd08e9e27c93c2e09eecd144 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Mar 2024 12:53:58 +0000 Subject: [PATCH 24/78] Remove unnecessary call to convertToFullColumnIfConst() --- src/Functions/array/arrayDistance.cpp | 3 --- src/Functions/array/arrayDotProduct.cpp | 3 --- src/Functions/array/arrayNorm.cpp | 3 +-- 3 files changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Functions/array/arrayDistance.cpp b/src/Functions/array/arrayDistance.cpp index 69e5e3712dd..6ed4bf24f99 100644 --- a/src/Functions/array/arrayDistance.cpp +++ b/src/Functions/array/arrayDistance.cpp @@ -455,9 +455,6 @@ private: return executeWithLeftArgConst(col_y, col_x, input_rows_count, arguments); } - col_x = col_x->convertToFullColumnIfConst(); - col_y = col_y->convertToFullColumnIfConst(); - const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index a9547ca90bb..783843a89d5 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -255,9 +255,6 @@ private: return executeWithLeftArgConst(col_y, col_x, input_rows_count); } - col_x = col_x->convertToFullColumnIfConst(); - col_y = col_y->convertToFullColumnIfConst(); - const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); diff --git a/src/Functions/array/arrayNorm.cpp b/src/Functions/array/arrayNorm.cpp index 027a33d094c..e87eff6add1 100644 --- a/src/Functions/array/arrayNorm.cpp +++ b/src/Functions/array/arrayNorm.cpp @@ -175,8 +175,7 @@ public: } } - ColumnPtr - executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { DataTypePtr type = typeid_cast(arguments[0].type.get())->getNestedType(); ColumnPtr column = arguments[0].column->convertToFullColumnIfConst(); From 7a8d79de234e4f83a4e6421e5de49e5800ff70bc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Sun, 10 Mar 2024 22:13:49 +0100 Subject: [PATCH 25/78] reload CI due to OOM in build From b43d1c4eb5c2cb661f3bb8aeb65b4549b8fea6bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Mar 2024 13:13:33 +0100 Subject: [PATCH 26/78] Fix macro --- base/base/itoa.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/itoa.h b/base/base/itoa.h index e69ce0ef17d..3461d679d43 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -26,5 +26,5 @@ INSTANTIATION(unsigned long) INSTANTIATION(long) #endif -#undef FOR_MISSING_INTEGER_TYPES +#undef FOR_INTEGER_TYPES #undef INSTANTIATION From 6f1b835a4e77df36e322ca16ee01b2c52f60edbf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 14 Mar 2024 20:46:54 +0100 Subject: [PATCH 27/78] Fix comment --- base/base/itoa.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/itoa.cpp b/base/base/itoa.cpp index 9bd2fcd1837..fd8fd8de025 100644 --- a/base/base/itoa.cpp +++ b/base/base/itoa.cpp @@ -1,6 +1,6 @@ // Based on https://github.com/amdn/itoa and combined with our optimizations // -//=== itoa.h - Fast integer to ascii conversion --*- C++ -*-// +//=== itoa.cpp - Fast integer to ascii conversion --*- C++ -*-// // // The MIT License (MIT) // Copyright (c) 2016 Arturo Martin-de-Nicolas From b9f6de095d5932c83abe0092c4b7b7e29a64e4d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 15 Mar 2024 18:11:35 +0100 Subject: [PATCH 28/78] Disable async_insert_use_adaptive_busy_timeout correctly with compatibility settings --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4914f97a6fb..5ecaf12647f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -104,7 +104,7 @@ static std::map sett {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", true, true, "Use adaptive asynchronous insert timeout"}, + {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, From cfb13d212e5e9093542fef7875fd4df764a8dd7d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 15 Mar 2024 18:28:04 +0100 Subject: [PATCH 29/78] Add a test --- .../0_stateless/03011_adaptative_timeout_compatibility.reference | 1 + .../0_stateless/03011_adaptative_timeout_compatibility.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/03011_adaptative_timeout_compatibility.reference create mode 100644 tests/queries/0_stateless/03011_adaptative_timeout_compatibility.sql diff --git a/tests/queries/0_stateless/03011_adaptative_timeout_compatibility.reference b/tests/queries/0_stateless/03011_adaptative_timeout_compatibility.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03011_adaptative_timeout_compatibility.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03011_adaptative_timeout_compatibility.sql b/tests/queries/0_stateless/03011_adaptative_timeout_compatibility.sql new file mode 100644 index 00000000000..cdd2da0f6d8 --- /dev/null +++ b/tests/queries/0_stateless/03011_adaptative_timeout_compatibility.sql @@ -0,0 +1 @@ +SELECT value from system.settings where name = 'async_insert_use_adaptive_busy_timeout' SETTINGS compatibility = '23.12'; From 5000e3e10b5caf188b8b3fc6cdf7d41d711ade1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 18 Mar 2024 02:16:24 +0100 Subject: [PATCH 30/78] Fix error in clickhouse-client --- src/Parsers/parseQuery.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 7aad0b010a5..51878efa706 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -92,7 +92,9 @@ void writeQueryWithHighlightedErrorPositions( } else { - size_t bytes_to_hilite = UTF8::seqLength(*current_position_to_hilite); + ssize_t bytes_to_hilite = UTF8::seqLength(*current_position_to_hilite); + if (bytes_to_hilite > end - current_position_to_hilite) + bytes_to_hilite = end - current_position_to_hilite; /// Bright on red background. out << "\033[41;1m"; From 0240ffa5a6dab4f747d17295648d030af907f8f6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Mar 2024 19:31:52 +0100 Subject: [PATCH 31/78] Fetch the changed files only once --- tests/ci/pr_info.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 6f4b400f7a8..9bd30f3c58e 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -337,6 +337,9 @@ class PRInfo: ) def fetch_changed_files(self): + if self.changed_files_requested: + return + if not getattr(self, "diff_urls", False): raise TypeError("The event does not have diff URLs") From 49017fc163174e4cb4b3e9253094f399503add42 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Mar 2024 20:39:47 +0100 Subject: [PATCH 32/78] Run the style-check on top of the merge-commit of PRs --- tests/ci/style_check.py | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 7fb5d22a3ce..4daa6921d6b 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -67,26 +67,6 @@ def parse_args(): return parser.parse_args() -def checkout_head(pr_info: PRInfo) -> None: - # It works ONLY for PRs, and only over ssh, so either - # ROBOT_CLICKHOUSE_SSH_KEY should be set or ssh-agent should work - assert pr_info.number - if not pr_info.head_name == pr_info.base_name: - # We can't push to forks, sorry folks - return - remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] - fetch_cmd = ( - f"{GIT_PREFIX} fetch --depth=1 " - f"{remote_url} {pr_info.head_ref}:head-{pr_info.head_ref}" - ) - if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): - with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): - git_runner(fetch_cmd) - else: - git_runner(fetch_cmd) - git_runner(f"git checkout -f head-{pr_info.head_ref}") - - def commit_push_staged(pr_info: PRInfo) -> None: # It works ONLY for PRs, and only over ssh, so either # ROBOT_CLICKHOUSE_SSH_KEY should be set or ssh-agent should work @@ -177,8 +157,6 @@ def main(): _ = future1.result() if run_pycheck: - if args.push: - checkout_head(pr_info) logging.info("Run py files check: %s", cmd_py) future2 = executor.submit(subprocess.run, cmd_py, shell=True) _ = future2.result() From c3f386d06a2c12f08d168accbf51457ad7a735c0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Mar 2024 20:57:35 +0100 Subject: [PATCH 33/78] Fix the autofixer --- tests/ci/style_check.py | 22 ++++++++++------------ 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 4daa6921d6b..0332383df91 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -78,24 +78,23 @@ def commit_push_staged(pr_info: PRInfo) -> None: if not git_staged: return remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] + head = git_runner("git rev-parse HEAD^{}") git_runner(f"{GIT_PREFIX} commit -m 'Automatic style fix'") - push_cmd = ( - f"{GIT_PREFIX} push {remote_url} head-{pr_info.head_ref}:{pr_info.head_ref}" + # The fetch to avoid issue 'pushed branch tip is behind its remote' + fetch_cmd = ( + f"{GIT_PREFIX} fetch {remote_url} --no-recurse-submodules --depth=2 {head}" ) + push_cmd = f"{GIT_PREFIX} push {remote_url} HEAD:{pr_info.head_ref}" if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): + git_runner(fetch_cmd) git_runner(push_cmd) - else: - git_runner(push_cmd) + return + + git_runner(fetch_cmd) + git_runner(push_cmd) -def checkout_last_ref(pr_info: PRInfo) -> None: - # Checkout the merge commit back to avoid special effects - assert pr_info.number - if not pr_info.head_name == pr_info.base_name: - # We can't push to forks, sorry folks - return - git_runner("git checkout -f -") def main(): @@ -162,7 +161,6 @@ def main(): _ = future2.result() if args.push: commit_push_staged(pr_info) - checkout_last_ref(pr_info) subprocess.check_call( f"python3 ../../utils/check-style/process_style_check_result.py --in-results-dir {temp_path} " From 41432c56a8f13a8914b65b265d9855847314db79 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Mar 2024 10:50:44 +0100 Subject: [PATCH 34/78] Enable shellcheck conditionally --- tests/ci/style_check.py | 84 +++++++++++-------- utils/check-style/check_shell.sh | 13 +++ .../check-style/process_style_check_result.py | 4 +- 3 files changed, 64 insertions(+), 37 deletions(-) create mode 100755 utils/check-style/check_shell.sh diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 0332383df91..797c0ef12d0 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -8,8 +8,9 @@ import subprocess import sys from concurrent.futures import ProcessPoolExecutor from pathlib import Path -from typing import List, Tuple +from typing import List, Tuple, Union +import magic from docker_images_helper import get_docker_image, pull_image from env_helper import CI, REPO_COPY, TEMP_PATH from git_helper import GIT_PREFIX, git_runner @@ -95,6 +96,24 @@ def commit_push_staged(pr_info: PRInfo) -> None: git_runner(push_cmd) +def is_python(file: Union[Path, str]) -> bool: + """returns if the changed file in the repository is python script""" + # WARNING: python-magic v2:0.4.24-2 is used in ubuntu 22.04, + # and `Support os.PathLike values in magic.from_file` is only from 0.4.25 + return bool( + magic.from_file(os.path.join(REPO_COPY, file), mime=True) + == "text/x-script.python" + ) + + +def is_shell(file: Union[Path, str]) -> bool: + """returns if the changed file in the repository is shell script""" + # WARNING: python-magic v2:0.4.24-2 is used in ubuntu 22.04, + # and `Support os.PathLike values in magic.from_file` is only from 0.4.25 + return bool( + magic.from_file(os.path.join(REPO_COPY, file), mime=True) + == "text/x-shellscript" + ) def main(): @@ -111,29 +130,28 @@ def main(): temp_path.mkdir(parents=True, exist_ok=True) pr_info = PRInfo() + run_cpp_check = True + run_shell_check = True + run_python_check = True + if CI and pr_info.number > 0: + pr_info.fetch_changed_files() + run_cpp_check = not any( + is_python(file) or is_shell(file) for file in pr_info.changed_files + ) + run_shell_check = any(is_shell(file) for file in pr_info.changed_files) + run_python_check = any(is_python(file) for file in pr_info.changed_files) IMAGE_NAME = "clickhouse/style-test" image = pull_image(get_docker_image(IMAGE_NAME)) - cmd_cpp = ( + docker_command = ( f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " - f"--entrypoint= -w/ClickHouse/utils/check-style " - f"{image} ./check_cpp.sh" - ) - - cmd_py = ( - f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " - f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " - f"--entrypoint= -w/ClickHouse/utils/check-style " - f"{image} ./check_py.sh" - ) - - cmd_docs = ( - f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE " - f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output " - f"--entrypoint= -w/ClickHouse/utils/check-style " - f"{image} ./check_docs.sh" + f"--entrypoint= -w/ClickHouse/utils/check-style {image}" ) + cmd_docs = f"{docker_command} ./check_docs.sh" + cmd_cpp = f"{docker_command} ./check_cpp.sh" + cmd_py = f"{docker_command} ./check_py.sh" + cmd_shell = f"{docker_command} ./check_shell.sh" with ProcessPoolExecutor(max_workers=2) as executor: logging.info("Run docs files check: %s", cmd_docs) @@ -141,26 +159,22 @@ def main(): # Parallelization does not make it faster - run subsequently _ = future.result() - run_cppcheck = True - run_pycheck = True - if CI and pr_info.number > 0: - pr_info.fetch_changed_files() - if not any(file.endswith(".py") for file in pr_info.changed_files): - run_pycheck = False - if all(file.endswith(".py") for file in pr_info.changed_files): - run_cppcheck = False - - if run_cppcheck: + if run_cpp_check: logging.info("Run source files check: %s", cmd_cpp) - future1 = executor.submit(subprocess.run, cmd_cpp, shell=True) - _ = future1.result() + future = executor.submit(subprocess.run, cmd_cpp, shell=True) + _ = future.result() - if run_pycheck: + if run_python_check: logging.info("Run py files check: %s", cmd_py) - future2 = executor.submit(subprocess.run, cmd_py, shell=True) - _ = future2.result() - if args.push: - commit_push_staged(pr_info) + future = executor.submit(subprocess.run, cmd_py, shell=True) + _ = future.result() + if run_shell_check: + logging.info("Run shellcheck check: %s", cmd_shell) + future = executor.submit(subprocess.run, cmd_shell, shell=True) + _ = future.result() + + if args.push: + commit_push_staged(pr_info) subprocess.check_call( f"python3 ../../utils/check-style/process_style_check_result.py --in-results-dir {temp_path} " diff --git a/utils/check-style/check_shell.sh b/utils/check-style/check_shell.sh new file mode 100755 index 00000000000..94f06220f88 --- /dev/null +++ b/utils/check-style/check_shell.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv + +start_total=$(date +%s) + +start=$(date +%s) +./shellcheck-run.sh |& tee /test_output/shellcheck.txt +runtime=$(($(date +%s)-start)) +echo "Check shellcheck. Done. $runtime seconds." + +runtime=$(($(date +%s)-start_total)) +echo "Check style total. Done. $runtime seconds." diff --git a/utils/check-style/process_style_check_result.py b/utils/check-style/process_style_check_result.py index 8c9837b4fc6..f1ca53eba3b 100755 --- a/utils/check-style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -13,8 +13,8 @@ def process_result(result_folder): description = "" test_results = [] checks = ( - # "duplicate includes", - # "shellcheck", + "duplicate includes", + "shellcheck", "style", "pylint", "black", From 0e6b5705fbdce4c7032d39dbdae452f266e1b0b3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Mar 2024 12:20:30 +0100 Subject: [PATCH 35/78] Install python-magic in style-check, split pip and apt --- docker/test/style/Dockerfile | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index b4ffcfb597c..5d53d03606f 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -8,20 +8,22 @@ ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ - aspell \ - curl \ - git \ - file \ - libxml2-utils \ - moreutils \ - python3-fuzzywuzzy \ - python3-pip \ - yamllint \ - locales \ - && pip3 install black==23.12.0 boto3 codespell==2.2.1 mypy==1.8.0 PyGithub unidiff pylint==3.1.0 \ - requests types-requests \ + aspell \ + curl \ + git \ + file \ + libxml2-utils \ + moreutils \ + python3-fuzzywuzzy \ + python3-pip \ + yamllint \ + locales \ && apt-get clean \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \ + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* + +# python-magic is the same version as in Ubuntu 22.04 +RUN pip3 install black==23.12.0 boto3 codespell==2.2.1 mypy==1.8.0 PyGithub unidiff pylint==3.1.0 \ + python-magic==0.4.24 requests types-requests \ && rm -rf /root/.cache/pip RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 From 97b0513f9928de5f0207978d8138b478bc1677bd Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Mar 2024 19:21:05 +0100 Subject: [PATCH 36/78] Enable duplicate includes back, rewrite and parallelize them --- utils/check-style/check-duplicate-includes.sh | 5 ++- utils/check-style/check_cpp.sh | 31 +++++++++---------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/utils/check-style/check-duplicate-includes.sh b/utils/check-style/check-duplicate-includes.sh index 64aca4d180d..ca8b98d9537 100755 --- a/utils/check-style/check-duplicate-includes.sh +++ b/utils/check-style/check-duplicate-includes.sh @@ -1,6 +1,9 @@ #!/usr/bin/env bash ROOT_PATH=$(git rev-parse --show-toplevel) +NPROC=$(($(nproc) + 3)) # Find duplicate include directives -find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | while read file; do grep -P '^#include ' $file | sort | uniq -c | grep -v -P '^\s+1\s' && echo $file; done | sed '/^[[:space:]]*$/d' +find "$ROOT_PATH"/{src,base,programs,utils} -type f '(' -name '*.h' -or -name '*.cpp' ')' -print0 | \ + xargs -0 -I {} -P "$NPROC" bash -c 'grep -P "^#include " "{}" | sort | uniq -c | grep -v -P "^\s+1\s" && echo "{}" '| \ + sed '/^\s*$/d' diff --git a/utils/check-style/check_cpp.sh b/utils/check-style/check_cpp.sh index ea90d79418c..dc1dae70e22 100755 --- a/utils/check-style/check_cpp.sh +++ b/utils/check-style/check_cpp.sh @@ -4,35 +4,32 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv -start_total=`date +%s` +start_total=$(date +%s) -# FIXME: 30 sec to wait -# echo "Check duplicates" | ts -# ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt +start=$(date +%s) +./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt +runtime=$(($(date +%s)-start)) +echo "Check duplicates. Done. $runtime seconds." -start=`date +%s` +start=$(date +%s) ./check-style -n |& tee /test_output/style_output.txt -runtime=$((`date +%s`-start)) +runtime=$(($(date +%s)-start)) echo "Check style. Done. $runtime seconds." -start=`date +%s` +start=$(date +%s) ./check-whitespaces -n |& tee /test_output/whitespaces_output.txt -runtime=$((`date +%s`-start)) +runtime=$(($(date +%s)-start)) echo "Check whitespaces. Done. $runtime seconds." -start=`date +%s` +start=$(date +%s) ./check-workflows |& tee /test_output/workflows_output.txt -runtime=$((`date +%s`-start)) +runtime=$(($(date +%s)-start)) echo "Check workflows. Done. $runtime seconds." -start=`date +%s` +start=$(date +%s) ./check-submodules |& tee /test_output/submodules_output.txt -runtime=$((`date +%s`-start)) +runtime=$(($(date +%s)-start)) echo "Check submodules. Done. $runtime seconds." -# FIXME: 6 min to wait -# echo "Check shell scripts with shellcheck" | ts -# ./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt - -runtime=$((`date +%s`-start_total)) +runtime=$(($(date +%s)-start_total)) echo "Check style total. Done. $runtime seconds." From a5331ea54462c5cdb300c7f48d28d8795fac1bc9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Mar 2024 13:49:05 +0100 Subject: [PATCH 37/78] Rewrite list-licenses.sh to use array+find --- utils/list-licenses/list-licenses.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/utils/list-licenses/list-licenses.sh b/utils/list-licenses/list-licenses.sh index f09168a0596..cc730464e8e 100755 --- a/utils/list-licenses/list-licenses.sh +++ b/utils/list-licenses/list-licenses.sh @@ -5,15 +5,16 @@ if [[ "$OSTYPE" == "darwin"* ]]; then GREP_CMD=ggrep FIND_CMD=gfind else - FIND_CMD=find - GREP_CMD=grep + FIND_CMD='find' + GREP_CMD='grep' fi ROOT_PATH="$(git rev-parse --show-toplevel)" LIBS_PATH="${ROOT_PATH}/contrib" -ls -1 -d ${LIBS_PATH}/*/ "${ROOT_PATH}/base/poco" | ${GREP_CMD} -F -v -- '-cmake' | LC_ALL=C sort | while read LIB; do - LIB_NAME=$(basename $LIB) +mapfile -t libs < <(echo "${ROOT_PATH}/base/poco"; find "${LIBS_PATH}" -type d -maxdepth 1 ! -name '*-cmake' | LC_ALL=C sort) +for LIB in "${libs[@]}"; do + LIB_NAME=$(basename "$LIB") LIB_LICENSE=$( LC_ALL=C ${FIND_CMD} "$LIB" -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not '(' -iname '*.html' -or -iname '*.htm' -or -iname '*.rtf' -or -name '*.cpp' -or -name '*.h' -or -iname '*.json' ')' -printf "%d\t%p\n" | From 070f7bee9b550962cba6768ffd3a74306e4e2107 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Mar 2024 13:56:43 +0100 Subject: [PATCH 38/78] Fix shellcheck issues in durability-test.sh --- utils/durability-test/durability-test.sh | 51 +++++++++++------------- 1 file changed, 24 insertions(+), 27 deletions(-) diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh index 97c39473b69..188ef3452e7 100755 --- a/utils/durability-test/durability-test.sh +++ b/utils/durability-test/durability-test.sh @@ -15,7 +15,6 @@ Usage: URL=http://cloud-images.ubuntu.com/bionic/current IMAGE=bionic-server-cloudimg-amd64.img SSH_PORT=11022 -CLICKHOUSE_PORT=9090 PASSWORD=root TABLE_NAME=$1 @@ -34,19 +33,18 @@ function run() function copy() { - sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 2>/dev/null + sshpass -p $PASSWORD scp -r -P $SSH_PORT "$1" "root@localhost:$2" 2>/dev/null } function wait_vm_for_start() { echo "Waiting until VM started..." started=0 - for i in {0..100}; do - run "exit" - if [ $? -eq 0 ]; then + for _ in {0..100}; do + if run "exit"; then started=1 break - fi + fi sleep 1s done @@ -62,9 +60,8 @@ function wait_clickhouse_for_start() { echo "Waiting until ClickHouse started..." started=0 - for i in {0..30}; do - run "clickhouse client --query 'select 1'" > /dev/null - if [ $? -eq 0 ]; then + for _ in {0..30}; do + if run "clickhouse client --query 'select 1'" > /dev/null; then started=1 break fi @@ -105,11 +102,11 @@ if [[ -z $CLICKHOUSE_CONFIG_DIR ]]; then CLICKHOUSE_CONFIG_DIR=/etc/clickhouse-server fi -echo "Using ClickHouse binary:" $CLICKHOUSE_BINARY -echo "Using ClickHouse config from:" $CLICKHOUSE_CONFIG_DIR +echo "Using ClickHouse binary: $CLICKHOUSE_BINARY" +echo "Using ClickHouse config from: $CLICKHOUSE_CONFIG_DIR" -copy $CLICKHOUSE_BINARY /usr/bin -copy $CLICKHOUSE_CONFIG_DIR /etc +copy "$CLICKHOUSE_BINARY" /usr/bin +copy "$CLICKHOUSE_CONFIG_DIR" /etc run "mv /etc/$CLICKHOUSE_CONFIG_DIR /etc/clickhouse-server" echo "Prepared VM" @@ -118,23 +115,23 @@ echo "Starting ClickHouse" run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & wait_clickhouse_for_start -query=`cat $CREATE_QUERY` -echo "Executing query:" $query +query=$(cat "$CREATE_QUERY") +echo "Executing query: $query" run "clickhouse client --query '$query'" -query=`cat $INSERT_QUERY` -echo "Will run in a loop query: " $query +query=$(cat "$INSERT_QUERY") +echo "Will run in a loop query: $query" run "clickhouse benchmark <<< '$query' -c 8" & echo "Running queries" -pid=`pidof qemu-system-x86_64` +pid=$(pidof qemu-system-x86_64) sec=$(( (RANDOM % 5) + 25 )) ms=$(( RANDOM % 1000 )) echo "Will kill VM in $sec.$ms sec" sleep $sec.$ms -kill -9 $pid +kill -9 "$pid" echo "Restarting" @@ -147,22 +144,22 @@ run "rm -r *data/system" run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & wait_clickhouse_for_start -pid=`pidof qemu-system-x86_64` -result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Caught exception while loading metadata'"` +pid=$(pidof qemu-system-x86_64) +result=$(run "grep $TABLE_NAME clickhouse-server.log | grep 'Caught exception while loading metadata'") if [[ -n $result ]]; then echo "FAIL. Can't attach table:" - echo $result - kill -9 $pid + echo "$result" + kill -9 "$pid" exit 1 fi -result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Considering to remove broken part'"` +result=$(run "grep $TABLE_NAME clickhouse-server.log | grep 'Considering to remove broken part'") if [[ -n $result ]]; then echo "FAIL. Have broken parts:" - echo $result - kill -9 $pid + echo "$result" + kill -9 "$pid" exit 1 fi -kill -9 $pid +kill -9 "$pid" echo OK From 4c27287c6d40f803121d2ef7263799c7460c4b6e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 18 Mar 2024 17:30:29 +0100 Subject: [PATCH 39/78] Fix crash in arrayPartialReverseSort --- src/Functions/array/arraySort.cpp | 5 ++++- ...013_fuzz_arrayPartialReverseSort.reference | 14 ++++++++++++++ .../03013_fuzz_arrayPartialReverseSort.sql | 19 +++++++++++++++++++ 3 files changed, 37 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03013_fuzz_arrayPartialReverseSort.reference create mode 100644 tests/queries/0_stateless/03013_fuzz_arrayPartialReverseSort.sql diff --git a/src/Functions/array/arraySort.cpp b/src/Functions/array/arraySort.cpp index 184b1f82280..6c8741e6eec 100644 --- a/src/Functions/array/arraySort.cpp +++ b/src/Functions/array/arraySort.cpp @@ -46,7 +46,10 @@ ColumnPtr ArraySortImpl::execute( ErrorCodes::LOGICAL_ERROR, "Expected fixed arguments to get the limit for partial array sort" ); - return fixed_arguments[0].column.get()->getUInt(0); + + /// During dryRun the input column might be empty + if (!fixed_arguments[0].column->empty()) + return fixed_arguments[0].column->getUInt(0); } return 0; }(); diff --git a/tests/queries/0_stateless/03013_fuzz_arrayPartialReverseSort.reference b/tests/queries/0_stateless/03013_fuzz_arrayPartialReverseSort.reference new file mode 100644 index 00000000000..5832f8b7a52 --- /dev/null +++ b/tests/queries/0_stateless/03013_fuzz_arrayPartialReverseSort.reference @@ -0,0 +1,14 @@ +[] +[] +[1,0] +[] +[3,2] +[] +[5,4] +[] +[] +[1,0] +[] +[3,2] +[] +[5,4] diff --git a/tests/queries/0_stateless/03013_fuzz_arrayPartialReverseSort.sql b/tests/queries/0_stateless/03013_fuzz_arrayPartialReverseSort.sql new file mode 100644 index 00000000000..f575ae952dd --- /dev/null +++ b/tests/queries/0_stateless/03013_fuzz_arrayPartialReverseSort.sql @@ -0,0 +1,19 @@ +SELECT res +FROM +( + SELECT + arrayPartialReverseSort(2, if(number % 2, emptyArrayUInt64(), range(number))) AS arr, + arrayResize(arr, if(empty(arr), 0, 2)) AS res + FROM system.numbers + LIMIT 7 +); + +SELECT res +FROM +( + SELECT + arrayPartialReverseSort(materialize(2), if(number % 2, emptyArrayUInt64(), range(number))) AS arr, + arrayResize(arr, if(empty(arr), 0, 2)) AS res + FROM system.numbers + LIMIT 7 +); From 84051d8604ce6cbd03510cf77bda53a3f95382fa Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Mar 2024 17:34:16 +0100 Subject: [PATCH 40/78] Don't run 01459_manual_write_to_replicas in debug build as it's too slow --- tests/queries/0_stateless/01459_manual_write_to_replicas.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index a9a6d27c145..56620d848a3 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: replica, no-parallel +# Tags: replica, no-parallel, no-debug set -e From a1a4bd3514f72b4545fbb734749dcb9d7f513007 Mon Sep 17 00:00:00 2001 From: nemonlou Date: Wed, 6 Mar 2024 10:42:30 +0800 Subject: [PATCH 41/78] make nulls direction configuable for FullSortingMergeJoin --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 7 +- src/Processors/QueryPlan/JoinStep.cpp | 8 +- src/Processors/QueryPlan/JoinStep.h | 4 +- .../Transforms/MergeJoinTransform.cpp | 82 ++++++++++++------- .../Transforms/MergeJoinTransform.h | 16 +++- src/QueryPipeline/QueryPipelineBuilder.cpp | 3 +- src/QueryPipeline/QueryPipelineBuilder.h | 1 + 8 files changed, 84 insertions(+), 38 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a3c5638d97f..8d48b3f5e68 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -891,6 +891,7 @@ class IColumn; M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ + M(Bool, nulls_biggest_in_smj, true, "Treat nulls as biggest in sort. Used in sort merge join for compare null keys.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7c87dadfce6..6f0a9fa9bfb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1693,9 +1693,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), joined_plan->getCurrentDataStream(), expressions.join, settings.max_block_size, max_streams, - analysis_result.optimize_read_in_order); + analysis_result.optimize_read_in_order, + null_direct_hint); join_step->setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); std::vector plans; diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 1931b1eb3a1..0c46ce7893d 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -44,8 +44,10 @@ JoinStep::JoinStep( JoinPtr join_, size_t max_block_size_, size_t max_streams_, - bool keep_left_read_in_order_) - : join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_) + bool keep_left_read_in_order_, + int null_direction_hint_) + : join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_), + null_direction_hint(null_direction_hint_) { updateInputStreams(DataStreams{left_stream_, right_stream_}); } @@ -58,7 +60,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (join->pipelineType() == JoinPipelineType::YShaped) { auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( - std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, &processors); + std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, null_direction_hint, &processors); joined_pipeline->resize(max_streams); return joined_pipeline; } diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index a9059a083fe..08909ce48a9 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -19,7 +19,8 @@ public: JoinPtr join_, size_t max_block_size_, size_t max_streams_, - bool keep_left_read_in_order_); + bool keep_left_read_in_order_, + int null_direction_hint_ = 1); String getName() const override { return "Join"; } @@ -42,6 +43,7 @@ private: size_t max_block_size; size_t max_streams; bool keep_left_read_in_order; + int null_direction_hint; }; /// Special step for the case when Join is already filled. diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 2d313d4ea5c..c8e3a806a9f 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -43,7 +43,7 @@ FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns) } template -int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint = 1) +int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos, int null_direction_hint) { if constexpr (has_left_nulls && has_right_nulls) { @@ -88,35 +88,36 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, } int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, size_t lpos, - const SortCursorImpl & rhs, size_t rpos) + const SortCursorImpl & rhs, size_t rpos, + int null_direction_hint) { for (size_t i = 0; i < lhs.sort_columns_size; ++i) { /// TODO(@vdimir): use nullableCompareAt only if there's nullable columns - int cmp = nullableCompareAt(*lhs.sort_columns[i], *rhs.sort_columns[i], lpos, rpos); + int cmp = nullableCompareAt(*lhs.sort_columns[i], *rhs.sort_columns[i], lpos, rpos, null_direction_hint); if (cmp != 0) return cmp; } return 0; } -int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImpl & rhs) +int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImpl & rhs, int null_direction_hint) { - return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow()); + return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow(), null_direction_hint); } -bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs) +bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs, int null_direction_hint) { /// The last row of left cursor is less than the current row of the right cursor. - int cmp = compareCursors(lhs, lhs.rows - 1, rhs, rhs.getRow()); + int cmp = compareCursors(lhs, lhs.rows - 1, rhs, rhs.getRow(), null_direction_hint); return cmp < 0; } -int ALWAYS_INLINE totallyCompare(SortCursorImpl & lhs, SortCursorImpl & rhs) +int ALWAYS_INLINE totallyCompare(SortCursorImpl & lhs, SortCursorImpl & rhs, int null_direction_hint) { - if (totallyLess(lhs, rhs)) + if (totallyLess(lhs, rhs, null_direction_hint)) return -1; - if (totallyLess(rhs, lhs)) + if (totallyLess(rhs, lhs, null_direction_hint)) return 1; return 0; } @@ -270,9 +271,11 @@ bool FullMergeJoinCursor::fullyCompleted() const MergeJoinAlgorithm::MergeJoinAlgorithm( JoinPtr table_join_, const Blocks & input_headers, - size_t max_block_size_) + size_t max_block_size_, + int null_direction_hint_) : table_join(table_join_) , max_block_size(max_block_size_) + , null_direction_hint(null_direction_hint_) , log(getLogger("MergeJoinAlgorithm")) { if (input_headers.size() != 2) @@ -356,7 +359,7 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) cursors[source_num]->setChunk(std::move(input.chunk)); } -template +template struct AllJoinImpl { constexpr static bool enabled = isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind); @@ -382,7 +385,7 @@ struct AllJoinImpl lpos = left_cursor->getRow(); rpos = right_cursor->getRow(); - cmp = compareCursors(left_cursor.cursor, right_cursor.cursor); + cmp = compareCursors(left_cursor.cursor, right_cursor.cursor, nullDirection(nullOrder)); if (cmp == 0) { size_t lnum = nextDistinct(left_cursor.cursor); @@ -432,19 +435,37 @@ struct AllJoinImpl } }; -template class Impl, typename ... Args> -void dispatchKind(JoinKind kind, Args && ... args) +template class Impl, typename ... Args> +void dispatchKind(JoinKind kind, int null_direction_hint, Args && ... args) { - if (Impl::enabled && kind == JoinKind::Inner) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Left) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Right) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Full) - return Impl::join(std::forward(args)...); + if (isSmall(null_direction_hint)) + { + if (Impl::enabled && kind == JoinKind::Inner) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Left) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Right) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Full) + return Impl::join(std::forward(args)...); + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); + + } else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); + { + if (Impl::enabled && kind == JoinKind::Inner) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Left) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Right) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Full) + return Impl::join(std::forward(args)...); + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); + + } } std::optional MergeJoinAlgorithm::handleAllJoinState() @@ -517,7 +538,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) { PaddedPODArray idx_map[2]; - dispatchKind(kind, *cursors[0], *cursors[1], max_block_size, idx_map[0], idx_map[1], all_join_state); + dispatchKind(kind, null_direction_hint, *cursors[0], *cursors[1], max_block_size, idx_map[0], idx_map[1], all_join_state); assert(idx_map[0].size() == idx_map[1].size()); Chunk result; @@ -567,7 +588,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) } -template +template struct AnyJoinImpl { constexpr static bool enabled = isInner(kind) || isLeft(kind) || isRight(kind); @@ -599,7 +620,7 @@ struct AnyJoinImpl lpos = left_cursor->getRow(); rpos = right_cursor->getRow(); - cmp = compareCursors(left_cursor.cursor, right_cursor.cursor); + cmp = compareCursors(left_cursor.cursor, right_cursor.cursor, nullDirection(order)); if (cmp == 0) { if constexpr (isLeftOrFull(kind)) @@ -723,7 +744,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) PaddedPODArray idx_map[2]; size_t prev_pos[] = {current_left.getRow(), current_right.getRow()}; - dispatchKind(kind, *cursors[0], *cursors[1], idx_map[0], idx_map[1], any_join_state); + dispatchKind(kind, null_direction_hint, *cursors[0], *cursors[1], idx_map[0], idx_map[1], any_join_state); assert(idx_map[0].empty() || idx_map[1].empty() || idx_map[0].size() == idx_map[1].size()); size_t num_result_rows = std::max(idx_map[0].size(), idx_map[1].size()); @@ -816,7 +837,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge() } /// check if blocks are not intersecting at all - if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor); cmp != 0) + if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor, null_direction_hint); cmp != 0) { if (cmp < 0) { @@ -851,6 +872,7 @@ MergeJoinTransform::MergeJoinTransform( const Blocks & input_headers, const Block & output_header, size_t max_block_size, + int null_direction_hint_, UInt64 limit_hint_) : IMergingTransform( input_headers, @@ -859,7 +881,7 @@ MergeJoinTransform::MergeJoinTransform( limit_hint_, /* always_read_till_end_= */ false, /* empty_chunk_on_finish_= */ true, - table_join, input_headers, max_block_size) + table_join, input_headers, max_block_size, null_direction_hint_) , log(getLogger("MergeJoinTransform")) { LOG_TRACE(log, "Use MergeJoinTransform"); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 959550067f7..43485321122 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -220,6 +220,17 @@ private: bool recieved_all_blocks = false; }; +/// Join method. +enum class NullOrder +{ + SMALLEST, /// null is treated as smallest + BIGGEST /// null is treated as biggest +}; + +inline constexpr bool isSmall(int null_direction) { return null_direction == 1; } + +inline constexpr int nullDirection(NullOrder order) {return order == NullOrder::SMALLEST ? 1 : -1;} + /* * This class is used to join chunks from two sorted streams. * It is used in MergeJoinTransform. @@ -227,7 +238,8 @@ private: class MergeJoinAlgorithm final : public IMergingAlgorithm { public: - explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers, size_t max_block_size_); + explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers, size_t max_block_size_, + int null_direction_hint = 1); const char * getName() const override { return "MergeJoinAlgorithm"; } void initialize(Inputs inputs) override; @@ -258,6 +270,7 @@ private: JoinPtr table_join; size_t max_block_size; + int null_direction_hint; struct Statistic { @@ -282,6 +295,7 @@ public: const Blocks & input_headers, const Block & output_header, size_t max_block_size, + int null_direction_hint, UInt64 limit_hint = 0); String getName() const override { return "MergeJoinTransform"; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 67a8fe5dcab..e338c3ce0fa 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -349,6 +349,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped JoinPtr join, const Block & out_header, size_t max_block_size, + int null_direction_hint, Processors * collected_processors) { left->checkInitializedAndNotCompleted(); @@ -376,7 +377,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped } else { - auto joining = std::make_shared(join, inputs, out_header, max_block_size); + auto joining = std::make_shared(join, inputs, out_header, max_block_size, null_direction_hint); return mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); } } diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index f0b2ead687e..4753f957a25 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -137,6 +137,7 @@ public: JoinPtr table_join, const Block & out_header, size_t max_block_size, + int null_direction_hint, Processors * collected_processors = nullptr); /// Add other pipeline and execute it before current one. From 0b5fc743f2e0711556ab4628aecd13e5fcd1a9b8 Mon Sep 17 00:00:00 2001 From: nemonlou Date: Tue, 12 Mar 2024 09:55:02 +0800 Subject: [PATCH 42/78] make nulls direction configuable for FullSortingMergeJoin(fix review comments) --- src/Core/Settings.h | 1 - src/Interpreters/InterpreterSelectQuery.cpp | 7 +-- .../Transforms/MergeJoinTransform.cpp | 56 +++++++------------ .../Transforms/MergeJoinTransform.h | 11 ---- 4 files changed, 22 insertions(+), 53 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8d48b3f5e68..a3c5638d97f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -891,7 +891,6 @@ class IColumn; M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ - M(Bool, nulls_biggest_in_smj, true, "Treat nulls as biggest in sort. Used in sort merge join for compare null keys.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6f0a9fa9bfb..7c87dadfce6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1693,10 +1693,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), joined_plan->getCurrentDataStream(), expressions.join, settings.max_block_size, max_streams, - analysis_result.optimize_read_in_order, - null_direct_hint); + analysis_result.optimize_read_in_order); join_step->setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); std::vector plans; diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index c8e3a806a9f..6288a850d76 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -359,7 +359,7 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) cursors[source_num]->setChunk(std::move(input.chunk)); } -template +template struct AllJoinImpl { constexpr static bool enabled = isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind); @@ -369,7 +369,8 @@ struct AllJoinImpl size_t max_block_size, PaddedPODArray & left_map, PaddedPODArray & right_map, - std::unique_ptr & state) + std::unique_ptr & state, + int null_direction_hint) { right_map.clear(); right_map.reserve(max_block_size); @@ -385,7 +386,7 @@ struct AllJoinImpl lpos = left_cursor->getRow(); rpos = right_cursor->getRow(); - cmp = compareCursors(left_cursor.cursor, right_cursor.cursor, nullDirection(nullOrder)); + cmp = compareCursors(left_cursor.cursor, right_cursor.cursor, null_direction_hint); if (cmp == 0) { size_t lnum = nextDistinct(left_cursor.cursor); @@ -435,37 +436,19 @@ struct AllJoinImpl } }; -template class Impl, typename ... Args> -void dispatchKind(JoinKind kind, int null_direction_hint, Args && ... args) +template class Impl, typename ... Args> +void dispatchKind(JoinKind kind, Args && ... args) { - if (isSmall(null_direction_hint)) - { - if (Impl::enabled && kind == JoinKind::Inner) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Left) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Right) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Full) - return Impl::join(std::forward(args)...); + if (Impl::enabled && kind == JoinKind::Inner) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Left) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Right) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Full) + return Impl::join(std::forward(args)...); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); - - } - else - { - if (Impl::enabled && kind == JoinKind::Inner) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Left) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Right) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Full) - return Impl::join(std::forward(args)...); - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); - - } } std::optional MergeJoinAlgorithm::handleAllJoinState() @@ -538,7 +521,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) { PaddedPODArray idx_map[2]; - dispatchKind(kind, null_direction_hint, *cursors[0], *cursors[1], max_block_size, idx_map[0], idx_map[1], all_join_state); + dispatchKind(kind, *cursors[0], *cursors[1], max_block_size, idx_map[0], idx_map[1], all_join_state, null_direction_hint); assert(idx_map[0].size() == idx_map[1].size()); Chunk result; @@ -588,7 +571,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) } -template +template struct AnyJoinImpl { constexpr static bool enabled = isInner(kind) || isLeft(kind) || isRight(kind); @@ -597,7 +580,8 @@ struct AnyJoinImpl FullMergeJoinCursor & right_cursor, PaddedPODArray & left_map, PaddedPODArray & right_map, - AnyJoinState & state) + AnyJoinState & state, + int null_direction_hint) { assert(enabled); @@ -620,7 +604,7 @@ struct AnyJoinImpl lpos = left_cursor->getRow(); rpos = right_cursor->getRow(); - cmp = compareCursors(left_cursor.cursor, right_cursor.cursor, nullDirection(order)); + cmp = compareCursors(left_cursor.cursor, right_cursor.cursor, null_direction_hint); if (cmp == 0) { if constexpr (isLeftOrFull(kind)) @@ -744,7 +728,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) PaddedPODArray idx_map[2]; size_t prev_pos[] = {current_left.getRow(), current_right.getRow()}; - dispatchKind(kind, null_direction_hint, *cursors[0], *cursors[1], idx_map[0], idx_map[1], any_join_state); + dispatchKind(kind, *cursors[0], *cursors[1], idx_map[0], idx_map[1], any_join_state, null_direction_hint); assert(idx_map[0].empty() || idx_map[1].empty() || idx_map[0].size() == idx_map[1].size()); size_t num_result_rows = std::max(idx_map[0].size(), idx_map[1].size()); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 43485321122..8af486ea34b 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -220,17 +220,6 @@ private: bool recieved_all_blocks = false; }; -/// Join method. -enum class NullOrder -{ - SMALLEST, /// null is treated as smallest - BIGGEST /// null is treated as biggest -}; - -inline constexpr bool isSmall(int null_direction) { return null_direction == 1; } - -inline constexpr int nullDirection(NullOrder order) {return order == NullOrder::SMALLEST ? 1 : -1;} - /* * This class is used to join chunks from two sorted streams. * It is used in MergeJoinTransform. From 5cf22bae6f40ab1beb258cf22a9b0627c601495d Mon Sep 17 00:00:00 2001 From: nemonlou Date: Tue, 12 Mar 2024 10:04:37 +0800 Subject: [PATCH 43/78] minor: fix style change --- .../Transforms/MergeJoinTransform.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 6288a850d76..37a178810cb 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -440,15 +440,15 @@ template class Impl, typename ... Args> void dispatchKind(JoinKind kind, Args && ... args) { if (Impl::enabled && kind == JoinKind::Inner) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Left) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Right) - return Impl::join(std::forward(args)...); - else if (Impl::enabled && kind == JoinKind::Full) - return Impl::join(std::forward(args)...); - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Left) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Right) + return Impl::join(std::forward(args)...); + else if (Impl::enabled && kind == JoinKind::Full) + return Impl::join(std::forward(args)...); + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); } std::optional MergeJoinAlgorithm::handleAllJoinState() From 8ff21d7e47c25637b87f3ab674421cfc0a2c4487 Mon Sep 17 00:00:00 2001 From: nemonlou Date: Tue, 19 Mar 2024 10:59:32 +0800 Subject: [PATCH 44/78] fix review comments --- src/Interpreters/FullSortingMergeJoin.h | 7 ++++++- src/Processors/QueryPlan/JoinStep.cpp | 8 +++----- src/Processors/QueryPlan/JoinStep.h | 4 +--- .../Transforms/MergeJoinTransform.cpp | 17 +++++++++++------ src/Processors/Transforms/MergeJoinTransform.h | 6 ++---- src/QueryPipeline/QueryPipelineBuilder.cpp | 3 +-- src/QueryPipeline/QueryPipelineBuilder.h | 1 - 7 files changed, 24 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 7688d44f7a9..7e07c2004b6 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -21,9 +21,11 @@ namespace ErrorCodes class FullSortingMergeJoin : public IJoin { public: - explicit FullSortingMergeJoin(std::shared_ptr table_join_, const Block & right_sample_block_) + explicit FullSortingMergeJoin(std::shared_ptr table_join_, const Block & right_sample_block_, + int null_direction_ = 1) : table_join(table_join_) , right_sample_block(right_sample_block_) + , null_direction(null_direction_) { LOG_TRACE(getLogger("FullSortingMergeJoin"), "Will use full sorting merge join"); } @@ -31,6 +33,8 @@ public: std::string getName() const override { return "FullSortingMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } + int getNullDirection() const { return null_direction; } + bool addBlockToJoin(const Block & /* block */, bool /* check_limits */) override { throw Exception(ErrorCodes::LOGICAL_ERROR, "FullSortingMergeJoin::addBlockToJoin should not be called"); @@ -119,6 +123,7 @@ private: std::shared_ptr table_join; Block right_sample_block; Block totals; + int null_direction; }; } diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 0c46ce7893d..1931b1eb3a1 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -44,10 +44,8 @@ JoinStep::JoinStep( JoinPtr join_, size_t max_block_size_, size_t max_streams_, - bool keep_left_read_in_order_, - int null_direction_hint_) - : join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_), - null_direction_hint(null_direction_hint_) + bool keep_left_read_in_order_) + : join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_) { updateInputStreams(DataStreams{left_stream_, right_stream_}); } @@ -60,7 +58,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (join->pipelineType() == JoinPipelineType::YShaped) { auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( - std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, null_direction_hint, &processors); + std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, &processors); joined_pipeline->resize(max_streams); return joined_pipeline; } diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 08909ce48a9..a9059a083fe 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -19,8 +19,7 @@ public: JoinPtr join_, size_t max_block_size_, size_t max_streams_, - bool keep_left_read_in_order_, - int null_direction_hint_ = 1); + bool keep_left_read_in_order_); String getName() const override { return "Join"; } @@ -43,7 +42,6 @@ private: size_t max_block_size; size_t max_streams; bool keep_left_read_in_order; - int null_direction_hint; }; /// Special step for the case when Join is already filled. diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 37a178810cb..b63598483ef 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -271,11 +272,9 @@ bool FullMergeJoinCursor::fullyCompleted() const MergeJoinAlgorithm::MergeJoinAlgorithm( JoinPtr table_join_, const Blocks & input_headers, - size_t max_block_size_, - int null_direction_hint_) + size_t max_block_size_) : table_join(table_join_) , max_block_size(max_block_size_) - , null_direction_hint(null_direction_hint_) , log(getLogger("MergeJoinAlgorithm")) { if (input_headers.size() != 2) @@ -305,6 +304,13 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( size_t right_idx = input_headers[1].getPositionByName(right_key); left_to_right_key_remap[left_idx] = right_idx; } + + auto smjPtr = typeid_cast(table_join.get()); + if (smjPtr) + { + null_direction_hint = smjPtr->getNullDirection(); + } + } void MergeJoinAlgorithm::logElapsed(double seconds) @@ -448,7 +454,7 @@ void dispatchKind(JoinKind kind, Args && ... args) else if (Impl::enabled && kind == JoinKind::Full) return Impl::join(std::forward(args)...); else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); } std::optional MergeJoinAlgorithm::handleAllJoinState() @@ -856,7 +862,6 @@ MergeJoinTransform::MergeJoinTransform( const Blocks & input_headers, const Block & output_header, size_t max_block_size, - int null_direction_hint_, UInt64 limit_hint_) : IMergingTransform( input_headers, @@ -865,7 +870,7 @@ MergeJoinTransform::MergeJoinTransform( limit_hint_, /* always_read_till_end_= */ false, /* empty_chunk_on_finish_= */ true, - table_join, input_headers, max_block_size, null_direction_hint_) + table_join, input_headers, max_block_size) , log(getLogger("MergeJoinTransform")) { LOG_TRACE(log, "Use MergeJoinTransform"); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 8af486ea34b..cf9331abd59 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -227,8 +227,7 @@ private: class MergeJoinAlgorithm final : public IMergingAlgorithm { public: - explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers, size_t max_block_size_, - int null_direction_hint = 1); + explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers, size_t max_block_size_); const char * getName() const override { return "MergeJoinAlgorithm"; } void initialize(Inputs inputs) override; @@ -259,7 +258,7 @@ private: JoinPtr table_join; size_t max_block_size; - int null_direction_hint; + int null_direction_hint = 1; struct Statistic { @@ -284,7 +283,6 @@ public: const Blocks & input_headers, const Block & output_header, size_t max_block_size, - int null_direction_hint, UInt64 limit_hint = 0); String getName() const override { return "MergeJoinTransform"; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index e338c3ce0fa..67a8fe5dcab 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -349,7 +349,6 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped JoinPtr join, const Block & out_header, size_t max_block_size, - int null_direction_hint, Processors * collected_processors) { left->checkInitializedAndNotCompleted(); @@ -377,7 +376,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped } else { - auto joining = std::make_shared(join, inputs, out_header, max_block_size, null_direction_hint); + auto joining = std::make_shared(join, inputs, out_header, max_block_size); return mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); } } diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 4753f957a25..f0b2ead687e 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -137,7 +137,6 @@ public: JoinPtr table_join, const Block & out_header, size_t max_block_size, - int null_direction_hint, Processors * collected_processors = nullptr); /// Add other pipeline and execute it before current one. From 6b43d2e1971ac9303ca30692740c1469ddd39504 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 19 Mar 2024 13:27:26 +0100 Subject: [PATCH 45/78] Improve logging in build_download_helper --- tests/ci/build_download_helper.py | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 66734f58b23..b8469767876 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -120,12 +120,14 @@ def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str for root, _, files in os.walk(reports_path): for file in files: if file.endswith(f"_{build_name}.json"): - logging.info("Found build report json %s", file) + logging.info("Found build report json %s for %s", file, build_name) with open( os.path.join(root, file), "r", encoding="utf-8" ) as file_handler: build_report = json.load(file_handler) return build_report["build_urls"] # type: ignore + + logging.info("A build report is not found for %s", build_name) return [] @@ -197,7 +199,7 @@ def download_builds_filter( ): build_name = get_build_name_for_check(check_name) urls = read_build_urls(build_name, reports_path) - print(urls) + logging.info("The build report for %s contains the next URLs: %s", build_name, urls) if not urls: raise DownloadException("No build URLs found") @@ -223,6 +225,21 @@ def download_clickhouse_binary(check_name, reports_path, result_path): ) +def get_clickhouse_binary_url(check_name, reports_path): + build_name = get_build_name_for_check(check_name) + urls = read_build_urls(build_name, reports_path) + logging.info("The build report for %s contains the next URLs: %s", build_name, urls) + for url in urls: + check_url = url + if "?" in check_url: + check_url = check_url.split("?")[0] + + if check_url.endswith("clickhouse"): + return url + + return None + + def download_performance_build(check_name, reports_path, result_path): download_builds_filter( check_name, From 4efddb5ed47e6c7c323e9138b82e47cc71e7831c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 19 Mar 2024 14:01:31 +0100 Subject: [PATCH 46/78] Add type checking to build_download_helper --- tests/ci/build_download_helper.py | 37 +++++++++++++++++------- tests/ci/performance_comparison_check.py | 2 +- tests/ci/unit_tests_check.py | 2 +- 3 files changed, 28 insertions(+), 13 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index b8469767876..0d89515d5d8 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -6,7 +6,7 @@ import os import sys import time from pathlib import Path -from typing import Any, Callable, List, Union +from typing import Any, Callable, List, Optional, Union # isort: off import requests @@ -185,18 +185,21 @@ def download_build_with_progress(url: str, path: Path) -> None: def download_builds( - result_path: str, build_urls: List[str], filter_fn: Callable[[str], bool] + result_path: Path, build_urls: List[str], filter_fn: Callable[[str], bool] ) -> None: for url in build_urls: if filter_fn(url): fname = os.path.basename(url.replace("%2B", "+").replace("%20", " ")) logging.info("Will download %s to %s", fname, result_path) - download_build_with_progress(url, Path(result_path) / fname) + download_build_with_progress(url, result_path / fname) def download_builds_filter( - check_name, reports_path, result_path, filter_fn=lambda _: True -): + check_name: str, + reports_path: Union[Path, str], + result_path: Path, + filter_fn: Callable[[str], bool] = lambda _: True, +) -> None: build_name = get_build_name_for_check(check_name) urls = read_build_urls(build_name, reports_path) logging.info("The build report for %s contains the next URLs: %s", build_name, urls) @@ -207,25 +210,33 @@ def download_builds_filter( download_builds(result_path, urls, filter_fn) -def download_all_deb_packages(check_name, reports_path, result_path): +def download_all_deb_packages( + check_name: str, reports_path: Union[Path, str], result_path: Path +) -> None: download_builds_filter( check_name, reports_path, result_path, lambda x: x.endswith("deb") ) -def download_unit_tests(check_name, reports_path, result_path): +def download_unit_tests( + check_name: str, reports_path: Union[Path, str], result_path: Path +) -> None: download_builds_filter( check_name, reports_path, result_path, lambda x: x.endswith("unit_tests_dbms") ) -def download_clickhouse_binary(check_name, reports_path, result_path): +def download_clickhouse_binary( + check_name: str, reports_path: Union[Path, str], result_path: Path +) -> None: download_builds_filter( check_name, reports_path, result_path, lambda x: x.endswith("clickhouse") ) -def get_clickhouse_binary_url(check_name, reports_path): +def get_clickhouse_binary_url( + check_name: str, reports_path: Union[Path, str] +) -> Optional[str]: build_name = get_build_name_for_check(check_name) urls = read_build_urls(build_name, reports_path) logging.info("The build report for %s contains the next URLs: %s", build_name, urls) @@ -240,7 +251,9 @@ def get_clickhouse_binary_url(check_name, reports_path): return None -def download_performance_build(check_name, reports_path, result_path): +def download_performance_build( + check_name: str, reports_path: Union[Path, str], result_path: Path +) -> None: download_builds_filter( check_name, reports_path, @@ -249,7 +262,9 @@ def download_performance_build(check_name, reports_path, result_path): ) -def download_fuzzers(check_name, reports_path, result_path): +def download_fuzzers( + check_name: str, reports_path: Union[Path, str], result_path: Path +) -> None: download_builds_filter( check_name, reports_path, diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index f0af15397c7..c238fbae603 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -155,7 +155,7 @@ def main(): } download_builds_filter( - check_name, REPORT_PATH, TEMP_PATH, lambda url: "performance.tar.zst" in url + check_name, REPORT_PATH, temp_path, lambda url: "performance.tar.zst" in url ) assert os.path.exists(f"{TEMP_PATH}/performance.tar.zst"), "Perf artifact not found" diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 2c2862d926a..b66a4312657 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -166,7 +166,7 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - download_unit_tests(check_name, REPORT_PATH, TEMP_PATH) + download_unit_tests(check_name, REPORT_PATH, temp_path) tests_binary = temp_path / "unit_tests_dbms" os.chmod(tests_binary, 0o777) From 56e48c5d64c6e9ab3441908c04014f3e3e87de74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 19 Mar 2024 16:56:46 +0100 Subject: [PATCH 47/78] Increase compiler memory for riscv --- CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 8c4e16eace2..d345705ae53 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -61,8 +61,8 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # set CPU time limit to 1000 seconds set (RLIMIT_CPU 1000) - # -fsanitize=memory and address are too heavy - if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE) + # Sanitizers are too heavy. Also RISCV has some extra memory requirements + if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE OR ARCH_RISCV64) set (RLIMIT_DATA 10000000000) # 10G endif() From 6987622737a6ecf8dc9d7d49a6cfd4867dc36a5b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Mar 2024 15:57:19 +0000 Subject: [PATCH 48/78] Remove some tests from analyzer_tech_debt --- tests/analyzer_tech_debt.txt | 4 +--- .../0_stateless/01083_expressions_in_engine_arguments.sql | 2 +- .../0_stateless/01925_join_materialized_columns.reference | 3 +++ .../0_stateless/01925_join_materialized_columns.sql | 8 +++++--- tests/queries/0_stateless/02354_vector_search_queries.sql | 2 -- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 7cd73705e77..7dd6d059a71 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -1,13 +1,11 @@ 00223_shard_distributed_aggregation_memory_efficient 00725_memory_tracking 01062_pm_all_join_with_block_continuation -01083_expressions_in_engine_arguments 01155_rename_move_materialized_view 01584_distributed_buffer_cannot_find_column 01624_soft_constraints -01747_join_view_filter_dictionary 01925_join_materialized_columns -02354_annoy +02354_vector_search_queries # Check after constants refactoring 02901_parallel_replicas_rollup # Flaky. Please don't delete them without fixing them: diff --git a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql index b162fdb21fd..7e4f77ccdb4 100644 --- a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql +++ b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql @@ -28,7 +28,7 @@ CREATE TABLE url (n UInt64, col String) ENGINE=URL ( replace ( - 'https://localhost:8443/?query=' || 'select n, _table from ' || currentDatabase() || '.merge format CSV', ' ', '+' + 'http://localhost:8123/?query=' || 'select n, _table from ' || currentDatabase() || '.merge format CSV', ' ', '+' ), CSV ); diff --git a/tests/queries/0_stateless/01925_join_materialized_columns.reference b/tests/queries/0_stateless/01925_join_materialized_columns.reference index 5125b322aed..1d2ab582038 100644 --- a/tests/queries/0_stateless/01925_join_materialized_columns.reference +++ b/tests/queries/0_stateless/01925_join_materialized_columns.reference @@ -10,6 +10,8 @@ - 2020-01-01 12:00:00 fact1 t1_val1 2020-01-01 12:00:00 fact1 t2_val2 2020-01-01 13:00:00 fact3 t1_val3 2020-01-01 12:00:00 fact1 t2_val2 +2020-01-01 2020-01-01 12:00:00 fact1 t1_val1 2020-01-01 12:00:00 fact1 t2_val2 +2020-01-01 2020-01-01 13:00:00 fact3 t1_val3 2020-01-01 12:00:00 fact1 t2_val2 - 2020-01-01 12:00:00 fact1 t1_val1 2019-01-01 12:00:00 fact4 t2_val2 2020-01-01 12:00:00 fact1 t1_val1 2020-01-01 12:00:00 fact1 t2_val2 @@ -32,3 +34,4 @@ fact2t1_val2 fact2t1_val2 2020-01-01 12:00:00 2020-01-01 12:00:00 2020-01-01 12:00:00 +2020-01-01 12:00:00 diff --git a/tests/queries/0_stateless/01925_join_materialized_columns.sql b/tests/queries/0_stateless/01925_join_materialized_columns.sql index 271c230c35b..abb6fda3bfb 100644 --- a/tests/queries/0_stateless/01925_join_materialized_columns.sql +++ b/tests/queries/0_stateless/01925_join_materialized_columns.sql @@ -32,7 +32,8 @@ SELECT t1.dt, t2.dt FROM t1 JOIN t2 ON t1.foo = t2.bar ORDER BY t1.dt; SELECT '-'; SELECT * FROM t1 ALL JOIN t2 ON t1.dt = t2.dt ORDER BY t1.time, t2.time; SELECT '-'; -SELECT * FROM t1 ALL JOIN t2 USING (dt) ORDER BY t1.time, t2.time; +SELECT * FROM t1 ALL JOIN t2 USING (dt) ORDER BY t1.time, t2.time settings allow_experimental_analyzer=0; +SELECT * FROM t1 ALL JOIN t2 USING (dt) ORDER BY t1.time, t2.time settings allow_experimental_analyzer=1; SELECT '-'; SELECT * FROM t1 JOIN t2 ON t1.dt1 = t2.dt2 ORDER BY t1.time, t1.dimension_1, t2.time, t2.dimension_2; SELECT '-'; @@ -51,5 +52,6 @@ SELECT t1.time as talias FROM t1 JOIN t2 ON talias = t2.time; SELECT t1.time as talias FROM t1 JOIN t2 ON talias = t2.time_alias; SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time = talias; SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time_alias = talias; -SELECT time as talias FROM t1 JOIN t2 ON t1.time = talias; -- { serverError AMBIGUOUS_COLUMN_NAME } -SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT time as talias FROM t1 JOIN t2 ON t1.time = talias; -- { serverError AMBIGUOUS_COLUMN_NAME, INVALID_JOIN_ON_EXPRESSION } +SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings allow_experimental_analyzer=0; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings allow_experimental_analyzer=1; diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 2ccf869fdba..64051aa8544 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -8,8 +8,6 @@ SET allow_experimental_annoy_index = 1; SET allow_experimental_usearch_index = 1; -SET allow_experimental_analyzer = 0; - SELECT 'ARRAY, 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block'; DROP TABLE IF EXISTS tab_annoy; From b1d8521a5bff044e2fb53eb548c9810723a892e9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Mar 2024 16:58:33 +0100 Subject: [PATCH 49/78] Update 01083_expressions_in_engine_arguments.sql --- .../0_stateless/01083_expressions_in_engine_arguments.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql index 7e4f77ccdb4..b162fdb21fd 100644 --- a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql +++ b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql @@ -28,7 +28,7 @@ CREATE TABLE url (n UInt64, col String) ENGINE=URL ( replace ( - 'http://localhost:8123/?query=' || 'select n, _table from ' || currentDatabase() || '.merge format CSV', ' ', '+' + 'https://localhost:8443/?query=' || 'select n, _table from ' || currentDatabase() || '.merge format CSV', ' ', '+' ), CSV ); From de855ca917798adea684d25bfb8d287303c30d70 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 19 Mar 2024 17:04:29 +0100 Subject: [PATCH 50/78] Reduce header dependencies --- base/base/IPv4andIPv6.h | 6 +- programs/benchmark/Benchmark.cpp | 3 +- programs/format/Format.cpp | 3 +- programs/server/Server.cpp | 1 + src/Access/Common/QuotaDefs.cpp | 1 + src/Access/IAccessStorage.h | 2 + src/Access/User.cpp | 1 + .../AggregateFunctionRetention.cpp | 8 +-- .../AggregateFunctionSequenceMatch.cpp | 2 + src/Analyzer/Passes/QueryAnalysisPass.cpp | 1 + src/Backups/BackupEntriesCollector.cpp | 1 + src/Backups/BackupIO_Default.h | 1 + src/Backups/BackupIO_Disk.h | 2 + src/Backups/BackupIO_S3.h | 1 + src/Client/ClientBase.cpp | 9 +-- src/Client/LocalConnection.cpp | 5 +- src/Common/HTTPConnectionPool.cpp | 11 ++-- src/Common/Scheduler/IResourceManager.h | 3 + src/Common/Scheduler/ResouceLink.cpp | 25 ++++++++ src/Common/Scheduler/ResourceGuard.h | 3 +- src/Common/Scheduler/ResourceLink.h | 24 ++----- src/Common/SymbolIndex.cpp | 4 +- src/Common/ThreadStatus.h | 6 +- src/Common/formatIPv6.h | 7 +- ..._proxy_configuration_resolver_provider.cpp | 2 + src/Core/BaseSettings.h | 64 ------------------- src/Core/BaseSettingsProgramOptions.h | 60 +++++++++++++++++ src/Core/LogsLevel.h | 16 +++++ src/Core/SettingsEnums.h | 19 ++---- src/Daemon/SentryWriter.cpp | 1 + src/DataTypes/DataTypeDomainBool.cpp | 5 +- src/DataTypes/DataTypeFactory.h | 1 - src/DataTypes/DataTypeTuple.cpp | 2 +- src/DataTypes/DataTypeTuple.h | 2 +- src/DataTypes/IDataType.cpp | 2 +- src/DataTypes/IDataType.h | 9 ++- src/DataTypes/ObjectUtils.h | 1 + .../Serializations/SerializationInfo.h | 26 ++++---- .../SerializationInfoSettings.h | 14 ++++ .../gtest_DataType_deserializeAsText.cpp | 1 + src/Databases/DatabaseAtomic.cpp | 1 + src/Databases/DatabaseMemory.cpp | 1 + src/Databases/DatabaseOnDisk.cpp | 3 +- src/Databases/DatabaseReplicated.cpp | 1 + src/Databases/DatabasesCommon.cpp | 1 + src/Databases/IDatabase.cpp | 9 +-- .../ObjectStorages/DiskObjectStorage.cpp | 1 + src/Formats/FormatFactory.h | 1 + src/Formats/FormatSchemaInfo.h | 1 + src/Formats/ReadSchemaUtils.cpp | 1 + src/Formats/SchemaInferenceUtils.h | 6 ++ src/Functions/DateTimeTransforms.h | 1 + src/Functions/FunctionJoinGet.cpp | 7 +- .../FunctionsStringHashFixedString.cpp | 2 +- src/Functions/appendTrailingCharIfAbsent.cpp | 3 +- src/Functions/array/arrayIntersect.cpp | 1 + src/Functions/dateDiff.cpp | 2 +- src/Functions/date_trunc.cpp | 3 +- src/Functions/hasColumnInTable.cpp | 1 + .../keyvaluepair/ArgumentExtractor.h | 1 + src/Functions/tupleConcat.cpp | 2 + src/IO/CachedInMemoryReadBufferFromFile.h | 3 +- src/IO/ReadHelpers.h | 9 ++- src/IO/S3/copyS3File.h | 1 + src/IO/examples/read_buffer_from_hdfs.cpp | 2 + src/Interpreters/ActionLocksManager.cpp | 1 + src/Interpreters/ActionsVisitor.cpp | 1 + src/Interpreters/ActionsVisitor.h | 5 +- src/Interpreters/AsynchronousInsertQueue.cpp | 1 + src/Interpreters/CollectJoinOnKeysVisitor.h | 7 +- src/Interpreters/Context.h | 15 ++++- src/Interpreters/DDLTask.cpp | 3 +- src/Interpreters/DatabaseCatalog.cpp | 40 ++++++++++++ src/Interpreters/DatabaseCatalog.h | 64 ------------------- src/Interpreters/IKeyValueEntity.h | 1 + .../InJoinSubqueriesPreprocessor.cpp | 1 + src/Interpreters/InterpreterCheckQuery.cpp | 1 + .../InterpreterCreateIndexQuery.cpp | 5 +- src/Interpreters/InterpreterDescribeQuery.cpp | 1 + .../InterpreterDropIndexQuery.cpp | 1 + src/Interpreters/InterpreterDropQuery.cpp | 1 + src/Interpreters/InterpreterExistsQuery.cpp | 1 + src/Interpreters/InterpreterInsertQuery.cpp | 1 + .../InterpreterKillQueryQuery.cpp | 1 + src/Interpreters/InterpreterOptimizeQuery.cpp | 1 + src/Interpreters/InterpreterRenameQuery.cpp | 1 + .../InterpreterShowCreateQuery.cpp | 1 + .../InterpreterShowFunctionsQuery.cpp | 6 +- src/Interpreters/InterpreterUndropQuery.cpp | 1 + src/Interpreters/InterpreterWatchQuery.cpp | 3 +- src/Interpreters/JoinedTables.cpp | 1 + ...writeSumFunctionWithSumAndCountVisitor.cpp | 2 + src/Interpreters/SystemLog.cpp | 1 + src/Interpreters/SystemLog.h | 3 +- src/Interpreters/TableNameHints.h | 39 +++++++++++ src/Interpreters/executeDDLQueryOnCluster.cpp | 1 + src/Interpreters/getTableExpressions.cpp | 1 + src/Interpreters/interpretSubquery.cpp | 5 +- src/Interpreters/loadMetadata.cpp | 5 +- src/Planner/PlannerJoinTree.cpp | 1 + .../Impl/JSONColumnsBlockInputFormatBase.h | 1 + src/Processors/Formats/Impl/Parquet/Write.h | 1 + .../CreateSetAndFilterOnTheFlyTransform.cpp | 2 + .../getSourceFromASTInsertQuery.cpp | 1 + src/Server/GRPCServer.cpp | 1 + src/Server/MySQLHandler.cpp | 5 +- src/Server/PrometheusMetricsWriter.cpp | 2 + .../Distributed/DistributedSettings.cpp | 7 +- src/Storages/ExecutableSettings.cpp | 6 +- src/Storages/ExecutableSettings.h | 1 + src/Storages/FileLog/StorageFileLog.cpp | 1 + .../extractZkPathFromCreateQuery.cpp | 7 +- src/Storages/RocksDB/StorageSystemRocksDB.cpp | 1 + src/Storages/S3Queue/StorageS3Queue.cpp | 3 +- src/Storages/StorageMaterializedView.cpp | 7 +- src/Storages/StorageNull.cpp | 3 +- src/Storages/System/StorageSystemClusters.cpp | 1 + src/Storages/System/StorageSystemColumns.cpp | 1 + .../System/StorageSystemDatabases.cpp | 1 + .../System/StorageSystemDistributionQueue.cpp | 1 + .../StorageSystemDroppedTablesParts.cpp | 1 + src/Storages/System/StorageSystemGraphite.cpp | 1 + .../System/StorageSystemMutations.cpp | 1 + .../System/StorageSystemPartsBase.cpp | 1 + src/Storages/System/StorageSystemTables.cpp | 1 + .../StorageSystemZooKeeperConnection.cpp | 2 + src/Storages/buildQueryTreeForShard.cpp | 1 + src/Storages/getStructureOfRemoteTable.cpp | 1 + src/TableFunctions/TableFunctionMerge.cpp | 1 + 129 files changed, 439 insertions(+), 266 deletions(-) create mode 100644 src/Common/Scheduler/ResouceLink.cpp create mode 100644 src/Core/BaseSettingsProgramOptions.h create mode 100644 src/Core/LogsLevel.h create mode 100644 src/DataTypes/Serializations/SerializationInfoSettings.h create mode 100644 src/Interpreters/TableNameHints.h diff --git a/base/base/IPv4andIPv6.h b/base/base/IPv4andIPv6.h index e2f93b54124..9b1e518c161 100644 --- a/base/base/IPv4andIPv6.h +++ b/base/base/IPv4andIPv6.h @@ -1,8 +1,7 @@ #pragma once -#include #include -#include +#include #include namespace DB @@ -62,7 +61,8 @@ namespace std { size_t operator()(const DB::IPv6 & x) const { - return std::hash{}(std::string_view(reinterpret_cast(&x.toUnderType()), IPV6_BINARY_LENGTH)); + return std::hash{}( + std::string_view(reinterpret_cast(&x.toUnderType()), sizeof(DB::IPv6::UnderlyingType))); } }; diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 45dadfef774..eecc352d073 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -34,6 +34,7 @@ #include #include #include +#include /** A tool for evaluating ClickHouse performance. @@ -623,7 +624,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ; Settings settings; - settings.addProgramOptions(desc); + addProgramOptions(settings, desc); boost::program_options::variables_map options; boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 50f801f2560..fc73eda6815 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -102,7 +103,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { std::string_view name = field.getName(); if (name == "max_parser_depth" || name == "max_query_size") - cmd_settings.addProgramOption(desc, name, field); + addProgramOption(cmd_settings, desc, name, field); } boost::program_options::variables_map options; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b67a4eccd15..59dadd14b20 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Access/Common/QuotaDefs.cpp b/src/Access/Common/QuotaDefs.cpp index 04c16a562d2..2a22b80f65d 100644 --- a/src/Access/Common/QuotaDefs.cpp +++ b/src/Access/Common/QuotaDefs.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index ad78bf92e02..b00c8bac849 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -13,6 +13,8 @@ #include #include +#include + namespace Poco { class Logger; } namespace Poco::Net { class IPAddress; } diff --git a/src/Access/User.cpp b/src/Access/User.cpp index 91477e5f3aa..39930c9cf76 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -1,4 +1,5 @@ #include +#include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionRetention.cpp b/src/AggregateFunctions/AggregateFunctionRetention.cpp index a004f3527a2..5eaa1a7a39c 100644 --- a/src/AggregateFunctions/AggregateFunctionRetention.cpp +++ b/src/AggregateFunctions/AggregateFunctionRetention.cpp @@ -1,16 +1,16 @@ #include +#include #include - -#include #include #include #include #include #include #include -#include +#include -#include +#include +#include namespace DB diff --git a/src/AggregateFunctions/AggregateFunctionSequenceMatch.cpp b/src/AggregateFunctions/AggregateFunctionSequenceMatch.cpp index ff9259e3aac..bc3d4da5e39 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceMatch.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceMatch.cpp @@ -10,6 +10,8 @@ #include #include #include +#include + #include #include diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 1da5d86edf3..d8a196ee3d4 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -48,6 +48,7 @@ #include #include #include +#include #include #include diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 5c0b80aa894..c71ce195388 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Backups/BackupIO_Default.h b/src/Backups/BackupIO_Default.h index 639293f22d9..4dba0324703 100644 --- a/src/Backups/BackupIO_Default.h +++ b/src/Backups/BackupIO_Default.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index 575ec3f5707..3d3253877bd 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -1,7 +1,9 @@ #pragma once #include +#include #include + #include diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 8ab8e1fb566..57108d122ea 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -4,6 +4,7 @@ #if USE_AWS_S3 #include +#include #include #include #include diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index d561a64895b..582dc2c9e6b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -2714,9 +2715,9 @@ private: void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) { if (allow_repeated_settings) - cmd_settings.addProgramOptionsAsMultitokens(options_description.main_description.value()); + addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); else - cmd_settings.addProgramOptions(options_description.main_description.value()); + addProgramOptions(cmd_settings, options_description.main_description.value()); if (allow_merge_tree_settings) { @@ -2737,9 +2738,9 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, return; if (allow_repeated_settings) - cmd_merge_tree_settings.addProgramOptionAsMultitoken(main_options, name, setting); + addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting); else - cmd_merge_tree_settings.addProgramOption(main_options, name, setting); + addProgramOption(cmd_merge_tree_settings, main_options, name, setting); }; const auto & setting_name = setting.getName(); diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index dbb115f44ef..afcaa4d6098 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -1,13 +1,14 @@ #include "LocalConnection.h" +#include +#include #include #include #include -#include #include +#include #include #include #include -#include namespace DB diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index a21438a11a2..f729b8ea8d0 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -10,14 +10,15 @@ #include #include -#include -#include -#include #include +#include +#include +#include +#include +#include #include -#include -#include +#include #include "config.h" diff --git a/src/Common/Scheduler/IResourceManager.h b/src/Common/Scheduler/IResourceManager.h index 0af74e00960..8a7077ac3d5 100644 --- a/src/Common/Scheduler/IResourceManager.h +++ b/src/Common/Scheduler/IResourceManager.h @@ -12,6 +12,9 @@ namespace DB { +class ISchedulerNode; +using SchedulerNodePtr = std::shared_ptr; + /* * Instance of derived class holds everything required for resource consumption, * including resources currently registered at `SchedulerRoot`. This is required to avoid diff --git a/src/Common/Scheduler/ResouceLink.cpp b/src/Common/Scheduler/ResouceLink.cpp new file mode 100644 index 00000000000..2da5dba62dc --- /dev/null +++ b/src/Common/Scheduler/ResouceLink.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + +namespace DB +{ +void ResourceLink::adjust(ResourceCost estimated_cost, ResourceCost real_cost) const +{ + if (queue) + queue->adjustBudget(estimated_cost, real_cost); +} + +void ResourceLink::consumed(ResourceCost cost) const +{ + if (queue) + queue->consumeBudget(cost); +} + +void ResourceLink::accumulate(DB::ResourceCost cost) const +{ + if (queue) + queue->accumulateBudget(cost); +} +} + diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 50f665a384b..3c29f588fba 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -2,9 +2,10 @@ #include +#include +#include #include #include -#include #include #include diff --git a/src/Common/Scheduler/ResourceLink.h b/src/Common/Scheduler/ResourceLink.h index 108f51ac399..450d9bc1efa 100644 --- a/src/Common/Scheduler/ResourceLink.h +++ b/src/Common/Scheduler/ResourceLink.h @@ -2,12 +2,10 @@ #include -#include -#include - - namespace DB { +class ISchedulerQueue; +using ResourceCost = Int64; /* * Everything required for resource consumption. Connection to a specific resource queue. @@ -17,23 +15,11 @@ struct ResourceLink ISchedulerQueue * queue = nullptr; bool operator==(const ResourceLink &) const = default; - void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const - { - if (queue) - queue->adjustBudget(estimated_cost, real_cost); - } + void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const; - void consumed(ResourceCost cost) const - { - if (queue) - queue->consumeBudget(cost); - } + void consumed(ResourceCost cost) const; - void accumulate(ResourceCost cost) const - { - if (queue) - queue->accumulateBudget(cost); - } + void accumulate(ResourceCost cost) const; }; } diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 8dde617fc74..a2d58fc7e8c 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include @@ -11,8 +13,6 @@ #include -#include - /** ELF object can contain three different places with symbol names and addresses: diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 77386af6c2b..48b52f8aa6e 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -1,12 +1,12 @@ #pragma once -#include -#include +#include #include +#include +#include #include #include #include -#include #include diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index fa7a9592c48..3451eda6b3c 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -1,13 +1,12 @@ #pragma once -#include -#include #include +#include #include #include -#include -#include #include +#include +#include #include constexpr size_t IPV4_BINARY_LENGTH = 4; diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index 9a7447b02e4..d5d6f86f661 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -4,6 +4,8 @@ #include #include +#include + using ConfigurationPtr = Poco::AutoPtr; class ProxyConfigurationResolverProviderTests : public ::testing::Test diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 6f3245c83e8..7191038a4ce 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace boost::program_options @@ -129,18 +128,6 @@ public: std::conditional_t custom_setting; }; - /// Adds program options to set the settings from a command line. - /// (Don't forget to call notify() on the `variables_map` after parsing it!) - void addProgramOptions(boost::program_options::options_description & options); - - /// Adds program options as to set the settings from a command line. - /// Allows to set one setting multiple times, the last value will be used. - /// (Don't forget to call notify() on the `variables_map` after parsing it!) - void addProgramOptionsAsMultitokens(boost::program_options::options_description & options); - - void addProgramOption(boost::program_options::options_description & options, std::string_view name, const SettingFieldRef & field); - void addProgramOptionAsMultitoken(boost::program_options::options_description & options, std::string_view name, const SettingFieldRef & field); - enum SkipFlags { SKIP_NONE = 0, @@ -561,57 +548,6 @@ String BaseSettings::toString() const return out.str(); } -template -void BaseSettings::addProgramOptions(boost::program_options::options_description & options) -{ - const auto & settings_to_aliases = TTraits::settingsToAliases(); - for (const auto & field : all()) - { - std::string_view name = field.getName(); - addProgramOption(options, name, field); - - if (auto it = settings_to_aliases.find(name); it != settings_to_aliases.end()) - { - for (const auto alias : it->second) - addProgramOption(options, alias, field); - } - } -} - -template -void BaseSettings::addProgramOptionsAsMultitokens(boost::program_options::options_description & options) -{ - const auto & settings_to_aliases = TTraits::settingsToAliases(); - for (const auto & field : all()) - { - std::string_view name = field.getName(); - addProgramOptionAsMultitoken(options, name, field); - - if (auto it = settings_to_aliases.find(name); it != settings_to_aliases.end()) - { - for (const auto alias : it->second) - addProgramOptionAsMultitoken(options, alias, field); - } - } -} - - -template -void BaseSettings::addProgramOption(boost::program_options::options_description & options, std::string_view name, const SettingFieldRef & field) -{ - auto on_program_option = boost::function1([this, name](const std::string & value) { set(name, value); }); - options.add(boost::shared_ptr(new boost::program_options::option_description( - name.data(), boost::program_options::value()->composing()->notifier(on_program_option), field.getDescription()))); -} - -template -void BaseSettings::addProgramOptionAsMultitoken(boost::program_options::options_description & options, std::string_view name, const SettingFieldRef & field) -{ - auto on_program_option = boost::function1([this, name](const Strings & values) { set(name, values.back()); }); - options.add(boost::shared_ptr(new boost::program_options::option_description( - name.data(), boost::program_options::value()->multitoken()->composing()->notifier(on_program_option), field.getDescription()))); -} - template bool operator==(const BaseSettings & left, const BaseSettings & right) { diff --git a/src/Core/BaseSettingsProgramOptions.h b/src/Core/BaseSettingsProgramOptions.h new file mode 100644 index 00000000000..6c8166fc119 --- /dev/null +++ b/src/Core/BaseSettingsProgramOptions.h @@ -0,0 +1,60 @@ +#pragma once + +#include +#include + +#include + +namespace DB +{ + +template +void addProgramOptionAsMultitoken(T &cmd_settings, boost::program_options::options_description & options, std::string_view name, const typename T::SettingFieldRef & field) +{ + auto on_program_option = boost::function1([&cmd_settings, name](const Strings & values) { cmd_settings.set(name, values.back()); }); + options.add(boost::shared_ptr(new boost::program_options::option_description( + name.data(), boost::program_options::value()->multitoken()->composing()->notifier(on_program_option), field.getDescription()))); +} + +template +void addProgramOptionsAsMultitokens(T &cmd_settings, boost::program_options::options_description & options) +{ + const auto & settings_to_aliases = T::Traits::settingsToAliases(); + for (const auto & field : cmd_settings.all()) + { + std::string_view name = field.getName(); + addProgramOptionAsMultitoken(cmd_settings, options, name, field); + + if (auto it = settings_to_aliases.find(name); it != settings_to_aliases.end()) + for (const auto alias : it->second) + addProgramOptionAsMultitoken(cmd_settings, options, alias, field); + } +} + +/// Adds program options to set the settings from a command line. +/// (Don't forget to call notify() on the `variables_map` after parsing it!) +template +void addProgramOption(T &cmd_settings, boost::program_options::options_description & options, std::string_view name, const typename T::SettingFieldRef & field) +{ + auto on_program_option = boost::function1([&cmd_settings, name](const std::string & value) { cmd_settings.set(name, value); }); + options.add(boost::shared_ptr(new boost::program_options::option_description( + name.data(), boost::program_options::value()->composing()->notifier(on_program_option), field.getDescription()))); +} + +template +void addProgramOptions(T &cmd_settings, boost::program_options::options_description & options) +{ + const auto & settings_to_aliases = T::Traits::settingsToAliases(); + for (const auto & field : cmd_settings.all()) + { + std::string_view name = field.getName(); + addProgramOption(cmd_settings, options, name, field); + + if (auto it = settings_to_aliases.find(name); it != settings_to_aliases.end()) + for (const auto alias : it->second) + addProgramOption(cmd_settings, options, alias, field); + } +} + + +} diff --git a/src/Core/LogsLevel.h b/src/Core/LogsLevel.h new file mode 100644 index 00000000000..95721820d9c --- /dev/null +++ b/src/Core/LogsLevel.h @@ -0,0 +1,16 @@ +#pragma once + +namespace DB +{ +enum class LogsLevel +{ + none = 0, /// Disable + fatal, + error, + warning, + information, + debug, + trace, + test, +}; +} diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 22fcf0389d8..421712ed868 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -1,12 +1,13 @@ #pragma once -#include #include -#include +#include +#include #include #include -#include #include +#include +#include namespace DB @@ -90,18 +91,6 @@ DECLARE_SETTING_ENUM_WITH_RENAME(IntervalOutputFormat, FormatSettings::IntervalO DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) -enum class LogsLevel -{ - none = 0, /// Disable - fatal, - error, - warning, - information, - debug, - trace, - test, -}; - DECLARE_SETTING_ENUM(LogsLevel) diff --git a/src/Daemon/SentryWriter.cpp b/src/Daemon/SentryWriter.cpp index 192e9952b9a..0fa8b99a6f7 100644 --- a/src/Daemon/SentryWriter.cpp +++ b/src/Daemon/SentryWriter.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include "config.h" #include diff --git a/src/DataTypes/DataTypeDomainBool.cpp b/src/DataTypes/DataTypeDomainBool.cpp index 245c5495299..3d19b6262d8 100644 --- a/src/DataTypes/DataTypeDomainBool.cpp +++ b/src/DataTypes/DataTypeDomainBool.cpp @@ -1,6 +1,7 @@ -#include -#include #include +#include +#include +#include namespace DB { diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index a2aeb6f3646..4727cb3ae5c 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index eb218d8efb7..5bbd79160d4 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -346,7 +346,7 @@ SerializationPtr DataTypeTuple::getSerialization(const SerializationInfo & info) return std::make_shared(std::move(serializations), have_explicit_names); } -MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfo::Settings & settings) const +MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfoSettings & settings) const { MutableSerializationInfos infos; infos.reserve(elems.size()); diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 4e5a0c1b33c..15561fe4286 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -58,7 +58,7 @@ public: SerializationPtr doGetDefaultSerialization() const override; SerializationPtr getSerialization(const SerializationInfo & info) const override; - MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const override; + MutableSerializationInfoPtr createSerializationInfo(const SerializationInfoSettings & settings) const override; SerializationInfoPtr getSerializationInfo(const IColumn & column) const override; const DataTypePtr & getElement(size_t i) const { return elems[i]; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 40915418aea..344b81be960 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -202,7 +202,7 @@ void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const custom_serialization = std::move(custom_desc_->serialization); } -MutableSerializationInfoPtr IDataType::createSerializationInfo(const SerializationInfo::Settings & settings) const +MutableSerializationInfoPtr IDataType::createSerializationInfo(const SerializationInfoSettings & settings) const { return std::make_shared(ISerialization::Kind::DEFAULT, settings); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 4403e3d9bd4..eaf798a3017 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -7,8 +7,6 @@ #include #include #include -#include - namespace DB { @@ -38,6 +36,11 @@ struct DataTypeWithConstInfo using DataTypesWithConstInfo = std::vector; +class SerializationInfo; +using SerializationInfoPtr = std::shared_ptr; +using MutableSerializationInfoPtr = std::shared_ptr; +struct SerializationInfoSettings; + /** Properties of data type. * * Contains methods for getting serialization instances. @@ -117,7 +120,7 @@ public: Names getSubcolumnNames() const; - virtual MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const; + virtual MutableSerializationInfoPtr createSerializationInfo(const SerializationInfoSettings & settings) const; virtual SerializationInfoPtr getSerializationInfo(const IColumn & column) const; /// TODO: support more types. diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index 2bfcaae09ca..9897f7d20f9 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -13,6 +13,7 @@ namespace DB struct StorageSnapshot; using StorageSnapshotPtr = std::shared_ptr; +class ColumnsDescription; /// Returns number of dimensions in Array type. 0 if type is not array. size_t getNumberOfDimensions(const IDataType & type); diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 3d8f4f1d00c..5a900a5521c 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -2,6 +2,8 @@ #include #include +#include + #include @@ -28,6 +30,8 @@ constexpr auto SERIALIZATION_INFO_VERSION = 0; class SerializationInfo { public: + using Settings = SerializationInfoSettings; + struct Data { size_t num_rows = 0; @@ -38,16 +42,8 @@ public: void addDefaults(size_t length); }; - struct Settings - { - const double ratio_of_defaults_for_sparse = 1.0; - const bool choose_kind = false; - - bool isAlwaysDefault() const { return ratio_of_defaults_for_sparse >= 1.0; } - }; - - SerializationInfo(ISerialization::Kind kind_, const Settings & settings_); - SerializationInfo(ISerialization::Kind kind_, const Settings & settings_, const Data & data_); + SerializationInfo(ISerialization::Kind kind_, const SerializationInfoSettings & settings_); + SerializationInfo(ISerialization::Kind kind_, const SerializationInfoSettings & settings_, const Data & data_); virtual ~SerializationInfo() = default; @@ -64,7 +60,7 @@ public: virtual std::shared_ptr createWithType( const IDataType & old_type, const IDataType & new_type, - const Settings & new_settings) const; + const SerializationInfoSettings & new_settings) const; virtual void serialializeKindBinary(WriteBuffer & out) const; virtual void deserializeFromKindsBinary(ReadBuffer & in); @@ -73,14 +69,14 @@ public: virtual void fromJSON(const Poco::JSON::Object & object); void setKind(ISerialization::Kind kind_) { kind = kind_; } - const Settings & getSettings() const { return settings; } + const SerializationInfoSettings & getSettings() const { return settings; } const Data & getData() const { return data; } ISerialization::Kind getKind() const { return kind; } - static ISerialization::Kind chooseKind(const Data & data, const Settings & settings); + static ISerialization::Kind chooseKind(const Data & data, const SerializationInfoSettings & settings); protected: - const Settings settings; + const SerializationInfoSettings settings; ISerialization::Kind kind; Data data; @@ -96,7 +92,7 @@ using MutableSerializationInfos = std::vector; class SerializationInfoByName : public std::map { public: - using Settings = SerializationInfo::Settings; + using Settings = SerializationInfoSettings; SerializationInfoByName() = default; SerializationInfoByName(const NamesAndTypesList & columns, const Settings & settings); diff --git a/src/DataTypes/Serializations/SerializationInfoSettings.h b/src/DataTypes/Serializations/SerializationInfoSettings.h new file mode 100644 index 00000000000..26f2c344bea --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInfoSettings.h @@ -0,0 +1,14 @@ +#pragma once + +namespace DB +{ + +struct SerializationInfoSettings +{ + const double ratio_of_defaults_for_sparse = 1.0; + const bool choose_kind = false; + + bool isAlwaysDefault() const { return ratio_of_defaults_for_sparse >= 1.0; } +}; + +} diff --git a/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp b/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp index 0373e55a62d..bf5337c89da 100644 --- a/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp +++ b/src/DataTypes/tests/gtest_DataType_deserializeAsText.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 0bf7c8af4b4..6f3f1151153 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 794eebbc399..4ff7b3c7f2b 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index dcfc1916450..896baa561b4 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -5,8 +5,9 @@ #include #include #include -#include #include +#include +#include #include #include #include diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3b6a712510d..59b3e52e139 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index f8d6ad69ba8..d40e5f98aaa 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index ae8fc58bf89..95d671d1960 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -1,11 +1,12 @@ #include #include -#include -#include -#include #include -#include +#include +#include +#include #include +#include +#include namespace CurrentMetrics diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 696a0ef7416..9d20427e0d5 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 46c1b8ddcdd..93f67e83831 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -23,6 +23,7 @@ namespace DB class Block; struct Settings; struct FormatFactorySettings; +struct ReadSettings; class ReadBuffer; class WriteBuffer; diff --git a/src/Formats/FormatSchemaInfo.h b/src/Formats/FormatSchemaInfo.h index e8758c3f761..0dd28699896 100644 --- a/src/Formats/FormatSchemaInfo.h +++ b/src/Formats/FormatSchemaInfo.h @@ -8,6 +8,7 @@ namespace DB { class Context; +class Block; /// Extracts information about where the format schema file is from passed context and keep it. class FormatSchemaInfo diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index b05b768899b..735b536986d 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Formats/SchemaInferenceUtils.h b/src/Formats/SchemaInferenceUtils.h index b492d9b22b6..bcf3d194825 100644 --- a/src/Formats/SchemaInferenceUtils.h +++ b/src/Formats/SchemaInferenceUtils.h @@ -3,9 +3,15 @@ #include #include +#include + namespace DB { +class Block; +class NamesAndTypesList; +using NamesAndTypesLists = std::vector; + /// Struct with some additional information about inferred types for JSON formats. struct JSONInferenceInfo { diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index f5072de24f3..558c309007c 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index 5602c88c60e..085c4db3f57 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -1,12 +1,13 @@ #include +#include #include #include -#include -#include #include +#include +#include +#include #include #include -#include namespace DB diff --git a/src/Functions/FunctionsStringHashFixedString.cpp b/src/Functions/FunctionsStringHashFixedString.cpp index bbb02de6c9f..f4160badd37 100644 --- a/src/Functions/FunctionsStringHashFixedString.cpp +++ b/src/Functions/FunctionsStringHashFixedString.cpp @@ -274,7 +274,7 @@ public: const typename ColumnIPv6::Container & data = col_from_ip->getData(); const auto size = col_from_ip->size(); auto & chars_to = col_to->getChars(); - const auto length = IPV6_BINARY_LENGTH; + const auto length = sizeof(IPv6::UnderlyingType); chars_to.resize(size * Impl::length); for (size_t i = 0; i < size; ++i) { diff --git a/src/Functions/appendTrailingCharIfAbsent.cpp b/src/Functions/appendTrailingCharIfAbsent.cpp index 7ff35e599be..a5554171aaa 100644 --- a/src/Functions/appendTrailingCharIfAbsent.cpp +++ b/src/Functions/appendTrailingCharIfAbsent.cpp @@ -1,9 +1,10 @@ #include -#include #include #include #include #include +#include +#include namespace DB diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index ffc7480219e..209441eb301 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index f75e6eb4fc8..8f318d1bc55 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -6,7 +6,7 @@ #include #include #include - +#include #include #include #include diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index f64848e9185..8493df17a2f 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -1,9 +1,10 @@ #include -#include #include +#include #include #include #include +#include #include #include diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 66ed515e490..48783a672e2 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include diff --git a/src/Functions/keyvaluepair/ArgumentExtractor.h b/src/Functions/keyvaluepair/ArgumentExtractor.h index e6538584d01..6ff8aa36f13 100644 --- a/src/Functions/keyvaluepair/ArgumentExtractor.h +++ b/src/Functions/keyvaluepair/ArgumentExtractor.h @@ -4,6 +4,7 @@ #include #include +#include #include namespace DB diff --git a/src/Functions/tupleConcat.cpp b/src/Functions/tupleConcat.cpp index 0556f4181e6..c48e4d61463 100644 --- a/src/Functions/tupleConcat.cpp +++ b/src/Functions/tupleConcat.cpp @@ -4,6 +4,8 @@ #include #include +#include + namespace DB { namespace ErrorCodes diff --git a/src/IO/CachedInMemoryReadBufferFromFile.h b/src/IO/CachedInMemoryReadBufferFromFile.h index 300c2e82386..a0d07486359 100644 --- a/src/IO/CachedInMemoryReadBufferFromFile.h +++ b/src/IO/CachedInMemoryReadBufferFromFile.h @@ -1,7 +1,8 @@ #pragma once -#include #include +#include +#include namespace DB { diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 49530f4787a..ca568c469b4 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -154,9 +154,12 @@ inline void readIPv6Binary(IPv6 & ip, ReadBuffer & buf) size_t size = 0; readVarUInt(size, buf); - if (size != IPV6_BINARY_LENGTH) - throw Exception(ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH, - "Size of the string {} doesn't match size of binary IPv6 {}", size, IPV6_BINARY_LENGTH); + if (size != sizeof(IPv6::UnderlyingType)) + throw Exception( + ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH, + "Size of the string {} doesn't match size of binary IPv6 {}", + size, + sizeof(IPv6::UnderlyingType)); buf.readStrict(reinterpret_cast(&ip.toUnderType()), size); } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 093d26ba7bb..5eb6f702473 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -14,6 +14,7 @@ namespace DB { +struct ReadSettings; class SeekableReadBuffer; using CreateReadBuffer = std::function()>; diff --git a/src/IO/examples/read_buffer_from_hdfs.cpp b/src/IO/examples/read_buffer_from_hdfs.cpp index 977dd2ae227..c499542fedb 100644 --- a/src/IO/examples/read_buffer_from_hdfs.cpp +++ b/src/IO/examples/read_buffer_from_hdfs.cpp @@ -6,6 +6,8 @@ #include #include +#include + using namespace DB; int main() diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index 65f13ebd66c..28803a94c80 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -1,5 +1,6 @@ #include "ActionLocksManager.h" #include +#include #include #include diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index f9a58c8c3ca..16e2449206d 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -44,6 +44,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 1e2ebaf6d87..643bf50ecd4 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -1,13 +1,14 @@ #pragma once #include +#include +#include #include #include #include #include #include -#include -#include +#include namespace DB { diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 9234d052d97..1c6039bbbf7 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 194ec01bcd5..ff10db3beb5 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -1,11 +1,12 @@ #pragma once +#include #include +#include +#include +#include #include #include -#include -#include -#include namespace DB diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index dbd9bbd1ff8..c2d6febc733 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -19,9 +19,8 @@ #include #include #include -#include +#include #include -#include #include #include #include @@ -149,6 +148,18 @@ template class MergeTreeBackgroundExecutor; class AsyncLoader; +struct TemporaryTableHolder; +using TemporaryTablesMapping = std::map>; + +class LoadTask; +using LoadTaskPtr = std::shared_ptr; +using LoadTaskPtrs = std::vector; + +class IClassifier; +using ClassifierPtr = std::shared_ptr; +class IResourceManager; +using ResourceManagerPtr = std::shared_ptr; + /// Scheduling policy can be changed using `background_merges_mutations_scheduling_policy` config option. /// By default concurrent merges are scheduled using "round_robin" to ensure fair and starvation-free operation. /// Previously in heavily overloaded shards big merges could possibly be starved by smaller diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index e10f3ecfbc9..a37b4db029a 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include #include @@ -14,7 +16,6 @@ #include #include #include -#include namespace DB diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c2f2003aabd..ec6c8b5924f 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -1705,4 +1706,43 @@ DDLGuard::~DDLGuard() releaseTableLock(); } +std::pair TableNameHints::getHintForTable(const String & table_name) const +{ + auto results = this->getHints(table_name, getAllRegisteredNames()); + if (results.empty()) + return getExtendedHintForTable(table_name); + return std::make_pair(database->getDatabaseName(), results[0]); +} + +std::pair TableNameHints::getExtendedHintForTable(const String & table_name) const +{ + /// load all available databases from the DatabaseCatalog instance + auto & database_catalog = DatabaseCatalog::instance(); + auto all_databases = database_catalog.getDatabases(); + + for (const auto & [db_name, db] : all_databases) + { + /// this case should be covered already by getHintForTable + if (db_name == database->getDatabaseName()) + continue; + + TableNameHints hints(db, context); + auto results = hints.getHints(table_name); + + /// if the results are not empty, return the first instance of the table_name + /// and the corresponding database_name that was found. + if (!results.empty()) + return std::make_pair(db_name, results[0]); + } + return {}; +} + +Names TableNameHints::getAllRegisteredNames() const +{ + Names result; + if (database) + for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) + result.emplace_back(table_it->name()); + return result; +} } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 6995fc51941..30b49cf1fed 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -1,13 +1,11 @@ #pragma once #include -#include #include #include #include #include #include -#include #include #include @@ -365,68 +363,6 @@ private: static constexpr time_t DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC = 5; }; -class TableNameHints : public IHints<> -{ -public: - TableNameHints(ConstDatabasePtr database_, ContextPtr context_) - : context(context_), - database(database_) - { - } - - /// getHintForTable tries to get a hint for the provided table_name in the provided - /// database. If the results are empty, it goes for extended hints for the table - /// with getExtendedHintForTable which looks for the table name in every database that's - /// available in the database catalog. It finally returns a single hint which is the database - /// name and table_name pair which is similar to the table_name provided. Perhaps something to - /// consider is should we return more than one pair of hint? - std::pair getHintForTable(const String & table_name) const - { - auto results = this->getHints(table_name, getAllRegisteredNames()); - if (results.empty()) - return getExtendedHintForTable(table_name); - return std::make_pair(database->getDatabaseName(), results[0]); - } - - /// getExtendedHintsForTable tries to get hint for the given table_name across all - /// the databases that are available in the database catalog. - std::pair getExtendedHintForTable(const String & table_name) const - { - /// load all available databases from the DatabaseCatalog instance - auto & database_catalog = DatabaseCatalog::instance(); - auto all_databases = database_catalog.getDatabases(); - - for (const auto & [db_name, db] : all_databases) - { - /// this case should be covered already by getHintForTable - if (db_name == database->getDatabaseName()) - continue; - - TableNameHints hints(db, context); - auto results = hints.getHints(table_name); - - /// if the results are not empty, return the first instance of the table_name - /// and the corresponding database_name that was found. - if (!results.empty()) - return std::make_pair(db_name, results[0]); - } - return {}; - } - - Names getAllRegisteredNames() const override - { - Names result; - if (database) - for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) - result.emplace_back(table_it->name()); - return result; - } - -private: - ContextPtr context; - ConstDatabasePtr database; -}; - /// This class is useful when creating a table or database. /// Usually we create IStorage/IDatabase object first and then add it to IDatabase/DatabaseCatalog. diff --git a/src/Interpreters/IKeyValueEntity.h b/src/Interpreters/IKeyValueEntity.h index d1ceda57f0e..856ce28bae7 100644 --- a/src/Interpreters/IKeyValueEntity.h +++ b/src/Interpreters/IKeyValueEntity.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include diff --git a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index ec4241a2740..3b3ef928b42 100644 --- a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 98a281bd5ad..ae8cef3f102 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index cd2f996c74c..aed4b0587b4 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -1,10 +1,11 @@ -#include -#include #include #include #include +#include #include +#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 04d44e34fff..87b0eb89302 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterDropIndexQuery.cpp b/src/Interpreters/InterpreterDropIndexQuery.cpp index 025677eeb91..f052aa201f1 100644 --- a/src/Interpreters/InterpreterDropIndexQuery.cpp +++ b/src/Interpreters/InterpreterDropIndexQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 72aa4cc63e3..a7040709034 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index e4176a44533..14ca9eaed0a 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 3e8bb268fe7..fc58f7b5098 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 26dae6a1df3..6d6b1085ffb 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index 1e2eaa50ab1..907a01b0432 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 06b6ebc9cbb..b63672f1757 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 9edac1fd8e1..0fca7b64d5a 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterShowFunctionsQuery.cpp b/src/Interpreters/InterpreterShowFunctionsQuery.cpp index 829670d7929..96a730f6527 100644 --- a/src/Interpreters/InterpreterShowFunctionsQuery.cpp +++ b/src/Interpreters/InterpreterShowFunctionsQuery.cpp @@ -1,8 +1,8 @@ -#include -#include - +#include #include #include +#include +#include #include #include diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp index f628a656947..920df3d6aed 100644 --- a/src/Interpreters/InterpreterUndropQuery.cpp +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index 2b68c5d7a10..4937d8660e0 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -12,9 +12,10 @@ limitations under the License. */ #include #include #include +#include +#include #include #include -#include #include #include #include diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 49693332280..5b549a19083 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -2,6 +2,7 @@ #include +#include #include #include #include diff --git a/src/Interpreters/RewriteSumFunctionWithSumAndCountVisitor.cpp b/src/Interpreters/RewriteSumFunctionWithSumAndCountVisitor.cpp index 2f5e597bdab..8e60603cca9 100644 --- a/src/Interpreters/RewriteSumFunctionWithSumAndCountVisitor.cpp +++ b/src/Interpreters/RewriteSumFunctionWithSumAndCountVisitor.cpp @@ -4,6 +4,8 @@ #include #include +#include + namespace DB { diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index efb2559ce42..db73fe038c0 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index c296b91e24a..e5b79585701 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -1,8 +1,9 @@ #pragma once +#include #include -#include +#include namespace DB { diff --git a/src/Interpreters/TableNameHints.h b/src/Interpreters/TableNameHints.h new file mode 100644 index 00000000000..d3d8d10c50c --- /dev/null +++ b/src/Interpreters/TableNameHints.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +class IDatabase; +using ConstDatabasePtr = std::shared_ptr; + +class TableNameHints : public IHints<> +{ +public: + TableNameHints(ConstDatabasePtr database_, ContextPtr context_) : context(context_), database(database_) { } + + /// getHintForTable tries to get a hint for the provided table_name in the provided + /// database. If the results are empty, it goes for extended hints for the table + /// with getExtendedHintForTable which looks for the table name in every database that's + /// available in the database catalog. It finally returns a single hint which is the database + /// name and table_name pair which is similar to the table_name provided. Perhaps something to + /// consider is should we return more than one pair of hint? + std::pair getHintForTable(const String & table_name) const; + + /// getExtendedHintsForTable tries to get hint for the given table_name across all + /// the databases that are available in the database catalog. + std::pair getExtendedHintForTable(const String & table_name) const; + + Names getAllRegisteredNames() const override; + +private: + ContextPtr context; + ConstDatabasePtr database; +}; + +} diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index df8236c11f4..fbcb57b6125 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 2853be4c05e..cd92d9ecbf3 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 5f00be07fa5..6b42345f1d6 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -10,9 +10,10 @@ #include #include -#include -#include #include +#include +#include +#include namespace DB { diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 226472175b3..9c3922b8bda 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -6,11 +6,12 @@ #include #include +#include +#include #include #include -#include -#include #include +#include #include #include diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 8ca8f0f258b..a8d455cf9ee 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -51,6 +51,7 @@ #include #include +#include #include #include #include diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h index 6d2532b350a..b1fd86bb806 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Processors/Formats/Impl/Parquet/Write.h b/src/Processors/Formats/Impl/Parquet/Write.h index 24733ac276b..f162984fd5e 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.h +++ b/src/Processors/Formats/Impl/Parquet/Write.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 83a75318d61..c38e9e97ed5 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -12,6 +12,8 @@ #include #include +#include + namespace DB { diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 8a13973b970..a2a42f27c3f 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index dfae812c620..72135157117 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 9efcebfc72d..ad5cae01540 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -22,10 +23,10 @@ #include #include #include -#include -#include #include +#include #include +#include #if USE_SSL # include diff --git a/src/Server/PrometheusMetricsWriter.cpp b/src/Server/PrometheusMetricsWriter.cpp index d0fdcd61493..85eafbe4808 100644 --- a/src/Server/PrometheusMetricsWriter.cpp +++ b/src/Server/PrometheusMetricsWriter.cpp @@ -1,7 +1,9 @@ #include "PrometheusMetricsWriter.h" #include +#include #include + #include #include "config.h" diff --git a/src/Storages/Distributed/DistributedSettings.cpp b/src/Storages/Distributed/DistributedSettings.cpp index e07b8da34af..1f6aa6c72fa 100644 --- a/src/Storages/Distributed/DistributedSettings.cpp +++ b/src/Storages/Distributed/DistributedSettings.cpp @@ -1,9 +1,12 @@ -#include +#include #include -#include #include +#include +#include #include +#include + namespace DB { diff --git a/src/Storages/ExecutableSettings.cpp b/src/Storages/ExecutableSettings.cpp index dc462350a06..d00e4098181 100644 --- a/src/Storages/ExecutableSettings.cpp +++ b/src/Storages/ExecutableSettings.cpp @@ -1,10 +1,10 @@ #include "ExecutableSettings.h" -#include - +#include #include -#include #include +#include +#include namespace DB { diff --git a/src/Storages/ExecutableSettings.h b/src/Storages/ExecutableSettings.h index 10dbae8ac9f..95627f08d16 100644 --- a/src/Storages/ExecutableSettings.h +++ b/src/Storages/ExecutableSettings.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index a5f2331a068..7b0cfdf6a6c 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp b/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp index 45d667047af..8ea732b0243 100644 --- a/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp +++ b/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp @@ -1,11 +1,12 @@ -#include -#include #include #include +#include +#include #include #include #include -#include +#include +#include namespace DB diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.cpp b/src/Storages/RocksDB/StorageSystemRocksDB.cpp index eec2f53381f..4406a7c3fd4 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.cpp +++ b/src/Storages/RocksDB/StorageSystemRocksDB.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 6e7ac2b47b8..e4cd166437e 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -24,8 +25,8 @@ #include #include #include -#include +#include namespace fs = std::filesystem; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 9958d65819b..16428d2dc1c 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -5,14 +5,15 @@ #include #include +#include #include +#include #include #include -#include #include -#include +#include #include -#include +#include #include #include diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index 5e4fde99306..7a8852ca2d6 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -2,8 +2,9 @@ #include #include -#include #include +#include +#include #include diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 3c01b4717cc..cb8d5caa50c 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 5c96c6502af..8c6d29a3b70 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index f5537b508ba..2351c3c6a2a 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index 50c6436f316..e2058448904 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.cpp b/src/Storages/System/StorageSystemDroppedTablesParts.cpp index 20baeee1d3b..344c653f7e4 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.cpp +++ b/src/Storages/System/StorageSystemDroppedTablesParts.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/System/StorageSystemGraphite.cpp b/src/Storages/System/StorageSystemGraphite.cpp index eaa386763c2..d8b760e1302 100644 --- a/src/Storages/System/StorageSystemGraphite.cpp +++ b/src/Storages/System/StorageSystemGraphite.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index 60b80e0b0ad..94656008029 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 6bdfdd357e8..812a4e6efbe 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -22,6 +22,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 639c1455b83..edfc7213dcd 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 8041370ee92..950e20512c0 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -8,6 +8,8 @@ #include #include +#include + namespace DB { diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index c87a1b216ca..41c8f475f03 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index c545367b63d..26e953c0578 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index ff1459ca34b..13b5c167e0e 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include From c6f0a434c04023290b67403eeab7136631573798 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 19 Mar 2024 17:40:27 +0100 Subject: [PATCH 51/78] Remove some magic_enum from headers --- src/Common/IntervalKind.cpp | 7 +++ src/Common/IntervalKind.h | 3 +- .../MergeTree/MergeTreeDataPartType.cpp | 47 +++++++++++++++++++ .../MergeTree/MergeTreeDataPartType.h | 28 ++--------- 4 files changed, 59 insertions(+), 26 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeDataPartType.cpp diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index 7ba32a689f5..22c7db504c3 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -1,6 +1,8 @@ #include #include +#include + namespace DB { @@ -10,6 +12,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +std::string_view IntervalKind::toString() const +{ + return magic_enum::enum_name(kind); +} + Int64 IntervalKind::toAvgNanoseconds() const { static constexpr Int64 NANOSECONDS_PER_MICROSECOND = 1000; diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index d77a39cb9fa..497dc83be0b 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB { @@ -27,7 +26,7 @@ struct IntervalKind IntervalKind(Kind kind_ = Kind::Second) : kind(kind_) {} /// NOLINT operator Kind() const { return kind; } /// NOLINT - constexpr std::string_view toString() const { return magic_enum::enum_name(kind); } + std::string_view toString() const; /// Returns number of nanoseconds in one interval. /// For `Month`, `Quarter` and `Year` the function returns an average number of nanoseconds. diff --git a/src/Storages/MergeTree/MergeTreeDataPartType.cpp b/src/Storages/MergeTree/MergeTreeDataPartType.cpp new file mode 100644 index 00000000000..fb5555cb13f --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartType.cpp @@ -0,0 +1,47 @@ +#include +#include + +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +template +requires std::is_enum_v +static E parseEnum(const String & str) +{ + auto value = magic_enum::enum_cast(str); + if (!value || *value == E::Unknown) + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected string {} for enum {}", str, magic_enum::enum_type_name()); + + return *value; +} + +String MergeTreeDataPartType::toString() const +{ + return String(magic_enum::enum_name(value)); +} + +void MergeTreeDataPartType::fromString(const String & str) +{ + value = parseEnum(str); +} + +String MergeTreeDataPartStorageType::toString() const +{ + return String(magic_enum::enum_name(value)); +} + +void MergeTreeDataPartStorageType::fromString(const String & str) +{ + value = parseEnum(str); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeDataPartType.h b/src/Storages/MergeTree/MergeTreeDataPartType.h index 64f3abee391..8177809d41e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartType.h +++ b/src/Storages/MergeTree/MergeTreeDataPartType.h @@ -1,29 +1,9 @@ #pragma once -#include #include -#include namespace DB { - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -template -requires std::is_enum_v -static E parseEnum(const String & str) -{ - auto value = magic_enum::enum_cast(str); - if (!value || *value == E::Unknown) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected string {} for enum {}", str, magic_enum::enum_type_name()); - - return *value; -} - /// It's a bug in clang with three-way comparison operator /// https://github.com/llvm/llvm-project/issues/55919 #pragma clang diagnostic push @@ -51,8 +31,8 @@ public: auto operator<=>(const MergeTreeDataPartType &) const = default; Value getValue() const { return value; } - String toString() const { return String(magic_enum::enum_name(value)); } - void fromString(const String & str) { value = parseEnum(str); } + String toString() const; + void fromString(const String & str); private: Value value; @@ -74,8 +54,8 @@ public: auto operator<=>(const MergeTreeDataPartStorageType &) const = default; Value getValue() const { return value; } - String toString() const { return String(magic_enum::enum_name(value)); } - void fromString(const String & str) { value = parseEnum(str); } + String toString() const; + void fromString(const String & str); private: Value value; From f60b5f3cb7f1c32fd6f5d9837c4d27401dbe5da8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 19 Mar 2024 18:45:07 +0100 Subject: [PATCH 52/78] Add test for #60317 --- .../03014_analyzer_groupby_fuzz_60317.reference | 1 + .../0_stateless/03014_analyzer_groupby_fuzz_60317.sql | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.reference create mode 100644 tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql diff --git a/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.reference b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.reference new file mode 100644 index 00000000000..4972904f87d --- /dev/null +++ b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.reference @@ -0,0 +1 @@ +30 30 1970-01-01 diff --git a/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql new file mode 100644 index 00000000000..2f54058526e --- /dev/null +++ b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql @@ -0,0 +1,9 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/60317 +SELECT + toNullable(materialize(_CAST(30, 'LowCardinality(UInt8)'))) as a, + _CAST(30, 'LowCardinality(UInt8)') as b, + makeDate(materialize(_CAST(30, 'LowCardinality(UInt8)')), 10, _CAST(30, 'Nullable(UInt8)')) as c +FROM system.one +GROUP BY + _CAST(30, 'Nullable(UInt8)') +SETTINGS allow_experimental_analyzer = 1; From e954f684823c98adfe5bc99af8de4c7a615a60a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 19 Mar 2024 18:51:09 +0100 Subject: [PATCH 53/78] Add test for #60772 --- .../03015_analyzer_groupby_fuzz_60772.reference | 1 + .../0_stateless/03015_analyzer_groupby_fuzz_60772.sql | 10 ++++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.reference create mode 100644 tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql diff --git a/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.reference b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.reference new file mode 100644 index 00000000000..611407ecd90 --- /dev/null +++ b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.reference @@ -0,0 +1 @@ +%W 2018-01-02 22:33:44 1 diff --git a/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql new file mode 100644 index 00000000000..c8b4eef50ff --- /dev/null +++ b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql @@ -0,0 +1,10 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/60772 +SELECT toFixedString(toFixedString(toFixedString(toFixedString(toFixedString(toFixedString('%W', 2), 2), 2),toLowCardinality(toLowCardinality(toNullable(2)))), 2), 2), + toFixedString(toFixedString('2018-01-02 22:33:44', 19), 19), + hasSubsequence(toNullable(materialize(toLowCardinality('garbage'))), 'gr') +GROUP BY + '2018-01-02 22:33:44', + toFixedString(toFixedString('2018-01-02 22:33:44', 19), 19), + 'gr', + '2018-01-02 22:33:44' +SETTINGS allow_experimental_analyzer = 1; From d56838080011e65ac2363862e8d6157bf2b35d2b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 19 Mar 2024 19:03:35 +0100 Subject: [PATCH 54/78] Add test for #59796 --- .../0_stateless/03016_analyzer_groupby_fuzz_59796.reference | 1 + .../0_stateless/03016_analyzer_groupby_fuzz_59796.sql | 6 ++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.reference create mode 100644 tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.sql diff --git a/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.reference b/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.reference new file mode 100644 index 00000000000..6f7389b482b --- /dev/null +++ b/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.reference @@ -0,0 +1 @@ +\0\0\0 key="v" \0 key="v" key="v" key="v" key="v" \0 key="v" diff --git a/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.sql b/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.sql new file mode 100644 index 00000000000..c00a75a631c --- /dev/null +++ b/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.sql @@ -0,0 +1,6 @@ +SELECT + concat(concat(unhex('00'), concat(unhex('00'), concat(unhex(toFixedString('00', 2)), toFixedString(toFixedString(' key="v" ', 9), 9), concat(unhex('00'), toFixedString(' key="v" ', 9)), toFixedString(materialize(toLowCardinality(' key="v" ')), 9)), toFixedString(' key="v" ', 9)), toFixedString(' key="v" ', 9)), unhex('00'), ' key="v" ') AS haystack +GROUP BY + concat(unhex('00'), toFixedString(materialize(toFixedString(' key="v" ', 9)), 9), toFixedString(toFixedString('00', 2), toNullable(2)), toFixedString(toFixedString(toFixedString(' key="v" ', 9), 9), 9)), + concat(' key="v" ') +SETTINGS allow_experimental_analyzer = 1; From 73d601d231a8825643492b8895e6d1b43f04915f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 19 Mar 2024 19:11:32 +0100 Subject: [PATCH 55/78] Add test for #61600 --- .../03017_analyzer_groupby_fuzz_61600.reference | 1 + .../03017_analyzer_groupby_fuzz_61600.sql | 14 ++++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.reference create mode 100644 tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql diff --git a/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.reference b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.reference new file mode 100644 index 00000000000..68acb650f8e --- /dev/null +++ b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.reference @@ -0,0 +1 @@ +38 \N diff --git a/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql new file mode 100644 index 00000000000..44b27750c16 --- /dev/null +++ b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql @@ -0,0 +1,14 @@ +CREATE TABLE set_index_not__fuzz_0 (`name` String, `status` Enum8('alive' = 0, 'rip' = 1), INDEX idx_status status TYPE set(2) GRANULARITY 1) +ENGINE = MergeTree ORDER BY name +SETTINGS index_granularity = 8192; + +INSERT INTO set_index_not__fuzz_0 SELECT * from generateRandom() limit 1; + +SELECT + 38, + concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3),NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3))) +FROM set_index_not__fuzz_0 +GROUP BY + toNullable(3), + concat(concat(NULLIF(1, 1), toNullable(toNullable(3)))) +SETTINGS allow_experimental_analyzer = 1; From c63a5566dc8f54ed28f07ddf7097bbfeb7f8b0a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 20 Mar 2024 01:25:46 +0100 Subject: [PATCH 56/78] Add a test --- .../03014_invalid_utf8_client.expect | 27 +++++++++++++++++++ .../03014_invalid_utf8_client.reference | 0 2 files changed, 27 insertions(+) create mode 100755 tests/queries/0_stateless/03014_invalid_utf8_client.expect create mode 100644 tests/queries/0_stateless/03014_invalid_utf8_client.reference diff --git a/tests/queries/0_stateless/03014_invalid_utf8_client.expect b/tests/queries/0_stateless/03014_invalid_utf8_client.expect new file mode 100755 index 00000000000..13cbfe408d4 --- /dev/null +++ b/tests/queries/0_stateless/03014_invalid_utf8_client.expect @@ -0,0 +1,27 @@ +#!/usr/bin/expect -f + +set basedir [file dirname $argv0] +set basename [file tail $argv0] +exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history + +log_user 0 +set timeout 60 +match_max 100000 + +expect_after { + # Do not ignore eof from expect + -i $any_spawn_id eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + -i $any_spawn_id timeout { exit 1 } +} + +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" +expect ":) " + +send -- "SELECT \x99\r" + +expect "Syntax error" + +send "exit\r" +expect eof diff --git a/tests/queries/0_stateless/03014_invalid_utf8_client.reference b/tests/queries/0_stateless/03014_invalid_utf8_client.reference new file mode 100644 index 00000000000..e69de29bb2d From 96e90438e0a1f47fc706792492cf86fe093d0b26 Mon Sep 17 00:00:00 2001 From: nemonlou Date: Wed, 20 Mar 2024 14:25:16 +0800 Subject: [PATCH 57/78] fix clang-tiny --- src/Processors/Transforms/MergeJoinTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index b63598483ef..62361bef5e2 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -305,7 +305,7 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( left_to_right_key_remap[left_idx] = right_idx; } - auto smjPtr = typeid_cast(table_join.get()); + const auto *smjPtr = typeid_cast(table_join.get()); if (smjPtr) { null_direction_hint = smjPtr->getNullDirection(); From 2312eb886f93f126be326abd6af1f5ad433d0f29 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 20 Mar 2024 08:53:26 +0100 Subject: [PATCH 58/78] Fix part log for async insert with deduplication --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 14 +++++++++++ ...14_async_with_dedup_part_log_rmt.reference | 5 ++++ .../03014_async_with_dedup_part_log_rmt.sql | 24 +++++++++++++++++++ 3 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.reference create mode 100644 tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.sql diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index f67076c741b..b43d47bf5f4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -487,7 +487,14 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa partition.temp_part.finalize(); auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num, false).first; if (conflict_block_ids.empty()) + { + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); + PartLog::addNewPart( + storage.getContext(), + PartLog::PartLogEntry(partition.temp_part.part, partition.elapsed_ns, counters_snapshot), + ExecutionStatus(0)); break; + } storage.async_block_ids_cache.triggerCacheUpdate(); ++retry_times; @@ -495,7 +502,14 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa /// partition clean conflict partition.filterBlockDuplicate(conflict_block_ids, false); if (partition.block_with_partition.block.rows() == 0) + { + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); + PartLog::addNewPart( + storage.getContext(), + PartLog::PartLogEntry(partition.temp_part.part, partition.elapsed_ns, counters_snapshot), + ExecutionStatus(ErrorCodes::INSERT_WAS_DEDUPLICATED)); break; + } partition.block_with_partition.partition = std::move(partition.temp_part.part->partition.value); /// partition.temp_part is already finalized, no need to call cancel partition.temp_part = storage.writer.writeTempPart(partition.block_with_partition, metadata_snapshot, context); diff --git a/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.reference b/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.reference new file mode 100644 index 00000000000..335c796a9cf --- /dev/null +++ b/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.reference @@ -0,0 +1,5 @@ +-- Inserted part -- +0 1 +-- Deduplicated part -- +0 1 +389 1 diff --git a/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.sql b/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.sql new file mode 100644 index 00000000000..e14c1635853 --- /dev/null +++ b/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.sql @@ -0,0 +1,24 @@ +CREATE TABLE 03014_async_with_dedup_part_log (x UInt64) +ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}/03014_async_with_dedup_part_log', 'r1') ORDER BY tuple(); + +SET async_insert = 1; +SET wait_for_async_insert = 1; +SET async_insert_deduplicate = 1; + +SELECT '-- Inserted part --'; +INSERT INTO 03014_async_with_dedup_part_log VALUES (2); + +SYSTEM FLUSH LOGS; +SELECT error, count() FROM system.part_log +WHERE table = '03014_async_with_dedup_part_log' and database = currentDatabase() +GROUP BY error +ORDER BY error; + +SELECT '-- Deduplicated part --'; +INSERT INTO 03014_async_with_dedup_part_log VALUES (2); + +SYSTEM FLUSH LOGS; +SELECT error, count() FROM system.part_log +WHERE table = '03014_async_with_dedup_part_log' and database = currentDatabase() +GROUP BY error +ORDER BY error; From 2c0612c42522007b493f0838dcde7b2030a2d708 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Mar 2024 09:54:49 +0000 Subject: [PATCH 59/78] Empty-Commit From 25bae3e0a5f1b7cb157eb2bc0aa9984e2e6b5274 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Mar 2024 10:58:43 +0000 Subject: [PATCH 60/78] Verbose error medssage about analyzer_compatibility_join_using_top_level_identifier --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 30 ++++++++++++++++++++--- 1 file changed, 26 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 0414ea93f94..c7362cc4cce 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -7409,18 +7409,40 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS if (!result_left_table_expression) result_left_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node_typed.getLeftTableExpression(), scope); - /// Here we may try to resolve identifier from projection in case it's not resolved from left table expression - /// and analyzer_compatibility_join_using_top_level_identifier is disabled. - /// For now we do not do this, because not all corner cases are clear. + /** Here we may try to resolve identifier from projection in case it's not resolved from left table expression + * and analyzer_compatibility_join_using_top_level_identifier is disabled. + * For now we do not do this, because not all corner cases are clear. + * But let's at least mention it in error message + */ /// if (!settings.analyzer_compatibility_join_using_top_level_identifier && !result_left_table_expression) /// result_left_table_expression = try_resolve_identifier_from_query_projection(identifier_full_name, join_node_typed.getLeftTableExpression(), scope); if (!result_left_table_expression) + { + String extra_message; + const QueryNode * query_node = scope.scope_node ? scope.scope_node->as() : nullptr; + if (settings.analyzer_compatibility_join_using_top_level_identifier && query_node) + { + for (const auto & projection_node : query_node->getProjection().getNodes()) + { + if (projection_node->hasAlias() && identifier_full_name == projection_node->getAlias()) + { + extra_message = fmt::format( + " , but alias '{}' is present in SELECT list." + " You may try to SET analyzer_compatibility_join_using_top_level_identifier = 1, to allow to use it in USING clause", + projection_node->formatASTForErrorMessage()); + break; + } + } + } + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "JOIN {} using identifier '{}' cannot be resolved from left table expression. In scope {}", + "JOIN {} using identifier '{}' cannot be resolved from left table expression{}. In scope {}", join_node_typed.formatASTForErrorMessage(), identifier_full_name, + extra_message, scope.scope_node->formatASTForErrorMessage()); + } if (result_left_table_expression->getNodeType() != QueryTreeNodeType::COLUMN) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, From 5c1a54f643b9c72bf87ab03358a6244a7adf692b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 18 Mar 2024 15:22:50 +0100 Subject: [PATCH 61/78] Fix `clickhouse-client -s` argument --- programs/client/Client.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 4203e4738dd..9e90596b4f8 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -933,7 +933,7 @@ void Client::addOptions(OptionsDescription & options_description) ("config,c", po::value(), "config-file path (another shorthand)") ("connection", po::value(), "connection to use (from the client config), by default connection name is hostname") ("secure,s", "Use TLS connection") - ("no-secure,s", "Don't use TLS connection") + ("no-secure", "Don't use TLS connection") ("user,u", po::value()->default_value("default"), "user") ("password", po::value(), "password") ("ask-password", "ask-password") From 55f7595c43f7f694dca65fad2880ba92b6d5de41 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 18 Mar 2024 15:24:55 +0100 Subject: [PATCH 62/78] Test `-s` argument for clickhouse-client --- tests/queries/0_stateless/00505_secure.reference | 1 + tests/queries/0_stateless/00505_secure.sh | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/00505_secure.reference b/tests/queries/0_stateless/00505_secure.reference index 9a8656bf491..73bdbdbafbe 100644 --- a/tests/queries/0_stateless/00505_secure.reference +++ b/tests/queries/0_stateless/00505_secure.reference @@ -1,3 +1,4 @@ +1 2 3 4 diff --git a/tests/queries/0_stateless/00505_secure.sh b/tests/queries/0_stateless/00505_secure.sh index b7c12911b90..dcda2c0f2e8 100755 --- a/tests/queries/0_stateless/00505_secure.sh +++ b/tests/queries/0_stateless/00505_secure.sh @@ -7,6 +7,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# Use $CLICKHOUSE_CLIENT_SECURE, but replace `--secure` by `-s` to test it +CLICKHOUSE_CLIENT_S=${CLICKHOUSE_CLIENT_SECURE/ --secure / -s } +$CLICKHOUSE_CLIENT_S -q "SELECT 1;" + $CLICKHOUSE_CLIENT_SECURE -q "SELECT 2;" #disable test From 463b90e058c1c6f0b5cd692e2fd699f5b5a1b3f8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 18 Mar 2024 16:23:07 +0100 Subject: [PATCH 63/78] Add test for `--no-secure` --- tests/queries/0_stateless/00505_secure.reference | 1 + tests/queries/0_stateless/00505_secure.sh | 2 ++ 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/00505_secure.reference b/tests/queries/0_stateless/00505_secure.reference index 73bdbdbafbe..24319e2c136 100644 --- a/tests/queries/0_stateless/00505_secure.reference +++ b/tests/queries/0_stateless/00505_secure.reference @@ -1,3 +1,4 @@ +0 1 2 3 diff --git a/tests/queries/0_stateless/00505_secure.sh b/tests/queries/0_stateless/00505_secure.sh index dcda2c0f2e8..eaa50bce6b1 100755 --- a/tests/queries/0_stateless/00505_secure.sh +++ b/tests/queries/0_stateless/00505_secure.sh @@ -7,6 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +$CLICKHOUSE_CLIENT --no-secure -q "SELECT 0;" + # Use $CLICKHOUSE_CLIENT_SECURE, but replace `--secure` by `-s` to test it CLICKHOUSE_CLIENT_S=${CLICKHOUSE_CLIENT_SECURE/ --secure / -s } $CLICKHOUSE_CLIENT_S -q "SELECT 1;" From 007775c9826853ddcec24282edbd1e1496911b0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 20 Mar 2024 13:09:26 +0100 Subject: [PATCH 64/78] Better fuzzer logs --- docker/test/fuzzer/run-fuzzer.sh | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index c9e1681401d..55f4d379005 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -173,16 +173,15 @@ function fuzz mkdir -p /var/run/clickhouse-server - # NOTE: we use process substitution here to preserve keep $! as a pid of clickhouse-server - # server.log -> CH logs - # stderr.log -> Process logs (sanitizer) + # server.log -> All server logs, including sanitizer + # stderr.log -> Process logs (sanitizer) only clickhouse-server \ --config-file db/config.xml \ --pid-file /var/run/clickhouse-server/clickhouse-server.pid \ -- --path db \ --logger.console=0 \ - --logger.log=server.log > stderr.log 2>&1 & - server_pid=$! + --logger.log=server.log 2>&1 | tee -a stderr.log >> server.log 2>&1 & + server_pid=$(pidof clickhouse-server) kill -0 $server_pid @@ -310,7 +309,7 @@ quit if [ "$server_died" == 1 ] then # The server has died. - if ! rg --text -o 'Received signal.*|Logical error.*|Assertion.*failed|Failed assertion.*|.*runtime error: .*|.*is located.*|(SUMMARY|ERROR): [a-zA-Z]+Sanitizer:.*|.*_LIBCPP_ASSERT.*|.*Child process was terminated by signal 9.*' server.log stderr.log > description.txt + if ! rg --text -o 'Received signal.*|Logical error.*|Assertion.*failed|Failed assertion.*|.*runtime error: .*|.*is located.*|(SUMMARY|ERROR): [a-zA-Z]+Sanitizer:.*|.*_LIBCPP_ASSERT.*|.*Child process was terminated by signal 9.*' server.log > description.txt then echo "Lost connection to server. See the logs." > description.txt fi From 15339091626617a224ec8a152696b4c4f8881c3a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 20 Mar 2024 13:22:30 +0100 Subject: [PATCH 65/78] Fix flaky 02122_parallel_formatting_Template --- tests/queries/0_stateless/02122_parallel_formatting_Template.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02122_parallel_formatting_Template.sh b/tests/queries/0_stateless/02122_parallel_formatting_Template.sh index c7d2329ec14..86a4c017bbe 100755 --- a/tests/queries/0_stateless/02122_parallel_formatting_Template.sh +++ b/tests/queries/0_stateless/02122_parallel_formatting_Template.sh @@ -21,7 +21,7 @@ $CLICKHOUSE_CLIENT -q "select number as x, number + 1 as y, concat('string: ', t diff $non_parallel_file $parallel_file -echo -ne '{prefix} \n${data}\n $$ suffix $$\n${totals}\n${min}\n${max}\n${rows:Quoted}\n${rows_before_limit:Quoted}\n${rows_read:Quoted}\n${bytes_read:Quoted}\n' > $resultset_path +echo -ne '{prefix} \n${data}\n $$ suffix $$\n${totals}\n${min}\n${max}\n${rows:Quoted}\n${rows_before_limit:Quoted}\n' > $resultset_path echo "Template-2" $CLICKHOUSE_CLIENT -q "select number as x, number + 1 as y, concat('string: ', toString(number)) as s from numbers(200000) group by number with totals order by number limit 190000 format Template $TEMPLATE_SETTINGS" --output_format_parallel_formatting=0 --extremes=1 > $non_parallel_file From 2c02476f4aea90906fc117067d4adc627b142cfe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 20 Mar 2024 14:08:00 +0100 Subject: [PATCH 66/78] Change riscv RLIMIT_CPU --- CMakeLists.txt | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d345705ae53..f646c830daa 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -61,11 +61,16 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # set CPU time limit to 1000 seconds set (RLIMIT_CPU 1000) - # Sanitizers are too heavy. Also RISCV has some extra memory requirements - if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE OR ARCH_RISCV64) + # Sanitizers are too heavy + if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE) set (RLIMIT_DATA 10000000000) # 10G endif() + # For some files currently building RISCV64 might be too slow. TODO: Improve compilation times per file + if (ARCH_RISCV64) + set (RLIMIT_CPU 1800) + endif() + set (CMAKE_CXX_COMPILER_LAUNCHER prlimit --as=${RLIMIT_AS} --data=${RLIMIT_DATA} --cpu=${RLIMIT_CPU} ${CMAKE_CXX_COMPILER_LAUNCHER}) endif () From 5ed895dbf529415af2896abfda72f9a096461af9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 20 Mar 2024 14:12:33 +0100 Subject: [PATCH 67/78] remove whitespace before coma --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c7362cc4cce..6014d761794 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -7428,7 +7428,7 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS if (projection_node->hasAlias() && identifier_full_name == projection_node->getAlias()) { extra_message = fmt::format( - " , but alias '{}' is present in SELECT list." + ", but alias '{}' is present in SELECT list." " You may try to SET analyzer_compatibility_join_using_top_level_identifier = 1, to allow to use it in USING clause", projection_node->formatASTForErrorMessage()); break; From 9bf50ac814f59a1936dbe1b64a717718af1cfea8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 20 Mar 2024 14:50:26 +0100 Subject: [PATCH 68/78] Fix aggregator when data is empty --- src/Interpreters/Aggregator.cpp | 7 ++++++- .../03015_aggregator_empty_data_multiple_blocks.reference | 0 .../03015_aggregator_empty_data_multiple_blocks.sql | 3 +++ 3 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03015_aggregator_empty_data_multiple_blocks.reference create mode 100644 tests/queries/0_stateless/03015_aggregator_empty_data_multiple_blocks.sql diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 1c0d4b2d671..7f3b961a598 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1867,7 +1867,12 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are if (data.empty()) { auto && out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, rows); - return {finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows)}; + auto finalized_block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows); + + if (return_single_block) + return std::move(finalized_block); + + return BlocksList{std::move(finalized_block)}; } ConvertToBlockResVariant res; bool use_compiled_functions = false; diff --git a/tests/queries/0_stateless/03015_aggregator_empty_data_multiple_blocks.reference b/tests/queries/0_stateless/03015_aggregator_empty_data_multiple_blocks.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03015_aggregator_empty_data_multiple_blocks.sql b/tests/queries/0_stateless/03015_aggregator_empty_data_multiple_blocks.sql new file mode 100644 index 00000000000..d2c1816f7e8 --- /dev/null +++ b/tests/queries/0_stateless/03015_aggregator_empty_data_multiple_blocks.sql @@ -0,0 +1,3 @@ +CREATE TABLE 03015_aggregator_empty_data_multiple_blocks (c0 Int32) ENGINE = Memory(); +INSERT INTO 03015_aggregator_empty_data_multiple_blocks SELECT * FROM generateRandom() LIMIT 1000; +SELECT radians(t1.c0) FROM 03015_aggregator_empty_data_multiple_blocks AS t1 RIGHT ANTI JOIN 03015_aggregator_empty_data_multiple_blocks AS right_0 ON t1.c0=right_0.c0 GROUP BY t1.c0; From 981c9e0c12ef7f554f48aa7e8402eb7e15368296 Mon Sep 17 00:00:00 2001 From: Oxide Computer Company Date: Wed, 22 Nov 2023 11:47:30 +0000 Subject: [PATCH 69/78] Restore poco SUN files These were accidentally deleted as unused in https://github.com/ClickHouse/ClickHouse/commit/44afecf083b2cfa3d77d2e227f --- .../include/Poco/FPEnvironment_SUN.h | 75 ++++++++++ .../poco/Foundation/src/FPEnvironment_SUN.cpp | 139 ++++++++++++++++++ 2 files changed, 214 insertions(+) create mode 100644 base/poco/Foundation/include/Poco/FPEnvironment_SUN.h create mode 100644 base/poco/Foundation/src/FPEnvironment_SUN.cpp diff --git a/base/poco/Foundation/include/Poco/FPEnvironment_SUN.h b/base/poco/Foundation/include/Poco/FPEnvironment_SUN.h new file mode 100644 index 00000000000..7b31307e1ca --- /dev/null +++ b/base/poco/Foundation/include/Poco/FPEnvironment_SUN.h @@ -0,0 +1,75 @@ +// +// FPEnvironment_SUN.h +// +// Library: Foundation +// Package: Core +// Module: FPEnvironment +// +// Definitions of class FPEnvironmentImpl for Solaris. +// +// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. +// and Contributors. +// +// SPDX-License-Identifier: BSL-1.0 +// + + +#ifndef Foundation_FPEnvironment_SUN_INCLUDED +#define Foundation_FPEnvironment_SUN_INCLUDED + + +#include +#include "Poco/Foundation.h" + + +namespace Poco +{ + + +class FPEnvironmentImpl +{ +protected: + enum RoundingModeImpl + { + FP_ROUND_DOWNWARD_IMPL = FP_RM, + FP_ROUND_UPWARD_IMPL = FP_RP, + FP_ROUND_TONEAREST_IMPL = FP_RN, + FP_ROUND_TOWARDZERO_IMPL = FP_RZ + }; + enum FlagImpl + { + FP_DIVIDE_BY_ZERO_IMPL = FP_X_DZ, + FP_INEXACT_IMPL = FP_X_IMP, + FP_OVERFLOW_IMPL = FP_X_OFL, + FP_UNDERFLOW_IMPL = FP_X_UFL, + FP_INVALID_IMPL = FP_X_INV + }; + FPEnvironmentImpl(); + FPEnvironmentImpl(const FPEnvironmentImpl & env); + ~FPEnvironmentImpl(); + FPEnvironmentImpl & operator=(const FPEnvironmentImpl & env); + void keepCurrentImpl(); + static void clearFlagsImpl(); + static bool isFlagImpl(FlagImpl flag); + static void setRoundingModeImpl(RoundingModeImpl mode); + static RoundingModeImpl getRoundingModeImpl(); + static bool isInfiniteImpl(float value); + static bool isInfiniteImpl(double value); + static bool isInfiniteImpl(long double value); + static bool isNaNImpl(float value); + static bool isNaNImpl(double value); + static bool isNaNImpl(long double value); + static float copySignImpl(float target, float source); + static double copySignImpl(double target, double source); + static long double copySignImpl(long double target, long double source); + +private: + fp_rnd _rnd; + fp_except _exc; +}; + + +} // namespace Poco + + +#endif // Foundation_FPEnvironment_SUN_INCLUDED diff --git a/base/poco/Foundation/src/FPEnvironment_SUN.cpp b/base/poco/Foundation/src/FPEnvironment_SUN.cpp new file mode 100644 index 00000000000..36ee36431df --- /dev/null +++ b/base/poco/Foundation/src/FPEnvironment_SUN.cpp @@ -0,0 +1,139 @@ +// +// FPEnvironment_SUN.cpp +// +// Library: Foundation +// Package: Core +// Module: FPEnvironment +// +// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. +// and Contributors. +// +// SPDX-License-Identifier: BSL-1.0 +// + + +#include +#include "Poco/FPEnvironment_SUN.h" + + +namespace Poco { + + +FPEnvironmentImpl::FPEnvironmentImpl() +{ + _rnd = fpgetround(); + _exc = fpgetmask(); +} + + +FPEnvironmentImpl::FPEnvironmentImpl(const FPEnvironmentImpl& env) +{ + _rnd = env._rnd; + _exc = env._exc; +} + + +FPEnvironmentImpl::~FPEnvironmentImpl() +{ + fpsetround(_rnd); + fpsetmask(_exc); +} + + +FPEnvironmentImpl& FPEnvironmentImpl::operator = (const FPEnvironmentImpl& env) +{ + _rnd = env._rnd; + _exc = env._exc; + return *this; +} + + +bool FPEnvironmentImpl::isInfiniteImpl(float value) +{ + int cls = fpclass(value); + return cls == FP_PINF || cls == FP_NINF; +} + + +bool FPEnvironmentImpl::isInfiniteImpl(double value) +{ + int cls = fpclass(value); + return cls == FP_PINF || cls == FP_NINF; +} + + +bool FPEnvironmentImpl::isInfiniteImpl(long double value) +{ + int cls = fpclass(value); + return cls == FP_PINF || cls == FP_NINF; +} + + +bool FPEnvironmentImpl::isNaNImpl(float value) +{ + return isnanf(value) != 0; +} + + +bool FPEnvironmentImpl::isNaNImpl(double value) +{ + return isnan(value) != 0; +} + + +bool FPEnvironmentImpl::isNaNImpl(long double value) +{ + return isnan((double) value) != 0; +} + + +float FPEnvironmentImpl::copySignImpl(float target, float source) +{ + return (float) copysign(target, source); +} + + +double FPEnvironmentImpl::copySignImpl(double target, double source) +{ + return (float) copysign(target, source); +} + + +long double FPEnvironmentImpl::copySignImpl(long double target, long double source) +{ + return (source > 0 && target > 0) || (source < 0 && target < 0) ? target : -target; +} + + +void FPEnvironmentImpl::keepCurrentImpl() +{ + fpsetround(_rnd); + fpsetmask(_exc); +} + + +void FPEnvironmentImpl::clearFlagsImpl() +{ + fpsetsticky(0); +} + + +bool FPEnvironmentImpl::isFlagImpl(FlagImpl flag) +{ + return (fpgetsticky() & flag) != 0; +} + + +void FPEnvironmentImpl::setRoundingModeImpl(RoundingModeImpl mode) +{ + fpsetround((fp_rnd) mode); +} + + +FPEnvironmentImpl::RoundingModeImpl FPEnvironmentImpl::getRoundingModeImpl() +{ + return (FPEnvironmentImpl::RoundingModeImpl) fpgetround(); +} + + +} // namespace Poco From da28eaf454e6fca09dcc6c9b7f616f716ce49e02 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 19 Mar 2024 13:08:38 +0000 Subject: [PATCH 70/78] CI: remove compose files from integration test docker #ci_set_integration --- docker/test/integration/runner/Dockerfile | 1 - tests/integration/README.md | 1 - .../compose/docker_compose_azurite.yml | 0 .../compose/docker_compose_cassandra.yml | 0 .../compose/docker_compose_clickhouse.yml | 0 .../compose/docker_compose_coredns.yml | 0 .../compose/docker_compose_dotnet_client.yml | 0 .../integration}/compose/docker_compose_hdfs.yml | 0 .../integration}/compose/docker_compose_hive.yml | 0 .../compose/docker_compose_jdbc_bridge.yml | 0 .../compose/docker_compose_kafka.yml | 0 .../compose/docker_compose_keeper.yml | 0 .../compose/docker_compose_kerberized_hdfs.yml | 0 .../compose/docker_compose_kerberized_kafka.yml | 0 .../compose/docker_compose_kerberos_kdc.yml | 0 .../integration}/compose/docker_compose_ldap.yml | 0 .../compose/docker_compose_minio.yml | 0 .../compose/docker_compose_mongo.yml | 0 .../compose/docker_compose_mongo_secure.yml | 0 .../compose/docker_compose_mysql.yml | 0 .../compose/docker_compose_mysql_8_0.yml | 0 .../compose/docker_compose_mysql_client.yml | 0 .../compose/docker_compose_mysql_cluster.yml | 0 .../docker_compose_mysql_golang_client.yml | 0 .../compose/docker_compose_mysql_java_client.yml | 0 .../compose/docker_compose_mysql_js_client.yml | 0 .../compose/docker_compose_mysql_php_client.yml | 0 .../integration}/compose/docker_compose_nats.yml | 0 .../integration}/compose/docker_compose_net.yml | 0 .../compose/docker_compose_nginx.yml | 0 .../compose/docker_compose_postgres.yml | 0 .../compose/docker_compose_postgres_cluster.yml | 0 .../compose/docker_compose_postgresql.yml | 0 .../docker_compose_postgresql_java_client.yml | 0 .../compose/docker_compose_rabbitmq.yml | 0 .../compose/docker_compose_redis.yml | 0 .../compose/docker_compose_zookeeper.yml | 0 .../compose/docker_compose_zookeeper_secure.yml | 0 tests/integration/helpers/cluster.py | 16 ++-------------- tests/integration/runner | 2 +- 40 files changed, 3 insertions(+), 17 deletions(-) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_azurite.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_cassandra.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_clickhouse.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_coredns.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_dotnet_client.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_hdfs.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_hive.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_jdbc_bridge.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_kafka.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_keeper.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_kerberized_hdfs.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_kerberized_kafka.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_kerberos_kdc.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_ldap.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_minio.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mongo.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mongo_secure.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mysql.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mysql_8_0.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mysql_client.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mysql_cluster.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mysql_golang_client.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mysql_java_client.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mysql_js_client.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_mysql_php_client.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_nats.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_net.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_nginx.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_postgres.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_postgres_cluster.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_postgresql.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_postgresql_java_client.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_rabbitmq.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_redis.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_zookeeper.yml (100%) rename {docker/test/integration/runner => tests/integration}/compose/docker_compose_zookeeper_secure.yml (100%) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index b876f7b9635..8297a7100d1 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -126,7 +126,6 @@ RUN set -x \ COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ -COPY compose/ /compose/ COPY misc/ /misc/ diff --git a/tests/integration/README.md b/tests/integration/README.md index ac01c43769e..cde4cb05aec 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -66,7 +66,6 @@ set the following environment variables: Please note that if you use separate build (`ENABLE_CLICKHOUSE_ALL=OFF`), you need to build different components, including but not limited to `ENABLE_CLICKHOUSE_LIBRARY_BRIDGE=ON ENABLE_CLICKHOUSE_ODBC_BRIDGE=ON ENABLE_CLICKHOUSE_KEEPER=ON`. So it is easier to use `ENABLE_CLICKHOUSE_ALL=ON` -For tests that use common docker compose files you may need to set up their path with environment variable: `DOCKER_COMPOSE_DIR=$HOME/ClickHouse/docker/test/integration/runner/compose` ### Running with runner script diff --git a/docker/test/integration/runner/compose/docker_compose_azurite.yml b/tests/integration/compose/docker_compose_azurite.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_azurite.yml rename to tests/integration/compose/docker_compose_azurite.yml diff --git a/docker/test/integration/runner/compose/docker_compose_cassandra.yml b/tests/integration/compose/docker_compose_cassandra.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_cassandra.yml rename to tests/integration/compose/docker_compose_cassandra.yml diff --git a/docker/test/integration/runner/compose/docker_compose_clickhouse.yml b/tests/integration/compose/docker_compose_clickhouse.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_clickhouse.yml rename to tests/integration/compose/docker_compose_clickhouse.yml diff --git a/docker/test/integration/runner/compose/docker_compose_coredns.yml b/tests/integration/compose/docker_compose_coredns.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_coredns.yml rename to tests/integration/compose/docker_compose_coredns.yml diff --git a/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml b/tests/integration/compose/docker_compose_dotnet_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_dotnet_client.yml rename to tests/integration/compose/docker_compose_dotnet_client.yml diff --git a/docker/test/integration/runner/compose/docker_compose_hdfs.yml b/tests/integration/compose/docker_compose_hdfs.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_hdfs.yml rename to tests/integration/compose/docker_compose_hdfs.yml diff --git a/docker/test/integration/runner/compose/docker_compose_hive.yml b/tests/integration/compose/docker_compose_hive.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_hive.yml rename to tests/integration/compose/docker_compose_hive.yml diff --git a/docker/test/integration/runner/compose/docker_compose_jdbc_bridge.yml b/tests/integration/compose/docker_compose_jdbc_bridge.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_jdbc_bridge.yml rename to tests/integration/compose/docker_compose_jdbc_bridge.yml diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/tests/integration/compose/docker_compose_kafka.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_kafka.yml rename to tests/integration/compose/docker_compose_kafka.yml diff --git a/docker/test/integration/runner/compose/docker_compose_keeper.yml b/tests/integration/compose/docker_compose_keeper.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_keeper.yml rename to tests/integration/compose/docker_compose_keeper.yml diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml b/tests/integration/compose/docker_compose_kerberized_hdfs.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml rename to tests/integration/compose/docker_compose_kerberized_hdfs.yml diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/tests/integration/compose/docker_compose_kerberized_kafka.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml rename to tests/integration/compose/docker_compose_kerberized_kafka.yml diff --git a/docker/test/integration/runner/compose/docker_compose_kerberos_kdc.yml b/tests/integration/compose/docker_compose_kerberos_kdc.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_kerberos_kdc.yml rename to tests/integration/compose/docker_compose_kerberos_kdc.yml diff --git a/docker/test/integration/runner/compose/docker_compose_ldap.yml b/tests/integration/compose/docker_compose_ldap.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_ldap.yml rename to tests/integration/compose/docker_compose_ldap.yml diff --git a/docker/test/integration/runner/compose/docker_compose_minio.yml b/tests/integration/compose/docker_compose_minio.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_minio.yml rename to tests/integration/compose/docker_compose_minio.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mongo.yml b/tests/integration/compose/docker_compose_mongo.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mongo.yml rename to tests/integration/compose/docker_compose_mongo.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml b/tests/integration/compose/docker_compose_mongo_secure.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mongo_secure.yml rename to tests/integration/compose/docker_compose_mongo_secure.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/tests/integration/compose/docker_compose_mysql.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mysql.yml rename to tests/integration/compose/docker_compose_mysql.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml b/tests/integration/compose/docker_compose_mysql_8_0.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml rename to tests/integration/compose/docker_compose_mysql_8_0.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_client.yml b/tests/integration/compose/docker_compose_mysql_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mysql_client.yml rename to tests/integration/compose/docker_compose_mysql_client.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml b/tests/integration/compose/docker_compose_mysql_cluster.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml rename to tests/integration/compose/docker_compose_mysql_cluster.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml b/tests/integration/compose/docker_compose_mysql_golang_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml rename to tests/integration/compose/docker_compose_mysql_golang_client.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml b/tests/integration/compose/docker_compose_mysql_java_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml rename to tests/integration/compose/docker_compose_mysql_java_client.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml b/tests/integration/compose/docker_compose_mysql_js_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml rename to tests/integration/compose/docker_compose_mysql_js_client.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml b/tests/integration/compose/docker_compose_mysql_php_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml rename to tests/integration/compose/docker_compose_mysql_php_client.yml diff --git a/docker/test/integration/runner/compose/docker_compose_nats.yml b/tests/integration/compose/docker_compose_nats.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_nats.yml rename to tests/integration/compose/docker_compose_nats.yml diff --git a/docker/test/integration/runner/compose/docker_compose_net.yml b/tests/integration/compose/docker_compose_net.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_net.yml rename to tests/integration/compose/docker_compose_net.yml diff --git a/docker/test/integration/runner/compose/docker_compose_nginx.yml b/tests/integration/compose/docker_compose_nginx.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_nginx.yml rename to tests/integration/compose/docker_compose_nginx.yml diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/tests/integration/compose/docker_compose_postgres.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgres.yml rename to tests/integration/compose/docker_compose_postgres.yml diff --git a/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml b/tests/integration/compose/docker_compose_postgres_cluster.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml rename to tests/integration/compose/docker_compose_postgres_cluster.yml diff --git a/docker/test/integration/runner/compose/docker_compose_postgresql.yml b/tests/integration/compose/docker_compose_postgresql.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgresql.yml rename to tests/integration/compose/docker_compose_postgresql.yml diff --git a/docker/test/integration/runner/compose/docker_compose_postgresql_java_client.yml b/tests/integration/compose/docker_compose_postgresql_java_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgresql_java_client.yml rename to tests/integration/compose/docker_compose_postgresql_java_client.yml diff --git a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml b/tests/integration/compose/docker_compose_rabbitmq.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_rabbitmq.yml rename to tests/integration/compose/docker_compose_rabbitmq.yml diff --git a/docker/test/integration/runner/compose/docker_compose_redis.yml b/tests/integration/compose/docker_compose_redis.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_redis.yml rename to tests/integration/compose/docker_compose_redis.yml diff --git a/docker/test/integration/runner/compose/docker_compose_zookeeper.yml b/tests/integration/compose/docker_compose_zookeeper.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_zookeeper.yml rename to tests/integration/compose/docker_compose_zookeeper.yml diff --git a/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml b/tests/integration/compose/docker_compose_zookeeper_secure.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml rename to tests/integration/compose/docker_compose_zookeeper_secure.yml diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index b695b493db7..152b0d066ee 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -55,9 +55,7 @@ from .config_cluster import * HELPERS_DIR = p.dirname(__file__) CLICKHOUSE_ROOT_DIR = p.join(p.dirname(__file__), "../../..") -LOCAL_DOCKER_COMPOSE_DIR = p.join( - CLICKHOUSE_ROOT_DIR, "docker/test/integration/runner/compose/" -) +LOCAL_DOCKER_COMPOSE_DIR = p.join(CLICKHOUSE_ROOT_DIR, "tests/integration/compose/") DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" @@ -186,17 +184,7 @@ def get_library_bridge_path(): def get_docker_compose_path(): - compose_path = os.environ.get("DOCKER_COMPOSE_DIR") - if compose_path is not None: - return os.path.dirname(compose_path) - else: - if os.path.exists(os.path.dirname("/compose/")): - return os.path.dirname("/compose/") # default in docker runner container - else: - logging.debug( - f"Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {LOCAL_DOCKER_COMPOSE_DIR}" - ) - return LOCAL_DOCKER_COMPOSE_DIR + return LOCAL_DOCKER_COMPOSE_DIR def check_kafka_is_available(kafka_id, kafka_port): diff --git a/tests/integration/runner b/tests/integration/runner index f1d5198f545..2834eaf311b 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -415,7 +415,7 @@ if __name__ == "__main__": cmd = cmd_base + " " + args.command cmd_pre_pull = ( - f"{cmd_base} find /compose -name docker_compose_*.yml " + f"{cmd_base} find /ClickHouse/tests/integration/compose -name docker_compose_*.yml " r"-exec docker-compose -f '{}' pull \;" ) From a343406da47f62f015a5b70590fae479a3894625 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 20 Mar 2024 15:33:45 +0100 Subject: [PATCH 71/78] Another fix for SumIfToCountIfPass --- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 10 ++-- ...3010_sum_to_to_count_if_nullable.reference | 46 +++++++++++++++++++ .../03010_sum_to_to_count_if_nullable.sql | 5 +- 3 files changed, 56 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index 1c2097e7be9..1a4712aa697 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -97,13 +97,15 @@ public: if (!if_true_condition_constant_node || !if_false_condition_constant_node) return; + if (auto constant_type = if_true_condition_constant_node->getResultType(); !isNativeInteger(constant_type)) + return; + + if (auto constant_type = if_false_condition_constant_node->getResultType(); !isNativeInteger(constant_type)) + return; + const auto & if_true_condition_constant_value_literal = if_true_condition_constant_node->getValue(); const auto & if_false_condition_constant_value_literal = if_false_condition_constant_node->getValue(); - if (!isInt64OrUInt64FieldType(if_true_condition_constant_value_literal.getType()) || - !isInt64OrUInt64FieldType(if_false_condition_constant_value_literal.getType())) - return; - auto if_true_condition_value = if_true_condition_constant_value_literal.get(); auto if_false_condition_value = if_false_condition_constant_value_literal.get(); diff --git a/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.reference b/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.reference index 89e5f639c66..d20634acf93 100644 --- a/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.reference +++ b/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.reference @@ -1,5 +1,6 @@ (5,NULL) (5,NULL) +((6150),3) (5,NULL) QUERY id: 0 PROJECTION COLUMNS @@ -66,3 +67,48 @@ QUERY id: 0 ARGUMENTS LIST id: 19, nodes: 1 CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt8 +((6150),3) +QUERY id: 0 + PROJECTION COLUMNS + ((sum(if(equals(modulo(number, 2), 0), toNullable(0), 123))), toUInt8(3)) Tuple(Tuple(Nullable(UInt64)), UInt8) + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: tuple, function_type: ordinary, result_type: Tuple(Tuple(Nullable(UInt64)), UInt8) + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: tuple, function_type: ordinary, result_type: Tuple(Nullable(UInt64)) + ARGUMENTS + LIST id: 5, nodes: 1 + FUNCTION id: 6, function_name: sum, function_type: aggregate, nulls_action : IGNORE_NULLS, result_type: Nullable(UInt64) + ARGUMENTS + LIST id: 7, nodes: 1 + FUNCTION id: 8, function_name: if, function_type: ordinary, result_type: Nullable(UInt8) + ARGUMENTS + LIST id: 9, nodes: 3 + FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: modulo, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: number, result_type: UInt64, source_id: 15 + CONSTANT id: 16, constant_value: UInt64_2, constant_value_type: UInt8 + CONSTANT id: 17, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 18, constant_value: UInt64_0, constant_value_type: Nullable(UInt8) + EXPRESSION + FUNCTION id: 19, function_name: toNullable, function_type: ordinary, result_type: Nullable(UInt8) + ARGUMENTS + LIST id: 20, nodes: 1 + CONSTANT id: 21, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 22, constant_value: UInt64_123, constant_value_type: UInt8 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 + EXPRESSION + FUNCTION id: 24, function_name: toUInt8, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 25, nodes: 1 + CONSTANT id: 26, constant_value: UInt64_3, constant_value_type: UInt8 + JOIN TREE + TABLE_FUNCTION id: 15, alias: __table1, table_function_name: numbers + ARGUMENTS + LIST id: 27, nodes: 1 + CONSTANT id: 28, constant_value: UInt64_100, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.sql b/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.sql index b283a69a020..71a175faac8 100644 --- a/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.sql +++ b/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.sql @@ -3,9 +3,12 @@ SET optimize_rewrite_sum_if_to_count_if = 1; SET allow_experimental_analyzer = 0; SELECT (sumIf(toNullable(1), (number % 2) = 0), NULL) FROM numbers(10); SELECT (sum(if((number % 2) = 0, toNullable(1), 0)), NULL) FROM numbers(10); +SELECT (tuple(sum(if((number % 2) = 0, toNullable(0), 123)) IGNORE NULLS), toUInt8(3)) FROM numbers(100); SET allow_experimental_analyzer = 1; SELECT (sumIf(toNullable(1), (number % 2) = 0), NULL) FROM numbers(10); EXPLAIN QUERY TREE SELECT (sumIf(toNullable(1), (number % 2) = 0), NULL) FROM numbers(10); SELECT (sum(if((number % 2) = 0, toNullable(1), 0)), NULL) FROM numbers(10); -EXPLAIN QUERY TREE SELECT (sum(if((number % 2) = 0, toNullable(1), 0)), NULL) FROM numbers(10); \ No newline at end of file +EXPLAIN QUERY TREE SELECT (sum(if((number % 2) = 0, toNullable(1), 0)), NULL) FROM numbers(10); +SELECT (tuple(sum(if((number % 2) = 0, toNullable(0), 123)) IGNORE NULLS), toUInt8(3)) FROM numbers(100); +EXPLAIN QUERY TREE SELECT (tuple(sum(if((number % 2) = 0, toNullable(0), 123)) IGNORE NULLS), toUInt8(3)) FROM numbers(100); From e6256fa380721a2d60d29658ae0f8977db1a639e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 20 Mar 2024 15:43:09 +0100 Subject: [PATCH 72/78] Fix build --- src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h index 7807f3d8c25..ccc88079daa 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h @@ -15,6 +15,7 @@ struct MergeTreeDataPartChecksums; struct MergeTreeIndexGranularityInfo; class ISerialization; using SerializationPtr = std::shared_ptr; +class SerializationInfoByName; /** * A class which contains all information about a data part that is required From aa1a210d5d793b16d162d0f49fc92f12de0d7e8c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Mar 2024 17:07:32 +0100 Subject: [PATCH 73/78] Fix logging of autoscaling lambda, add test for effective_capacity --- tests/ci/autoscale_runners_lambda/app.py | 26 ++++++++--------- .../test_autoscale.py | 29 ++++++++++++++++++- 2 files changed, 41 insertions(+), 14 deletions(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 6c3d71708e9..520abcb41bb 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -120,13 +120,13 @@ def set_capacity( # Let's calculate a new desired capacity # (capacity_deficit + scale_up - 1) // scale_up : will increase min by 1 # if there is any capacity_deficit - desired_capacity = ( + new_capacity = ( asg["DesiredCapacity"] + (capacity_deficit + scale_up - 1) // scale_up ) - desired_capacity = max(desired_capacity, asg["MinSize"]) - desired_capacity = min(desired_capacity, asg["MaxSize"]) + new_capacity = max(new_capacity, asg["MinSize"]) + new_capacity = min(new_capacity, asg["MaxSize"]) # Finally, should the capacity be even changed - stop = stop or asg["DesiredCapacity"] == desired_capacity + stop = stop or asg["DesiredCapacity"] == new_capacity if stop: logging.info( "Do not increase ASG %s capacity, current capacity=%s, effective " @@ -144,9 +144,9 @@ def set_capacity( "The ASG %s capacity will be increased to %s, current capacity=%s, " "effective capacity=%s, maximum capacity=%s, running jobs=%s, queue size=%s", asg["AutoScalingGroupName"], - desired_capacity, - effective_capacity, + new_capacity, asg["DesiredCapacity"], + effective_capacity, asg["MaxSize"], running, queued, @@ -154,16 +154,16 @@ def set_capacity( if not dry_run: client.set_desired_capacity( AutoScalingGroupName=asg["AutoScalingGroupName"], - DesiredCapacity=desired_capacity, + DesiredCapacity=new_capacity, ) return # Now we will calculate if we need to scale down stop = stop or asg["DesiredCapacity"] == asg["MinSize"] - desired_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) - desired_capacity = max(desired_capacity, asg["MinSize"]) - desired_capacity = min(desired_capacity, asg["MaxSize"]) - stop = stop or asg["DesiredCapacity"] == desired_capacity + new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) + new_capacity = max(new_capacity, asg["MinSize"]) + new_capacity = min(new_capacity, asg["MaxSize"]) + stop = stop or asg["DesiredCapacity"] == new_capacity if stop: logging.info( "Do not decrease ASG %s capacity, current capacity=%s, effective " @@ -181,7 +181,7 @@ def set_capacity( "The ASG %s capacity will be decreased to %s, current capacity=%s, effective " "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", asg["AutoScalingGroupName"], - desired_capacity, + new_capacity, asg["DesiredCapacity"], effective_capacity, asg["MinSize"], @@ -191,7 +191,7 @@ def set_capacity( if not dry_run: client.set_desired_capacity( AutoScalingGroupName=asg["AutoScalingGroupName"], - DesiredCapacity=desired_capacity, + DesiredCapacity=new_capacity, ) diff --git a/tests/ci/autoscale_runners_lambda/test_autoscale.py b/tests/ci/autoscale_runners_lambda/test_autoscale.py index 75f178ac394..3aeab49ffc7 100644 --- a/tests/ci/autoscale_runners_lambda/test_autoscale.py +++ b/tests/ci/autoscale_runners_lambda/test_autoscale.py @@ -35,7 +35,7 @@ class TestSetCapacity(unittest.TestCase): @property def expected_capacity(self) -> int: - """one-time property""" + """a one-time property""" capacity, self._expected_capacity = self._expected_capacity, -1 return capacity @@ -117,6 +117,33 @@ class TestSetCapacity(unittest.TestCase): set_capacity(t.name, t.queues, self.client, False) self.assertEqual(t.expected_capacity, self.client.expected_capacity, t.name) + def test_effective_capacity(self): + """Normal cases test increasing w/o considering + effective_capacity much lower than DesiredCapacity""" + test_cases = ( + TestCase( + "desired-overwritten", + 1, + 20, # DesiredCapacity, overwritten by effective_capacity + 50, + [ + Queue("in_progress", 30, "desired-overwritten"), + Queue("queued", 60, "desired-overwritten"), + ], + 40, + ), + ) + for t in test_cases: + self.client.data_helper(t.name, t.min_size, t.desired_capacity, t.max_size) + # we test that effective_capacity is 30 (a half of 60) + data_with_instances = self.client.expected_data + data_with_instances["AutoScalingGroups"][0]["Instances"] = [ + {"HealthStatus": "Healthy" if i % 2 else "Unhealthy"} for i in range(60) + ] + self.client.expected_data = data_with_instances + set_capacity(t.name, t.queues, self.client, False) + self.assertEqual(t.expected_capacity, self.client.expected_capacity, t.name) + def test_exceptions(self): test_cases = ( ( From 54fae360f0a43ba644e667cefbcc28a72e0bc4f9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Mar 2024 17:50:40 +0100 Subject: [PATCH 74/78] Disable duplicate-inclides style check --- utils/check-style/check_cpp.sh | 9 +++++---- utils/check-style/process_style_check_result.py | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/utils/check-style/check_cpp.sh b/utils/check-style/check_cpp.sh index dc1dae70e22..d647f5eefe1 100755 --- a/utils/check-style/check_cpp.sh +++ b/utils/check-style/check_cpp.sh @@ -6,10 +6,11 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_ou start_total=$(date +%s) -start=$(date +%s) -./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt -runtime=$(($(date +%s)-start)) -echo "Check duplicates. Done. $runtime seconds." +# We decided to have the regexp-based check disabled in favor of clang-tidy +# start=$(date +%s) +# ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt +# runtime=$(($(date +%s)-start)) +# echo "Check duplicates. Done. $runtime seconds." start=$(date +%s) ./check-style -n |& tee /test_output/style_output.txt diff --git a/utils/check-style/process_style_check_result.py b/utils/check-style/process_style_check_result.py index f1ca53eba3b..e603084732d 100755 --- a/utils/check-style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -13,7 +13,7 @@ def process_result(result_folder): description = "" test_results = [] checks = ( - "duplicate includes", + # "duplicate includes", # disabled in favor of clang-tidy "shellcheck", "style", "pylint", From 952b6b85143b89cc507b6f08a46d662a7d9d87a3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Mar 2024 18:00:02 +0100 Subject: [PATCH 75/78] Use the same features for black check --- utils/check-style/check_py.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 13f4e754ed3..2e645d2f19a 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -4,9 +4,11 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_ou start_total=`date +%s` -# FIXME: 1 min to wait + head checkout -echo "Check python formatting with black" | ts +start=`date +%s` +echo "Check " | ts ./check-black -n |& tee /test_output/black_output.txt +runtime=$((`date +%s`-start)) +echo "Check python formatting with black. Done. $runtime seconds." start=`date +%s` ./check-pylint -n |& tee /test_output/pylint_output.txt From 35cc3355971a20d86561aec0a40da00e8cdb4001 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Mar 2024 18:04:35 +0100 Subject: [PATCH 76/78] Revert "Un-flake `test_undrop_query`" --- .../settings.md | 4 +- docs/en/sql-reference/statements/undrop.md | 52 ++++++++++++------- src/Interpreters/DatabaseCatalog.cpp | 4 +- src/Interpreters/InterpreterUndropQuery.cpp | 6 +-- tests/integration/test_undrop_query/test.py | 25 +++------ .../0_stateless/02681_undrop_query.sql | 2 +- 6 files changed, 48 insertions(+), 45 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index f20dcb9025e..07c9a2b88ab 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -933,9 +933,9 @@ Hard limit is configured via system tools ## database_atomic_delay_before_drop_table_sec {#database_atomic_delay_before_drop_table_sec} -The delay before a table data is dropped in seconds. If the `DROP TABLE` query has a `SYNC` modifier, this setting is ignored. +Sets the delay before remove table data in seconds. If the query has `SYNC` modifier, this setting is ignored. -Default value: `480` (8 minutes). +Default value: `480` (8 minute). ## database_catalog_unused_dir_hide_timeout_sec {#database_catalog_unused_dir_hide_timeout_sec} diff --git a/docs/en/sql-reference/statements/undrop.md b/docs/en/sql-reference/statements/undrop.md index 4b138bfe679..40ac1ab4f99 100644 --- a/docs/en/sql-reference/statements/undrop.md +++ b/docs/en/sql-reference/statements/undrop.md @@ -13,6 +13,13 @@ a system table called `system.dropped_tables`. If you have a materialized view without a `TO` clause associated with the dropped table, then you will also have to UNDROP the inner table of that view. +:::note +UNDROP TABLE is experimental. To use it add this setting: +```sql +set allow_experimental_undrop_table_query = 1; +``` +::: + :::tip Also see [DROP TABLE](/docs/en/sql-reference/statements/drop.md) ::: @@ -25,53 +32,60 @@ UNDROP TABLE [db.]name [UUID ''] [ON CLUSTER cluster] **Example** +``` sql +set allow_experimental_undrop_table_query = 1; +``` + ```sql -CREATE TABLE tab +CREATE TABLE undropMe ( `id` UInt8 ) ENGINE = MergeTree -ORDER BY id; - -DROP TABLE tab; - -SELECT * -FROM system.dropped_tables -FORMAT Vertical; +ORDER BY id ``` +```sql +DROP TABLE undropMe +``` +```sql +SELECT * +FROM system.dropped_tables +FORMAT Vertical +``` ```response Row 1: ────── index: 0 database: default -table: tab +table: undropMe uuid: aa696a1a-1d70-4e60-a841-4c80827706cc engine: MergeTree -metadata_dropped_path: /var/lib/clickhouse/metadata_dropped/default.tab.aa696a1a-1d70-4e60-a841-4c80827706cc.sql +metadata_dropped_path: /var/lib/clickhouse/metadata_dropped/default.undropMe.aa696a1a-1d70-4e60-a841-4c80827706cc.sql table_dropped_time: 2023-04-05 14:12:12 1 row in set. Elapsed: 0.001 sec. ``` - ```sql -UNDROP TABLE tab; - +UNDROP TABLE undropMe +``` +```response +Ok. +``` +```sql SELECT * FROM system.dropped_tables -FORMAT Vertical; - +FORMAT Vertical +``` ```response Ok. 0 rows in set. Elapsed: 0.001 sec. ``` - ```sql -DESCRIBE TABLE tab -FORMAT Vertical; +DESCRIBE TABLE undropMe +FORMAT Vertical ``` - ```response Row 1: ────── diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c2f2003aabd..a5a523b658b 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1142,7 +1142,7 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) TableMarkedAsDropped dropped_table; { std::lock_guard lock(tables_marked_dropped_mutex); - auto latest_drop_time = std::numeric_limits::min(); + time_t latest_drop_time = std::numeric_limits::min(); auto it_dropped_table = tables_marked_dropped.end(); for (auto it = tables_marked_dropped.begin(); it != tables_marked_dropped.end(); ++it) { @@ -1167,7 +1167,7 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) } if (it_dropped_table == tables_marked_dropped.end()) throw Exception(ErrorCodes::UNKNOWN_TABLE, - "Table {} is being dropped, has been dropped, or the database engine does not support UNDROP", + "The drop task of table {} is in progress, has been dropped or the database engine doesn't support it", table_id.getNameForLogs()); latest_metadata_dropped_path = it_dropped_table->metadata_path; String table_metadata_path = getPathForMetadata(it_dropped_table->table_id); diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp index f628a656947..8401c47df6b 100644 --- a/src/Interpreters/InterpreterUndropQuery.cpp +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -17,16 +17,14 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } -InterpreterUndropQuery::InterpreterUndropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) - : WithMutableContext(context_) - , query_ptr(query_ptr_) +InterpreterUndropQuery::InterpreterUndropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) { } + BlockIO InterpreterUndropQuery::execute() { getContext()->checkAccess(AccessType::UNDROP_TABLE); - auto & undrop = query_ptr->as(); if (!undrop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) { diff --git a/tests/integration/test_undrop_query/test.py b/tests/integration/test_undrop_query/test.py index d57aa8c2dc7..590a5690e55 100644 --- a/tests/integration/test_undrop_query/test.py +++ b/tests/integration/test_undrop_query/test.py @@ -29,39 +29,30 @@ def test_undrop_drop_and_undrop_loop(started_cluster): logging.info( "random_sec: " + random_sec.__str__() + ", table_uuid: " + table_uuid ) - node.query( - "CREATE TABLE test_undrop_loop" + "create table test_undrop_loop" + count.__str__() + " UUID '" + table_uuid - + "' (id Int32) ENGINE = MergeTree() ORDER BY id;" + + "' (id Int32) Engine=MergeTree() order by id;" ) - - node.query("DROP TABLE test_undrop_loop" + count.__str__() + ";") - + node.query("drop table test_undrop_loop" + count.__str__() + ";") time.sleep(random_sec) - if random_sec >= 5: error = node.query_and_get_error( - "UNDROP TABLE test_undrop_loop" + "undrop table test_undrop_loop" + count.__str__() - + " UUID '" + + " uuid '" + table_uuid + "';" ) assert "UNKNOWN_TABLE" in error - elif random_sec <= 3: - # (*) + else: node.query( - "UNDROP TABLE test_undrop_loop" + "undrop table test_undrop_loop" + count.__str__() - + " UUID '" + + " uuid '" + table_uuid + "';" ) count = count + 1 - else: - pass - # ignore random_sec = 4 to account for communication delay with the database. - # if we don't do that, then the second case (*) may find the table already dropped and receive an unexpected exception from the database (Bug #55167) diff --git a/tests/queries/0_stateless/02681_undrop_query.sql b/tests/queries/0_stateless/02681_undrop_query.sql index d038a383690..66447fc6c44 100644 --- a/tests/queries/0_stateless/02681_undrop_query.sql +++ b/tests/queries/0_stateless/02681_undrop_query.sql @@ -85,5 +85,5 @@ drop table 02681_undrop_multiple; select table from system.dropped_tables where table = '02681_undrop_multiple' limit 1; undrop table 02681_undrop_multiple; select * from 02681_undrop_multiple order by id; -undrop table 02681_undrop_multiple; -- { serverError TABLE_ALREADY_EXISTS } +undrop table 02681_undrop_multiple; -- { serverError 57 } drop table 02681_undrop_multiple sync; From ed9db6d392ed8b68c6d39ba8ffab81f875bc6949 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 20 Mar 2024 18:08:58 +0100 Subject: [PATCH 77/78] Add fuzz queries with CUBE and ROLLUP --- ...03014_analyzer_groupby_fuzz_60317.reference | 10 ++++++++++ .../03014_analyzer_groupby_fuzz_60317.sql | 18 ++++++++++++++++++ ...03015_analyzer_groupby_fuzz_60772.reference | 8 ++++++++ .../03015_analyzer_groupby_fuzz_60772.sql | 13 +++++++++++++ ...03017_analyzer_groupby_fuzz_61600.reference | 3 +++ .../03017_analyzer_groupby_fuzz_61600.sql | 11 +++++++++++ 6 files changed, 63 insertions(+) diff --git a/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.reference b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.reference index 4972904f87d..5e56482a470 100644 --- a/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.reference +++ b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.reference @@ -1 +1,11 @@ 30 30 1970-01-01 +30 1970-01-01 30 30 1970-01-01 -1980.1 +30 1970-01-01 30 30 1970-01-01 -1980.1 +30 1970-01-01 30 30 1970-01-01 -1980.1 +30 1970-01-01 30 30 1970-01-01 -1980.1 +30 1970-01-01 30 30 1970-01-01 -1980.1 +30 1970-01-01 30 30 1970-01-01 -1980.1 +30 1970-01-01 30 30 1970-01-01 -1980.1 +30 1970-01-01 30 30 1970-01-01 -1980.1 + +30 1970-01-01 30 30 1970-01-01 -1980.1 diff --git a/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql index 2f54058526e..094614cb78d 100644 --- a/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql +++ b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql @@ -7,3 +7,21 @@ FROM system.one GROUP BY _CAST(30, 'Nullable(UInt8)') SETTINGS allow_experimental_analyzer = 1; + +-- WITH CUBE (note that result is different with the analyzer (analyzer is correct including all combinations) +SELECT + toNullable(toNullable(materialize(_CAST(30, 'LowCardinality(UInt8)')))) AS `toNullable(toNullable(materialize(toLowCardinality(30))))`, + _CAST(0, 'Date') AS `makeDate(-1980.1, -1980.1, 10)`, + _CAST(30, 'LowCardinality(UInt8)') AS `toLowCardinality(30)`, + 30 AS `30`, + makeDate(materialize(_CAST(30, 'LowCardinality(UInt8)')), 10, _CAST(30, 'Nullable(UInt8)')) AS `makeDate(materialize(toLowCardinality(30)), 10, toNullable(toNullable(30)))`, + -1980.1 AS `-1980.1` +FROM system.one AS __table1 +GROUP BY + _CAST(30, 'Nullable(UInt8)'), + -1980.1, + materialize(30), + _CAST(30, 'Nullable(UInt8)') +WITH CUBE +WITH TOTALS +SETTINGS allow_experimental_analyzer = 1; diff --git a/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.reference b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.reference index 611407ecd90..5cdf256a24a 100644 --- a/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.reference +++ b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.reference @@ -1 +1,9 @@ %W 2018-01-02 22:33:44 1 +%W 2018-01-02 22:33:44 1 +%W 2018-01-02 22:33:44 1 +%W 2018-01-02 22:33:44 1 +%W 2018-01-02 22:33:44 1 +%W 2018-01-02 22:33:44 1 +%W 2018-01-02 22:33:44 1 +%W 2018-01-02 22:33:44 1 +%W 2018-01-02 22:33:44 1 diff --git a/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql index c8b4eef50ff..d3bd9ef0ce3 100644 --- a/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql +++ b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql @@ -8,3 +8,16 @@ GROUP BY 'gr', '2018-01-02 22:33:44' SETTINGS allow_experimental_analyzer = 1; + +-- WITH CUBE (note that result is different with the analyzer (analyzer is correct including all combinations) +SELECT + toFixedString(toFixedString(toFixedString(toFixedString(toFixedString(toFixedString('%W', 2), 2), 2), toLowCardinality(toLowCardinality(toNullable(2)))), 2), 2), + toFixedString(toFixedString('2018-01-02 22:33:44', 19), 19), + hasSubsequence(toNullable(materialize(toLowCardinality('garbage'))), 'gr') +GROUP BY + '2018-01-02 22:33:44', + toFixedString(toFixedString('2018-01-02 22:33:44', 19), 19), + 'gr', + '2018-01-02 22:33:44' +WITH CUBE +SETTINGS allow_experimental_analyzer = 1; diff --git a/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.reference b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.reference index 68acb650f8e..eaa1097a734 100644 --- a/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.reference +++ b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.reference @@ -1 +1,4 @@ 38 \N +38 \N +38 \N +38 \N diff --git a/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql index 44b27750c16..53a5cfe9b1a 100644 --- a/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql +++ b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql @@ -12,3 +12,14 @@ GROUP BY toNullable(3), concat(concat(NULLIF(1, 1), toNullable(toNullable(3)))) SETTINGS allow_experimental_analyzer = 1; + +-- WITH ROLLUP (note that result is different with the analyzer (analyzer is correct including all combinations) +SELECT + 38, + concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3))) +FROM set_index_not__fuzz_0 +GROUP BY + toNullable(3), + concat(concat(NULLIF(1, 1), toNullable(toNullable(3)))) +WITH ROLLUP +SETTINGS allow_experimental_analyzer = 1; From a31cf43499b1d02827ee3ca08e3fc961c3911eaa Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 20 Mar 2024 23:13:17 +0100 Subject: [PATCH 78/78] Update the dictionary for OSSFuzz (#61672) --- tests/fuzz/README.md | 18 +- tests/fuzz/all.dict | 3455 +++++++++++++++- tests/fuzz/dictionaries/datatypes.dict | 287 +- tests/fuzz/dictionaries/functions.dict | 5264 +++++++++++++++++++----- tests/fuzz/dictionaries/key_words.dict | 426 +- 5 files changed, 8089 insertions(+), 1361 deletions(-) diff --git a/tests/fuzz/README.md b/tests/fuzz/README.md index 5b864babde3..6b5b161b2d5 100644 --- a/tests/fuzz/README.md +++ b/tests/fuzz/README.md @@ -1,13 +1,23 @@ -The list of functions generated via following query +The list of functions generated via the following query ``` - clickhouse-client -q "select concat('\"', name, '\"') from system.functions union all select concat('\"', alias_to, '\"') from system.functions where alias_to != '' " > functions.dict + clickhouse client -q "SELECT * FROM (SELECT DISTINCT concat('\"', name, '\"') as res FROM system.functions ORDER BY name UNION ALL SELECT concat('\"', a.name, b.name, '\"') as res FROM system.functions as a CROSS JOIN system.aggregate_function_combinators as b WHERE a.is_aggregate = 1) ORDER BY res" > functions.dict ``` -The list of datatypes generated via following query: +The list of datatypes generated via the following query: ``` - clickhouse-client -q "select concat('\"', name, '\"') from system.data_type_families union all select concat('\"', alias_to, '\"') from system.data_type_families where alias_to != '' " > datatypes.dict + clickhouse client -q "SELECT DISTINCT concat('\"', name, '\"') as res FROM system.data_type_families ORDER BY name" > datatypes.dict +``` + +The list of keywords generated via the following query: + +``` + clickhouse client -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > key_words.dict ``` Then merge all dictionaries into one (all.dict) + +``` + cat ./dictionaries/* | sort | uniq > all.dict +``` \ No newline at end of file diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index b38e2c11a1c..f08e319f0d4 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -1,59 +1,333 @@ - "abs" -"accurate_Cast" "accurateCast" -"accurate_CastOrNull" +"accurateCastOrDefault" "accurateCastOrNull" "acos" "acosh" "ADD" "ADD COLUMN" "ADD CONSTRAINT" +"addDate" "addDays" "addHours" "ADD INDEX" +"addInterval" +"addMicroseconds" +"addMilliseconds" "addMinutes" "addMonths" +"addNanoseconds" +"ADD PROJECTION" "addQuarters" "addressToLine" +"addressToLineWithInlines" "addressToSymbol" "addSeconds" +"ADD STATISTIC" +"addTupleOfIntervals" "addWeeks" "addYears" +"ADMIN OPTION FOR" "aes_decrypt_mysql" "aes_encrypt_mysql" "AFTER" +"age" "AggregateFunction" "aggThrow" +"aggThrowArgMax" +"aggThrowArgMin" +"aggThrowArray" +"aggThrowDistinct" +"aggThrowForEach" +"aggThrowIf" +"aggThrowMap" +"aggThrowMerge" +"aggThrowNull" +"aggThrowOrDefault" +"aggThrowOrNull" +"aggThrowResample" +"aggThrowSimpleState" +"aggThrowState" +"ALGORITHM" "ALIAS" "ALL" +"ALLOWED_LATENESS" "alphaTokens" "ALTER" +"ALTER COLUMN" +"ALTER DATABASE" "ALTER LIVE VIEW" +"ALTER POLICY" +"ALTER PROFILE" +"ALTER QUOTA" +"ALTER ROLE" +"ALTER ROW POLICY" +"ALTER SETTINGS PROFILE" "ALTER TABLE" +"ALTER TEMPORARY TABLE" +"ALTER USER" +"analysisOfVariance" +"analysisOfVarianceArgMax" +"analysisOfVarianceArgMin" +"analysisOfVarianceArray" +"analysisOfVarianceDistinct" +"analysisOfVarianceForEach" +"analysisOfVarianceIf" +"analysisOfVarianceMap" +"analysisOfVarianceMerge" +"analysisOfVarianceNull" +"analysisOfVarianceOrDefault" +"analysisOfVarianceOrNull" +"analysisOfVarianceResample" +"analysisOfVarianceSimpleState" +"analysisOfVarianceState" "and" "AND" +"AND STDOUT" +"anova" +"anovaArgMax" +"anovaArgMin" +"anovaArray" +"anovaDistinct" +"anovaForEach" +"anovaIf" +"anovaMap" +"anovaMerge" +"anovaNull" +"anovaOrDefault" +"anovaOrNull" +"anovaResample" +"anovaSimpleState" +"anovaState" "ANTI" "any" "ANY" +"anyArgMax" +"anyArgMin" +"anyArray" +"anyDistinct" +"anyForEach" "anyHeavy" +"anyHeavyArgMax" +"anyHeavyArgMin" +"anyHeavyArray" +"anyHeavyDistinct" +"anyHeavyForEach" +"anyHeavyIf" +"anyHeavyMap" +"anyHeavyMerge" +"anyHeavyNull" +"anyHeavyOrDefault" +"anyHeavyOrNull" +"anyHeavyResample" +"anyHeavySimpleState" +"anyHeavyState" +"anyIf" "anyLast" +"anyLastArgMax" +"anyLastArgMin" +"anyLastArray" +"anyLastDistinct" +"anyLastForEach" +"anyLastIf" +"anyLastMap" +"anyLastMerge" +"anyLastNull" +"anyLastOrDefault" +"anyLastOrNull" +"anyLastResample" +"anyLast_respect_nulls" +"anyLast_respect_nullsArgMax" +"anyLast_respect_nullsArgMin" +"anyLast_respect_nullsArray" +"anyLast_respect_nullsDistinct" +"anyLast_respect_nullsForEach" +"anyLast_respect_nullsIf" +"anyLast_respect_nullsMap" +"anyLast_respect_nullsMerge" +"anyLast_respect_nullsNull" +"anyLast_respect_nullsOrDefault" +"anyLast_respect_nullsOrNull" +"anyLast_respect_nullsResample" +"anyLast_respect_nullsSimpleState" +"anyLast_respect_nullsState" +"anyLastSimpleState" +"anyLastState" +"anyMap" +"anyMerge" +"anyNull" +"anyOrDefault" +"anyOrNull" +"anyResample" +"any_respect_nulls" +"any_respect_nullsArgMax" +"any_respect_nullsArgMin" +"any_respect_nullsArray" +"any_respect_nullsDistinct" +"any_respect_nullsForEach" +"any_respect_nullsIf" +"any_respect_nullsMap" +"any_respect_nullsMerge" +"any_respect_nullsNull" +"any_respect_nullsOrDefault" +"any_respect_nullsOrNull" +"any_respect_nullsResample" +"any_respect_nullsSimpleState" +"any_respect_nullsState" +"anySimpleState" +"anyState" +"any_value" +"any_valueArgMax" +"any_valueArgMin" +"any_valueArray" +"any_valueDistinct" +"any_valueForEach" +"any_valueIf" +"any_valueMap" +"any_valueMerge" +"any_valueNull" +"any_valueOrDefault" +"any_valueOrNull" +"any_valueResample" +"any_value_respect_nulls" +"any_value_respect_nullsArgMax" +"any_value_respect_nullsArgMin" +"any_value_respect_nullsArray" +"any_value_respect_nullsDistinct" +"any_value_respect_nullsForEach" +"any_value_respect_nullsIf" +"any_value_respect_nullsMap" +"any_value_respect_nullsMerge" +"any_value_respect_nullsNull" +"any_value_respect_nullsOrDefault" +"any_value_respect_nullsOrNull" +"any_value_respect_nullsResample" +"any_value_respect_nullsSimpleState" +"any_value_respect_nullsState" +"any_valueSimpleState" +"any_valueState" +"APPEND" "appendTrailingCharIfAbsent" +"APPLY" +"APPLY DELETED MASK" +"approx_top_count" +"approx_top_countArgMax" +"approx_top_countArgMin" +"approx_top_countArray" +"approx_top_countDistinct" +"approx_top_countForEach" +"approx_top_countIf" +"approx_top_countMap" +"approx_top_countMerge" +"approx_top_countNull" +"approx_top_countOrDefault" +"approx_top_countOrNull" +"approx_top_countResample" +"approx_top_countSimpleState" +"approx_top_countState" +"approx_top_k" +"approx_top_kArgMax" +"approx_top_kArgMin" +"approx_top_kArray" +"approx_top_kDistinct" +"approx_top_kForEach" +"approx_top_kIf" +"approx_top_kMap" +"approx_top_kMerge" +"approx_top_kNull" +"approx_top_kOrDefault" +"approx_top_kOrNull" +"approx_top_kResample" +"approx_top_kSimpleState" +"approx_top_kState" +"approx_top_sum" +"approx_top_sumArgMax" +"approx_top_sumArgMin" +"approx_top_sumArray" +"approx_top_sumDistinct" +"approx_top_sumForEach" +"approx_top_sumIf" +"approx_top_sumMap" +"approx_top_sumMerge" +"approx_top_sumNull" +"approx_top_sumOrDefault" +"approx_top_sumOrNull" +"approx_top_sumResample" +"approx_top_sumSimpleState" +"approx_top_sumState" "argMax" +"argMaxArgMax" +"argMaxArgMin" +"argMaxArray" +"argMaxDistinct" +"argMaxForEach" +"argMaxIf" +"argMaxMap" +"argMaxMerge" +"argMaxNull" +"argMaxOrDefault" +"argMaxOrNull" +"argMaxResample" +"argMaxSimpleState" +"argMaxState" "argMin" +"argMinArgMax" +"argMinArgMin" +"argMinArray" +"argMinDistinct" +"argMinForEach" +"argMinIf" +"argMinMap" +"argMinMerge" +"argMinNull" +"argMinOrDefault" +"argMinOrNull" +"argMinResample" +"argMinSimpleState" +"argMinState" "array" "Array" -"ARRAY" +"array_agg" +"array_aggArgMax" +"array_aggArgMin" +"array_aggArray" +"array_aggDistinct" +"array_aggForEach" +"array_aggIf" +"array_aggMap" +"array_aggMerge" +"array_aggNull" +"array_aggOrDefault" +"array_aggOrNull" +"array_aggResample" +"array_aggSimpleState" +"array_aggState" "arrayAll" "arrayAUC" "arrayAvg" "arrayCompact" "arrayConcat" +"array_concat_agg" +"array_concat_aggArgMax" +"array_concat_aggArgMin" +"array_concat_aggArray" +"array_concat_aggDistinct" +"array_concat_aggForEach" +"array_concat_aggIf" +"array_concat_aggMap" +"array_concat_aggMerge" +"array_concat_aggNull" +"array_concat_aggOrDefault" +"array_concat_aggOrNull" +"array_concat_aggResample" +"array_concat_aggSimpleState" +"array_concat_aggState" "arrayCount" "arrayCumSum" "arrayCumSumNonNegative" "arrayDifference" "arrayDistinct" +"arrayDotProduct" "arrayElement" "arrayEnumerate" "arrayEnumerateDense" @@ -65,10 +339,16 @@ "arrayFilter" "arrayFirst" "arrayFirstIndex" +"arrayFirstOrNull" "arrayFlatten" +"arrayFold" "arrayIntersect" +"arrayJaccardIndex" "arrayJoin" "ARRAY JOIN" +"arrayLast" +"arrayLastIndex" +"arrayLastOrNull" "arrayMap" "arrayMax" "arrayMin" @@ -80,6 +360,7 @@ "arrayProduct" "arrayPushBack" "arrayPushFront" +"arrayRandomSample" "arrayReduce" "arrayReduceInRanges" "arrayResize" @@ -87,6 +368,11 @@ "arrayReverseFill" "arrayReverseSort" "arrayReverseSplit" +"arrayRotateLeft" +"arrayRotateRight" +"arrayShiftLeft" +"arrayShiftRight" +"arrayShingles" "arrayShuffle" "arraySlice" "arraySort" @@ -99,9 +385,11 @@ "AS" "ASC" "ASCENDING" +"ascii" "asin" "asinh" "ASOF" +"ASSUME" "assumeNotNull" "AST" "ASYNC" @@ -111,14 +399,59 @@ "ATTACH" "ATTACH PART" "ATTACH PARTITION" +"ATTACH POLICY" +"ATTACH PROFILE" +"ATTACH QUOTA" +"ATTACH ROLE" +"ATTACH ROW POLICY" +"ATTACH SETTINGS PROFILE" +"ATTACH USER" +"AUTO_INCREMENT" "avg" +"avgArgMax" +"avgArgMin" +"avgArray" +"avgDistinct" +"avgForEach" +"avgIf" +"avgMap" +"avgMerge" +"avgNull" +"avgOrDefault" +"avgOrNull" +"avgResample" +"avgSimpleState" +"avgState" "avgWeighted" +"avgWeightedArgMax" +"avgWeightedArgMin" +"avgWeightedArray" +"avgWeightedDistinct" +"avgWeightedForEach" +"avgWeightedIf" +"avgWeightedMap" +"avgWeightedMerge" +"avgWeightedNull" +"avgWeightedOrDefault" +"avgWeightedOrNull" +"avgWeightedResample" +"avgWeightedSimpleState" +"avgWeightedState" +"AZURE" +"BACKUP" +"bagexpansion" "bar" +"base58Decode" +"base58Encode" "base64Decode" "base64Encode" +"base_backup" "basename" -"bayesAB" +"BCRYPT_HASH" +"BCRYPT_PASSWORD" +"BEGIN TRANSACTION" "BETWEEN" +"BIDIRECTIONAL" "BIGINT" "BIGINT SIGNED" "BIGINT UNSIGNED" @@ -126,8 +459,23 @@ "BINARY" "BINARY LARGE OBJECT" "BINARY VARYING" +"BIT" "bitAnd" "BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" "__bitBoolMaskAnd" "__bitBoolMaskOr" "bitCount" @@ -156,11 +504,26 @@ "bitNot" "bitOr" "BIT_OR" +"BIT_ORArgMax" +"BIT_ORArgMin" +"BIT_ORArray" +"BIT_ORDistinct" +"BIT_ORForEach" +"BIT_ORIf" +"BIT_ORMap" +"BIT_ORMerge" +"BIT_ORNull" +"BIT_OROrDefault" +"BIT_OROrNull" +"BIT_ORResample" +"BIT_ORSimpleState" +"BIT_ORState" "bitPositionsToArray" "bitRotateLeft" "bitRotateRight" "bitShiftLeft" "bitShiftRight" +"bitSlice" "__bitSwapLastTwo" "bitTest" "bitTestAll" @@ -168,19 +531,53 @@ "__bitWrapperFunc" "bitXor" "BIT_XOR" +"BIT_XORArgMax" +"BIT_XORArgMin" +"BIT_XORArray" +"BIT_XORDistinct" +"BIT_XORForEach" +"BIT_XORIf" +"BIT_XORMap" +"BIT_XORMerge" +"BIT_XORNull" +"BIT_XOROrDefault" +"BIT_XOROrNull" +"BIT_XORResample" +"BIT_XORSimpleState" +"BIT_XORState" +"BLAKE3" "BLOB" "blockNumber" "blockSerializedSize" "blockSize" -"BOOL" -"BOOLEAN" +"bool" +"Bool" +"boolean" "BOTH" "boundingRatio" +"boundingRatioArgMax" +"boundingRatioArgMin" +"boundingRatioArray" +"boundingRatioDistinct" +"boundingRatioForEach" +"boundingRatioIf" +"boundingRatioMap" +"boundingRatioMerge" +"boundingRatioNull" +"boundingRatioOrDefault" +"boundingRatioOrNull" +"boundingRatioResample" +"boundingRatioSimpleState" +"boundingRatioState" "buildId" "BY" "BYTE" "BYTEA" +"byteHammingDistance" "byteSize" +"byteSlice" +"byteSwap" +"CASCADE" "CASE" "caseWithExpr" "caseWithExpression" @@ -188,10 +585,28 @@ "caseWithoutExpression" "_CAST" "CAST" +"catboostEvaluate" "categoricalInformationValue" +"categoricalInformationValueArgMax" +"categoricalInformationValueArgMin" +"categoricalInformationValueArray" +"categoricalInformationValueDistinct" +"categoricalInformationValueForEach" +"categoricalInformationValueIf" +"categoricalInformationValueMap" +"categoricalInformationValueMerge" +"categoricalInformationValueNull" +"categoricalInformationValueOrDefault" +"categoricalInformationValueOrNull" +"categoricalInformationValueResample" +"categoricalInformationValueSimpleState" +"categoricalInformationValueState" "cbrt" "ceil" "ceiling" +"CHANGE" +"CHANGEABLE_IN_READONLY" +"CHANGED" "char" "CHAR" "CHARACTER" @@ -202,13 +617,19 @@ "CHAR_LENGTH" "CHAR VARYING" "CHECK" +"CHECK ALL TABLES" "CHECK TABLE" "cityHash64" -"CLEAR" +"CLEANUP" "CLEAR COLUMN" "CLEAR INDEX" +"CLEAR PROJECTION" +"CLEAR STATISTIC" "CLOB" "CLUSTER" +"cluster_host_ids" +"CLUSTERS" +"CN" "coalesce" "CODEC" "COLLATE" @@ -216,52 +637,301 @@ "COLUMNS" "COMMENT" "COMMENT COLUMN" +"COMMIT" +"COMPRESSION" "concat" "concatAssumeInjective" +"concatWithSeparator" +"concatWithSeparatorAssumeInjective" +"concat_ws" "connection_id" -"connectionid" "connectionId" +"CONST" "CONSTRAINT" +"contingency" +"contingencyArgMax" +"contingencyArgMin" +"contingencyArray" +"contingencyDistinct" +"contingencyForEach" +"contingencyIf" +"contingencyMap" +"contingencyMerge" +"contingencyNull" +"contingencyOrDefault" +"contingencyOrNull" +"contingencyResample" +"contingencySimpleState" +"contingencyState" "convertCharset" "corr" +"corrArgMax" +"corrArgMin" +"corrArray" +"corrDistinct" +"corrForEach" +"corrIf" +"corrMap" +"corrMatrix" +"corrMatrixArgMax" +"corrMatrixArgMin" +"corrMatrixArray" +"corrMatrixDistinct" +"corrMatrixForEach" +"corrMatrixIf" +"corrMatrixMap" +"corrMatrixMerge" +"corrMatrixNull" +"corrMatrixOrDefault" +"corrMatrixOrNull" +"corrMatrixResample" +"corrMatrixSimpleState" +"corrMatrixState" +"corrMerge" +"corrNull" +"corrOrDefault" +"corrOrNull" +"corrResample" +"corrSimpleState" "corrStable" +"corrStableArgMax" +"corrStableArgMin" +"corrStableArray" +"corrStableDistinct" +"corrStableForEach" +"corrStableIf" +"corrStableMap" +"corrStableMerge" +"corrStableNull" +"corrStableOrDefault" +"corrStableOrNull" +"corrStableResample" +"corrStableSimpleState" +"corrStableState" +"corrState" "cos" "cosh" +"cosineDistance" "count" +"countArgMax" +"countArgMin" +"countArray" "countDigits" +"countDistinct" "countEqual" +"countForEach" +"countIf" +"countMap" "countMatches" "countMatchesCaseInsensitive" +"countMerge" +"countNull" +"countOrDefault" +"countOrNull" +"countResample" +"countSimpleState" +"countState" "countSubstrings" "countSubstringsCaseInsensitive" "countSubstringsCaseInsensitiveUTF8" "covarPop" "COVAR_POP" +"covarPopArgMax" +"COVAR_POPArgMax" +"covarPopArgMin" +"COVAR_POPArgMin" +"covarPopArray" +"COVAR_POPArray" +"covarPopDistinct" +"COVAR_POPDistinct" +"covarPopForEach" +"COVAR_POPForEach" +"covarPopIf" +"COVAR_POPIf" +"covarPopMap" +"COVAR_POPMap" +"covarPopMatrix" +"covarPopMatrixArgMax" +"covarPopMatrixArgMin" +"covarPopMatrixArray" +"covarPopMatrixDistinct" +"covarPopMatrixForEach" +"covarPopMatrixIf" +"covarPopMatrixMap" +"covarPopMatrixMerge" +"covarPopMatrixNull" +"covarPopMatrixOrDefault" +"covarPopMatrixOrNull" +"covarPopMatrixResample" +"covarPopMatrixSimpleState" +"covarPopMatrixState" +"covarPopMerge" +"COVAR_POPMerge" +"covarPopNull" +"COVAR_POPNull" +"covarPopOrDefault" +"COVAR_POPOrDefault" +"covarPopOrNull" +"COVAR_POPOrNull" +"covarPopResample" +"COVAR_POPResample" +"covarPopSimpleState" +"COVAR_POPSimpleState" "covarPopStable" +"covarPopStableArgMax" +"covarPopStableArgMin" +"covarPopStableArray" +"covarPopStableDistinct" +"covarPopStableForEach" +"covarPopStableIf" +"covarPopStableMap" +"covarPopStableMerge" +"covarPopStableNull" +"covarPopStableOrDefault" +"covarPopStableOrNull" +"covarPopStableResample" +"covarPopStableSimpleState" +"covarPopStableState" +"covarPopState" +"COVAR_POPState" "covarSamp" "COVAR_SAMP" +"covarSampArgMax" +"COVAR_SAMPArgMax" +"covarSampArgMin" +"COVAR_SAMPArgMin" +"covarSampArray" +"COVAR_SAMPArray" +"covarSampDistinct" +"COVAR_SAMPDistinct" +"covarSampForEach" +"COVAR_SAMPForEach" +"covarSampIf" +"COVAR_SAMPIf" +"covarSampMap" +"COVAR_SAMPMap" +"covarSampMatrix" +"covarSampMatrixArgMax" +"covarSampMatrixArgMin" +"covarSampMatrixArray" +"covarSampMatrixDistinct" +"covarSampMatrixForEach" +"covarSampMatrixIf" +"covarSampMatrixMap" +"covarSampMatrixMerge" +"covarSampMatrixNull" +"covarSampMatrixOrDefault" +"covarSampMatrixOrNull" +"covarSampMatrixResample" +"covarSampMatrixSimpleState" +"covarSampMatrixState" +"covarSampMerge" +"COVAR_SAMPMerge" +"covarSampNull" +"COVAR_SAMPNull" +"covarSampOrDefault" +"COVAR_SAMPOrDefault" +"covarSampOrNull" +"COVAR_SAMPOrNull" +"covarSampResample" +"COVAR_SAMPResample" +"covarSampSimpleState" +"COVAR_SAMPSimpleState" "covarSampStable" +"covarSampStableArgMax" +"covarSampStableArgMin" +"covarSampStableArray" +"covarSampStableDistinct" +"covarSampStableForEach" +"covarSampStableIf" +"covarSampStableMap" +"covarSampStableMerge" +"covarSampStableNull" +"covarSampStableOrDefault" +"covarSampStableOrNull" +"covarSampStableResample" +"covarSampStableSimpleState" +"covarSampStableState" +"covarSampState" +"COVAR_SAMPState" +"cramersV" +"cramersVArgMax" +"cramersVArgMin" +"cramersVArray" +"cramersVBiasCorrected" +"cramersVBiasCorrectedArgMax" +"cramersVBiasCorrectedArgMin" +"cramersVBiasCorrectedArray" +"cramersVBiasCorrectedDistinct" +"cramersVBiasCorrectedForEach" +"cramersVBiasCorrectedIf" +"cramersVBiasCorrectedMap" +"cramersVBiasCorrectedMerge" +"cramersVBiasCorrectedNull" +"cramersVBiasCorrectedOrDefault" +"cramersVBiasCorrectedOrNull" +"cramersVBiasCorrectedResample" +"cramersVBiasCorrectedSimpleState" +"cramersVBiasCorrectedState" +"cramersVDistinct" +"cramersVForEach" +"cramersVIf" +"cramersVMap" +"cramersVMerge" +"cramersVNull" +"cramersVOrDefault" +"cramersVOrNull" +"cramersVResample" +"cramersVSimpleState" +"cramersVState" "CRC32" "CRC32IEEE" "CRC64" "CREATE" +"CREATE POLICY" +"CREATE PROFILE" +"CREATE QUOTA" +"CREATE ROLE" +"CREATE ROW POLICY" +"CREATE SETTINGS PROFILE" +"CREATE TABLE" +"CREATE TEMPORARY TABLE" +"CREATE USER" "CROSS" "CUBE" +"curdate" +"current_database" "currentDatabase" +"current_date" +"CURRENT GRANTS" "currentProfiles" +"CURRENT QUOTA" "currentRoles" +"CURRENT ROLES" +"CURRENT ROW" +"current_schemas" +"currentSchemas" +"current_timestamp" +"CURRENT TRANSACTION" "currentUser" +"CURRENT_USER" +"CURRENTUSER" "cutFragment" "cutIPv6" "cutQueryString" "cutQueryStringAndFragment" "cutToFirstSignificantSubdomain" "cutToFirstSignificantSubdomainCustom" +"cutToFirstSignificantSubdomainCustomRFC" "cutToFirstSignificantSubdomainCustomWithWWW" +"cutToFirstSignificantSubdomainCustomWithWWWRFC" +"cutToFirstSignificantSubdomainRFC" "cutToFirstSignificantSubdomainWithWWW" +"cutToFirstSignificantSubdomainWithWWWRFC" "cutURLParameter" "cutWWW" "D" +"damerauLevenshteinDistance" "DATABASE" "DATABASES" "Date" @@ -269,9 +939,11 @@ "Date32" "DATE_ADD" "DATEADD" +"date_diff" "dateDiff" "DATE_DIFF" "DATEDIFF" +"DATE_FORMAT" "dateName" "DATE_SUB" "DATESUB" @@ -280,12 +952,13 @@ "DateTime64" "dateTime64ToSnowflake" "dateTimeToSnowflake" -"date_trunc" "dateTrunc" +"DATE_TRUNC" "DAY" "DAYOFMONTH" "DAYOFWEEK" "DAYOFYEAR" +"DAYS" "DD" "DEC" "Decimal" @@ -293,28 +966,84 @@ "Decimal256" "Decimal32" "Decimal64" +"decodeHTMLComponent" "decodeURLComponent" +"decodeURLFormComponent" "decodeXMLComponent" "decrypt" "DEDUPLICATE" "DEFAULT" +"DEFAULT DATABASE" "defaultProfiles" +"DEFAULT ROLE" "defaultRoles" "defaultValueOfArgumentType" "defaultValueOfTypeName" -"DELAY" +"DEFINER" +"degrees" "DELETE" "DELETE WHERE" "deltaSum" +"deltaSumArgMax" +"deltaSumArgMin" +"deltaSumArray" +"deltaSumDistinct" +"deltaSumForEach" +"deltaSumIf" +"deltaSumMap" +"deltaSumMerge" +"deltaSumNull" +"deltaSumOrDefault" +"deltaSumOrNull" +"deltaSumResample" +"deltaSumSimpleState" +"deltaSumState" "deltaSumTimestamp" +"deltaSumTimestampArgMax" +"deltaSumTimestampArgMin" +"deltaSumTimestampArray" +"deltaSumTimestampDistinct" +"deltaSumTimestampForEach" +"deltaSumTimestampIf" +"deltaSumTimestampMap" +"deltaSumTimestampMerge" +"deltaSumTimestampNull" +"deltaSumTimestampOrDefault" +"deltaSumTimestampOrNull" +"deltaSumTimestampResample" +"deltaSumTimestampSimpleState" +"deltaSumTimestampState" "demangle" "dense_rank" +"dense_rankArgMax" +"dense_rankArgMin" +"dense_rankArray" +"dense_rankDistinct" +"dense_rankForEach" +"dense_rankIf" +"dense_rankMap" +"dense_rankMerge" +"dense_rankNull" +"dense_rankOrDefault" +"dense_rankOrNull" +"dense_rankResample" +"dense_rankSimpleState" +"dense_rankState" +"DEPENDS ON" "DESC" "DESCENDING" "DESCRIBE" "DETACH" +"DETACH PART" "DETACH PARTITION" +"detectCharset" +"detectLanguage" +"detectLanguageMixed" +"detectLanguageUnknown" +"detectProgrammingLanguage" +"detectTonality" "dictGet" +"dictGetAll" "dictGetChildren" "dictGetDate" "dictGetDateOrDefault" @@ -334,6 +1063,10 @@ "dictGetInt64OrDefault" "dictGetInt8" "dictGetInt8OrDefault" +"dictGetIPv4" +"dictGetIPv4OrDefault" +"dictGetIPv6" +"dictGetIPv6OrDefault" "dictGetOrDefault" "dictGetOrNull" "dictGetString" @@ -353,24 +1086,45 @@ "DICTIONARY" "dictIsIn" "DISK" +"displayName" +"distanceL1" +"distanceL2" +"distanceL2Squared" +"distanceLinf" +"distanceLp" "DISTINCT" -"DISTRIBUTED" +"DISTINCT ON" +"DIV" "divide" +"divideDecimal" "domain" +"domainRFC" "domainWithoutWWW" +"domainWithoutWWWRFC" +"dotProduct" "DOUBLE" "DOUBLE PRECISION" +"DOUBLE_SHA1_HASH" +"DOUBLE_SHA1_PASSWORD" "DROP" "DROP COLUMN" "DROP CONSTRAINT" +"DROP DEFAULT" "DROP DETACHED PART" "DROP DETACHED PARTITION" "DROP INDEX" +"DROP PART" "DROP PARTITION" +"DROP PROJECTION" +"DROP STATISTIC" +"DROP TABLE" +"DROP TEMPORARY TABLE" "dumpColumnStructure" "e" +"editDistance" "ELSE" "empty" +"EMPTY" "emptyArrayDate" "emptyArrayDateTime" "emptyArrayFloat32" @@ -385,24 +1139,55 @@ "emptyArrayUInt32" "emptyArrayUInt64" "emptyArrayUInt8" +"EMPTY AS" "enabledProfiles" "enabledRoles" +"ENABLED ROLES" +"encodeURLComponent" +"encodeURLFormComponent" "encodeXMLComponent" "encrypt" "END" "endsWith" +"endsWithUTF8" +"ENFORCED" "ENGINE" "entropy" +"entropyArgMax" +"entropyArgMin" +"entropyArray" +"entropyDistinct" +"entropyForEach" +"entropyIf" +"entropyMap" +"entropyMerge" +"entropyNull" +"entropyOrDefault" +"entropyOrNull" +"entropyResample" +"entropySimpleState" +"entropyState" "Enum" "ENUM" "Enum16" "Enum8" +"EPHEMERAL" +"EPHEMERAL SEQUENTIAL" "equals" "erf" "erfc" "errorCodeToName" +"ESTIMATE" "evalMLMethod" +"EVENT" "EVENTS" +"EVERY" +"EXCEPT" +"EXCEPT DATABASE" +"EXCEPT DATABASES" +"EXCEPT TABLE" +"EXCEPT TABLES" +"EXCHANGE DICTIONARIES" "EXCHANGE TABLES" "EXISTS" "exp" @@ -410,7 +1195,83 @@ "exp2" "EXPLAIN" "exponentialMovingAverage" +"exponentialMovingAverageArgMax" +"exponentialMovingAverageArgMin" +"exponentialMovingAverageArray" +"exponentialMovingAverageDistinct" +"exponentialMovingAverageForEach" +"exponentialMovingAverageIf" +"exponentialMovingAverageMap" +"exponentialMovingAverageMerge" +"exponentialMovingAverageNull" +"exponentialMovingAverageOrDefault" +"exponentialMovingAverageOrNull" +"exponentialMovingAverageResample" +"exponentialMovingAverageSimpleState" +"exponentialMovingAverageState" +"exponentialTimeDecayedAvg" +"exponentialTimeDecayedAvgArgMax" +"exponentialTimeDecayedAvgArgMin" +"exponentialTimeDecayedAvgArray" +"exponentialTimeDecayedAvgDistinct" +"exponentialTimeDecayedAvgForEach" +"exponentialTimeDecayedAvgIf" +"exponentialTimeDecayedAvgMap" +"exponentialTimeDecayedAvgMerge" +"exponentialTimeDecayedAvgNull" +"exponentialTimeDecayedAvgOrDefault" +"exponentialTimeDecayedAvgOrNull" +"exponentialTimeDecayedAvgResample" +"exponentialTimeDecayedAvgSimpleState" +"exponentialTimeDecayedAvgState" +"exponentialTimeDecayedCount" +"exponentialTimeDecayedCountArgMax" +"exponentialTimeDecayedCountArgMin" +"exponentialTimeDecayedCountArray" +"exponentialTimeDecayedCountDistinct" +"exponentialTimeDecayedCountForEach" +"exponentialTimeDecayedCountIf" +"exponentialTimeDecayedCountMap" +"exponentialTimeDecayedCountMerge" +"exponentialTimeDecayedCountNull" +"exponentialTimeDecayedCountOrDefault" +"exponentialTimeDecayedCountOrNull" +"exponentialTimeDecayedCountResample" +"exponentialTimeDecayedCountSimpleState" +"exponentialTimeDecayedCountState" +"exponentialTimeDecayedMax" +"exponentialTimeDecayedMaxArgMax" +"exponentialTimeDecayedMaxArgMin" +"exponentialTimeDecayedMaxArray" +"exponentialTimeDecayedMaxDistinct" +"exponentialTimeDecayedMaxForEach" +"exponentialTimeDecayedMaxIf" +"exponentialTimeDecayedMaxMap" +"exponentialTimeDecayedMaxMerge" +"exponentialTimeDecayedMaxNull" +"exponentialTimeDecayedMaxOrDefault" +"exponentialTimeDecayedMaxOrNull" +"exponentialTimeDecayedMaxResample" +"exponentialTimeDecayedMaxSimpleState" +"exponentialTimeDecayedMaxState" +"exponentialTimeDecayedSum" +"exponentialTimeDecayedSumArgMax" +"exponentialTimeDecayedSumArgMin" +"exponentialTimeDecayedSumArray" +"exponentialTimeDecayedSumDistinct" +"exponentialTimeDecayedSumForEach" +"exponentialTimeDecayedSumIf" +"exponentialTimeDecayedSumMap" +"exponentialTimeDecayedSumMerge" +"exponentialTimeDecayedSumNull" +"exponentialTimeDecayedSumOrDefault" +"exponentialTimeDecayedSumOrNull" +"exponentialTimeDecayedSumResample" +"exponentialTimeDecayedSumSimpleState" +"exponentialTimeDecayedSumState" "EXPRESSION" +"EXTENDED" +"EXTERNAL DDL FROM" "extract" "EXTRACT" "extractAll" @@ -418,40 +1279,106 @@ "extractAllGroupsHorizontal" "extractAllGroupsVertical" "extractGroups" +"extractKeyValuePairs" +"extractKeyValuePairsWithEscaping" "extractTextFromHTML" "extractURLParameter" "extractURLParameterNames" "extractURLParameters" +"factorial" +"FALSE" "farmFingerprint64" "farmHash64" -"FETCHES" +"FETCH" "FETCH PART" "FETCH PARTITION" +"FIELDS" "file" +"FILE" "filesystemAvailable" +"FILESYSTEM CACHE" +"FILESYSTEM CACHES" "filesystemCapacity" -"filesystemFree" +"filesystemUnreserved" +"FILTER" "FINAL" "finalizeAggregation" "FIRST" +"firstLine" "firstSignificantSubdomain" "firstSignificantSubdomainCustom" +"firstSignificantSubdomainCustomRFC" +"firstSignificantSubdomainRFC" "first_value" +"first_valueArgMax" +"first_valueArgMin" +"first_valueArray" +"first_valueDistinct" +"first_valueForEach" +"first_valueIf" +"first_valueMap" +"first_valueMerge" +"first_valueNull" +"first_valueOrDefault" +"first_valueOrNull" +"first_valueResample" +"first_value_respect_nulls" +"first_value_respect_nullsArgMax" +"first_value_respect_nullsArgMin" +"first_value_respect_nullsArray" +"first_value_respect_nullsDistinct" +"first_value_respect_nullsForEach" +"first_value_respect_nullsIf" +"first_value_respect_nullsMap" +"first_value_respect_nullsMerge" +"first_value_respect_nullsNull" +"first_value_respect_nullsOrDefault" +"first_value_respect_nullsOrNull" +"first_value_respect_nullsResample" +"first_value_respect_nullsSimpleState" +"first_value_respect_nullsState" +"first_valueSimpleState" +"first_valueState" "FIXED" "FixedString" +"flameGraph" +"flameGraphArgMax" +"flameGraphArgMin" +"flameGraphArray" +"flameGraphDistinct" +"flameGraphForEach" +"flameGraphIf" +"flameGraphMap" +"flameGraphMerge" +"flameGraphNull" +"flameGraphOrDefault" +"flameGraphOrNull" +"flameGraphResample" +"flameGraphSimpleState" +"flameGraphState" "flatten" +"flattenTuple" "FLOAT" "Float32" "Float64" "floor" -"FLUSH" +"FOLLOWING" "FOR" "ForEach" +"FOREIGN" +"FOREIGN KEY" +"FORGET PARTITION" "format" "FORMAT" +"FORMAT_BYTES" "formatDateTime" -"formatReadableQuantity" +"formatDateTimeInJodaSyntax" +"formatQuery" +"formatQueryOrNull" +"formatQuerySingleLine" +"formatQuerySingleLineOrNull" "formatReadableDecimalSize" +"formatReadableQuantity" "formatReadableSize" "formatReadableTimeDelta" "formatRow" @@ -461,132 +1388,437 @@ "FREEZE" "FROM" "FROM_BASE64" -"toDaysSinceYearZero" +"FROM_DAYS" "fromDaysSinceYearZero" "fromDaysSinceYearZero32" +"FROM INFILE" "fromModifiedJulianDay" "fromModifiedJulianDayOrNull" +"FROM SHARD" "FROM_UNIXTIME" "fromUnixTimestamp" "fromUnixTimestamp64Micro" "fromUnixTimestamp64Milli" "fromUnixTimestamp64Nano" +"fromUnixTimestampInJodaSyntax" +"from_utc_timestamp" +"fromUTCTimestamp" "FULL" "fullHostName" +"FULLTEXT" "FUNCTION" "fuzzBits" "gccMurmurHash" "gcd" +"generateRandomStructure" +"generateULID" "generateUUIDv4" "geoDistance" "geohashDecode" "geohashEncode" "geohashesInBox" +"GEOMETRY" "geoToH3" "geoToS2" "getMacro" +"getOSKernelVersion" "__getScalar" "getServerPort" "getSetting" "getSizeOfEnumType" +"getSubcolumn" +"getTypeSerializationStreams" "GLOBAL" "globalIn" +"GLOBAL IN" "globalInIgnoreSet" "globalNotIn" +"GLOBAL NOT IN" "globalNotInIgnoreSet" "globalNotNullIn" "globalNotNullInIgnoreSet" "globalNullIn" "globalNullInIgnoreSet" "globalVariable" +"GRANT" +"GRANTEES" +"GRANT OPTION FOR" "GRANULARITY" "greatCircleAngle" "greatCircleDistance" "greater" "greaterOrEquals" "greatest" -"GROUP" "groupArray" +"groupArrayArgMax" +"groupArrayArgMin" +"groupArrayArray" +"groupArrayDistinct" +"groupArrayForEach" +"groupArrayIf" "groupArrayInsertAt" +"groupArrayInsertAtArgMax" +"groupArrayInsertAtArgMin" +"groupArrayInsertAtArray" +"groupArrayInsertAtDistinct" +"groupArrayInsertAtForEach" +"groupArrayInsertAtIf" +"groupArrayInsertAtMap" +"groupArrayInsertAtMerge" +"groupArrayInsertAtNull" +"groupArrayInsertAtOrDefault" +"groupArrayInsertAtOrNull" +"groupArrayInsertAtResample" +"groupArrayInsertAtSimpleState" +"groupArrayInsertAtState" +"groupArrayIntersect" +"groupArrayIntersectArgMax" +"groupArrayIntersectArgMin" +"groupArrayIntersectArray" +"groupArrayIntersectDistinct" +"groupArrayIntersectForEach" +"groupArrayIntersectIf" +"groupArrayIntersectMap" +"groupArrayIntersectMerge" +"groupArrayIntersectNull" +"groupArrayIntersectOrDefault" +"groupArrayIntersectOrNull" +"groupArrayIntersectResample" +"groupArrayIntersectSimpleState" +"groupArrayIntersectState" +"groupArrayLast" +"groupArrayLastArgMax" +"groupArrayLastArgMin" +"groupArrayLastArray" +"groupArrayLastDistinct" +"groupArrayLastForEach" +"groupArrayLastIf" +"groupArrayLastMap" +"groupArrayLastMerge" +"groupArrayLastNull" +"groupArrayLastOrDefault" +"groupArrayLastOrNull" +"groupArrayLastResample" +"groupArrayLastSimpleState" +"groupArrayLastState" +"groupArrayMap" +"groupArrayMerge" "groupArrayMovingAvg" +"groupArrayMovingAvgArgMax" +"groupArrayMovingAvgArgMin" +"groupArrayMovingAvgArray" +"groupArrayMovingAvgDistinct" +"groupArrayMovingAvgForEach" +"groupArrayMovingAvgIf" +"groupArrayMovingAvgMap" +"groupArrayMovingAvgMerge" +"groupArrayMovingAvgNull" +"groupArrayMovingAvgOrDefault" +"groupArrayMovingAvgOrNull" +"groupArrayMovingAvgResample" +"groupArrayMovingAvgSimpleState" +"groupArrayMovingAvgState" "groupArrayMovingSum" +"groupArrayMovingSumArgMax" +"groupArrayMovingSumArgMin" +"groupArrayMovingSumArray" +"groupArrayMovingSumDistinct" +"groupArrayMovingSumForEach" +"groupArrayMovingSumIf" +"groupArrayMovingSumMap" +"groupArrayMovingSumMerge" +"groupArrayMovingSumNull" +"groupArrayMovingSumOrDefault" +"groupArrayMovingSumOrNull" +"groupArrayMovingSumResample" +"groupArrayMovingSumSimpleState" +"groupArrayMovingSumState" +"groupArrayNull" +"groupArrayOrDefault" +"groupArrayOrNull" +"groupArrayResample" "groupArraySample" +"groupArraySampleArgMax" +"groupArraySampleArgMin" +"groupArraySampleArray" +"groupArraySampleDistinct" +"groupArraySampleForEach" +"groupArraySampleIf" +"groupArraySampleMap" +"groupArraySampleMerge" +"groupArraySampleNull" +"groupArraySampleOrDefault" +"groupArraySampleOrNull" +"groupArraySampleResample" +"groupArraySampleSimpleState" +"groupArraySampleState" +"groupArraySimpleState" +"groupArraySorted" +"groupArraySortedArgMax" +"groupArraySortedArgMin" +"groupArraySortedArray" +"groupArraySortedDistinct" +"groupArraySortedForEach" +"groupArraySortedIf" +"groupArraySortedMap" +"groupArraySortedMerge" +"groupArraySortedNull" +"groupArraySortedOrDefault" +"groupArraySortedOrNull" +"groupArraySortedResample" +"groupArraySortedSimpleState" +"groupArraySortedState" +"groupArrayState" "groupBitAnd" +"groupBitAndArgMax" +"groupBitAndArgMin" +"groupBitAndArray" +"groupBitAndDistinct" +"groupBitAndForEach" +"groupBitAndIf" +"groupBitAndMap" +"groupBitAndMerge" +"groupBitAndNull" +"groupBitAndOrDefault" +"groupBitAndOrNull" +"groupBitAndResample" +"groupBitAndSimpleState" +"groupBitAndState" "groupBitmap" "groupBitmapAnd" +"groupBitmapAndArgMax" +"groupBitmapAndArgMin" +"groupBitmapAndArray" +"groupBitmapAndDistinct" +"groupBitmapAndForEach" +"groupBitmapAndIf" +"groupBitmapAndMap" +"groupBitmapAndMerge" +"groupBitmapAndNull" +"groupBitmapAndOrDefault" +"groupBitmapAndOrNull" +"groupBitmapAndResample" +"groupBitmapAndSimpleState" +"groupBitmapAndState" +"groupBitmapArgMax" +"groupBitmapArgMin" +"groupBitmapArray" +"groupBitmapDistinct" +"groupBitmapForEach" +"groupBitmapIf" +"groupBitmapMap" +"groupBitmapMerge" +"groupBitmapNull" "groupBitmapOr" +"groupBitmapOrArgMax" +"groupBitmapOrArgMin" +"groupBitmapOrArray" +"groupBitmapOrDefault" +"groupBitmapOrDistinct" +"groupBitmapOrForEach" +"groupBitmapOrIf" +"groupBitmapOrMap" +"groupBitmapOrMerge" +"groupBitmapOrNull" +"groupBitmapOrOrDefault" +"groupBitmapOrOrNull" +"groupBitmapOrResample" +"groupBitmapOrSimpleState" +"groupBitmapOrState" +"groupBitmapResample" +"groupBitmapSimpleState" +"groupBitmapState" "groupBitmapXor" +"groupBitmapXorArgMax" +"groupBitmapXorArgMin" +"groupBitmapXorArray" +"groupBitmapXorDistinct" +"groupBitmapXorForEach" +"groupBitmapXorIf" +"groupBitmapXorMap" +"groupBitmapXorMerge" +"groupBitmapXorNull" +"groupBitmapXorOrDefault" +"groupBitmapXorOrNull" +"groupBitmapXorResample" +"groupBitmapXorSimpleState" +"groupBitmapXorState" "groupBitOr" +"groupBitOrArgMax" +"groupBitOrArgMin" +"groupBitOrArray" +"groupBitOrDistinct" +"groupBitOrForEach" +"groupBitOrIf" +"groupBitOrMap" +"groupBitOrMerge" +"groupBitOrNull" +"groupBitOrOrDefault" +"groupBitOrOrNull" +"groupBitOrResample" +"groupBitOrSimpleState" +"groupBitOrState" "groupBitXor" +"groupBitXorArgMax" +"groupBitXorArgMin" +"groupBitXorArray" +"groupBitXorDistinct" +"groupBitXorForEach" +"groupBitXorIf" +"groupBitXorMap" +"groupBitXorMerge" +"groupBitXorNull" +"groupBitXorOrDefault" +"groupBitXorOrNull" +"groupBitXorResample" +"groupBitXorSimpleState" +"groupBitXorState" "GROUP BY" +"GROUPING SETS" +"GROUPS" "groupUniqArray" +"groupUniqArrayArgMax" +"groupUniqArrayArgMin" +"groupUniqArrayArray" +"groupUniqArrayDistinct" +"groupUniqArrayForEach" +"groupUniqArrayIf" +"groupUniqArrayMap" +"groupUniqArrayMerge" +"groupUniqArrayNull" +"groupUniqArrayOrDefault" +"groupUniqArrayOrNull" +"groupUniqArrayResample" +"groupUniqArraySimpleState" +"groupUniqArrayState" +"H" +"h3CellAreaM2" +"h3CellAreaRads2" +"h3Distance" "h3EdgeAngle" +"h3EdgeLengthKm" "h3EdgeLengthM" +"h3ExactEdgeLengthKm" +"h3ExactEdgeLengthM" +"h3ExactEdgeLengthRads" "h3GetBaseCell" +"h3GetDestinationIndexFromUnidirectionalEdge" "h3GetFaces" +"h3GetIndexesFromUnidirectionalEdge" +"h3GetOriginIndexFromUnidirectionalEdge" +"h3GetPentagonIndexes" +"h3GetRes0Indexes" "h3GetResolution" +"h3GetUnidirectionalEdge" +"h3GetUnidirectionalEdgeBoundary" +"h3GetUnidirectionalEdgesFromHexagon" +"h3HexAreaKm2" "h3HexAreaM2" +"h3HexRing" "h3IndexesAreNeighbors" "h3IsPentagon" "h3IsResClassIII" "h3IsValid" "h3kRing" +"h3Line" +"h3NumHexagons" +"h3PointDistKm" +"h3PointDistM" +"h3PointDistRads" +"h3ToCenterChild" "h3ToChildren" "h3ToGeo" "h3ToGeoBoundary" "h3ToParent" "h3ToString" +"h3UnidirectionalEdgeIsValid" "halfMD5" "has" "hasAll" "hasAny" "hasColumnInTable" +"HASH" +"hasSubsequence" +"hasSubsequenceCaseInsensitive" +"hasSubsequenceCaseInsensitiveUTF8" +"hasSubsequenceUTF8" "hasSubstr" "hasThreadFuzzer" "hasToken" "hasTokenCaseInsensitive" +"hasTokenCaseInsensitiveOrNull" +"hasTokenOrNull" "HAVING" +"HDFS" "hex" "HH" "HIERARCHICAL" "histogram" +"histogramArgMax" +"histogramArgMin" +"histogramArray" +"histogramDistinct" +"histogramForEach" +"histogramIf" +"histogramMap" +"histogramMerge" +"histogramNull" +"histogramOrDefault" +"histogramOrNull" +"histogramResample" +"histogramSimpleState" +"histogramState" "hiveHash" +"hop" +"hopEnd" +"hopStart" +"HOST" "hostname" "hostName" "HOUR" +"HOURS" +"HTTP" "hypot" "ID" +"IDENTIFIED" "identity" +"idnaDecode" +"idnaEncode" "if" -"IF" +"IF EMPTY" "IF EXISTS" "IF NOT EXISTS" "ifNotFinite" "ifNull" "ignore" +"IGNORE NULLS" "ilike" "ILIKE" "in" "IN" "INDEX" +"INDEXES" "indexHint" "indexOf" +"INDICES" "INET4" "INET6" "INET6_ATON" "INET6_NTOA" "INET_ATON" "INET_NTOA" -"INF" +"INHERIT" "inIgnoreSet" +"initcap" +"initcapUTF8" "initializeAggregation" "initial_query_id" "initialQueryID" "INJECTIVE" "INNER" "IN PARTITION" -"INSERT" "INSERT INTO" +"instr" "INT" "INT1" "Int128" @@ -602,12 +1834,31 @@ "INTEGER" "INTEGER SIGNED" "INTEGER UNSIGNED" +"INTERPOLATE" +"INTERSECT" "INTERVAL" "IntervalDay" "IntervalHour" "intervalLengthSum" +"intervalLengthSumArgMax" +"intervalLengthSumArgMin" +"intervalLengthSumArray" +"intervalLengthSumDistinct" +"intervalLengthSumForEach" +"intervalLengthSumIf" +"intervalLengthSumMap" +"intervalLengthSumMerge" +"intervalLengthSumNull" +"intervalLengthSumOrDefault" +"intervalLengthSumOrNull" +"intervalLengthSumResample" +"intervalLengthSumSimpleState" +"intervalLengthSumState" +"IntervalMicrosecond" +"IntervalMillisecond" "IntervalMinute" "IntervalMonth" +"IntervalNanosecond" "IntervalQuarter" "IntervalSecond" "IntervalWeek" @@ -616,21 +1867,26 @@ "intExp2" "intHash32" "intHash64" -"INTO" "INTO OUTFILE" "INT SIGNED" "INT UNSIGNED" +"INVISIBLE" +"INVOKER" +"IP" "IPv4" "IPv4CIDRToRange" "IPv4NumToString" "IPv4NumToStringClassC" "IPv4StringToNum" +"IPv4StringToNumOrDefault" +"IPv4StringToNumOrNull" "IPv4ToIPv6" "IPv6" "IPv6CIDRToRange" "IPv6NumToString" "IPv6StringToNum" -"IS" +"IPv6StringToNumOrDefault" +"IPv6StringToNumOrNull" "isConstant" "isDecimalOverflow" "isFinite" @@ -639,64 +1895,220 @@ "isIPv4String" "isIPv6String" "isNaN" +"isNotDistinctFrom" +"IS NOT DISTINCT FROM" "isNotNull" +"IS NOT NULL" "isNull" +"IS NULL" +"isNullable" "IS_OBJECT_ID" "isValidJSON" "isValidUTF8" "isZeroOrNull" +"jaroSimilarity" +"jaroWinklerSimilarity" "javaHash" "javaHashUTF16LE" "JOIN" "joinGet" "joinGetOrNull" +"JSON" +"JSONArrayLength" +"JSON_ARRAY_LENGTH" "JSON_EXISTS" "JSONExtract" "JSONExtractArrayRaw" "JSONExtractBool" "JSONExtractFloat" "JSONExtractInt" +"JSONExtractKeys" "JSONExtractKeysAndValues" "JSONExtractKeysAndValuesRaw" -"JSONExtractKeys" "JSONExtractRaw" "JSONExtractString" "JSONExtractUInt" "JSONHas" "JSONKey" "JSONLength" +"jsonMergePatch" "JSON_QUERY" "JSONType" "JSON_VALUE" "jumpConsistentHash" +"kafkaMurmurHash" +"KERBEROS" "KEY" -# Key words (based on keywords from antlr parser) +"KEY BY" +"KEYED BY" +"KEYS" "KILL" +"KIND" +"kolmogorovSmirnovTest" +"kolmogorovSmirnovTestArgMax" +"kolmogorovSmirnovTestArgMin" +"kolmogorovSmirnovTestArray" +"kolmogorovSmirnovTestDistinct" +"kolmogorovSmirnovTestForEach" +"kolmogorovSmirnovTestIf" +"kolmogorovSmirnovTestMap" +"kolmogorovSmirnovTestMerge" +"kolmogorovSmirnovTestNull" +"kolmogorovSmirnovTestOrDefault" +"kolmogorovSmirnovTestOrNull" +"kolmogorovSmirnovTestResample" +"kolmogorovSmirnovTestSimpleState" +"kolmogorovSmirnovTestState" +"kostikConsistentHash" +"kql_array_sort_asc" +"kql_array_sort_desc" "kurtPop" +"kurtPopArgMax" +"kurtPopArgMin" +"kurtPopArray" +"kurtPopDistinct" +"kurtPopForEach" +"kurtPopIf" +"kurtPopMap" +"kurtPopMerge" +"kurtPopNull" +"kurtPopOrDefault" +"kurtPopOrNull" +"kurtPopResample" +"kurtPopSimpleState" +"kurtPopState" "kurtSamp" +"kurtSampArgMax" +"kurtSampArgMin" +"kurtSampArray" +"kurtSampDistinct" +"kurtSampForEach" +"kurtSampIf" +"kurtSampMap" +"kurtSampMerge" +"kurtSampNull" +"kurtSampOrDefault" +"kurtSampOrNull" +"kurtSampResample" +"kurtSampSimpleState" +"kurtSampState" +"L1Distance" +"L1Norm" +"L1Normalize" +"L2Distance" +"L2Norm" +"L2Normalize" +"L2SquaredDistance" +"L2SquaredNorm" "lagInFrame" +"lagInFrameArgMax" +"lagInFrameArgMin" +"lagInFrameArray" +"lagInFrameDistinct" +"lagInFrameForEach" +"lagInFrameIf" +"lagInFrameMap" +"lagInFrameMerge" +"lagInFrameNull" +"lagInFrameOrDefault" +"lagInFrameOrNull" +"lagInFrameResample" +"lagInFrameSimpleState" +"lagInFrameState" +"LARGE OBJECT" +"largestTriangleThreeBuckets" +"largestTriangleThreeBucketsArgMax" +"largestTriangleThreeBucketsArgMin" +"largestTriangleThreeBucketsArray" +"largestTriangleThreeBucketsDistinct" +"largestTriangleThreeBucketsForEach" +"largestTriangleThreeBucketsIf" +"largestTriangleThreeBucketsMap" +"largestTriangleThreeBucketsMerge" +"largestTriangleThreeBucketsNull" +"largestTriangleThreeBucketsOrDefault" +"largestTriangleThreeBucketsOrNull" +"largestTriangleThreeBucketsResample" +"largestTriangleThreeBucketsSimpleState" +"largestTriangleThreeBucketsState" "LAST" +"LAST_DAY" "last_value" +"last_valueArgMax" +"last_valueArgMin" +"last_valueArray" +"last_valueDistinct" +"last_valueForEach" +"last_valueIf" +"last_valueMap" +"last_valueMerge" +"last_valueNull" +"last_valueOrDefault" +"last_valueOrNull" +"last_valueResample" +"last_value_respect_nulls" +"last_value_respect_nullsArgMax" +"last_value_respect_nullsArgMin" +"last_value_respect_nullsArray" +"last_value_respect_nullsDistinct" +"last_value_respect_nullsForEach" +"last_value_respect_nullsIf" +"last_value_respect_nullsMap" +"last_value_respect_nullsMerge" +"last_value_respect_nullsNull" +"last_value_respect_nullsOrDefault" +"last_value_respect_nullsOrNull" +"last_value_respect_nullsResample" +"last_value_respect_nullsSimpleState" +"last_value_respect_nullsState" +"last_valueSimpleState" +"last_valueState" "LAYOUT" "lcase" "lcm" +"LDAP" "leadInFrame" +"leadInFrameArgMax" +"leadInFrameArgMin" +"leadInFrameArray" +"leadInFrameDistinct" +"leadInFrameForEach" +"leadInFrameIf" +"leadInFrameMap" +"leadInFrameMerge" +"leadInFrameNull" +"leadInFrameOrDefault" +"leadInFrameOrNull" +"leadInFrameResample" +"leadInFrameSimpleState" +"leadInFrameState" "LEADING" "least" +"left" "LEFT" "LEFT ARRAY JOIN" "leftPad" "leftPadUTF8" +"leftUTF8" "lemmatize" "length" "lengthUTF8" "less" "lessOrEquals" +"LESS THAN" +"LEVEL" +"levenshteinDistance" "lgamma" "LIFETIME" +"LIGHTWEIGHT" "like" "LIKE" "LIMIT" +"LINEAR" +"LinfDistance" +"LinfNorm" +"LinfNormalize" +"LIST" "LIVE" "ln" "LOCAL" @@ -705,7 +2117,6 @@ "log10" "log1p" "log2" -"LOGS" "logTrace" "LONGBLOB" "LONGTEXT" @@ -715,76 +2126,467 @@ "lower" "lowerUTF8" "lpad" +"LpDistance" +"LpNorm" +"LpNormalize" +"ltrim" "LTRIM" +"lttb" +"lttbArgMax" +"lttbArgMin" +"lttbArray" +"lttbDistinct" +"lttbForEach" +"lttbIf" +"lttbMap" +"lttbMerge" +"lttbNull" +"lttbOrDefault" +"lttbOrNull" +"lttbResample" +"lttbSimpleState" +"lttbState" "M" "MACNumToString" "MACStringToNum" "MACStringToOUI" +"makeDate" +"makeDate32" +"makeDateTime" +"makeDateTime64" "mannWhitneyUTest" +"mannWhitneyUTestArgMax" +"mannWhitneyUTestArgMin" +"mannWhitneyUTestArray" +"mannWhitneyUTestDistinct" +"mannWhitneyUTestForEach" +"mannWhitneyUTestIf" +"mannWhitneyUTestMap" +"mannWhitneyUTestMerge" +"mannWhitneyUTestNull" +"mannWhitneyUTestOrDefault" +"mannWhitneyUTestOrNull" +"mannWhitneyUTestResample" +"mannWhitneyUTestSimpleState" +"mannWhitneyUTestState" "map" "Map" "mapAdd" +"mapAll" +"mapApply" +"mapConcat" "mapContains" +"mapContainsKeyLike" +"mapExists" +"mapExtractKeyLike" +"mapFilter" +"mapFromArrays" +"MAP_FROM_ARRAYS" +"mapFromString" "mapKeys" +"mapPartialReverseSort" +"mapPartialSort" "mapPopulateSeries" +"mapReverseSort" +"mapSort" "mapSubtract" +"mapUpdate" "mapValues" "match" +"MATCH" "materialize" "MATERIALIZE" +"MATERIALIZE COLUMN" "MATERIALIZED" "MATERIALIZE INDEX" +"MATERIALIZE PROJECTION" +"MATERIALIZE STATISTIC" "MATERIALIZE TTL" "max" "MAX" +"max2" +"maxArgMax" +"maxArgMin" +"maxArray" +"maxDistinct" +"maxForEach" +"maxIf" "maxIntersections" +"maxIntersectionsArgMax" +"maxIntersectionsArgMin" +"maxIntersectionsArray" +"maxIntersectionsDistinct" +"maxIntersectionsForEach" +"maxIntersectionsIf" +"maxIntersectionsMap" +"maxIntersectionsMerge" +"maxIntersectionsNull" +"maxIntersectionsOrDefault" +"maxIntersectionsOrNull" "maxIntersectionsPosition" +"maxIntersectionsPositionArgMax" +"maxIntersectionsPositionArgMin" +"maxIntersectionsPositionArray" +"maxIntersectionsPositionDistinct" +"maxIntersectionsPositionForEach" +"maxIntersectionsPositionIf" +"maxIntersectionsPositionMap" +"maxIntersectionsPositionMerge" +"maxIntersectionsPositionNull" +"maxIntersectionsPositionOrDefault" +"maxIntersectionsPositionOrNull" +"maxIntersectionsPositionResample" +"maxIntersectionsPositionSimpleState" +"maxIntersectionsPositionState" +"maxIntersectionsResample" +"maxIntersectionsSimpleState" +"maxIntersectionsState" "maxMap" +"maxMappedArrays" +"maxMappedArraysArgMax" +"maxMappedArraysArgMin" +"maxMappedArraysArray" +"maxMappedArraysDistinct" +"maxMappedArraysForEach" +"maxMappedArraysIf" +"maxMappedArraysMap" +"maxMappedArraysMerge" +"maxMappedArraysNull" +"maxMappedArraysOrDefault" +"maxMappedArraysOrNull" +"maxMappedArraysResample" +"maxMappedArraysSimpleState" +"maxMappedArraysState" +"maxMerge" +"maxNull" +"maxOrDefault" +"maxOrNull" +"maxResample" +"maxSimpleState" +"maxState" +"MCS" "MD4" "MD5" +"meanZTest" +"meanZTestArgMax" +"meanZTestArgMin" +"meanZTestArray" +"meanZTestDistinct" +"meanZTestForEach" +"meanZTestIf" +"meanZTestMap" +"meanZTestMerge" +"meanZTestNull" +"meanZTestOrDefault" +"meanZTestOrNull" +"meanZTestResample" +"meanZTestSimpleState" +"meanZTestState" "median" +"medianArgMax" +"medianArgMin" +"medianArray" "medianBFloat16" +"medianBFloat16ArgMax" +"medianBFloat16ArgMin" +"medianBFloat16Array" +"medianBFloat16Distinct" +"medianBFloat16ForEach" +"medianBFloat16If" +"medianBFloat16Map" +"medianBFloat16Merge" +"medianBFloat16Null" +"medianBFloat16OrDefault" +"medianBFloat16OrNull" +"medianBFloat16Resample" +"medianBFloat16SimpleState" +"medianBFloat16State" "medianBFloat16Weighted" +"medianBFloat16WeightedArgMax" +"medianBFloat16WeightedArgMin" +"medianBFloat16WeightedArray" +"medianBFloat16WeightedDistinct" +"medianBFloat16WeightedForEach" +"medianBFloat16WeightedIf" +"medianBFloat16WeightedMap" +"medianBFloat16WeightedMerge" +"medianBFloat16WeightedNull" +"medianBFloat16WeightedOrDefault" +"medianBFloat16WeightedOrNull" +"medianBFloat16WeightedResample" +"medianBFloat16WeightedSimpleState" +"medianBFloat16WeightedState" +"medianDD" +"medianDDArgMax" +"medianDDArgMin" +"medianDDArray" +"medianDDDistinct" +"medianDDForEach" +"medianDDIf" +"medianDDMap" +"medianDDMerge" +"medianDDNull" +"medianDDOrDefault" +"medianDDOrNull" +"medianDDResample" +"medianDDSimpleState" +"medianDDState" "medianDeterministic" +"medianDeterministicArgMax" +"medianDeterministicArgMin" +"medianDeterministicArray" +"medianDeterministicDistinct" +"medianDeterministicForEach" +"medianDeterministicIf" +"medianDeterministicMap" +"medianDeterministicMerge" +"medianDeterministicNull" +"medianDeterministicOrDefault" +"medianDeterministicOrNull" +"medianDeterministicResample" +"medianDeterministicSimpleState" +"medianDeterministicState" +"medianDistinct" "medianExact" +"medianExactArgMax" +"medianExactArgMin" +"medianExactArray" +"medianExactDistinct" +"medianExactForEach" "medianExactHigh" +"medianExactHighArgMax" +"medianExactHighArgMin" +"medianExactHighArray" +"medianExactHighDistinct" +"medianExactHighForEach" +"medianExactHighIf" +"medianExactHighMap" +"medianExactHighMerge" +"medianExactHighNull" +"medianExactHighOrDefault" +"medianExactHighOrNull" +"medianExactHighResample" +"medianExactHighSimpleState" +"medianExactHighState" +"medianExactIf" "medianExactLow" +"medianExactLowArgMax" +"medianExactLowArgMin" +"medianExactLowArray" +"medianExactLowDistinct" +"medianExactLowForEach" +"medianExactLowIf" +"medianExactLowMap" +"medianExactLowMerge" +"medianExactLowNull" +"medianExactLowOrDefault" +"medianExactLowOrNull" +"medianExactLowResample" +"medianExactLowSimpleState" +"medianExactLowState" +"medianExactMap" +"medianExactMerge" +"medianExactNull" +"medianExactOrDefault" +"medianExactOrNull" +"medianExactResample" +"medianExactSimpleState" +"medianExactState" "medianExactWeighted" +"medianExactWeightedArgMax" +"medianExactWeightedArgMin" +"medianExactWeightedArray" +"medianExactWeightedDistinct" +"medianExactWeightedForEach" +"medianExactWeightedIf" +"medianExactWeightedMap" +"medianExactWeightedMerge" +"medianExactWeightedNull" +"medianExactWeightedOrDefault" +"medianExactWeightedOrNull" +"medianExactWeightedResample" +"medianExactWeightedSimpleState" +"medianExactWeightedState" +"medianForEach" +"medianGK" +"medianGKArgMax" +"medianGKArgMin" +"medianGKArray" +"medianGKDistinct" +"medianGKForEach" +"medianGKIf" +"medianGKMap" +"medianGKMerge" +"medianGKNull" +"medianGKOrDefault" +"medianGKOrNull" +"medianGKResample" +"medianGKSimpleState" +"medianGKState" +"medianIf" +"medianInterpolatedWeighted" +"medianInterpolatedWeightedArgMax" +"medianInterpolatedWeightedArgMin" +"medianInterpolatedWeightedArray" +"medianInterpolatedWeightedDistinct" +"medianInterpolatedWeightedForEach" +"medianInterpolatedWeightedIf" +"medianInterpolatedWeightedMap" +"medianInterpolatedWeightedMerge" +"medianInterpolatedWeightedNull" +"medianInterpolatedWeightedOrDefault" +"medianInterpolatedWeightedOrNull" +"medianInterpolatedWeightedResample" +"medianInterpolatedWeightedSimpleState" +"medianInterpolatedWeightedState" +"medianMap" +"medianMerge" +"medianNull" +"medianOrDefault" +"medianOrNull" +"medianResample" +"medianSimpleState" +"medianState" "medianTDigest" +"medianTDigestArgMax" +"medianTDigestArgMin" +"medianTDigestArray" +"medianTDigestDistinct" +"medianTDigestForEach" +"medianTDigestIf" +"medianTDigestMap" +"medianTDigestMerge" +"medianTDigestNull" +"medianTDigestOrDefault" +"medianTDigestOrNull" +"medianTDigestResample" +"medianTDigestSimpleState" +"medianTDigestState" "medianTDigestWeighted" +"medianTDigestWeightedArgMax" +"medianTDigestWeightedArgMin" +"medianTDigestWeightedArray" +"medianTDigestWeightedDistinct" +"medianTDigestWeightedForEach" +"medianTDigestWeightedIf" +"medianTDigestWeightedMap" +"medianTDigestWeightedMerge" +"medianTDigestWeightedNull" +"medianTDigestWeightedOrDefault" +"medianTDigestWeightedOrNull" +"medianTDigestWeightedResample" +"medianTDigestWeightedSimpleState" +"medianTDigestWeightedState" "medianTiming" +"medianTimingArgMax" +"medianTimingArgMin" +"medianTimingArray" +"medianTimingDistinct" +"medianTimingForEach" +"medianTimingIf" +"medianTimingMap" +"medianTimingMerge" +"medianTimingNull" +"medianTimingOrDefault" +"medianTimingOrNull" +"medianTimingResample" +"medianTimingSimpleState" +"medianTimingState" "medianTimingWeighted" +"medianTimingWeightedArgMax" +"medianTimingWeightedArgMin" +"medianTimingWeightedArray" +"medianTimingWeightedDistinct" +"medianTimingWeightedForEach" +"medianTimingWeightedIf" +"medianTimingWeightedMap" +"medianTimingWeightedMerge" +"medianTimingWeightedNull" +"medianTimingWeightedOrDefault" +"medianTimingWeightedOrNull" +"medianTimingWeightedResample" +"medianTimingWeightedSimpleState" +"medianTimingWeightedState" "MEDIUMBLOB" "MEDIUMINT" "MEDIUMINT SIGNED" "MEDIUMINT UNSIGNED" "MEDIUMTEXT" +"MEMORY" "Merge" "MERGES" "metroHash64" "MI" +"MICROSECOND" +"MICROSECONDS" "mid" +"MILLISECOND" +"MILLISECONDS" "min" "MIN" +"min2" +"minArgMax" +"minArgMin" +"minArray" +"minDistinct" +"minForEach" +"minIf" "minMap" +"minMappedArrays" +"minMappedArraysArgMax" +"minMappedArraysArgMin" +"minMappedArraysArray" +"minMappedArraysDistinct" +"minMappedArraysForEach" +"minMappedArraysIf" +"minMappedArraysMap" +"minMappedArraysMerge" +"minMappedArraysNull" +"minMappedArraysOrDefault" +"minMappedArraysOrNull" +"minMappedArraysResample" +"minMappedArraysSimpleState" +"minMappedArraysState" +"minMerge" +"minNull" +"minOrDefault" +"minOrNull" +"minResample" +"minSampleSizeContinous" +"minSampleSizeContinuous" +"minSampleSizeConversion" +"minSimpleState" +"minState" "minus" "MINUTE" +"MINUTES" +"mismatches" "MM" "mod" +"MOD" "MODIFY" "MODIFY COLUMN" +"MODIFY COMMENT" "MODIFY ORDER BY" "MODIFY QUERY" +"MODIFY REFRESH" +"MODIFY SAMPLE BY" "MODIFY SETTING" +"MODIFY SQL SECURITY" "MODIFY TTL" "modulo" "moduloLegacy" "moduloOrZero" "MONTH" +"monthName" +"MONTHS" +"mortonDecode" +"mortonEncode" "MOVE" "MOVE PART" "MOVE PARTITION" "movingXXX" +"MS" "multiFuzzyMatchAllIndices" "multiFuzzyMatchAny" "multiFuzzyMatchAnyIndex" @@ -793,6 +2595,7 @@ "multiMatchAny" "multiMatchAnyIndex" "multiply" +"multiplyDecimal" "MultiPolygon" "multiSearchAllPositions" "multiSearchAllPositionsCaseInsensitive" @@ -818,7 +2621,9 @@ "MUTATION" "N" "NAME" -"NAN_SQL" +"NAMED COLLECTION" +"NANOSECOND" +"NANOSECONDS" "NATIONAL CHAR" "NATIONAL CHARACTER" "NATIONAL CHARACTER LARGE OBJECT" @@ -829,8 +2634,10 @@ "NCHAR VARYING" "negate" "neighbor" +"nested" "Nested" "netloc" +"NEXT" "ngramDistance" "ngramDistanceCaseInsensitive" "ngramDistanceCaseInsensitiveUTF8" @@ -843,6 +2650,7 @@ "ngramMinHashCaseInsensitive" "ngramMinHashCaseInsensitiveUTF8" "ngramMinHashUTF8" +"ngrams" "ngramSearch" "ngramSearchCaseInsensitive" "ngramSearchCaseInsensitiveUTF8" @@ -851,68 +2659,207 @@ "ngramSimHashCaseInsensitive" "ngramSimHashCaseInsensitiveUTF8" "ngramSimHashUTF8" -"NO" +"NO ACTION" "NO DELAY" +"NO LIMITS" "NONE" +"nonNegativeDerivative" +"nonNegativeDerivativeArgMax" +"nonNegativeDerivativeArgMin" +"nonNegativeDerivativeArray" +"nonNegativeDerivativeDistinct" +"nonNegativeDerivativeForEach" +"nonNegativeDerivativeIf" +"nonNegativeDerivativeMap" +"nonNegativeDerivativeMerge" +"nonNegativeDerivativeNull" +"nonNegativeDerivativeOrDefault" +"nonNegativeDerivativeOrNull" +"nonNegativeDerivativeResample" +"nonNegativeDerivativeSimpleState" +"nonNegativeDerivativeState" +"NO_PASSWORD" "normalizedQueryHash" "normalizedQueryHashKeepNames" +"normalizeL1" +"normalizeL2" +"normalizeLinf" +"normalizeLp" "normalizeQuery" "normalizeQueryKeepNames" +"normalizeUTF8NFC" +"normalizeUTF8NFD" +"normalizeUTF8NFKC" +"normalizeUTF8NFKD" +"normL1" +"normL2" +"normL2Squared" +"normLinf" +"normLp" "not" "NOT" +"NOT BETWEEN" "notEmpty" "notEquals" "nothing" "Nothing" +"nothingArgMax" +"nothingArgMin" +"nothingArray" +"nothingDistinct" +"nothingForEach" +"nothingIf" +"nothingMap" +"nothingMerge" +"nothingNull" +"nothingNullArgMax" +"nothingNullArgMin" +"nothingNullArray" +"nothingNullDistinct" +"nothingNullForEach" +"nothingNullIf" +"nothingNullMap" +"nothingNullMerge" +"nothingNullNull" +"nothingNullOrDefault" +"nothingNullOrNull" +"nothingNullResample" +"nothingNullSimpleState" +"nothingNullState" +"nothingOrDefault" +"nothingOrNull" +"nothingResample" +"nothingSimpleState" +"nothingState" +"nothingUInt64" +"nothingUInt64ArgMax" +"nothingUInt64ArgMin" +"nothingUInt64Array" +"nothingUInt64Distinct" +"nothingUInt64ForEach" +"nothingUInt64If" +"nothingUInt64Map" +"nothingUInt64Merge" +"nothingUInt64Null" +"nothingUInt64OrDefault" +"nothingUInt64OrNull" +"nothingUInt64Resample" +"nothingUInt64SimpleState" +"nothingUInt64State" +"NOT IDENTIFIED" "notILike" +"NOT ILIKE" "notIn" +"NOT IN" "notInIgnoreSet" +"NOT KEYED" "notLike" +"NOT LIKE" "notNullIn" "notNullInIgnoreSet" +"NOT OVERRIDABLE" "now" "now64" +"nowInBlock" +"NS" +"nth_value" +"nth_valueArgMax" +"nth_valueArgMin" +"nth_valueArray" +"nth_valueDistinct" +"nth_valueForEach" +"nth_valueIf" +"nth_valueMap" +"nth_valueMerge" +"nth_valueNull" +"nth_valueOrDefault" +"nth_valueOrNull" +"nth_valueResample" +"nth_valueSimpleState" +"nth_valueState" +"ntile" +"ntileArgMax" +"ntileArgMin" +"ntileArray" +"ntileDistinct" +"ntileForEach" +"ntileIf" +"ntileMap" +"ntileMerge" +"ntileNull" +"ntileOrDefault" +"ntileOrNull" +"ntileResample" +"ntileSimpleState" +"ntileState" "Null" +"NULL" "Nullable" "nullIf" "nullIn" "nullInIgnoreSet" "NULLS" -"NULL_SQL" "NUMERIC" "NVARCHAR" +"Object" +"OCTET_LENGTH" "OFFSET" "ON" +"ON DELETE" "ONLY" -"OPTIMIZE" +"ON UPDATE" +"ON VOLUME" "OPTIMIZE TABLE" "or" "OR" -"ORDER" "ORDER BY" "OR REPLACE" "OUTER" -"OUTFILE" +"OVER" +"OVERRIDABLE" +"parseDateTime" "parseDateTime32BestEffort" "parseDateTime32BestEffortOrNull" "parseDateTime32BestEffortOrZero" "parseDateTime64BestEffort" "parseDateTime64BestEffortOrNull" "parseDateTime64BestEffortOrZero" +"parseDateTime64BestEffortUS" +"parseDateTime64BestEffortUSOrNull" +"parseDateTime64BestEffortUSOrZero" "parseDateTimeBestEffort" "parseDateTimeBestEffortOrNull" "parseDateTimeBestEffortOrZero" "parseDateTimeBestEffortUS" "parseDateTimeBestEffortUSOrNull" "parseDateTimeBestEffortUSOrZero" +"parseDateTimeInJodaSyntax" +"parseDateTimeInJodaSyntaxOrNull" +"parseDateTimeInJodaSyntaxOrZero" +"parseDateTimeOrNull" +"parseDateTimeOrZero" "parseTimeDelta" +"PART" +"PARTIAL" "PARTITION" "PARTITION BY" "partitionId" +"PARTITIONS" +"PART_MOVE_TO_SHARD" +"PASTE" "path" "pathFull" +"PERIODIC REFRESH" +"PERMANENTLY" +"PERMISSIVE" +"PERSISTENT" +"PERSISTENT SEQUENTIAL" "pi" +"PIPELINE" +"PLAINTEXT_PASSWORD" +"PLAN" "plus" +"pmod" "Point" "pointInEllipses" "pointInPolygon" @@ -935,71 +2882,620 @@ "polygonsWithinSpherical" "POPULATE" "port" +"portRFC" "position" "positionCaseInsensitive" "positionCaseInsensitiveUTF8" "positionUTF8" +"positive_modulo" +"positiveModulo" "pow" "power" +"PRECEDING" +"PRECISION" "PREWHERE" "PRIMARY" "PRIMARY KEY" +"PROFILE" "PROJECTION" +"proportionsZTest" +"Protobuf" "protocol" +"PULL" +"punycodeDecode" +"punycodeEncode" "Q" "QQ" "quantile" +"quantileArgMax" +"quantileArgMin" +"quantileArray" "quantileBFloat16" +"quantileBFloat16ArgMax" +"quantileBFloat16ArgMin" +"quantileBFloat16Array" +"quantileBFloat16Distinct" +"quantileBFloat16ForEach" +"quantileBFloat16If" +"quantileBFloat16Map" +"quantileBFloat16Merge" +"quantileBFloat16Null" +"quantileBFloat16OrDefault" +"quantileBFloat16OrNull" +"quantileBFloat16Resample" +"quantileBFloat16SimpleState" +"quantileBFloat16State" "quantileBFloat16Weighted" +"quantileBFloat16WeightedArgMax" +"quantileBFloat16WeightedArgMin" +"quantileBFloat16WeightedArray" +"quantileBFloat16WeightedDistinct" +"quantileBFloat16WeightedForEach" +"quantileBFloat16WeightedIf" +"quantileBFloat16WeightedMap" +"quantileBFloat16WeightedMerge" +"quantileBFloat16WeightedNull" +"quantileBFloat16WeightedOrDefault" +"quantileBFloat16WeightedOrNull" +"quantileBFloat16WeightedResample" +"quantileBFloat16WeightedSimpleState" +"quantileBFloat16WeightedState" +"quantileDD" +"quantileDDArgMax" +"quantileDDArgMin" +"quantileDDArray" +"quantileDDDistinct" +"quantileDDForEach" +"quantileDDIf" +"quantileDDMap" +"quantileDDMerge" +"quantileDDNull" +"quantileDDOrDefault" +"quantileDDOrNull" +"quantileDDResample" +"quantileDDSimpleState" +"quantileDDState" "quantileDeterministic" +"quantileDeterministicArgMax" +"quantileDeterministicArgMin" +"quantileDeterministicArray" +"quantileDeterministicDistinct" +"quantileDeterministicForEach" +"quantileDeterministicIf" +"quantileDeterministicMap" +"quantileDeterministicMerge" +"quantileDeterministicNull" +"quantileDeterministicOrDefault" +"quantileDeterministicOrNull" +"quantileDeterministicResample" +"quantileDeterministicSimpleState" +"quantileDeterministicState" +"quantileDistinct" "quantileExact" +"quantileExactArgMax" +"quantileExactArgMin" +"quantileExactArray" +"quantileExactDistinct" "quantileExactExclusive" +"quantileExactExclusiveArgMax" +"quantileExactExclusiveArgMin" +"quantileExactExclusiveArray" +"quantileExactExclusiveDistinct" +"quantileExactExclusiveForEach" +"quantileExactExclusiveIf" +"quantileExactExclusiveMap" +"quantileExactExclusiveMerge" +"quantileExactExclusiveNull" +"quantileExactExclusiveOrDefault" +"quantileExactExclusiveOrNull" +"quantileExactExclusiveResample" +"quantileExactExclusiveSimpleState" +"quantileExactExclusiveState" +"quantileExactForEach" "quantileExactHigh" +"quantileExactHighArgMax" +"quantileExactHighArgMin" +"quantileExactHighArray" +"quantileExactHighDistinct" +"quantileExactHighForEach" +"quantileExactHighIf" +"quantileExactHighMap" +"quantileExactHighMerge" +"quantileExactHighNull" +"quantileExactHighOrDefault" +"quantileExactHighOrNull" +"quantileExactHighResample" +"quantileExactHighSimpleState" +"quantileExactHighState" +"quantileExactIf" "quantileExactInclusive" +"quantileExactInclusiveArgMax" +"quantileExactInclusiveArgMin" +"quantileExactInclusiveArray" +"quantileExactInclusiveDistinct" +"quantileExactInclusiveForEach" +"quantileExactInclusiveIf" +"quantileExactInclusiveMap" +"quantileExactInclusiveMerge" +"quantileExactInclusiveNull" +"quantileExactInclusiveOrDefault" +"quantileExactInclusiveOrNull" +"quantileExactInclusiveResample" +"quantileExactInclusiveSimpleState" +"quantileExactInclusiveState" "quantileExactLow" +"quantileExactLowArgMax" +"quantileExactLowArgMin" +"quantileExactLowArray" +"quantileExactLowDistinct" +"quantileExactLowForEach" +"quantileExactLowIf" +"quantileExactLowMap" +"quantileExactLowMerge" +"quantileExactLowNull" +"quantileExactLowOrDefault" +"quantileExactLowOrNull" +"quantileExactLowResample" +"quantileExactLowSimpleState" +"quantileExactLowState" +"quantileExactMap" +"quantileExactMerge" +"quantileExactNull" +"quantileExactOrDefault" +"quantileExactOrNull" +"quantileExactResample" +"quantileExactSimpleState" +"quantileExactState" "quantileExactWeighted" +"quantileExactWeightedArgMax" +"quantileExactWeightedArgMin" +"quantileExactWeightedArray" +"quantileExactWeightedDistinct" +"quantileExactWeightedForEach" +"quantileExactWeightedIf" +"quantileExactWeightedMap" +"quantileExactWeightedMerge" +"quantileExactWeightedNull" +"quantileExactWeightedOrDefault" +"quantileExactWeightedOrNull" +"quantileExactWeightedResample" +"quantileExactWeightedSimpleState" +"quantileExactWeightedState" +"quantileForEach" +"quantileGK" +"quantileGKArgMax" +"quantileGKArgMin" +"quantileGKArray" +"quantileGKDistinct" +"quantileGKForEach" +"quantileGKIf" +"quantileGKMap" +"quantileGKMerge" +"quantileGKNull" +"quantileGKOrDefault" +"quantileGKOrNull" +"quantileGKResample" +"quantileGKSimpleState" +"quantileGKState" +"quantileIf" +"quantileInterpolatedWeighted" +"quantileInterpolatedWeightedArgMax" +"quantileInterpolatedWeightedArgMin" +"quantileInterpolatedWeightedArray" +"quantileInterpolatedWeightedDistinct" +"quantileInterpolatedWeightedForEach" +"quantileInterpolatedWeightedIf" +"quantileInterpolatedWeightedMap" +"quantileInterpolatedWeightedMerge" +"quantileInterpolatedWeightedNull" +"quantileInterpolatedWeightedOrDefault" +"quantileInterpolatedWeightedOrNull" +"quantileInterpolatedWeightedResample" +"quantileInterpolatedWeightedSimpleState" +"quantileInterpolatedWeightedState" +"quantileMap" +"quantileMerge" +"quantileNull" +"quantileOrDefault" +"quantileOrNull" +"quantileResample" "quantiles" +"quantilesArgMax" +"quantilesArgMin" +"quantilesArray" "quantilesBFloat16" +"quantilesBFloat16ArgMax" +"quantilesBFloat16ArgMin" +"quantilesBFloat16Array" +"quantilesBFloat16Distinct" +"quantilesBFloat16ForEach" +"quantilesBFloat16If" +"quantilesBFloat16Map" +"quantilesBFloat16Merge" +"quantilesBFloat16Null" +"quantilesBFloat16OrDefault" +"quantilesBFloat16OrNull" +"quantilesBFloat16Resample" +"quantilesBFloat16SimpleState" +"quantilesBFloat16State" "quantilesBFloat16Weighted" +"quantilesBFloat16WeightedArgMax" +"quantilesBFloat16WeightedArgMin" +"quantilesBFloat16WeightedArray" +"quantilesBFloat16WeightedDistinct" +"quantilesBFloat16WeightedForEach" +"quantilesBFloat16WeightedIf" +"quantilesBFloat16WeightedMap" +"quantilesBFloat16WeightedMerge" +"quantilesBFloat16WeightedNull" +"quantilesBFloat16WeightedOrDefault" +"quantilesBFloat16WeightedOrNull" +"quantilesBFloat16WeightedResample" +"quantilesBFloat16WeightedSimpleState" +"quantilesBFloat16WeightedState" +"quantilesDD" +"quantilesDDArgMax" +"quantilesDDArgMin" +"quantilesDDArray" +"quantilesDDDistinct" +"quantilesDDForEach" +"quantilesDDIf" +"quantilesDDMap" +"quantilesDDMerge" +"quantilesDDNull" +"quantilesDDOrDefault" +"quantilesDDOrNull" +"quantilesDDResample" +"quantilesDDSimpleState" +"quantilesDDState" "quantilesDeterministic" +"quantilesDeterministicArgMax" +"quantilesDeterministicArgMin" +"quantilesDeterministicArray" +"quantilesDeterministicDistinct" +"quantilesDeterministicForEach" +"quantilesDeterministicIf" +"quantilesDeterministicMap" +"quantilesDeterministicMerge" +"quantilesDeterministicNull" +"quantilesDeterministicOrDefault" +"quantilesDeterministicOrNull" +"quantilesDeterministicResample" +"quantilesDeterministicSimpleState" +"quantilesDeterministicState" +"quantilesDistinct" "quantilesExact" +"quantilesExactArgMax" +"quantilesExactArgMin" +"quantilesExactArray" +"quantilesExactDistinct" "quantilesExactExclusive" +"quantilesExactExclusiveArgMax" +"quantilesExactExclusiveArgMin" +"quantilesExactExclusiveArray" +"quantilesExactExclusiveDistinct" +"quantilesExactExclusiveForEach" +"quantilesExactExclusiveIf" +"quantilesExactExclusiveMap" +"quantilesExactExclusiveMerge" +"quantilesExactExclusiveNull" +"quantilesExactExclusiveOrDefault" +"quantilesExactExclusiveOrNull" +"quantilesExactExclusiveResample" +"quantilesExactExclusiveSimpleState" +"quantilesExactExclusiveState" +"quantilesExactForEach" "quantilesExactHigh" +"quantilesExactHighArgMax" +"quantilesExactHighArgMin" +"quantilesExactHighArray" +"quantilesExactHighDistinct" +"quantilesExactHighForEach" +"quantilesExactHighIf" +"quantilesExactHighMap" +"quantilesExactHighMerge" +"quantilesExactHighNull" +"quantilesExactHighOrDefault" +"quantilesExactHighOrNull" +"quantilesExactHighResample" +"quantilesExactHighSimpleState" +"quantilesExactHighState" +"quantilesExactIf" "quantilesExactInclusive" +"quantilesExactInclusiveArgMax" +"quantilesExactInclusiveArgMin" +"quantilesExactInclusiveArray" +"quantilesExactInclusiveDistinct" +"quantilesExactInclusiveForEach" +"quantilesExactInclusiveIf" +"quantilesExactInclusiveMap" +"quantilesExactInclusiveMerge" +"quantilesExactInclusiveNull" +"quantilesExactInclusiveOrDefault" +"quantilesExactInclusiveOrNull" +"quantilesExactInclusiveResample" +"quantilesExactInclusiveSimpleState" +"quantilesExactInclusiveState" "quantilesExactLow" +"quantilesExactLowArgMax" +"quantilesExactLowArgMin" +"quantilesExactLowArray" +"quantilesExactLowDistinct" +"quantilesExactLowForEach" +"quantilesExactLowIf" +"quantilesExactLowMap" +"quantilesExactLowMerge" +"quantilesExactLowNull" +"quantilesExactLowOrDefault" +"quantilesExactLowOrNull" +"quantilesExactLowResample" +"quantilesExactLowSimpleState" +"quantilesExactLowState" +"quantilesExactMap" +"quantilesExactMerge" +"quantilesExactNull" +"quantilesExactOrDefault" +"quantilesExactOrNull" +"quantilesExactResample" +"quantilesExactSimpleState" +"quantilesExactState" "quantilesExactWeighted" +"quantilesExactWeightedArgMax" +"quantilesExactWeightedArgMin" +"quantilesExactWeightedArray" +"quantilesExactWeightedDistinct" +"quantilesExactWeightedForEach" +"quantilesExactWeightedIf" +"quantilesExactWeightedMap" +"quantilesExactWeightedMerge" +"quantilesExactWeightedNull" +"quantilesExactWeightedOrDefault" +"quantilesExactWeightedOrNull" +"quantilesExactWeightedResample" +"quantilesExactWeightedSimpleState" +"quantilesExactWeightedState" +"quantilesForEach" +"quantilesGK" +"quantilesGKArgMax" +"quantilesGKArgMin" +"quantilesGKArray" +"quantilesGKDistinct" +"quantilesGKForEach" +"quantilesGKIf" +"quantilesGKMap" +"quantilesGKMerge" +"quantilesGKNull" +"quantilesGKOrDefault" +"quantilesGKOrNull" +"quantilesGKResample" +"quantilesGKSimpleState" +"quantilesGKState" +"quantilesIf" +"quantileSimpleState" +"quantilesInterpolatedWeighted" +"quantilesInterpolatedWeightedArgMax" +"quantilesInterpolatedWeightedArgMin" +"quantilesInterpolatedWeightedArray" +"quantilesInterpolatedWeightedDistinct" +"quantilesInterpolatedWeightedForEach" +"quantilesInterpolatedWeightedIf" +"quantilesInterpolatedWeightedMap" +"quantilesInterpolatedWeightedMerge" +"quantilesInterpolatedWeightedNull" +"quantilesInterpolatedWeightedOrDefault" +"quantilesInterpolatedWeightedOrNull" +"quantilesInterpolatedWeightedResample" +"quantilesInterpolatedWeightedSimpleState" +"quantilesInterpolatedWeightedState" +"quantilesMap" +"quantilesMerge" +"quantilesNull" +"quantilesOrDefault" +"quantilesOrNull" +"quantilesResample" +"quantilesSimpleState" +"quantilesState" +"quantileState" "quantilesTDigest" +"quantilesTDigestArgMax" +"quantilesTDigestArgMin" +"quantilesTDigestArray" +"quantilesTDigestDistinct" +"quantilesTDigestForEach" +"quantilesTDigestIf" +"quantilesTDigestMap" +"quantilesTDigestMerge" +"quantilesTDigestNull" +"quantilesTDigestOrDefault" +"quantilesTDigestOrNull" +"quantilesTDigestResample" +"quantilesTDigestSimpleState" +"quantilesTDigestState" "quantilesTDigestWeighted" +"quantilesTDigestWeightedArgMax" +"quantilesTDigestWeightedArgMin" +"quantilesTDigestWeightedArray" +"quantilesTDigestWeightedDistinct" +"quantilesTDigestWeightedForEach" +"quantilesTDigestWeightedIf" +"quantilesTDigestWeightedMap" +"quantilesTDigestWeightedMerge" +"quantilesTDigestWeightedNull" +"quantilesTDigestWeightedOrDefault" +"quantilesTDigestWeightedOrNull" +"quantilesTDigestWeightedResample" +"quantilesTDigestWeightedSimpleState" +"quantilesTDigestWeightedState" "quantilesTiming" +"quantilesTimingArgMax" +"quantilesTimingArgMin" +"quantilesTimingArray" +"quantilesTimingDistinct" +"quantilesTimingForEach" +"quantilesTimingIf" +"quantilesTimingMap" +"quantilesTimingMerge" +"quantilesTimingNull" +"quantilesTimingOrDefault" +"quantilesTimingOrNull" +"quantilesTimingResample" +"quantilesTimingSimpleState" +"quantilesTimingState" "quantilesTimingWeighted" +"quantilesTimingWeightedArgMax" +"quantilesTimingWeightedArgMin" +"quantilesTimingWeightedArray" +"quantilesTimingWeightedDistinct" +"quantilesTimingWeightedForEach" +"quantilesTimingWeightedIf" +"quantilesTimingWeightedMap" +"quantilesTimingWeightedMerge" +"quantilesTimingWeightedNull" +"quantilesTimingWeightedOrDefault" +"quantilesTimingWeightedOrNull" +"quantilesTimingWeightedResample" +"quantilesTimingWeightedSimpleState" +"quantilesTimingWeightedState" "quantileTDigest" +"quantileTDigestArgMax" +"quantileTDigestArgMin" +"quantileTDigestArray" +"quantileTDigestDistinct" +"quantileTDigestForEach" +"quantileTDigestIf" +"quantileTDigestMap" +"quantileTDigestMerge" +"quantileTDigestNull" +"quantileTDigestOrDefault" +"quantileTDigestOrNull" +"quantileTDigestResample" +"quantileTDigestSimpleState" +"quantileTDigestState" "quantileTDigestWeighted" +"quantileTDigestWeightedArgMax" +"quantileTDigestWeightedArgMin" +"quantileTDigestWeightedArray" +"quantileTDigestWeightedDistinct" +"quantileTDigestWeightedForEach" +"quantileTDigestWeightedIf" +"quantileTDigestWeightedMap" +"quantileTDigestWeightedMerge" +"quantileTDigestWeightedNull" +"quantileTDigestWeightedOrDefault" +"quantileTDigestWeightedOrNull" +"quantileTDigestWeightedResample" +"quantileTDigestWeightedSimpleState" +"quantileTDigestWeightedState" "quantileTiming" +"quantileTimingArgMax" +"quantileTimingArgMin" +"quantileTimingArray" +"quantileTimingDistinct" +"quantileTimingForEach" +"quantileTimingIf" +"quantileTimingMap" +"quantileTimingMerge" +"quantileTimingNull" +"quantileTimingOrDefault" +"quantileTimingOrNull" +"quantileTimingResample" +"quantileTimingSimpleState" +"quantileTimingState" "quantileTimingWeighted" +"quantileTimingWeightedArgMax" +"quantileTimingWeightedArgMin" +"quantileTimingWeightedArray" +"quantileTimingWeightedDistinct" +"quantileTimingWeightedForEach" +"quantileTimingWeightedIf" +"quantileTimingWeightedMap" +"quantileTimingWeightedMerge" +"quantileTimingWeightedNull" +"quantileTimingWeightedOrDefault" +"quantileTimingWeightedOrNull" +"quantileTimingWeightedResample" +"quantileTimingWeightedSimpleState" +"quantileTimingWeightedState" "QUARTER" +"QUARTERS" +"QUERY" "query_id" "queryID" "queryString" "queryStringAndFragment" +"QUERY TREE" +"QUOTA" +"radians" "rand" "rand32" "rand64" +"randBernoulli" +"randBinomial" +"randCanonical" +"randChiSquared" "randConstant" +"randExponential" +"randFisherF" +"randLogNormal" +"randNegativeBinomial" +"randNormal" "randomFixedString" +"RANDOMIZED" +"RANDOMIZE FOR" "randomPrintableASCII" "randomString" "randomStringUTF8" +"randPoisson" +"randStudentT" +"randUniform" "range" "RANGE" "rank" +"rankArgMax" +"rankArgMin" +"rankArray" "rankCorr" +"rankCorrArgMax" +"rankCorrArgMin" +"rankCorrArray" +"rankCorrDistinct" +"rankCorrForEach" +"rankCorrIf" +"rankCorrMap" +"rankCorrMerge" +"rankCorrNull" +"rankCorrOrDefault" +"rankCorrOrNull" +"rankCorrResample" +"rankCorrSimpleState" +"rankCorrState" +"rankDistinct" +"rankForEach" +"rankIf" +"rankMap" +"rankMerge" +"rankNull" +"rankOrDefault" +"rankOrNull" +"rankResample" +"rankSimpleState" +"rankState" +"READONLY" "readWKTMultiPolygon" "readWKTPoint" "readWKTPolygon" "readWKTRing" "REAL" +"REALM" +"RECOMPRESS" +"REFERENCES" "REFRESH" +"REGEXP" +"regexpExtract" +"REGEXP_EXTRACT" +"REGEXP_MATCHES" "regexpQuoteMeta" +"REGEXP_REPLACE" "regionHierarchy" "regionIn" "regionToArea" @@ -1030,11 +3526,15 @@ "reinterpretAsUInt64" "reinterpretAsUInt8" "reinterpretAsUUID" -"RELOAD" "REMOVE" +"REMOVE SAMPLE BY" +"REMOVE TTL" "RENAME" "RENAME COLUMN" +"RENAME DATABASE" +"RENAME DICTIONARY" "RENAME TABLE" +"RENAME TO" "repeat" "replace" "REPLACE" @@ -1043,18 +3543,40 @@ "REPLACE PARTITION" "replaceRegexpAll" "replaceRegexpOne" -"REPLICA" "replicate" -"REPLICATED" "Resample" +"RESET SETTING" +"RESPECT NULLS" +"RESTORE" +"RESTRICT" +"RESTRICTIVE" "RESUME" "retention" +"retentionArgMax" +"retentionArgMin" +"retentionArray" +"retentionDistinct" +"retentionForEach" +"retentionIf" +"retentionMap" +"retentionMerge" +"retentionNull" +"retentionOrDefault" +"retentionOrNull" +"retentionResample" +"retentionSimpleState" +"retentionState" "reverse" "reverseUTF8" +"revision" +"REVOKE" +"right" "RIGHT" "rightPad" "rightPadUTF8" +"rightUTF8" "Ring" +"ROLLBACK" "ROLLUP" "round" "roundAge" @@ -1062,10 +3584,27 @@ "roundDown" "roundDuration" "roundToExp2" +"ROW" "row_number" +"row_numberArgMax" +"row_numberArgMin" +"row_numberArray" +"row_numberDistinct" +"row_numberForEach" +"row_numberIf" "rowNumberInAllBlocks" "rowNumberInBlock" +"row_numberMap" +"row_numberMerge" +"row_numberNull" +"row_numberOrDefault" +"row_numberOrNull" +"row_numberResample" +"row_numberSimpleState" +"row_numberState" +"ROWS" "rpad" +"rtrim" "RTRIM" "runningAccumulate" "runningConcurrency" @@ -1081,29 +3620,103 @@ "s2RectIntersection" "s2RectUnion" "s2ToGeo" +"S3" +"SALT" "SAMPLE" "SAMPLE BY" +"scalarProduct" +"__scalarSubqueryResult" +"SCHEMA" +"SCHEME" "SECOND" +"SECONDS" "SELECT" "SEMI" -"SENDS" "sequenceCount" +"sequenceCountArgMax" +"sequenceCountArgMin" +"sequenceCountArray" +"sequenceCountDistinct" +"sequenceCountForEach" +"sequenceCountIf" +"sequenceCountMap" +"sequenceCountMerge" +"sequenceCountNull" +"sequenceCountOrDefault" +"sequenceCountOrNull" +"sequenceCountResample" +"sequenceCountSimpleState" +"sequenceCountState" "sequenceMatch" +"sequenceMatchArgMax" +"sequenceMatchArgMin" +"sequenceMatchArray" +"sequenceMatchDistinct" +"sequenceMatchForEach" +"sequenceMatchIf" +"sequenceMatchMap" +"sequenceMatchMerge" +"sequenceMatchNull" +"sequenceMatchOrDefault" +"sequenceMatchOrNull" +"sequenceMatchResample" +"sequenceMatchSimpleState" +"sequenceMatchState" "sequenceNextNode" +"sequenceNextNodeArgMax" +"sequenceNextNodeArgMin" +"sequenceNextNodeArray" +"sequenceNextNodeDistinct" +"sequenceNextNodeForEach" +"sequenceNextNodeIf" +"sequenceNextNodeMap" +"sequenceNextNodeMerge" +"sequenceNextNodeNull" +"sequenceNextNodeOrDefault" +"sequenceNextNodeOrNull" +"sequenceNextNodeResample" +"sequenceNextNodeSimpleState" +"sequenceNextNodeState" +"seriesDecomposeSTL" +"seriesOutliersDetectTukey" +"seriesPeriodDetectFFT" +"SERVER" +"serverTimezone" +"serverTimeZone" "serverUUID" "SET" +"SET DEFAULT" +"SET DEFAULT ROLE" +"SET FAKE TIME" +"SET NULL" +"SET ROLE" +"SET ROLE DEFAULT" "SETTINGS" +"SET TRANSACTION SNAPSHOT" "SHA1" "SHA224" "SHA256" +"SHA256_HASH" +"SHA256_PASSWORD" "SHA384" "SHA512" +"SHA512_256" "shardCount" "shardNum" "SHOW" +"SHOW ACCESS" +"showCertificate" +"SHOW CREATE" +"SHOW ENGINES" +"SHOW FUNCTIONS" +"SHOW GRANTS" +"SHOW PRIVILEGES" "SHOW PROCESSLIST" +"SHOW SETTING" "sigmoid" "sign" +"SIGNED" +"SIMPLE" "SimpleAggregateFunction" "simpleJSONExtractBool" "simpleJSONExtractFloat" @@ -1113,14 +3726,74 @@ "simpleJSONExtractUInt" "simpleJSONHas" "simpleLinearRegression" +"simpleLinearRegressionArgMax" +"simpleLinearRegressionArgMin" +"simpleLinearRegressionArray" +"simpleLinearRegressionDistinct" +"simpleLinearRegressionForEach" +"simpleLinearRegressionIf" +"simpleLinearRegressionMap" +"simpleLinearRegressionMerge" +"simpleLinearRegressionNull" +"simpleLinearRegressionOrDefault" +"simpleLinearRegressionOrNull" +"simpleLinearRegressionResample" +"simpleLinearRegressionSimpleState" +"simpleLinearRegressionState" "sin" "SINGLE" "singleValueOrNull" +"singleValueOrNullArgMax" +"singleValueOrNullArgMin" +"singleValueOrNullArray" +"singleValueOrNullDistinct" +"singleValueOrNullForEach" +"singleValueOrNullIf" +"singleValueOrNullMap" +"singleValueOrNullMerge" +"singleValueOrNullNull" +"singleValueOrNullOrDefault" +"singleValueOrNullOrNull" +"singleValueOrNullResample" +"singleValueOrNullSimpleState" +"singleValueOrNullState" "sinh" "sipHash128" +"sipHash128Keyed" +"sipHash128Reference" +"sipHash128ReferenceKeyed" "sipHash64" +"sipHash64Keyed" "skewPop" +"skewPopArgMax" +"skewPopArgMin" +"skewPopArray" +"skewPopDistinct" +"skewPopForEach" +"skewPopIf" +"skewPopMap" +"skewPopMerge" +"skewPopNull" +"skewPopOrDefault" +"skewPopOrNull" +"skewPopResample" +"skewPopSimpleState" +"skewPopState" "skewSamp" +"skewSampArgMax" +"skewSampArgMin" +"skewSampArray" +"skewSampDistinct" +"skewSampForEach" +"skewSampIf" +"skewSampMap" +"skewSampMerge" +"skewSampNull" +"skewSampOrDefault" +"skewSampOrNull" +"skewSampResample" +"skewSampSimpleState" +"skewSampState" "sleep" "sleepEachRow" "SMALLINT" @@ -1128,61 +3801,377 @@ "SMALLINT UNSIGNED" "snowflakeToDateTime" "snowflakeToDateTime64" +"soundex" "SOURCE" +"space" "sparkbar" +"sparkBar" +"sparkbarArgMax" +"sparkBarArgMax" +"sparkbarArgMin" +"sparkBarArgMin" +"sparkbarArray" +"sparkBarArray" +"sparkbarDistinct" +"sparkBarDistinct" +"sparkbarForEach" +"sparkBarForEach" +"sparkbarIf" +"sparkBarIf" +"sparkbarMap" +"sparkBarMap" +"sparkbarMerge" +"sparkBarMerge" +"sparkbarNull" +"sparkBarNull" +"sparkbarOrDefault" +"sparkBarOrDefault" +"sparkbarOrNull" +"sparkBarOrNull" +"sparkbarResample" +"sparkBarResample" +"sparkbarSimpleState" +"sparkBarSimpleState" +"sparkbarState" +"sparkBarState" +"SPATIAL" +"splitByAlpha" "splitByChar" "splitByNonAlpha" "splitByRegexp" "splitByString" "splitByWhitespace" +"sqid" +"sqidDecode" +"sqidEncode" +"SQL SECURITY" "SQL_TSI_DAY" "SQL_TSI_HOUR" +"SQL_TSI_MICROSECOND" +"SQL_TSI_MILLISECOND" "SQL_TSI_MINUTE" "SQL_TSI_MONTH" +"SQL_TSI_NANOSECOND" "SQL_TSI_QUARTER" "SQL_TSI_SECOND" "SQL_TSI_WEEK" "SQL_TSI_YEAR" "sqrt" "SS" -"START" +"SSH_KEY" +"SSL_CERTIFICATE" "startsWith" +"startsWithUTF8" "State" +"STATISTIC" +"STD" +"STDArgMax" +"STDArgMin" +"STDArray" "stddevPop" "STDDEV_POP" +"stddevPopArgMax" +"STDDEV_POPArgMax" +"stddevPopArgMin" +"STDDEV_POPArgMin" +"stddevPopArray" +"STDDEV_POPArray" +"stddevPopDistinct" +"STDDEV_POPDistinct" +"stddevPopForEach" +"STDDEV_POPForEach" +"stddevPopIf" +"STDDEV_POPIf" +"stddevPopMap" +"STDDEV_POPMap" +"stddevPopMerge" +"STDDEV_POPMerge" +"stddevPopNull" +"STDDEV_POPNull" +"stddevPopOrDefault" +"STDDEV_POPOrDefault" +"stddevPopOrNull" +"STDDEV_POPOrNull" +"stddevPopResample" +"STDDEV_POPResample" +"stddevPopSimpleState" +"STDDEV_POPSimpleState" "stddevPopStable" +"stddevPopStableArgMax" +"stddevPopStableArgMin" +"stddevPopStableArray" +"stddevPopStableDistinct" +"stddevPopStableForEach" +"stddevPopStableIf" +"stddevPopStableMap" +"stddevPopStableMerge" +"stddevPopStableNull" +"stddevPopStableOrDefault" +"stddevPopStableOrNull" +"stddevPopStableResample" +"stddevPopStableSimpleState" +"stddevPopStableState" +"stddevPopState" +"STDDEV_POPState" "stddevSamp" "STDDEV_SAMP" +"stddevSampArgMax" +"STDDEV_SAMPArgMax" +"stddevSampArgMin" +"STDDEV_SAMPArgMin" +"stddevSampArray" +"STDDEV_SAMPArray" +"stddevSampDistinct" +"STDDEV_SAMPDistinct" +"stddevSampForEach" +"STDDEV_SAMPForEach" +"stddevSampIf" +"STDDEV_SAMPIf" +"stddevSampMap" +"STDDEV_SAMPMap" +"stddevSampMerge" +"STDDEV_SAMPMerge" +"stddevSampNull" +"STDDEV_SAMPNull" +"stddevSampOrDefault" +"STDDEV_SAMPOrDefault" +"stddevSampOrNull" +"STDDEV_SAMPOrNull" +"stddevSampResample" +"STDDEV_SAMPResample" +"stddevSampSimpleState" +"STDDEV_SAMPSimpleState" "stddevSampStable" +"stddevSampStableArgMax" +"stddevSampStableArgMin" +"stddevSampStableArray" +"stddevSampStableDistinct" +"stddevSampStableForEach" +"stddevSampStableIf" +"stddevSampStableMap" +"stddevSampStableMerge" +"stddevSampStableNull" +"stddevSampStableOrDefault" +"stddevSampStableOrNull" +"stddevSampStableResample" +"stddevSampStableSimpleState" +"stddevSampStableState" +"stddevSampState" +"STDDEV_SAMPState" +"STDDistinct" +"STDForEach" +"STDIf" +"STDMap" +"STDMerge" +"STDNull" +"STDOrDefault" +"STDOrNull" +"STDResample" +"STDSimpleState" +"STDState" "stem" "STEP" "stochasticLinearRegression" +"stochasticLinearRegressionArgMax" +"stochasticLinearRegressionArgMin" +"stochasticLinearRegressionArray" +"stochasticLinearRegressionDistinct" +"stochasticLinearRegressionForEach" +"stochasticLinearRegressionIf" +"stochasticLinearRegressionMap" +"stochasticLinearRegressionMerge" +"stochasticLinearRegressionNull" +"stochasticLinearRegressionOrDefault" +"stochasticLinearRegressionOrNull" +"stochasticLinearRegressionResample" +"stochasticLinearRegressionSimpleState" +"stochasticLinearRegressionState" "stochasticLogisticRegression" -"STOP" +"stochasticLogisticRegressionArgMax" +"stochasticLogisticRegressionArgMin" +"stochasticLogisticRegressionArray" +"stochasticLogisticRegressionDistinct" +"stochasticLogisticRegressionForEach" +"stochasticLogisticRegressionIf" +"stochasticLogisticRegressionMap" +"stochasticLogisticRegressionMerge" +"stochasticLogisticRegressionNull" +"stochasticLogisticRegressionOrDefault" +"stochasticLogisticRegressionOrNull" +"stochasticLogisticRegressionResample" +"stochasticLogisticRegressionSimpleState" +"stochasticLogisticRegressionState" +"STORAGE" +"STRICT" +"STRICTLY_ASCENDING" "String" +"stringJaccardIndex" +"stringJaccardIndexUTF8" "stringToH3" +"str_to_date" +"str_to_map" +"structureToCapnProtoSchema" +"structureToProtobufSchema" "studentTTest" +"studentTTestArgMax" +"studentTTestArgMin" +"studentTTestArray" +"studentTTestDistinct" +"studentTTestForEach" +"studentTTestIf" +"studentTTestMap" +"studentTTestMerge" +"studentTTestNull" +"studentTTestOrDefault" +"studentTTestOrNull" +"studentTTestResample" +"studentTTestSimpleState" +"studentTTestState" "subBitmap" +"subDate" +"SUBPARTITION" +"SUBPARTITION BY" +"SUBPARTITIONS" "substr" "substring" "SUBSTRING" +"substringIndex" +"SUBSTRING_INDEX" +"substringIndexUTF8" "substringUTF8" "subtractDays" "subtractHours" +"subtractInterval" +"subtractMicroseconds" +"subtractMilliseconds" "subtractMinutes" "subtractMonths" +"subtractNanoseconds" "subtractQuarters" "subtractSeconds" +"subtractTupleOfIntervals" "subtractWeeks" "subtractYears" "sum" +"sumArgMax" +"sumArgMin" +"sumArray" "sumCount" +"sumCountArgMax" +"sumCountArgMin" +"sumCountArray" +"sumCountDistinct" +"sumCountForEach" +"sumCountIf" +"sumCountMap" +"sumCountMerge" +"sumCountNull" +"sumCountOrDefault" +"sumCountOrNull" +"sumCountResample" +"sumCountSimpleState" +"sumCountState" +"sumDistinct" +"sumForEach" +"sumIf" "sumKahan" +"sumKahanArgMax" +"sumKahanArgMin" +"sumKahanArray" +"sumKahanDistinct" +"sumKahanForEach" +"sumKahanIf" +"sumKahanMap" +"sumKahanMerge" +"sumKahanNull" +"sumKahanOrDefault" +"sumKahanOrNull" +"sumKahanResample" +"sumKahanSimpleState" +"sumKahanState" "sumMap" "sumMapFiltered" +"sumMapFilteredArgMax" +"sumMapFilteredArgMin" +"sumMapFilteredArray" +"sumMapFilteredDistinct" +"sumMapFilteredForEach" +"sumMapFilteredIf" +"sumMapFilteredMap" +"sumMapFilteredMerge" +"sumMapFilteredNull" +"sumMapFilteredOrDefault" +"sumMapFilteredOrNull" +"sumMapFilteredResample" +"sumMapFilteredSimpleState" +"sumMapFilteredState" "sumMapFilteredWithOverflow" +"sumMapFilteredWithOverflowArgMax" +"sumMapFilteredWithOverflowArgMin" +"sumMapFilteredWithOverflowArray" +"sumMapFilteredWithOverflowDistinct" +"sumMapFilteredWithOverflowForEach" +"sumMapFilteredWithOverflowIf" +"sumMapFilteredWithOverflowMap" +"sumMapFilteredWithOverflowMerge" +"sumMapFilteredWithOverflowNull" +"sumMapFilteredWithOverflowOrDefault" +"sumMapFilteredWithOverflowOrNull" +"sumMapFilteredWithOverflowResample" +"sumMapFilteredWithOverflowSimpleState" +"sumMapFilteredWithOverflowState" +"sumMappedArrays" +"sumMappedArraysArgMax" +"sumMappedArraysArgMin" +"sumMappedArraysArray" +"sumMappedArraysDistinct" +"sumMappedArraysForEach" +"sumMappedArraysIf" +"sumMappedArraysMap" +"sumMappedArraysMerge" +"sumMappedArraysNull" +"sumMappedArraysOrDefault" +"sumMappedArraysOrNull" +"sumMappedArraysResample" +"sumMappedArraysSimpleState" +"sumMappedArraysState" "sumMapWithOverflow" +"sumMapWithOverflowArgMax" +"sumMapWithOverflowArgMin" +"sumMapWithOverflowArray" +"sumMapWithOverflowDistinct" +"sumMapWithOverflowForEach" +"sumMapWithOverflowIf" +"sumMapWithOverflowMap" +"sumMapWithOverflowMerge" +"sumMapWithOverflowNull" +"sumMapWithOverflowOrDefault" +"sumMapWithOverflowOrNull" +"sumMapWithOverflowResample" +"sumMapWithOverflowSimpleState" +"sumMapWithOverflowState" +"sumMerge" +"sumNull" +"sumOrDefault" +"sumOrNull" +"sumResample" +"sumSimpleState" +"sumState" "sumWithOverflow" +"sumWithOverflowArgMax" +"sumWithOverflowArgMin" +"sumWithOverflowArray" +"sumWithOverflowDistinct" +"sumWithOverflowForEach" +"sumWithOverflowIf" +"sumWithOverflowMap" +"sumWithOverflowMerge" +"sumWithOverflowNull" +"sumWithOverflowOrDefault" +"sumWithOverflowOrNull" +"sumWithOverflowResample" +"sumWithOverflowSimpleState" +"sumWithOverflowState" "SUSPEND" "svg" "SVG" @@ -1191,24 +4180,44 @@ "SYNTAX" "SYSTEM" "TABLE" +"TABLE OVERRIDE" "TABLES" "tan" "tanh" "tcpPort" "TEMPORARY" +"TEMPORARY TABLE" "TEST" "TEXT" "tgamma" +"theilsU" +"theilsUArgMax" +"theilsUArgMin" +"theilsUArray" +"theilsUDistinct" +"theilsUForEach" +"theilsUIf" +"theilsUMap" +"theilsUMerge" +"theilsUNull" +"theilsUOrDefault" +"theilsUOrNull" +"theilsUResample" +"theilsUSimpleState" +"theilsUState" "THEN" "throwIf" "tid" -"TIES" -"TIMEOUT" +"TIME" +"timeDiff" "timeSlot" "timeSlots" +"timestamp" "TIMESTAMP" "TIMESTAMP_ADD" "TIMESTAMPADD" +"timestamp_diff" +"timestampDiff" "TIMESTAMP_DIFF" "TIMESTAMPDIFF" "TIMESTAMP_SUB" @@ -1226,77 +4235,111 @@ "TINYTEXT" "TO" "TO_BASE64" +"toBool" "toColumnTypeName" "toDate" "toDate32" +"toDate32OrDefault" "toDate32OrNull" "toDate32OrZero" +"toDateOrDefault" "toDateOrNull" "toDateOrZero" "toDateTime" "toDateTime32" "toDateTime64" +"toDateTime64OrDefault" "toDateTime64OrNull" "toDateTime64OrZero" +"toDateTimeOrDefault" "toDateTimeOrNull" "toDateTimeOrZero" "today" "toDayOfMonth" "toDayOfWeek" "toDayOfYear" +"TO_DAYS" +"toDaysSinceYearZero" "toDecimal128" +"toDecimal128OrDefault" "toDecimal128OrNull" "toDecimal128OrZero" "toDecimal256" +"toDecimal256OrDefault" "toDecimal256OrNull" "toDecimal256OrZero" "toDecimal32" +"toDecimal32OrDefault" "toDecimal32OrNull" "toDecimal32OrZero" "toDecimal64" +"toDecimal64OrDefault" "toDecimal64OrNull" "toDecimal64OrZero" +"toDecimalString" "TO DISK" "toFixedString" "toFloat32" +"toFloat32OrDefault" "toFloat32OrNull" "toFloat32OrZero" "toFloat64" +"toFloat64OrDefault" "toFloat64OrNull" "toFloat64OrZero" "toHour" +"TO INNER UUID" "toInt128" +"toInt128OrDefault" "toInt128OrNull" "toInt128OrZero" "toInt16" +"toInt16OrDefault" "toInt16OrNull" "toInt16OrZero" "toInt256" +"toInt256OrDefault" "toInt256OrNull" "toInt256OrZero" "toInt32" +"toInt32OrDefault" "toInt32OrNull" "toInt32OrZero" "toInt64" +"toInt64OrDefault" "toInt64OrNull" "toInt64OrZero" "toInt8" +"toInt8OrDefault" "toInt8OrNull" "toInt8OrZero" "toIntervalDay" "toIntervalHour" +"toIntervalMicrosecond" +"toIntervalMillisecond" "toIntervalMinute" "toIntervalMonth" +"toIntervalNanosecond" "toIntervalQuarter" "toIntervalSecond" "toIntervalWeek" "toIntervalYear" "toIPv4" +"toIPv4OrDefault" +"toIPv4OrNull" +"toIPv4OrZero" "toIPv6" +"toIPv6OrDefault" +"toIPv6OrNull" +"toIPv6OrZero" "toISOWeek" "toISOYear" "toJSONString" +"tokens" +"toLastDayOfMonth" +"toLastDayOfWeek" "toLowCardinality" +"toMillisecond" "toMinute" "toModifiedJulianDay" "toModifiedJulianDayOrNull" @@ -1305,8 +4348,37 @@ "toNullable" "TOP" "topK" +"topKArgMax" +"topKArgMin" +"topKArray" +"topKDistinct" +"topKForEach" +"topKIf" +"topKMap" +"topKMerge" +"topKNull" +"topKOrDefault" +"topKOrNull" +"topKResample" +"topKSimpleState" +"topKState" "topKWeighted" +"topKWeightedArgMax" +"topKWeightedArgMin" +"topKWeightedArray" +"topKWeightedDistinct" +"topKWeightedForEach" +"topKWeightedIf" +"topKWeightedMap" +"topKWeightedMerge" +"topKWeightedNull" +"topKWeightedOrDefault" +"topKWeightedOrNull" +"topKWeightedResample" +"topKWeightedSimpleState" +"topKWeightedState" "topLevelDomain" +"topLevelDomainRFC" "toQuarter" "toRelativeDayNum" "toRelativeHourNum" @@ -1317,14 +4389,19 @@ "toRelativeWeekNum" "toRelativeYearNum" "toSecond" +"TO SHARD" "toStartOfDay" "toStartOfFifteenMinutes" +"toStartOfFiveMinute" "toStartOfFiveMinutes" "toStartOfHour" "toStartOfInterval" "toStartOfISOYear" +"toStartOfMicrosecond" +"toStartOfMillisecond" "toStartOfMinute" "toStartOfMonth" +"toStartOfNanosecond" "toStartOfQuarter" "toStartOfSecond" "toStartOfTenMinutes" @@ -1339,28 +4416,38 @@ "toTimeZone" "toTypeName" "toUInt128" +"toUInt128OrDefault" "toUInt128OrNull" "toUInt128OrZero" "toUInt16" +"toUInt16OrDefault" "toUInt16OrNull" "toUInt16OrZero" "toUInt256" +"toUInt256OrDefault" "toUInt256OrNull" "toUInt256OrZero" "toUInt32" +"toUInt32OrDefault" "toUInt32OrNull" "toUInt32OrZero" "toUInt64" +"toUInt64OrDefault" "toUInt64OrNull" "toUInt64OrZero" "toUInt8" +"toUInt8OrDefault" "toUInt8OrNull" "toUInt8OrZero" +"TO_UNIXTIME" "toUnixTimestamp" "toUnixTimestamp64Micro" "toUnixTimestamp64Milli" "toUnixTimestamp64Nano" +"to_utc_timestamp" +"toUTCTimestamp" "toUUID" +"toUUIDOrDefault" "toUUIDOrNull" "toUUIDOrZero" "toValidUTF8" @@ -1371,23 +4458,55 @@ "toYYYYMM" "toYYYYMMDD" "toYYYYMMDDhhmmss" +"TRACKING ONLY" "TRAILING" +"TRANSACTION" +"transactionID" +"transactionLatestSnapshot" +"transactionOldestSnapshot" "transform" +"translate" +"translateUTF8" +"TRIGGER" +"trim" "TRIM" "trimBoth" "trimLeft" "trimRight" +"TRUE" "trunc" "truncate" "TRUNCATE" +"tryBase58Decode" "tryBase64Decode" +"tryDecrypt" +"tryIdnaEncode" +"tryPunycodeDecode" "TTL" +"tumble" +"tumbleEnd" +"tumbleStart" "tuple" "Tuple" +"tupleConcat" +"tupleDivide" +"tupleDivideByNumber" "tupleElement" "tupleHammingDistance" +"tupleIntDiv" +"tupleIntDivByNumber" +"tupleIntDivOrZero" +"tupleIntDivOrZeroByNumber" +"tupleMinus" +"tupleModulo" +"tupleModuloByNumber" +"tupleMultiply" +"tupleMultiplyByNumber" +"tupleNegate" +"tuplePlus" "tupleToNameValuePairs" "TYPE" +"TYPEOF" "ucase" "UInt128" "UInt16" @@ -1395,41 +4514,245 @@ "UInt32" "UInt64" "UInt8" +"ULIDStringToDateTime" "unbin" +"UNBOUNDED" +"UNDROP" +"UNFREEZE" "unhex" "UNION" "uniq" +"uniqArgMax" +"uniqArgMin" +"uniqArray" "uniqCombined" "uniqCombined64" +"uniqCombined64ArgMax" +"uniqCombined64ArgMin" +"uniqCombined64Array" +"uniqCombined64Distinct" +"uniqCombined64ForEach" +"uniqCombined64If" +"uniqCombined64Map" +"uniqCombined64Merge" +"uniqCombined64Null" +"uniqCombined64OrDefault" +"uniqCombined64OrNull" +"uniqCombined64Resample" +"uniqCombined64SimpleState" +"uniqCombined64State" +"uniqCombinedArgMax" +"uniqCombinedArgMin" +"uniqCombinedArray" +"uniqCombinedDistinct" +"uniqCombinedForEach" +"uniqCombinedIf" +"uniqCombinedMap" +"uniqCombinedMerge" +"uniqCombinedNull" +"uniqCombinedOrDefault" +"uniqCombinedOrNull" +"uniqCombinedResample" +"uniqCombinedSimpleState" +"uniqCombinedState" +"uniqDistinct" "uniqExact" +"uniqExactArgMax" +"uniqExactArgMin" +"uniqExactArray" +"uniqExactDistinct" +"uniqExactForEach" +"uniqExactIf" +"uniqExactMap" +"uniqExactMerge" +"uniqExactNull" +"uniqExactOrDefault" +"uniqExactOrNull" +"uniqExactResample" +"uniqExactSimpleState" +"uniqExactState" +"uniqForEach" "uniqHLL12" +"uniqHLL12ArgMax" +"uniqHLL12ArgMin" +"uniqHLL12Array" +"uniqHLL12Distinct" +"uniqHLL12ForEach" +"uniqHLL12If" +"uniqHLL12Map" +"uniqHLL12Merge" +"uniqHLL12Null" +"uniqHLL12OrDefault" +"uniqHLL12OrNull" +"uniqHLL12Resample" +"uniqHLL12SimpleState" +"uniqHLL12State" +"uniqIf" +"uniqMap" +"uniqMerge" +"uniqNull" +"uniqOrDefault" +"uniqOrNull" +"uniqResample" +"uniqSimpleState" +"uniqState" "uniqTheta" +"uniqThetaArgMax" +"uniqThetaArgMin" +"uniqThetaArray" +"uniqThetaDistinct" +"uniqThetaForEach" +"uniqThetaIf" +"uniqThetaIntersect" +"uniqThetaMap" +"uniqThetaMerge" +"uniqThetaNot" +"uniqThetaNull" +"uniqThetaOrDefault" +"uniqThetaOrNull" +"uniqThetaResample" +"uniqThetaSimpleState" +"uniqThetaState" +"uniqThetaUnion" +"UNIQUE" "uniqUpTo" +"uniqUpToArgMax" +"uniqUpToArgMin" +"uniqUpToArray" +"uniqUpToDistinct" +"uniqUpToForEach" +"uniqUpToIf" +"uniqUpToMap" +"uniqUpToMerge" +"uniqUpToNull" +"uniqUpToOrDefault" +"uniqUpToOrNull" +"uniqUpToResample" +"uniqUpToSimpleState" +"uniqUpToState" +"UNSET FAKE TIME" +"UNSIGNED" "UPDATE" "upper" "upperUTF8" "uptime" +"URL" "URLHash" "URLHierarchy" "URLPathHierarchy" "USE" "user" "USING" +"UTC_timestamp" +"UTCTimestamp" "UUID" "UUIDNumToString" "UUIDStringToNum" "validateNestedArraySizes" +"VALID UNTIL" "VALUES" +"VARBINARY" "VARCHAR" "VARCHAR2" +"Variant" +"variantElement" +"variantType" "varPop" "VAR_POP" +"varPopArgMax" +"VAR_POPArgMax" +"varPopArgMin" +"VAR_POPArgMin" +"varPopArray" +"VAR_POPArray" +"varPopDistinct" +"VAR_POPDistinct" +"varPopForEach" +"VAR_POPForEach" +"varPopIf" +"VAR_POPIf" +"varPopMap" +"VAR_POPMap" +"varPopMerge" +"VAR_POPMerge" +"varPopNull" +"VAR_POPNull" +"varPopOrDefault" +"VAR_POPOrDefault" +"varPopOrNull" +"VAR_POPOrNull" +"varPopResample" +"VAR_POPResample" +"varPopSimpleState" +"VAR_POPSimpleState" "varPopStable" +"varPopStableArgMax" +"varPopStableArgMin" +"varPopStableArray" +"varPopStableDistinct" +"varPopStableForEach" +"varPopStableIf" +"varPopStableMap" +"varPopStableMerge" +"varPopStableNull" +"varPopStableOrDefault" +"varPopStableOrNull" +"varPopStableResample" +"varPopStableSimpleState" +"varPopStableState" +"varPopState" +"VAR_POPState" "varSamp" "VAR_SAMP" +"varSampArgMax" +"VAR_SAMPArgMax" +"varSampArgMin" +"VAR_SAMPArgMin" +"varSampArray" +"VAR_SAMPArray" +"varSampDistinct" +"VAR_SAMPDistinct" +"varSampForEach" +"VAR_SAMPForEach" +"varSampIf" +"VAR_SAMPIf" +"varSampMap" +"VAR_SAMPMap" +"varSampMerge" +"VAR_SAMPMerge" +"varSampNull" +"VAR_SAMPNull" +"varSampOrDefault" +"VAR_SAMPOrDefault" +"varSampOrNull" +"VAR_SAMPOrNull" +"varSampResample" +"VAR_SAMPResample" +"varSampSimpleState" +"VAR_SAMPSimpleState" "varSampStable" +"varSampStableArgMax" +"varSampStableArgMin" +"varSampStableArray" +"varSampStableDistinct" +"varSampStableForEach" +"varSampStableIf" +"varSampStableMap" +"varSampStableMerge" +"varSampStableNull" +"varSampStableOrDefault" +"varSampStableOrNull" +"varSampStableResample" +"varSampStableSimpleState" +"varSampStableState" +"varSampState" +"VAR_SAMPState" +"VARYING" +"vectorDifference" +"vectorSum" "version" "VIEW" +"VISIBLE" "visibleWidth" "visitParamExtractBool" "visitParamExtractFloat" @@ -1438,16 +4761,55 @@ "visitParamExtractString" "visitParamExtractUInt" "visitParamHas" -"VOLUME" "WATCH" +"WATERMARK" "week" "WEEK" +"WEEKS" "welchTTest" +"welchTTestArgMax" +"welchTTestArgMin" +"welchTTestArray" +"welchTTestDistinct" +"welchTTestForEach" +"welchTTestIf" +"welchTTestMap" +"welchTTestMerge" +"welchTTestNull" +"welchTTestOrDefault" +"welchTTestOrNull" +"welchTTestResample" +"welchTTestSimpleState" +"welchTTestState" "WHEN" "WHERE" +"width_bucket" +"widthBucket" +"WINDOW" "windowFunnel" +"windowFunnelArgMax" +"windowFunnelArgMin" +"windowFunnelArray" +"windowFunnelDistinct" +"windowFunnelForEach" +"windowFunnelIf" +"windowFunnelMap" +"windowFunnelMerge" +"windowFunnelNull" +"windowFunnelOrDefault" +"windowFunnelOrNull" +"windowFunnelResample" +"windowFunnelSimpleState" +"windowFunnelState" +"windowID" "WITH" +"WITH ADMIN OPTION" +"WITH CHECK" "WITH FILL" +"WITH GRANT OPTION" +"with_itemindex" +"WITH NAME" +"WITH REPLACE OPTION" "WITH TIES" "WK" "wkt" @@ -1463,14 +4825,23 @@ "wordShingleSimHashCaseInsensitive" "wordShingleSimHashCaseInsensitiveUTF8" "wordShingleSimHashUTF8" +"WRITABLE" "WW" +"wyHash64" "xor" +"xxh3" "xxHash32" "xxHash64" -"kostikConsistentHash" +"yandexConsistentHash" "YEAR" +"YEARS" "yearweek" "yesterday" "YY" "YYYY" +"YYYYMMDDhhmmssToDateTime" +"YYYYMMDDhhmmssToDateTime64" +"YYYYMMDDToDate" +"YYYYMMDDToDate32" +"ZKPATH" "zookeeperSessionUptime" diff --git a/tests/fuzz/dictionaries/datatypes.dict b/tests/fuzz/dictionaries/datatypes.dict index e4983ae5f68..232e89db0c0 100644 --- a/tests/fuzz/dictionaries/datatypes.dict +++ b/tests/fuzz/dictionaries/datatypes.dict @@ -1,185 +1,134 @@ -"Polygon" -"Ring" -"Point" -"SimpleAggregateFunction" -"MultiPolygon" -"IPv6" -"IntervalSecond" +"AggregateFunction" +"Array" +"BIGINT" +"BIGINT SIGNED" +"BIGINT UNSIGNED" +"BINARY" +"BINARY LARGE OBJECT" +"BINARY VARYING" +"BIT" +"BLOB" +"BYTE" +"BYTEA" +"Bool" +"CHAR" +"CHAR LARGE OBJECT" +"CHAR VARYING" +"CHARACTER" +"CHARACTER LARGE OBJECT" +"CHARACTER VARYING" +"CLOB" +"DEC" +"DOUBLE" +"DOUBLE PRECISION" +"Date" +"Date32" +"DateTime" +"DateTime32" +"DateTime64" +"Decimal" +"Decimal128" +"Decimal256" +"Decimal32" +"Decimal64" +"ENUM" +"Enum" +"Enum16" +"Enum8" +"FIXED" +"FLOAT" +"FixedString" +"Float32" +"Float64" +"GEOMETRY" +"INET4" +"INET6" +"INT" +"INT SIGNED" +"INT UNSIGNED" +"INT1" +"INT1 SIGNED" +"INT1 UNSIGNED" +"INTEGER" +"INTEGER SIGNED" +"INTEGER UNSIGNED" "IPv4" -"UInt32" -"IntervalYear" -"IntervalQuarter" -"IntervalMonth" +"IPv6" +"Int128" +"Int16" +"Int256" +"Int32" "Int64" +"Int8" "IntervalDay" "IntervalHour" -"Int16" -"UInt256" -"LowCardinality" -"AggregateFunction" -"Nothing" -"Decimal256" -"Tuple" -"Array" -"Enum16" +"IntervalMicrosecond" +"IntervalMillisecond" "IntervalMinute" -"FixedString" -"String" -"DateTime" -"Map" -"UUID" -"Decimal64" -"Nullable" -"Enum" -"Int32" -"UInt8" -"Date" -"Decimal32" -"UInt128" -"Float64" -"Nested" -"Int128" -"Decimal128" -"Int8" -"Decimal" -"Int256" -"DateTime64" -"Enum8" -"DateTime32" -"Date32" +"IntervalMonth" +"IntervalNanosecond" +"IntervalQuarter" +"IntervalSecond" "IntervalWeek" -"UInt64" -"UInt16" -"Float32" -"INET6" -"INET4" -"ENUM" -"BINARY" -"NATIONAL CHAR VARYING" -"BINARY VARYING" -"NCHAR LARGE OBJECT" -"NATIONAL CHARACTER VARYING" -"NATIONAL CHARACTER LARGE OBJECT" -"NATIONAL CHARACTER" -"NATIONAL CHAR" -"CHARACTER VARYING" +"IntervalYear" +"JSON" "LONGBLOB" -"MEDIUMTEXT" -"TEXT" -"TINYBLOB" -"VARCHAR2" -"CHARACTER LARGE OBJECT" -"DOUBLE PRECISION" "LONGTEXT" -"NVARCHAR" -"INT1 UNSIGNED" -"VARCHAR" -"CHAR VARYING" +"LowCardinality" "MEDIUMBLOB" -"NCHAR" -"CHAR" -"SMALLINT UNSIGNED" -"TIMESTAMP" -"FIXED" -"TINYTEXT" -"NUMERIC" -"DEC" -"TINYINT UNSIGNED" -"INTEGER UNSIGNED" -"INT UNSIGNED" -"CLOB" -"MEDIUMINT UNSIGNED" -"BOOL" -"SMALLINT" -"INTEGER SIGNED" -"NCHAR VARYING" -"INT SIGNED" -"TINYINT SIGNED" -"BIGINT SIGNED" -"BINARY LARGE OBJECT" -"SMALLINT SIGNED" "MEDIUMINT" -"INTEGER" -"INT1 SIGNED" -"BIGINT UNSIGNED" -"BYTEA" -"INT" -"SINGLE" -"FLOAT" "MEDIUMINT SIGNED" -"BOOLEAN" -"DOUBLE" -"INT1" -"CHAR LARGE OBJECT" -"TINYINT" -"BIGINT" -"CHARACTER" -"BYTE" -"BLOB" +"MEDIUMINT UNSIGNED" +"MEDIUMTEXT" +"Map" +"MultiPolygon" +"NATIONAL CHAR" +"NATIONAL CHAR VARYING" +"NATIONAL CHARACTER" +"NATIONAL CHARACTER LARGE OBJECT" +"NATIONAL CHARACTER VARYING" +"NCHAR" +"NCHAR LARGE OBJECT" +"NCHAR VARYING" +"NUMERIC" +"NVARCHAR" +"Nested" +"Nothing" +"Nullable" +"Object" +"Point" +"Polygon" "REAL" -"IPv6" -"IPv4" -"Enum" -"FixedString" -"String" -"String" -"String" -"String" -"String" -"String" -"String" -"String" -"String" -"String" -"String" -"String" -"String" -"String" -"Float64" -"String" -"String" -"UInt8" -"String" -"String" -"String" -"String" +"Ring" +"SET" +"SIGNED" +"SINGLE" +"SMALLINT" +"SMALLINT SIGNED" +"SMALLINT UNSIGNED" +"SimpleAggregateFunction" "String" +"TEXT" +"TIME" +"TIMESTAMP" +"TINYBLOB" +"TINYINT" +"TINYINT SIGNED" +"TINYINT UNSIGNED" +"TINYTEXT" +"Tuple" +"UInt128" "UInt16" -"DateTime" -"Decimal" -"String" -"Decimal" -"Decimal" -"UInt8" +"UInt256" "UInt32" -"UInt32" -"String" -"UInt32" -"Int8" -"Int16" -"Int32" -"String" -"Int32" -"Int8" -"Int64" -"String" -"Int16" -"Int32" -"Int32" -"Int8" "UInt64" -"String" -"Int32" -"Float32" -"Float32" -"Int32" -"Int8" -"Float64" -"Int8" -"String" -"Int8" -"Int64" -"String" -"Int8" -"String" -"Float32" +"UInt8" +"UNSIGNED" +"UUID" +"VARBINARY" +"VARCHAR" +"VARCHAR2" +"Variant" +"YEAR" +"bool" +"boolean" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index 3fe8dbfe501..ec7f8017fb2 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -1,1134 +1,4172 @@ -"logTrace" -"aes_decrypt_mysql" -"aes_encrypt_mysql" -"decrypt" -"encrypt" -"bayesAB" -"snowflakeToDateTime64" -"snowflakeToDateTime" -"validateNestedArraySizes" -"wordShingleMinHashArgCaseInsensitiveUTF8" -"wordShingleMinHashArgUTF8" -"wordShingleMinHashArgCaseInsensitive" -"wordShingleMinHashArg" -"ngramMinHashArgCaseInsensitiveUTF8" -"ngramMinHashArgUTF8" -"ngramMinHashArgCaseInsensitive" -"wordShingleMinHashCaseInsensitiveUTF8" -"wordShingleMinHashUTF8" -"ngramMinHashCaseInsensitiveUTF8" -"ngramMinHashUTF8" -"ngramMinHashCaseInsensitive" -"wordShingleSimHashUTF8" -"ngramSimHashUTF8" -"ngramSimHashCaseInsensitive" -"ngramSimHash" -"toUnixTimestamp64Nano" -"toUnixTimestamp64Micro" -"jumpConsistentHash" -"kostikConsistentHash" -"addressToSymbol" -"toJSONString" -"JSON_VALUE" -"JSONExtractKeysAndValuesRaw" +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" +"BIT_OR" +"BIT_ORArgMax" +"BIT_ORArgMin" +"BIT_ORArray" +"BIT_ORDistinct" +"BIT_ORForEach" +"BIT_ORIf" +"BIT_ORMap" +"BIT_ORMerge" +"BIT_ORNull" +"BIT_OROrDefault" +"BIT_OROrNull" +"BIT_ORResample" +"BIT_ORSimpleState" +"BIT_ORState" +"BIT_XOR" +"BIT_XORArgMax" +"BIT_XORArgMin" +"BIT_XORArray" +"BIT_XORDistinct" +"BIT_XORForEach" +"BIT_XORIf" +"BIT_XORMap" +"BIT_XORMerge" +"BIT_XORNull" +"BIT_XOROrDefault" +"BIT_XOROrNull" +"BIT_XORResample" +"BIT_XORSimpleState" +"BIT_XORState" +"BLAKE3" +"CAST" +"CHARACTER_LENGTH" +"CHAR_LENGTH" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" +"COVAR_SAMPState" +"CRC32" +"CRC32IEEE" +"CRC64" +"DATABASE" +"DATE" +"DATE_DIFF" +"DATE_FORMAT" +"DATE_TRUNC" +"DAY" +"DAYOFMONTH" +"DAYOFWEEK" +"DAYOFYEAR" +"FORMAT_BYTES" +"FQDN" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"HOUR" +"INET6_ATON" +"INET6_NTOA" +"INET_ATON" +"INET_NTOA" +"IPv4CIDRToRange" +"IPv4NumToString" +"IPv4NumToStringClassC" +"IPv4StringToNum" +"IPv4StringToNumOrDefault" +"IPv4StringToNumOrNull" +"IPv4ToIPv6" +"IPv6CIDRToRange" +"IPv6NumToString" +"IPv6StringToNum" +"IPv6StringToNumOrDefault" +"IPv6StringToNumOrNull" +"JSONArrayLength" +"JSONExtract" +"JSONExtractArrayRaw" +"JSONExtractBool" +"JSONExtractFloat" +"JSONExtractInt" "JSONExtractKeys" +"JSONExtractKeysAndValues" +"JSONExtractKeysAndValuesRaw" +"JSONExtractRaw" "JSONExtractString" -"JSONType" +"JSONExtractUInt" +"JSONHas" "JSONKey" "JSONLength" -"isValidJSON" -"isZeroOrNull" -"assumeNotNull" -"s2CapUnion" -"s2CapContains" -"s2CellsIntersect" -"s2GetNeighbors" -"s2ToGeo" -"h3GetFaces" -"h3IsResClassIII" -"h3ToString" -"stringToH3" -"h3ToParent" -"h3GetResolution" -"h3EdgeLengthM" -"svg" -"SVG" -"equals" -"geohashesInBox" -"polygonsIntersectionCartesian" -"polygonPerimeterSpherical" -"bitHammingDistance" -"polygonsDistanceSpherical" -"polygonsSymDifferenceSpherical" -"polygonAreaSpherical" -"greatCircleDistance" -"toInt256" -"mapKeys" -"sign" -"reinterpretAsUInt8" -"atanh" -"formatReadableTimeDelta" -"parseTimeDelta" -"geohashEncode" -"atan2" -"acos" +"JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" +"JSON_VALUE" +"L1Distance" +"L1Norm" +"L1Normalize" +"L2Distance" +"L2Norm" +"L2Normalize" +"L2SquaredDistance" +"L2SquaredNorm" +"LAST_DAY" +"LinfDistance" +"LinfNorm" +"LinfNormalize" +"LpDistance" +"LpNorm" +"LpNormalize" +"MACNumToString" +"MACStringToNum" +"MACStringToOUI" +"MAP_FROM_ARRAYS" +"MD4" +"MD5" +"MILLISECOND" +"MINUTE" +"MONTH" +"OCTET_LENGTH" +"QUARTER" +"REGEXP_EXTRACT" +"REGEXP_MATCHES" +"REGEXP_REPLACE" +"SCHEMA" +"SECOND" +"SHA1" "SHA224" -"visitParamExtractBool" -"h3kRing" -"asin" -"sin" -"reinterpretAsUInt128" -"lgamma" -"log2" -"geoToH3" -"toUnixTimestamp64Milli" -"firstSignificantSubdomainCustom" -"simpleJSONExtractBool" -"visitParamExtractFloat" -"visitParamExtractInt" -"simpleJSONHas" -"visitParamHas" -"sigmoid" -"cutQueryStringAndFragment" -"dictGetInt16OrDefault" -"cutToFirstSignificantSubdomainCustom" +"SHA256" +"SHA384" +"SHA512" +"SHA512_256" +"STD" +"STDArgMax" +"STDArgMin" +"STDArray" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" +"STDDEV_SAMPState" +"STDDistinct" +"STDForEach" +"STDIf" +"STDMap" +"STDMerge" +"STDNull" +"STDOrDefault" +"STDOrNull" +"STDResample" +"STDSimpleState" +"STDState" +"SUBSTRING_INDEX" +"SVG" +"TIMESTAMP_DIFF" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"ULIDStringToDateTime" +"URLHash" "URLHierarchy" -"extractURLParameterNames" -"toDecimal128OrZero" -"extractURLParameters" -"path" -"pathFull" -"port" -"domain" +"URLPathHierarchy" +"UTCTimestamp" +"UTC_timestamp" +"UUIDNumToString" +"UUIDStringToNum" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" +"YEAR" +"YYYYMMDDToDate" +"YYYYMMDDToDate32" +"YYYYMMDDhhmmssToDateTime" +"YYYYMMDDhhmmssToDateTime64" +"_CAST" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" +"abs" +"accurateCast" +"accurateCastOrDefault" +"accurateCastOrNull" +"acos" +"acosh" +"addDate" +"addDays" +"addHours" +"addInterval" +"addMicroseconds" +"addMilliseconds" +"addMinutes" +"addMonths" +"addNanoseconds" +"addQuarters" +"addSeconds" +"addTupleOfIntervals" +"addWeeks" +"addYears" +"addressToLine" +"addressToLineWithInlines" +"addressToSymbol" +"aes_decrypt_mysql" +"aes_encrypt_mysql" +"age" +"aggThrow" +"aggThrowArgMax" +"aggThrowArgMin" +"aggThrowArray" +"aggThrowDistinct" +"aggThrowForEach" +"aggThrowIf" +"aggThrowMap" +"aggThrowMerge" +"aggThrowNull" +"aggThrowOrDefault" +"aggThrowOrNull" +"aggThrowResample" +"aggThrowSimpleState" +"aggThrowState" +"alphaTokens" +"analysisOfVariance" +"analysisOfVarianceArgMax" +"analysisOfVarianceArgMin" +"analysisOfVarianceArray" +"analysisOfVarianceDistinct" +"analysisOfVarianceForEach" +"analysisOfVarianceIf" +"analysisOfVarianceMap" +"analysisOfVarianceMerge" +"analysisOfVarianceNull" +"analysisOfVarianceOrDefault" +"analysisOfVarianceOrNull" +"analysisOfVarianceResample" +"analysisOfVarianceSimpleState" +"analysisOfVarianceState" +"and" +"anova" +"anovaArgMax" +"anovaArgMin" +"anovaArray" +"anovaDistinct" +"anovaForEach" +"anovaIf" +"anovaMap" +"anovaMerge" +"anovaNull" +"anovaOrDefault" +"anovaOrNull" +"anovaResample" +"anovaSimpleState" +"anovaState" +"any" +"anyArgMax" +"anyArgMin" +"anyArray" +"anyDistinct" +"anyForEach" +"anyHeavy" +"anyHeavyArgMax" +"anyHeavyArgMin" +"anyHeavyArray" +"anyHeavyDistinct" +"anyHeavyForEach" +"anyHeavyIf" +"anyHeavyMap" +"anyHeavyMerge" +"anyHeavyNull" +"anyHeavyOrDefault" +"anyHeavyOrNull" +"anyHeavyResample" +"anyHeavySimpleState" +"anyHeavyState" +"anyIf" +"anyLast" +"anyLastArgMax" +"anyLastArgMin" +"anyLastArray" +"anyLastDistinct" +"anyLastForEach" +"anyLastIf" +"anyLastMap" +"anyLastMerge" +"anyLastNull" +"anyLastOrDefault" +"anyLastOrNull" +"anyLastResample" +"anyLastSimpleState" +"anyLastState" +"anyLast_respect_nulls" +"anyLast_respect_nullsArgMax" +"anyLast_respect_nullsArgMin" +"anyLast_respect_nullsArray" +"anyLast_respect_nullsDistinct" +"anyLast_respect_nullsForEach" +"anyLast_respect_nullsIf" +"anyLast_respect_nullsMap" +"anyLast_respect_nullsMerge" +"anyLast_respect_nullsNull" +"anyLast_respect_nullsOrDefault" +"anyLast_respect_nullsOrNull" +"anyLast_respect_nullsResample" +"anyLast_respect_nullsSimpleState" +"anyLast_respect_nullsState" +"anyMap" +"anyMerge" +"anyNull" +"anyOrDefault" +"anyOrNull" +"anyResample" +"anySimpleState" +"anyState" +"any_respect_nulls" +"any_respect_nullsArgMax" +"any_respect_nullsArgMin" +"any_respect_nullsArray" +"any_respect_nullsDistinct" +"any_respect_nullsForEach" +"any_respect_nullsIf" +"any_respect_nullsMap" +"any_respect_nullsMerge" +"any_respect_nullsNull" +"any_respect_nullsOrDefault" +"any_respect_nullsOrNull" +"any_respect_nullsResample" +"any_respect_nullsSimpleState" +"any_respect_nullsState" +"any_value" +"any_valueArgMax" +"any_valueArgMin" +"any_valueArray" +"any_valueDistinct" +"any_valueForEach" +"any_valueIf" +"any_valueMap" +"any_valueMerge" +"any_valueNull" +"any_valueOrDefault" +"any_valueOrNull" +"any_valueResample" +"any_valueSimpleState" +"any_valueState" +"any_value_respect_nulls" +"any_value_respect_nullsArgMax" +"any_value_respect_nullsArgMin" +"any_value_respect_nullsArray" +"any_value_respect_nullsDistinct" +"any_value_respect_nullsForEach" +"any_value_respect_nullsIf" +"any_value_respect_nullsMap" +"any_value_respect_nullsMerge" +"any_value_respect_nullsNull" +"any_value_respect_nullsOrDefault" +"any_value_respect_nullsOrNull" +"any_value_respect_nullsResample" +"any_value_respect_nullsSimpleState" +"any_value_respect_nullsState" +"appendTrailingCharIfAbsent" +"approx_top_count" +"approx_top_countArgMax" +"approx_top_countArgMin" +"approx_top_countArray" +"approx_top_countDistinct" +"approx_top_countForEach" +"approx_top_countIf" +"approx_top_countMap" +"approx_top_countMerge" +"approx_top_countNull" +"approx_top_countOrDefault" +"approx_top_countOrNull" +"approx_top_countResample" +"approx_top_countSimpleState" +"approx_top_countState" +"approx_top_k" +"approx_top_kArgMax" +"approx_top_kArgMin" +"approx_top_kArray" +"approx_top_kDistinct" +"approx_top_kForEach" +"approx_top_kIf" +"approx_top_kMap" +"approx_top_kMerge" +"approx_top_kNull" +"approx_top_kOrDefault" +"approx_top_kOrNull" +"approx_top_kResample" +"approx_top_kSimpleState" +"approx_top_kState" +"approx_top_sum" +"approx_top_sumArgMax" +"approx_top_sumArgMin" +"approx_top_sumArray" +"approx_top_sumDistinct" +"approx_top_sumForEach" +"approx_top_sumIf" +"approx_top_sumMap" +"approx_top_sumMerge" +"approx_top_sumNull" +"approx_top_sumOrDefault" +"approx_top_sumOrNull" +"approx_top_sumResample" +"approx_top_sumSimpleState" +"approx_top_sumState" +"argMax" +"argMaxArgMax" +"argMaxArgMin" +"argMaxArray" +"argMaxDistinct" +"argMaxForEach" +"argMaxIf" +"argMaxMap" +"argMaxMerge" +"argMaxNull" +"argMaxOrDefault" +"argMaxOrNull" +"argMaxResample" +"argMaxSimpleState" +"argMaxState" +"argMin" +"argMinArgMax" +"argMinArgMin" +"argMinArray" +"argMinDistinct" +"argMinForEach" +"argMinIf" +"argMinMap" +"argMinMerge" +"argMinNull" +"argMinOrDefault" +"argMinOrNull" +"argMinResample" +"argMinSimpleState" +"argMinState" +"array" +"arrayAUC" +"arrayAll" +"arrayAvg" +"arrayCompact" +"arrayConcat" +"arrayCount" +"arrayCumSum" +"arrayCumSumNonNegative" +"arrayDifference" +"arrayDistinct" +"arrayDotProduct" +"arrayElement" +"arrayEnumerate" +"arrayEnumerateDense" +"arrayEnumerateDenseRanked" +"arrayEnumerateUniq" +"arrayEnumerateUniqRanked" +"arrayExists" +"arrayFill" +"arrayFilter" +"arrayFirst" +"arrayFirstIndex" +"arrayFirstOrNull" +"arrayFlatten" +"arrayFold" +"arrayIntersect" +"arrayJaccardIndex" +"arrayJoin" +"arrayLast" +"arrayLastIndex" +"arrayLastOrNull" +"arrayMap" +"arrayMax" +"arrayMin" +"arrayPartialReverseSort" +"arrayPartialShuffle" +"arrayPartialSort" +"arrayPopBack" +"arrayPopFront" +"arrayProduct" +"arrayPushBack" +"arrayPushFront" +"arrayRandomSample" +"arrayReduce" +"arrayReduceInRanges" +"arrayResize" +"arrayReverse" +"arrayReverseFill" +"arrayReverseSort" +"arrayReverseSplit" +"arrayRotateLeft" +"arrayRotateRight" +"arrayShiftLeft" +"arrayShiftRight" +"arrayShingles" +"arrayShuffle" +"arraySlice" +"arraySort" +"arraySplit" +"arrayStringConcat" +"arraySum" +"arrayUniq" +"arrayWithConstant" +"arrayZip" +"array_agg" +"array_aggArgMax" +"array_aggArgMin" +"array_aggArray" +"array_aggDistinct" +"array_aggForEach" +"array_aggIf" +"array_aggMap" +"array_aggMerge" +"array_aggNull" +"array_aggOrDefault" +"array_aggOrNull" +"array_aggResample" +"array_aggSimpleState" +"array_aggState" +"array_concat_agg" +"array_concat_aggArgMax" +"array_concat_aggArgMin" +"array_concat_aggArray" +"array_concat_aggDistinct" +"array_concat_aggForEach" +"array_concat_aggIf" +"array_concat_aggMap" +"array_concat_aggMerge" +"array_concat_aggNull" +"array_concat_aggOrDefault" +"array_concat_aggOrNull" +"array_concat_aggResample" +"array_concat_aggSimpleState" +"array_concat_aggState" +"ascii" +"asin" +"asinh" +"assumeNotNull" +"atan" +"atan2" +"atanh" +"avg" +"avgArgMax" +"avgArgMin" +"avgArray" +"avgDistinct" +"avgForEach" +"avgIf" +"avgMap" +"avgMerge" +"avgNull" +"avgOrDefault" +"avgOrNull" +"avgResample" +"avgSimpleState" +"avgState" +"avgWeighted" +"avgWeightedArgMax" +"avgWeightedArgMin" +"avgWeightedArray" +"avgWeightedDistinct" +"avgWeightedForEach" +"avgWeightedIf" +"avgWeightedMap" +"avgWeightedMerge" +"avgWeightedNull" +"avgWeightedOrDefault" +"avgWeightedOrNull" +"avgWeightedResample" +"avgWeightedSimpleState" +"avgWeightedState" +"bar" +"base58Decode" +"base58Encode" +"base64Decode" +"base64Encode" +"basename" +"bin" +"bitAnd" +"bitCount" +"bitHammingDistance" +"bitNot" +"bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" "bitTest" -"ngramSearchCaseInsensitiveUTF8" +"bitTestAll" +"bitTestAny" +"bitXor" +"bitmapAnd" +"bitmapAndCardinality" +"bitmapAndnot" +"bitmapAndnotCardinality" +"bitmapBuild" +"bitmapCardinality" +"bitmapContains" +"bitmapHasAll" +"bitmapHasAny" "bitmapMax" -"ngramDistanceCaseInsensitiveUTF8" -"ngramDistance" +"bitmapMin" +"bitmapOr" +"bitmapOrCardinality" +"bitmapSubsetInRange" +"bitmapSubsetLimit" +"bitmapToArray" +"bitmapTransform" +"bitmapXor" +"bitmapXorCardinality" +"bitmaskToArray" +"bitmaskToList" +"blockNumber" +"blockSerializedSize" +"blockSize" +"boundingRatio" +"boundingRatioArgMax" +"boundingRatioArgMin" +"boundingRatioArray" +"boundingRatioDistinct" +"boundingRatioForEach" +"boundingRatioIf" +"boundingRatioMap" +"boundingRatioMerge" +"boundingRatioNull" +"boundingRatioOrDefault" +"boundingRatioOrNull" +"boundingRatioResample" +"boundingRatioSimpleState" +"boundingRatioState" +"buildId" +"byteHammingDistance" +"byteSize" +"byteSlice" +"byteSwap" +"caseWithExpr" +"caseWithExpression" +"caseWithoutExpr" +"caseWithoutExpression" +"catboostEvaluate" +"categoricalInformationValue" +"categoricalInformationValueArgMax" +"categoricalInformationValueArgMin" +"categoricalInformationValueArray" +"categoricalInformationValueDistinct" +"categoricalInformationValueForEach" +"categoricalInformationValueIf" +"categoricalInformationValueMap" +"categoricalInformationValueMerge" +"categoricalInformationValueNull" +"categoricalInformationValueOrDefault" +"categoricalInformationValueOrNull" +"categoricalInformationValueResample" +"categoricalInformationValueSimpleState" +"categoricalInformationValueState" +"cbrt" +"ceil" +"ceiling" +"char" +"cityHash64" +"coalesce" +"concat" +"concatAssumeInjective" +"concatWithSeparator" +"concatWithSeparatorAssumeInjective" +"concat_ws" +"connectionId" +"connection_id" +"contingency" +"contingencyArgMax" +"contingencyArgMin" +"contingencyArray" +"contingencyDistinct" +"contingencyForEach" +"contingencyIf" +"contingencyMap" +"contingencyMerge" +"contingencyNull" +"contingencyOrDefault" +"contingencyOrNull" +"contingencyResample" +"contingencySimpleState" +"contingencyState" +"convertCharset" +"corr" +"corrArgMax" +"corrArgMin" +"corrArray" +"corrDistinct" +"corrForEach" +"corrIf" +"corrMap" +"corrMatrix" +"corrMatrixArgMax" +"corrMatrixArgMin" +"corrMatrixArray" +"corrMatrixDistinct" +"corrMatrixForEach" +"corrMatrixIf" +"corrMatrixMap" +"corrMatrixMerge" +"corrMatrixNull" +"corrMatrixOrDefault" +"corrMatrixOrNull" +"corrMatrixResample" +"corrMatrixSimpleState" +"corrMatrixState" +"corrMerge" +"corrNull" +"corrOrDefault" +"corrOrNull" +"corrResample" +"corrSimpleState" +"corrStable" +"corrStableArgMax" +"corrStableArgMin" +"corrStableArray" +"corrStableDistinct" +"corrStableForEach" +"corrStableIf" +"corrStableMap" +"corrStableMerge" +"corrStableNull" +"corrStableOrDefault" +"corrStableOrNull" +"corrStableResample" +"corrStableSimpleState" +"corrStableState" +"corrState" +"cos" +"cosh" +"cosineDistance" +"count" +"countArgMax" +"countArgMin" +"countArray" +"countDigits" +"countDistinct" +"countEqual" +"countForEach" +"countIf" +"countMap" +"countMatches" +"countMatchesCaseInsensitive" +"countMerge" +"countNull" +"countOrDefault" +"countOrNull" +"countResample" +"countSimpleState" +"countState" +"countSubstrings" +"countSubstringsCaseInsensitive" +"countSubstringsCaseInsensitiveUTF8" +"covarPop" +"covarPopArgMax" +"covarPopArgMin" +"covarPopArray" +"covarPopDistinct" +"covarPopForEach" +"covarPopIf" +"covarPopMap" +"covarPopMatrix" +"covarPopMatrixArgMax" +"covarPopMatrixArgMin" +"covarPopMatrixArray" +"covarPopMatrixDistinct" +"covarPopMatrixForEach" +"covarPopMatrixIf" +"covarPopMatrixMap" +"covarPopMatrixMerge" +"covarPopMatrixNull" +"covarPopMatrixOrDefault" +"covarPopMatrixOrNull" +"covarPopMatrixResample" +"covarPopMatrixSimpleState" +"covarPopMatrixState" +"covarPopMerge" +"covarPopNull" +"covarPopOrDefault" +"covarPopOrNull" +"covarPopResample" +"covarPopSimpleState" +"covarPopStable" +"covarPopStableArgMax" +"covarPopStableArgMin" +"covarPopStableArray" +"covarPopStableDistinct" +"covarPopStableForEach" +"covarPopStableIf" +"covarPopStableMap" +"covarPopStableMerge" +"covarPopStableNull" +"covarPopStableOrDefault" +"covarPopStableOrNull" +"covarPopStableResample" +"covarPopStableSimpleState" +"covarPopStableState" +"covarPopState" +"covarSamp" +"covarSampArgMax" +"covarSampArgMin" +"covarSampArray" +"covarSampDistinct" +"covarSampForEach" +"covarSampIf" +"covarSampMap" +"covarSampMatrix" +"covarSampMatrixArgMax" +"covarSampMatrixArgMin" +"covarSampMatrixArray" +"covarSampMatrixDistinct" +"covarSampMatrixForEach" +"covarSampMatrixIf" +"covarSampMatrixMap" +"covarSampMatrixMerge" +"covarSampMatrixNull" +"covarSampMatrixOrDefault" +"covarSampMatrixOrNull" +"covarSampMatrixResample" +"covarSampMatrixSimpleState" +"covarSampMatrixState" +"covarSampMerge" +"covarSampNull" +"covarSampOrDefault" +"covarSampOrNull" +"covarSampResample" +"covarSampSimpleState" +"covarSampStable" +"covarSampStableArgMax" +"covarSampStableArgMin" +"covarSampStableArray" +"covarSampStableDistinct" +"covarSampStableForEach" +"covarSampStableIf" +"covarSampStableMap" +"covarSampStableMerge" +"covarSampStableNull" +"covarSampStableOrDefault" +"covarSampStableOrNull" +"covarSampStableResample" +"covarSampStableSimpleState" +"covarSampStableState" +"covarSampState" +"cramersV" +"cramersVArgMax" +"cramersVArgMin" +"cramersVArray" +"cramersVBiasCorrected" +"cramersVBiasCorrectedArgMax" +"cramersVBiasCorrectedArgMin" +"cramersVBiasCorrectedArray" +"cramersVBiasCorrectedDistinct" +"cramersVBiasCorrectedForEach" +"cramersVBiasCorrectedIf" +"cramersVBiasCorrectedMap" +"cramersVBiasCorrectedMerge" +"cramersVBiasCorrectedNull" +"cramersVBiasCorrectedOrDefault" +"cramersVBiasCorrectedOrNull" +"cramersVBiasCorrectedResample" +"cramersVBiasCorrectedSimpleState" +"cramersVBiasCorrectedState" +"cramersVDistinct" +"cramersVForEach" +"cramersVIf" +"cramersVMap" +"cramersVMerge" +"cramersVNull" +"cramersVOrDefault" +"cramersVOrNull" +"cramersVResample" +"cramersVSimpleState" +"cramersVState" +"curdate" +"currentDatabase" +"currentProfiles" +"currentRoles" +"currentSchemas" +"currentUser" +"current_database" +"current_date" +"current_schemas" +"current_timestamp" +"cutFragment" +"cutIPv6" +"cutQueryString" +"cutQueryStringAndFragment" +"cutToFirstSignificantSubdomain" +"cutToFirstSignificantSubdomainCustom" +"cutToFirstSignificantSubdomainCustomRFC" +"cutToFirstSignificantSubdomainCustomWithWWW" +"cutToFirstSignificantSubdomainCustomWithWWWRFC" +"cutToFirstSignificantSubdomainRFC" +"cutToFirstSignificantSubdomainWithWWW" +"cutToFirstSignificantSubdomainWithWWWRFC" +"cutURLParameter" +"cutWWW" +"damerauLevenshteinDistance" +"dateDiff" +"dateName" +"dateTime64ToSnowflake" +"dateTimeToSnowflake" +"dateTrunc" +"date_diff" +"decodeHTMLComponent" +"decodeURLComponent" +"decodeURLFormComponent" +"decodeXMLComponent" +"decrypt" +"defaultProfiles" +"defaultRoles" +"defaultValueOfArgumentType" +"defaultValueOfTypeName" +"degrees" +"deltaSum" +"deltaSumArgMax" +"deltaSumArgMin" +"deltaSumArray" +"deltaSumDistinct" +"deltaSumForEach" +"deltaSumIf" +"deltaSumMap" +"deltaSumMerge" +"deltaSumNull" +"deltaSumOrDefault" +"deltaSumOrNull" +"deltaSumResample" +"deltaSumSimpleState" +"deltaSumState" +"deltaSumTimestamp" +"deltaSumTimestampArgMax" +"deltaSumTimestampArgMin" +"deltaSumTimestampArray" +"deltaSumTimestampDistinct" +"deltaSumTimestampForEach" +"deltaSumTimestampIf" +"deltaSumTimestampMap" +"deltaSumTimestampMerge" +"deltaSumTimestampNull" +"deltaSumTimestampOrDefault" +"deltaSumTimestampOrNull" +"deltaSumTimestampResample" +"deltaSumTimestampSimpleState" +"deltaSumTimestampState" +"demangle" +"dense_rank" +"dense_rankArgMax" +"dense_rankArgMin" +"dense_rankArray" +"dense_rankDistinct" +"dense_rankForEach" +"dense_rankIf" +"dense_rankMap" +"dense_rankMerge" +"dense_rankNull" +"dense_rankOrDefault" +"dense_rankOrNull" +"dense_rankResample" +"dense_rankSimpleState" +"dense_rankState" +"detectCharset" +"detectLanguage" +"detectLanguageMixed" +"detectLanguageUnknown" +"detectProgrammingLanguage" +"detectTonality" +"dictGet" +"dictGetAll" +"dictGetChildren" +"dictGetDate" +"dictGetDateOrDefault" +"dictGetDateTime" +"dictGetDateTimeOrDefault" +"dictGetDescendants" +"dictGetFloat32" +"dictGetFloat32OrDefault" +"dictGetFloat64" +"dictGetFloat64OrDefault" +"dictGetHierarchy" +"dictGetIPv4" +"dictGetIPv4OrDefault" +"dictGetIPv6" +"dictGetIPv6OrDefault" +"dictGetInt16" +"dictGetInt16OrDefault" +"dictGetInt32" +"dictGetInt32OrDefault" +"dictGetInt64" +"dictGetInt64OrDefault" +"dictGetInt8" +"dictGetInt8OrDefault" +"dictGetOrDefault" +"dictGetOrNull" +"dictGetString" +"dictGetStringOrDefault" +"dictGetUInt16" +"dictGetUInt16OrDefault" +"dictGetUInt32" +"dictGetUInt32OrDefault" +"dictGetUInt64" +"dictGetUInt64OrDefault" +"dictGetUInt8" +"dictGetUInt8OrDefault" +"dictGetUUID" +"dictGetUUIDOrDefault" +"dictHas" +"dictIsIn" +"displayName" +"distanceL1" +"distanceL2" +"distanceL2Squared" +"distanceLinf" +"distanceLp" +"divide" +"divideDecimal" +"domain" +"domainRFC" +"domainWithoutWWW" +"domainWithoutWWWRFC" +"dotProduct" +"dumpColumnStructure" +"e" +"editDistance" +"empty" +"emptyArrayDate" +"emptyArrayDateTime" +"emptyArrayFloat32" +"emptyArrayFloat64" +"emptyArrayInt16" +"emptyArrayInt32" +"emptyArrayInt64" +"emptyArrayInt8" +"emptyArrayString" +"emptyArrayToSingle" +"emptyArrayUInt16" +"emptyArrayUInt32" +"emptyArrayUInt64" +"emptyArrayUInt8" +"enabledProfiles" +"enabledRoles" +"encodeURLComponent" +"encodeURLFormComponent" +"encodeXMLComponent" +"encrypt" +"endsWith" +"endsWithUTF8" +"entropy" +"entropyArgMax" +"entropyArgMin" +"entropyArray" +"entropyDistinct" +"entropyForEach" +"entropyIf" +"entropyMap" +"entropyMerge" +"entropyNull" +"entropyOrDefault" +"entropyOrNull" +"entropyResample" +"entropySimpleState" +"entropyState" +"equals" +"erf" +"erfc" +"errorCodeToName" +"evalMLMethod" +"exp" +"exp10" +"exp2" +"exponentialMovingAverage" +"exponentialMovingAverageArgMax" +"exponentialMovingAverageArgMin" +"exponentialMovingAverageArray" +"exponentialMovingAverageDistinct" +"exponentialMovingAverageForEach" +"exponentialMovingAverageIf" +"exponentialMovingAverageMap" +"exponentialMovingAverageMerge" +"exponentialMovingAverageNull" +"exponentialMovingAverageOrDefault" +"exponentialMovingAverageOrNull" +"exponentialMovingAverageResample" +"exponentialMovingAverageSimpleState" +"exponentialMovingAverageState" +"exponentialTimeDecayedAvg" +"exponentialTimeDecayedAvgArgMax" +"exponentialTimeDecayedAvgArgMin" +"exponentialTimeDecayedAvgArray" +"exponentialTimeDecayedAvgDistinct" +"exponentialTimeDecayedAvgForEach" +"exponentialTimeDecayedAvgIf" +"exponentialTimeDecayedAvgMap" +"exponentialTimeDecayedAvgMerge" +"exponentialTimeDecayedAvgNull" +"exponentialTimeDecayedAvgOrDefault" +"exponentialTimeDecayedAvgOrNull" +"exponentialTimeDecayedAvgResample" +"exponentialTimeDecayedAvgSimpleState" +"exponentialTimeDecayedAvgState" +"exponentialTimeDecayedCount" +"exponentialTimeDecayedCountArgMax" +"exponentialTimeDecayedCountArgMin" +"exponentialTimeDecayedCountArray" +"exponentialTimeDecayedCountDistinct" +"exponentialTimeDecayedCountForEach" +"exponentialTimeDecayedCountIf" +"exponentialTimeDecayedCountMap" +"exponentialTimeDecayedCountMerge" +"exponentialTimeDecayedCountNull" +"exponentialTimeDecayedCountOrDefault" +"exponentialTimeDecayedCountOrNull" +"exponentialTimeDecayedCountResample" +"exponentialTimeDecayedCountSimpleState" +"exponentialTimeDecayedCountState" +"exponentialTimeDecayedMax" +"exponentialTimeDecayedMaxArgMax" +"exponentialTimeDecayedMaxArgMin" +"exponentialTimeDecayedMaxArray" +"exponentialTimeDecayedMaxDistinct" +"exponentialTimeDecayedMaxForEach" +"exponentialTimeDecayedMaxIf" +"exponentialTimeDecayedMaxMap" +"exponentialTimeDecayedMaxMerge" +"exponentialTimeDecayedMaxNull" +"exponentialTimeDecayedMaxOrDefault" +"exponentialTimeDecayedMaxOrNull" +"exponentialTimeDecayedMaxResample" +"exponentialTimeDecayedMaxSimpleState" +"exponentialTimeDecayedMaxState" +"exponentialTimeDecayedSum" +"exponentialTimeDecayedSumArgMax" +"exponentialTimeDecayedSumArgMin" +"exponentialTimeDecayedSumArray" +"exponentialTimeDecayedSumDistinct" +"exponentialTimeDecayedSumForEach" +"exponentialTimeDecayedSumIf" +"exponentialTimeDecayedSumMap" +"exponentialTimeDecayedSumMerge" +"exponentialTimeDecayedSumNull" +"exponentialTimeDecayedSumOrDefault" +"exponentialTimeDecayedSumOrNull" +"exponentialTimeDecayedSumResample" +"exponentialTimeDecayedSumSimpleState" +"exponentialTimeDecayedSumState" +"extract" +"extractAll" +"extractAllGroups" "extractAllGroupsHorizontal" "extractAllGroupsVertical" -"multiFuzzyMatchAllIndices" -"multiFuzzyMatchAnyIndex" -"multiMatchAnyIndex" -"replaceRegexpAll" -"emptyArrayInt16" -"dictGetOrDefault" -"replaceOne" -"emptyArrayInt32" -"extract" -"readWKTPolygon" -"notILike" -"geohashDecode" -"toModifiedJulianDay" -"notLike" -"countSubstringsCaseInsensitive" -"IPv4StringToNum" -"positionUTF8" -"hasToken" -"dictGetDate" -"multiSearchFirstPosition" -"multiSearchFirstIndexCaseInsensitiveUTF8" -"atan" -"rowNumberInAllBlocks" -"multiSearchFirstIndexCaseInsensitive" -"multiSearchAnyUTF8" -"parseDateTime32BestEffortOrZero" -"blockNumber" -"cutURLParameter" -"multiSearchFirstPositionCaseInsensitive" -"positionCaseInsensitiveUTF8" -"positionCaseInsensitive" -"splitByRegexp" -"toInt256OrZero" -"pointInPolygon" -"splitByWhitespace" -"lemmatize" -"synonyms" -"tryBase64Decode" -"multiSearchAllPositions" -"FROM_BASE64" -"replaceRegexpOne" -"subBitmap" -"defaultValueOfArgumentType" -"base64Decode" -"regionToContinent" -"ilike" -"fromModifiedJulianDay" -"TO_BASE64" -"dictIsIn" -"base64Encode" -"decodeXMLComponent" -"countMatchesCaseInsensitive" -"normalizeQuery" -"erf" -"trimBoth" -"fromUnixTimestamp64Nano" -"lessOrEquals" -"subtractQuarters" -"ngramSearch" -"readWKTRing" -"trimRight" -"endsWith" -"ngramDistanceCaseInsensitive" -"connectionId" -"initialQueryID" -"startsWith" -"tan" -"substring" +"extractGroups" +"extractKeyValuePairs" +"extractKeyValuePairsWithEscaping" +"extractTextFromHTML" +"extractURLParameter" +"extractURLParameterNames" +"extractURLParameters" +"factorial" +"farmFingerprint64" +"farmHash64" +"file" +"filesystemAvailable" +"filesystemCapacity" +"filesystemUnreserved" +"finalizeAggregation" +"firstLine" +"firstSignificantSubdomain" +"firstSignificantSubdomainCustom" +"firstSignificantSubdomainCustomRFC" +"firstSignificantSubdomainRFC" +"first_value" +"first_valueArgMax" +"first_valueArgMin" +"first_valueArray" +"first_valueDistinct" +"first_valueForEach" +"first_valueIf" +"first_valueMap" +"first_valueMerge" +"first_valueNull" +"first_valueOrDefault" +"first_valueOrNull" +"first_valueResample" +"first_valueSimpleState" +"first_valueState" +"first_value_respect_nulls" +"first_value_respect_nullsArgMax" +"first_value_respect_nullsArgMin" +"first_value_respect_nullsArray" +"first_value_respect_nullsDistinct" +"first_value_respect_nullsForEach" +"first_value_respect_nullsIf" +"first_value_respect_nullsMap" +"first_value_respect_nullsMerge" +"first_value_respect_nullsNull" +"first_value_respect_nullsOrDefault" +"first_value_respect_nullsOrNull" +"first_value_respect_nullsResample" +"first_value_respect_nullsSimpleState" +"first_value_respect_nullsState" +"flameGraph" +"flameGraphArgMax" +"flameGraphArgMin" +"flameGraphArray" +"flameGraphDistinct" +"flameGraphForEach" +"flameGraphIf" +"flameGraphMap" +"flameGraphMerge" +"flameGraphNull" +"flameGraphOrDefault" +"flameGraphOrNull" +"flameGraphResample" +"flameGraphSimpleState" +"flameGraphState" +"flatten" +"flattenTuple" +"floor" +"format" +"formatDateTime" +"formatDateTimeInJodaSyntax" +"formatQuery" +"formatQueryOrNull" +"formatQuerySingleLine" +"formatQuerySingleLineOrNull" +"formatReadableDecimalSize" +"formatReadableQuantity" +"formatReadableSize" +"formatReadableTimeDelta" "formatRow" "formatRowNoNewline" -"dictGetHierarchy" -"notEmpty" -"format" -"murmurHash3_64" -"reverseUTF8" -"arrayEnumerateUniqRanked" -"CRC64" -"pow" -"toInt16" -"reverse" -"lower" -"toStartOfDay" -"caseWithoutExpr" -"CHAR_LENGTH" -"lengthUTF8" -"length" -"bitTestAny" -"floor" -"round" -"reinterpretAsFixedString" -"reinterpretAsString" -"h3ToChildren" -"emptyArrayToSingle" -"tgamma" -"reinterpretAsInt128" -"visitParamExtractRaw" -"reinterpretAsInt64" -"reinterpretAsUInt32" -"reinterpretAsUInt16" -"fuzzBits" -"randomFixedString" -"tupleHammingDistance" -"JSONExtractBool" -"ngramSearchUTF8" -"generateUUIDv4" -"multiSearchAllPositionsCaseInsensitive" -"randConstant" -"leftPad" -"convertCharset" -"extractURLParameter" -"zookeeperSessionUptime" -"serverUUID" -"firstSignificantSubdomain" -"rand64" -"JSON_EXISTS" -"isIPAddressInRange" -"wordShingleSimHashCaseInsensitive" -"parseDateTime32BestEffortOrNull" -"cutToFirstSignificantSubdomainWithWWW" -"roundToExp2" -"mapSubtract" -"partitionId" -"intExp2" -"byteSize" -"errorCodeToName" -"toDateTime64" -"yesterday" -"initializeAggregation" -"globalVariable" -"countDigits" -"toStartOfHour" -"toWeek" -"isDecimalOverflow" -"visitParamExtractString" -"isConstant" -"getSetting" -"shardNum" -"__getScalar" -"position" -"__bitBoolMaskOr" -"cosh" -"basename" -"evalMLMethod" -"filesystemUnreserved" -"filesystemCapacity" -"reinterpretAsDate" -"filesystemAvailable" -"joinGet" -"globalNullInIgnoreSet" -"globalNotInIgnoreSet" -"toRelativeDayNum" -"splitByChar" -"notInIgnoreSet" -"globalInIgnoreSet" -"notNullIn" -"ngramSearchCaseInsensitive" -"notIn" -"trunc" -"JSONExtractArrayRaw" -"version" -"joinGetOrNull" -"globalIn" -"toDate32" -"lowCardinalityIndices" -"toLowCardinality" -"runningConcurrency" -"runningDifferenceStartingWithFirstValue" -"bitCount" -"timezoneOf" -"timezone" -"uptime" -"pi" -"extractTextFromHTML" -"UUIDStringToNum" -"roundDown" -"IPv6NumToString" -"throwIf" -"ifNotFinite" -"FROM_UNIXTIME" -"isInfinite" -"arrayProduct" -"polygonsIntersectionSpherical" -"toRelativeHourNum" -"hasColumnInTable" -"bar" -"JSONExtractKeysAndValues" -"replicate" -"arrayJoin" -"s2RectUnion" -"ngramDistanceUTF8" -"identity" -"indexHint" -"ignore" -"regionHierarchy" -"toDateTime64OrNull" -"polygonsWithinCartesian" -"materialize" -"sleepEachRow" -"parseDateTimeBestEffortUSOrNull" -"neighbor" -"splitByString" -"rowNumberInBlock" -"blockSize" -"defaultValueOfTypeName" -"hiveHash" -"randomStringUTF8" -"blockSerializedSize" -"visibleWidth" -"fullHostName" -"hostName" -"arrayFilter" -"defaultRoles" -"currentRoles" -"toUInt64OrZero" -"dictGetUInt16" -"DATABASE" -"toUInt8OrZero" -"reinterpretAsInt8" -"not" -"log1p" -"and" -"wordShingleMinHash" -"arrayDifference" -"arrayCumSumNonNegative" -"wordShingleSimHash" -"arrayCumSum" -"arraySort" -"arrayPartialSort" -"arrayPartialReverseSort" -"toDaysSinceYearZero" +"fragment" "fromDaysSinceYearZero" "fromDaysSinceYearZero32" -"dumpColumnStructure" -"multiSearchFirstIndex" -"arrayReverseSplit" -"ifNull" -"normalizeQueryKeepNames" -"xxHash64" -"in" -"arrayReverseFill" -"erfc" -"cutToFirstSignificantSubdomainCustomWithWWW" -"arrayFill" -"substringUTF8" -"arrayFirst" -"arrayAvg" -"cos" -"arrayMax" -"toColumnTypeName" -"arrayMin" -"toIntervalDay" -"reinterpretAsFloat32" -"arrayAll" -"or" -"arrayExists" -"trimLeft" -"arrayMap" -"polygonsSymDifferenceCartesian" -"gccMurmurHash" -"exp2" -"polygonPerimeterCartesian" -"SHA384" -"murmurHash3_128" -"polygonConvexHullCartesian" -"murmurHash3_32" -"h3GetBaseCell" -"murmurHash2_64" -"globalNullIn" -"javaHashUTF16LE" -"URLHash" -"reinterpretAsInt16" -"regionToCountry" -"toUUID" -"intHash32" -"metroHash64" -"arrayPushFront" -"bitmapAndnotCardinality" -"toFloat32" -"sipHash64" -"JSONExtractFloat" -"SHA512" -"JSONExtractUInt" -"arrayReduce" -"SHA256" -"array" -"parseDateTimeBestEffortUSOrZero" -"toInt32OrNull" -"MD5" -"randomString" -"__bitBoolMaskAnd" -"hasSubstr" -"formatReadableQuantity" -"replaceAll" -"toIntervalWeek" -"toRelativeSecondNum" -"demangle" -"toNullable" -"concat" -"formatReadableDecimalSize" -"formatReadableSize" -"shardCount" +"fromModifiedJulianDay" "fromModifiedJulianDayOrNull" -"toInt128OrZero" -"bitmaskToList" -"xor" -"bitPositionsToArray" -"empty" -"ngramMinHashArg" -"_CAST" -"dictGetStringOrDefault" -"arrayZip" -"dictGetUUIDOrDefault" -"dictGetFloat64OrDefault" -"dictGetInt64OrDefault" -"today" -"dictGetUInt32OrDefault" -"countSubstringsCaseInsensitiveUTF8" -"h3IsPentagon" -"dictGetUInt16OrDefault" -"dictGetDescendants" -"dictGetString" -"inIgnoreSet" -"dictGetUUID" -"arraySlice" -"reinterpretAsInt32" -"dictGetFloat64" -"globalNotNullInIgnoreSet" -"arrayUniq" -"polygonAreaCartesian" -"intDiv" -"cityHash64" -"regionToTopContinent" -"toInt32" -"e" -"wordShingleSimHashCaseInsensitiveUTF8" -"JSON_QUERY" -"isIPv4String" -"UUIDNumToString" -"modulo" -"reinterpret" -"unbin" -"regionToDistrict" -"sinh" -"date_trunc" -"__bitSwapLastTwo" -"formatDateTime" -"dateDiff" -"tanh" -"multiSearchAnyCaseInsensitiveUTF8" -"upperUTF8" -"bitmaskToArray" -"arrayAUC" -"multiMatchAllIndices" -"hasThreadFuzzer" -"toDayOfWeek" -"isFinite" -"transform" -"timezoneOffset" -"toTimezone" -"cutQueryString" -"subtractYears" -"tcpPort" -"subtractHours" -"tid" -"toDateTime64OrZero" -"subtractSeconds" -"alphaTokens" -"negate" -"file" -"roundAge" -"MACStringToOUI" -"addQuarters" -"arrayResize" -"addWeeks" -"HOUR" -"addHours" -"visitParamExtractUInt" -"caseWithExpression" -"rand" -"addSeconds" -"splitByNonAlpha" -"arrayStringConcat" -"randomPrintableASCII" -"h3HexAreaM2" -"toYYYYMMDDhhmmss" -"toInt16OrNull" -"timeSlots" -"timeSlot" -"dictGetInt16" -"queryString" -"javaHash" -"acosh" -"toRelativeMinuteNum" -"toRelativeYearNum" -"unhex" -"toInt8OrNull" -"regionToPopulation" -"toStartOfISOYear" -"getSizeOfEnumType" -"toStartOfInterval" -"toYYYYMMDD" -"multiSearchFirstPositionUTF8" -"CRC32" -"toStartOfFifteenMinutes" -"emptyArrayUInt8" -"dictGetUInt8" -"toStartOfFiveMinutes" -"cbrt" -"toStartOfMinute" -"dictGet" -"toStartOfSecond" -"hasTokenCaseInsensitive" -"toStartOfYear" -"farmHash64" -"toStartOfMonth" -"queryID" -"wordShingleMinHashCaseInsensitive" -"dictGetDateOrDefault" -"dictGetUInt64OrDefault" -"arrayElement" -"sleep" -"dictHas" -"enabledProfiles" -"toDecimal32OrNull" -"multiFuzzyMatchAny" -"toStartOfWeek" -"isValidUTF8" -"dictGetInt8" -"dictGetInt32OrDefault" -"toMonday" -"multiSearchFirstIndexUTF8" -"lowCardinalityKeys" -"halfMD5" -"domainWithoutWWW" -"MINUTE" -"addDays" -"toMinute" -"JSONExtractRaw" -"toUInt16OrNull" -"bin" -"toDayOfYear" -"log" -"DAYOFMONTH" -"toUInt32" -"toDecimal32" -"s2RectAdd" -"dictGetFloat32OrDefault" -"xxHash32" -"DAY" -"toISOWeek" -"ngramMinHash" -"toMonth" -"countMatches" -"toQuarter" -"toISOYear" -"YEAR" -"toIntervalYear" -"hypot" -"multiSearchAny" -"toIntervalQuarter" -"toSecond" -"accurate_CastOrNull" -"reinterpretAsUInt64" -"defaultProfiles" -"toIntervalMonth" -"bitRotateRight" -"arrayReverseSort" -"toInt8OrZero" -"SHA1" -"multiSearchFirstPositionCaseInsensitiveUTF8" -"cutFragment" -"toDecimal256" -"toIntervalSecond" -"toUInt256OrZero" -"toStringCutToZero" -"normalizedQueryHashKeepNames" -"coalesce" -"parseDateTime64BestEffortOrNull" -"toIntervalHour" -"bitmapSubsetLimit" -"parseDateTimeBestEffortOrNull" -"match" -"fragment" -"repeat" -"parseDateTimeBestEffortOrZero" -"toDecimal128OrNull" -"parseDateTimeBestEffort" -"simpleJSONExtractUInt" -"multiMatchAny" -"toDecimal256OrNull" -"roundDuration" -"addMinutes" -"toStartOfQuarter" -"exp10" -"regionIn" -"toDecimal64OrNull" -"toDateTimeOrNull" -"simpleJSONExtractFloat" -"reinterpretAsUUID" -"dictGetDateTimeOrDefault" -"greater" -"toRelativeMonthNum" -"toDate32OrNull" -"h3IndexesAreNeighbors" -"toFloat32OrNull" -"toInt256OrNull" -"leftPadUTF8" -"regionToArea" -"toInt64OrNull" -"toModifiedJulianDayOrNull" -"toUInt64OrNull" -"polygonsEqualsCartesian" -"accurateCast" -"toUInt32OrNull" -"encodeXMLComponent" -"toUInt8OrNull" -"toUUIDOrZero" -"arrayEnumerateDenseRanked" -"toYYYYMM" -"toDecimal256OrZero" -"fromUnixTimestamp64Milli" -"CHARACTER_LENGTH" -"isNull" -"isNaN" -"multiSearchAllPositionsCaseInsensitiveUTF8" -"FQDN" -"currentProfiles" -"toDateTime32" -"arraySplit" -"toIPv6" -"toDecimal32OrZero" -"bitmapCardinality" -"arrayFlatten" -"dateName" -"s2RectIntersection" -"CAST" -"caseWithExpr" -"nullIf" -"toDateTimeOrZero" -"bitOr" -"CRC32IEEE" -"emptyArrayDate" -"toUInt128OrZero" -"toUInt128OrNull" -"toFloat32OrZero" -"SECOND" -"arrayCompact" -"bitmapMin" -"toInt16OrZero" -"reinterpretAsUInt256" -"dictGetOrNull" -"map" -"bitmapHasAny" -"emptyArrayUInt16" -"cutIPv6" -"toUInt32OrZero" -"toTypeName" -"polygonsUnionCartesian" -"char" -"mapContains" -"emptyArrayDateTime" -"toUInt16OrZero" -"toInt64" -"wkt" -"dictGetUInt64" -"JSONHas" -"runningDifference" -"getServerPort" -"IPv4ToIPv6" -"__bitWrapperFunc" -"multiply" -"accurate_Cast" -"toDecimal64" -"toString" -"arrayIntersect" -"dictGetUInt8OrDefault" -"toTime" -"hasAll" -"now" -"parseDateTime64BestEffortOrZero" -"rightPadUTF8" -"emptyArrayInt64" -"simpleJSONExtractInt" -"toDecimal128" -"toRelativeWeekNum" -"toDateOrNull" -"topLevelDomain" -"greatest" -"isIPv6String" -"toHour" -"toFixedString" -"bitmapOrCardinality" -"toInt128" -"toDateOrZero" -"emptyArrayUInt64" -"toUInt128" -"isNotNull" -"toUInt16" -"MACNumToString" -"multiSearchAllPositionsUTF8" -"toUInt8" -"regionToCity" -"DAYOFYEAR" -"simpleJSONExtractRaw" -"appendTrailingCharIfAbsent" -"toUInt64" -"regionToName" -"IPv4NumToStringClassC" -"upper" -"greaterOrEquals" -"s2RectContains" -"toDate" -"regexpQuoteMeta" -"readWKTMultiPolygon" -"emptyArrayString" -"bitmapOr" -"cutWWW" -"emptyArrayInt8" -"less" -"readWKTPoint" -"reinterpretAsDateTime" -"notEquals" -"geoToS2" -"nullInIgnoreSet" -"globalNotNullIn" -"arrayConcat" -"mapPopulateSeries" -"parseDateTimeBestEffortUS" -"toDayOfMonth" -"gcd" -"protocol" -"currentUser" -"if" -"caseWithoutExpression" -"currentDatabase" -"log10" -"moduloOrZero" -"MD4" -"toInt64OrZero" -"arrayWithConstant" -"IPv4NumToString" -"dictGetUInt32" -"multiSearchAnyCaseInsensitive" -"countSubstrings" -"reinterpretAsFloat64" -"plus" -"dictGetChildren" -"IPv6CIDRToRange" -"toIntervalMinute" -"bitRotateLeft" -"geoDistance" -"mapValues" -"toIPv4" -"extractGroups" -"toYearWeek" -"polygonsDistanceCartesian" -"lowerUTF8" -"toUInt256OrNull" -"IPv4CIDRToRange" -"MACStringToNum" -"toFloat64OrNull" -"sipHash128" -"toFloat64" -"arrayPopFront" -"bitmapAnd" -"bitmapHasAll" -"subtractWeeks" -"arrayFirstIndex" -"bitmapXorCardinality" -"bitmapAndnot" -"asinh" -"dictGetInt32" -"toFloat64OrZero" -"toInt8" -"polygonsWithinSpherical" -"bitmapXor" -"intExp10" -"dictGetDateTime" -"h3ToGeoBoundary" -"decodeURLComponent" -"bitAnd" -"toUUIDOrNull" -"notNullInIgnoreSet" -"tupleElement" -"tupleToNameValuePairs" -"tuple" -"ceil" -"DAYOFWEEK" -"toDateTime" -"bitNot" -"rightPad" -"has" -"bitmapToArray" -"arrayCount" -"arrayDistinct" -"greatCircleAngle" -"IPv6StringToNum" -"dictGetInt64" -"toUnixTimestamp" -"getMacro" -"runningAccumulate" -"multiIf" -"abs" -"toValidUTF8" -"arrayEnumerateUniq" -"moduloLegacy" -"JSONExtractInt" -"subtractMonths" -"dateTimeToSnowflake" -"cutToFirstSignificantSubdomain" -"roundBankers" -"toUInt256" -"arrayEnumerateDense" -"stem" -"bitmapAndCardinality" -"bitTestAll" -"normalizedQueryHash" -"toYear" -"emptyArrayFloat64" -"bitmapTransform" -"JSONExtract" -"bitShiftRight" -"bitmapContains" -"sqrt" -"minus" -"QUARTER" -"parseDateTime64BestEffort" -"h3IsValid" -"h3EdgeAngle" -"concatAssumeInjective" -"emptyArrayUInt32" -"intHash64" -"enabledRoles" -"fromUnixTimestamp64Micro" -"range" -"subtractMinutes" -"farmFingerprint64" -"arrayReduceInRanges" -"bitmapBuild" -"bitmapSubsetInRange" -"arrayPopBack" -"h3ToGeo" -"hex" -"arrayPushBack" -"subtractDays" -"buildId" -"URLPathHierarchy" -"addMonths" -"parseDateTime32BestEffort" -"indexOf" -"polygonsUnionSpherical" -"arraySum" -"dictGetFloat32" -"ngramSimHashCaseInsensitiveUTF8" -"finalizeAggregation" -"divide" -"netloc" -"bitShiftLeft" -"toInt32OrZero" -"extractAll" -"toInt128OrNull" -"bitXor" -"lcm" -"accurateCastOrNull" -"toDate32OrZero" -"simpleJSONExtractString" -"least" -"nullIn" -"MONTH" -"arrayReverse" -"arrayShuffle" -"arrayPartialShuffle" -"now64" -"DATE" -"addressToLine" -"globalNotIn" -"dateTime64ToSnowflake" -"dictGetInt8OrDefault" -"exp" -"toRelativeQuarterNum" -"arrayEnumerate" -"mapAdd" -"murmurHash2_32" -"toStartOfTenMinutes" -"intDivOrZero" -"addYears" -"queryStringAndFragment" -"emptyArrayFloat32" -"countEqual" -"pointInEllipses" -"like" -"toDecimal64OrZero" -"reinterpretAsInt256" -"hasAny" -"ln" -"replace" -"rpad" -"mid" -"lcase" +"fromUTCTimestamp" "fromUnixTimestamp" -"truncate" -"ceiling" -"ucase" -"rand32" -"power" -"initial_query_id" -"timeZone" -"hostname" -"yearweek" -"timeZoneOffset" -"extractAllGroups" -"lpad" -"substr" -"INET6_NTOA" -"INET6_ATON" -"INET_ATON" -"timeZoneOf" -"locate" -"INET_NTOA" -"connection_id" -"user" -"toTimeZone" -"week" -"flatten" -"query_id" -"dateTrunc" -"mod" -"lagInFrame" -"dense_rank" -"rank" -"exponentialMovingAverage" -"sparkbar" -"singleValueOrNull" -"studentTTest" -"aggThrow" -"categoricalInformationValue" -"groupArrayMovingAvg" -"groupArrayMovingSum" -"simpleLinearRegression" -"entropy" -"quantilesBFloat16" -"maxIntersectionsPosition" -"groupBitmapXor" -"groupBitmap" -"skewPop" -"groupBitXor" -"groupBitOr" -"groupBitmapAnd" -"topKWeighted" -"stochasticLinearRegression" -"corr" -"corrMatrix" -"uniqCombined64" -"intervalLengthSum" -"uniqCombined" -"rankCorr" -"maxMap" -"minMap" -"sumMapFiltered" -"quantileExactWeighted" -"sumMapFilteredWithOverflow" -"sumMap" -"histogram" -"quantiles" -"sum" -"covarPop" -"covarPopMatrix" -"row_number" -"kurtPop" -"kurtSamp" -"skewSamp" -"uniqExact" -"sumMapWithOverflow" -"stddevSamp" -"varPop" -"corrStable" -"quantileTimingWeighted" -"covarPopStable" -"stddevSampStable" -"varSamp" -"topK" -"last_value" -"mannWhitneyUTest" -"maxIntersections" -"quantilesExact" -"uniqHLL12" -"quantileBFloat16" -"uniq" -"min" -"sequenceNextNode" -"quantilesTimingWeighted" -"boundingRatio" -"any" -"anyLast" -"deltaSum" -"retention" -"sequenceMatch" -"uniqUpTo" -"windowFunnel" -"deltaSumTimestamp" -"varSampStable" -"uniqTheta" -"quantilesExactWeighted" -"max" -"quantilesBFloat16Weighted" -"quantileBFloat16Weighted" -"sumKahan" -"quantilesTDigestWeighted" -"groupBitAnd" -"quantileTDigest" -"quantileTDigestWeighted" -"argMax" -"quantileDeterministic" -"quantilesTDigest" -"stochasticLogisticRegression" -"argMin" -"avg" -"covarSampStable" -"quantilesTiming" -"welchTTest" -"covarSamp" -"covarSampMatrix" -"varPopStable" -"quantileTiming" -"quantileExactInclusive" -"quantileExactHigh" -"groupArraySample" -"quantilesExactLow" -"groupBitmapOr" -"first_value" -"quantileExactExclusive" -"quantileExact" -"sumCount" -"groupArrayInsertAt" -"quantilesExactHigh" -"sumWithOverflow" -"sequenceCount" -"quantilesDeterministic" -"groupUniqArray" +"fromUnixTimestamp64Micro" +"fromUnixTimestamp64Milli" +"fromUnixTimestamp64Nano" +"fromUnixTimestampInJodaSyntax" +"from_utc_timestamp" +"fullHostName" +"fuzzBits" +"gccMurmurHash" +"gcd" +"generateRandomStructure" +"generateULID" +"generateUUIDv4" +"geoDistance" +"geoToH3" +"geoToS2" +"geohashDecode" +"geohashEncode" +"geohashesInBox" +"getMacro" +"getOSKernelVersion" +"getServerPort" +"getSetting" +"getSizeOfEnumType" +"getSubcolumn" +"getTypeSerializationStreams" +"globalIn" +"globalInIgnoreSet" +"globalNotIn" +"globalNotInIgnoreSet" +"globalNotNullIn" +"globalNotNullInIgnoreSet" +"globalNullIn" +"globalNullInIgnoreSet" +"globalVariable" +"greatCircleAngle" +"greatCircleDistance" +"greater" +"greaterOrEquals" +"greatest" "groupArray" -"anyHeavy" -"stddevPop" -"quantile" -"leadInFrame" -"quantilesExactExclusive" -"count" -"quantilesExactInclusive" -"stddevPopStable" -"quantileExactLow" -"avgWeighted" -"BIT_AND" -"VAR_SAMP" -"COVAR_SAMP" -"VAR_POP" -"medianTDigest" -"medianBFloat16" -"medianTimingWeighted" -"medianTiming" -"medianExactHigh" -"BIT_OR" -"medianDeterministic" -"STDDEV_POP" -"STDDEV_SAMP" -"medianExactLow" -"medianTDigestWeighted" -"medianExact" -"COVAR_POP" -"medianBFloat16Weighted" -"medianExactWeighted" -"BIT_XOR" -"median" -"log" -"replaceAll" -"rightPad" -"substring" -"lower" -"FROM_UNIXTIME" -"trunc" -"ceil" -"upper" -"rand" -"pow" -"initialQueryID" -"timezone" -"hostName" -"toYearWeek" -"timezoneOffset" -"extractAllGroupsVertical" -"leftPad" -"substring" -"IPv6NumToString" -"IPv6StringToNum" -"IPv4StringToNum" -"timezoneOf" -"position" -"IPv4NumToString" -"connectionid" -"currentUser" -"toTimezone" -"toWeek" -"arrayFlatten" -"queryID" -"date_trunc" -"modulo" +"groupArrayArgMax" +"groupArrayArgMin" +"groupArrayArray" +"groupArrayDistinct" +"groupArrayForEach" +"groupArrayIf" +"groupArrayInsertAt" +"groupArrayInsertAtArgMax" +"groupArrayInsertAtArgMin" +"groupArrayInsertAtArray" +"groupArrayInsertAtDistinct" +"groupArrayInsertAtForEach" +"groupArrayInsertAtIf" +"groupArrayInsertAtMap" +"groupArrayInsertAtMerge" +"groupArrayInsertAtNull" +"groupArrayInsertAtOrDefault" +"groupArrayInsertAtOrNull" +"groupArrayInsertAtResample" +"groupArrayInsertAtSimpleState" +"groupArrayInsertAtState" +"groupArrayIntersect" +"groupArrayIntersectArgMax" +"groupArrayIntersectArgMin" +"groupArrayIntersectArray" +"groupArrayIntersectDistinct" +"groupArrayIntersectForEach" +"groupArrayIntersectIf" +"groupArrayIntersectMap" +"groupArrayIntersectMerge" +"groupArrayIntersectNull" +"groupArrayIntersectOrDefault" +"groupArrayIntersectOrNull" +"groupArrayIntersectResample" +"groupArrayIntersectSimpleState" +"groupArrayIntersectState" +"groupArrayLast" +"groupArrayLastArgMax" +"groupArrayLastArgMin" +"groupArrayLastArray" +"groupArrayLastDistinct" +"groupArrayLastForEach" +"groupArrayLastIf" +"groupArrayLastMap" +"groupArrayLastMerge" +"groupArrayLastNull" +"groupArrayLastOrDefault" +"groupArrayLastOrNull" +"groupArrayLastResample" +"groupArrayLastSimpleState" +"groupArrayLastState" +"groupArrayMap" +"groupArrayMerge" +"groupArrayMovingAvg" +"groupArrayMovingAvgArgMax" +"groupArrayMovingAvgArgMin" +"groupArrayMovingAvgArray" +"groupArrayMovingAvgDistinct" +"groupArrayMovingAvgForEach" +"groupArrayMovingAvgIf" +"groupArrayMovingAvgMap" +"groupArrayMovingAvgMerge" +"groupArrayMovingAvgNull" +"groupArrayMovingAvgOrDefault" +"groupArrayMovingAvgOrNull" +"groupArrayMovingAvgResample" +"groupArrayMovingAvgSimpleState" +"groupArrayMovingAvgState" +"groupArrayMovingSum" +"groupArrayMovingSumArgMax" +"groupArrayMovingSumArgMin" +"groupArrayMovingSumArray" +"groupArrayMovingSumDistinct" +"groupArrayMovingSumForEach" +"groupArrayMovingSumIf" +"groupArrayMovingSumMap" +"groupArrayMovingSumMerge" +"groupArrayMovingSumNull" +"groupArrayMovingSumOrDefault" +"groupArrayMovingSumOrNull" +"groupArrayMovingSumResample" +"groupArrayMovingSumSimpleState" +"groupArrayMovingSumState" +"groupArrayNull" +"groupArrayOrDefault" +"groupArrayOrNull" +"groupArrayResample" +"groupArraySample" +"groupArraySampleArgMax" +"groupArraySampleArgMin" +"groupArraySampleArray" +"groupArraySampleDistinct" +"groupArraySampleForEach" +"groupArraySampleIf" +"groupArraySampleMap" +"groupArraySampleMerge" +"groupArraySampleNull" +"groupArraySampleOrDefault" +"groupArraySampleOrNull" +"groupArraySampleResample" +"groupArraySampleSimpleState" +"groupArraySampleState" +"groupArraySimpleState" +"groupArraySorted" +"groupArraySortedArgMax" +"groupArraySortedArgMin" +"groupArraySortedArray" +"groupArraySortedDistinct" +"groupArraySortedForEach" +"groupArraySortedIf" +"groupArraySortedMap" +"groupArraySortedMerge" +"groupArraySortedNull" +"groupArraySortedOrDefault" +"groupArraySortedOrNull" +"groupArraySortedResample" +"groupArraySortedSimpleState" +"groupArraySortedState" +"groupArrayState" "groupBitAnd" -"varSamp" -"covarSamp" -"varPop" -"quantileTDigest" -"quantileBFloat16" -"quantileTimingWeighted" -"quantileTiming" -"quantileExactHigh" +"groupBitAndArgMax" +"groupBitAndArgMin" +"groupBitAndArray" +"groupBitAndDistinct" +"groupBitAndForEach" +"groupBitAndIf" +"groupBitAndMap" +"groupBitAndMerge" +"groupBitAndNull" +"groupBitAndOrDefault" +"groupBitAndOrNull" +"groupBitAndResample" +"groupBitAndSimpleState" +"groupBitAndState" "groupBitOr" -"quantileDeterministic" -"stddevPop" -"stddevSamp" -"quantileExactLow" -"quantileTDigestWeighted" -"quantileExact" -"covarPop" -"quantileBFloat16Weighted" -"quantileExactWeighted" +"groupBitOrArgMax" +"groupBitOrArgMin" +"groupBitOrArray" +"groupBitOrDistinct" +"groupBitOrForEach" +"groupBitOrIf" +"groupBitOrMap" +"groupBitOrMerge" +"groupBitOrNull" +"groupBitOrOrDefault" +"groupBitOrOrNull" +"groupBitOrResample" +"groupBitOrSimpleState" +"groupBitOrState" "groupBitXor" +"groupBitXorArgMax" +"groupBitXorArgMin" +"groupBitXorArray" +"groupBitXorDistinct" +"groupBitXorForEach" +"groupBitXorIf" +"groupBitXorMap" +"groupBitXorMerge" +"groupBitXorNull" +"groupBitXorOrDefault" +"groupBitXorOrNull" +"groupBitXorResample" +"groupBitXorSimpleState" +"groupBitXorState" +"groupBitmap" +"groupBitmapAnd" +"groupBitmapAndArgMax" +"groupBitmapAndArgMin" +"groupBitmapAndArray" +"groupBitmapAndDistinct" +"groupBitmapAndForEach" +"groupBitmapAndIf" +"groupBitmapAndMap" +"groupBitmapAndMerge" +"groupBitmapAndNull" +"groupBitmapAndOrDefault" +"groupBitmapAndOrNull" +"groupBitmapAndResample" +"groupBitmapAndSimpleState" +"groupBitmapAndState" +"groupBitmapArgMax" +"groupBitmapArgMin" +"groupBitmapArray" +"groupBitmapDistinct" +"groupBitmapForEach" +"groupBitmapIf" +"groupBitmapMap" +"groupBitmapMerge" +"groupBitmapNull" +"groupBitmapOr" +"groupBitmapOrArgMax" +"groupBitmapOrArgMin" +"groupBitmapOrArray" +"groupBitmapOrDefault" +"groupBitmapOrDistinct" +"groupBitmapOrForEach" +"groupBitmapOrIf" +"groupBitmapOrMap" +"groupBitmapOrMerge" +"groupBitmapOrNull" +"groupBitmapOrNull" +"groupBitmapOrOrDefault" +"groupBitmapOrOrNull" +"groupBitmapOrResample" +"groupBitmapOrSimpleState" +"groupBitmapOrState" +"groupBitmapResample" +"groupBitmapSimpleState" +"groupBitmapState" +"groupBitmapXor" +"groupBitmapXorArgMax" +"groupBitmapXorArgMin" +"groupBitmapXorArray" +"groupBitmapXorDistinct" +"groupBitmapXorForEach" +"groupBitmapXorIf" +"groupBitmapXorMap" +"groupBitmapXorMerge" +"groupBitmapXorNull" +"groupBitmapXorOrDefault" +"groupBitmapXorOrNull" +"groupBitmapXorResample" +"groupBitmapXorSimpleState" +"groupBitmapXorState" +"groupUniqArray" +"groupUniqArrayArgMax" +"groupUniqArrayArgMin" +"groupUniqArrayArray" +"groupUniqArrayDistinct" +"groupUniqArrayForEach" +"groupUniqArrayIf" +"groupUniqArrayMap" +"groupUniqArrayMerge" +"groupUniqArrayNull" +"groupUniqArrayOrDefault" +"groupUniqArrayOrNull" +"groupUniqArrayResample" +"groupUniqArraySimpleState" +"groupUniqArrayState" +"h3CellAreaM2" +"h3CellAreaRads2" +"h3Distance" +"h3EdgeAngle" +"h3EdgeLengthKm" +"h3EdgeLengthM" +"h3ExactEdgeLengthKm" +"h3ExactEdgeLengthM" +"h3ExactEdgeLengthRads" +"h3GetBaseCell" +"h3GetDestinationIndexFromUnidirectionalEdge" +"h3GetFaces" +"h3GetIndexesFromUnidirectionalEdge" +"h3GetOriginIndexFromUnidirectionalEdge" +"h3GetPentagonIndexes" +"h3GetRes0Indexes" +"h3GetResolution" +"h3GetUnidirectionalEdge" +"h3GetUnidirectionalEdgeBoundary" +"h3GetUnidirectionalEdgesFromHexagon" +"h3HexAreaKm2" +"h3HexAreaM2" +"h3HexRing" +"h3IndexesAreNeighbors" +"h3IsPentagon" +"h3IsResClassIII" +"h3IsValid" +"h3Line" +"h3NumHexagons" +"h3PointDistKm" +"h3PointDistM" +"h3PointDistRads" +"h3ToCenterChild" +"h3ToChildren" +"h3ToGeo" +"h3ToGeoBoundary" +"h3ToParent" +"h3ToString" +"h3UnidirectionalEdgeIsValid" +"h3kRing" +"halfMD5" +"has" +"hasAll" +"hasAny" +"hasColumnInTable" +"hasSubsequence" +"hasSubsequenceCaseInsensitive" +"hasSubsequenceCaseInsensitiveUTF8" +"hasSubsequenceUTF8" +"hasSubstr" +"hasThreadFuzzer" +"hasToken" +"hasTokenCaseInsensitive" +"hasTokenCaseInsensitiveOrNull" +"hasTokenOrNull" +"hex" +"histogram" +"histogramArgMax" +"histogramArgMin" +"histogramArray" +"histogramDistinct" +"histogramForEach" +"histogramIf" +"histogramMap" +"histogramMerge" +"histogramNull" +"histogramOrDefault" +"histogramOrNull" +"histogramResample" +"histogramSimpleState" +"histogramState" +"hiveHash" +"hop" +"hopEnd" +"hopStart" +"hostName" +"hostname" +"hypot" +"identity" +"idnaDecode" +"idnaEncode" +"if" +"ifNotFinite" +"ifNull" +"ignore" +"ilike" +"in" +"inIgnoreSet" +"indexHint" +"indexOf" +"initcap" +"initcapUTF8" +"initialQueryID" +"initial_query_id" +"initializeAggregation" +"instr" +"intDiv" +"intDivOrZero" +"intExp10" +"intExp2" +"intHash32" +"intHash64" +"intervalLengthSum" +"intervalLengthSumArgMax" +"intervalLengthSumArgMin" +"intervalLengthSumArray" +"intervalLengthSumDistinct" +"intervalLengthSumForEach" +"intervalLengthSumIf" +"intervalLengthSumMap" +"intervalLengthSumMerge" +"intervalLengthSumNull" +"intervalLengthSumOrDefault" +"intervalLengthSumOrNull" +"intervalLengthSumResample" +"intervalLengthSumSimpleState" +"intervalLengthSumState" +"isConstant" +"isDecimalOverflow" +"isFinite" +"isIPAddressInRange" +"isIPv4String" +"isIPv6String" +"isInfinite" +"isNaN" +"isNotDistinctFrom" +"isNotNull" +"isNull" +"isNullable" +"isValidJSON" +"isValidUTF8" +"isZeroOrNull" +"jaroSimilarity" +"jaroWinklerSimilarity" +"javaHash" +"javaHashUTF16LE" +"joinGet" +"joinGetOrNull" +"jsonMergePatch" +"jumpConsistentHash" +"kafkaMurmurHash" +"kolmogorovSmirnovTest" +"kolmogorovSmirnovTestArgMax" +"kolmogorovSmirnovTestArgMin" +"kolmogorovSmirnovTestArray" +"kolmogorovSmirnovTestDistinct" +"kolmogorovSmirnovTestForEach" +"kolmogorovSmirnovTestIf" +"kolmogorovSmirnovTestMap" +"kolmogorovSmirnovTestMerge" +"kolmogorovSmirnovTestNull" +"kolmogorovSmirnovTestOrDefault" +"kolmogorovSmirnovTestOrNull" +"kolmogorovSmirnovTestResample" +"kolmogorovSmirnovTestSimpleState" +"kolmogorovSmirnovTestState" +"kostikConsistentHash" +"kql_array_sort_asc" +"kql_array_sort_desc" +"kurtPop" +"kurtPopArgMax" +"kurtPopArgMin" +"kurtPopArray" +"kurtPopDistinct" +"kurtPopForEach" +"kurtPopIf" +"kurtPopMap" +"kurtPopMerge" +"kurtPopNull" +"kurtPopOrDefault" +"kurtPopOrNull" +"kurtPopResample" +"kurtPopSimpleState" +"kurtPopState" +"kurtSamp" +"kurtSampArgMax" +"kurtSampArgMin" +"kurtSampArray" +"kurtSampDistinct" +"kurtSampForEach" +"kurtSampIf" +"kurtSampMap" +"kurtSampMerge" +"kurtSampNull" +"kurtSampOrDefault" +"kurtSampOrNull" +"kurtSampResample" +"kurtSampSimpleState" +"kurtSampState" +"lagInFrame" +"lagInFrameArgMax" +"lagInFrameArgMin" +"lagInFrameArray" +"lagInFrameDistinct" +"lagInFrameForEach" +"lagInFrameIf" +"lagInFrameMap" +"lagInFrameMerge" +"lagInFrameNull" +"lagInFrameOrDefault" +"lagInFrameOrNull" +"lagInFrameResample" +"lagInFrameSimpleState" +"lagInFrameState" +"largestTriangleThreeBuckets" +"largestTriangleThreeBucketsArgMax" +"largestTriangleThreeBucketsArgMin" +"largestTriangleThreeBucketsArray" +"largestTriangleThreeBucketsDistinct" +"largestTriangleThreeBucketsForEach" +"largestTriangleThreeBucketsIf" +"largestTriangleThreeBucketsMap" +"largestTriangleThreeBucketsMerge" +"largestTriangleThreeBucketsNull" +"largestTriangleThreeBucketsOrDefault" +"largestTriangleThreeBucketsOrNull" +"largestTriangleThreeBucketsResample" +"largestTriangleThreeBucketsSimpleState" +"largestTriangleThreeBucketsState" +"last_value" +"last_valueArgMax" +"last_valueArgMin" +"last_valueArray" +"last_valueDistinct" +"last_valueForEach" +"last_valueIf" +"last_valueMap" +"last_valueMerge" +"last_valueNull" +"last_valueOrDefault" +"last_valueOrNull" +"last_valueResample" +"last_valueSimpleState" +"last_valueState" +"last_value_respect_nulls" +"last_value_respect_nullsArgMax" +"last_value_respect_nullsArgMin" +"last_value_respect_nullsArray" +"last_value_respect_nullsDistinct" +"last_value_respect_nullsForEach" +"last_value_respect_nullsIf" +"last_value_respect_nullsMap" +"last_value_respect_nullsMerge" +"last_value_respect_nullsNull" +"last_value_respect_nullsOrDefault" +"last_value_respect_nullsOrNull" +"last_value_respect_nullsResample" +"last_value_respect_nullsSimpleState" +"last_value_respect_nullsState" +"lcase" +"lcm" +"leadInFrame" +"leadInFrameArgMax" +"leadInFrameArgMin" +"leadInFrameArray" +"leadInFrameDistinct" +"leadInFrameForEach" +"leadInFrameIf" +"leadInFrameMap" +"leadInFrameMerge" +"leadInFrameNull" +"leadInFrameOrDefault" +"leadInFrameOrNull" +"leadInFrameResample" +"leadInFrameSimpleState" +"leadInFrameState" +"least" +"left" +"leftPad" +"leftPadUTF8" +"leftUTF8" +"lemmatize" +"length" +"lengthUTF8" +"less" +"lessOrEquals" +"levenshteinDistance" +"lgamma" +"like" +"ln" +"locate" +"log" +"log10" +"log1p" +"log2" +"logTrace" +"lowCardinalityIndices" +"lowCardinalityKeys" +"lower" +"lowerUTF8" +"lpad" +"ltrim" +"lttb" +"lttbArgMax" +"lttbArgMin" +"lttbArray" +"lttbDistinct" +"lttbForEach" +"lttbIf" +"lttbMap" +"lttbMerge" +"lttbNull" +"lttbOrDefault" +"lttbOrNull" +"lttbResample" +"lttbSimpleState" +"lttbState" +"makeDate" +"makeDate32" +"makeDateTime" +"makeDateTime64" +"mannWhitneyUTest" +"mannWhitneyUTestArgMax" +"mannWhitneyUTestArgMin" +"mannWhitneyUTestArray" +"mannWhitneyUTestDistinct" +"mannWhitneyUTestForEach" +"mannWhitneyUTestIf" +"mannWhitneyUTestMap" +"mannWhitneyUTestMerge" +"mannWhitneyUTestNull" +"mannWhitneyUTestOrDefault" +"mannWhitneyUTestOrNull" +"mannWhitneyUTestResample" +"mannWhitneyUTestSimpleState" +"mannWhitneyUTestState" +"map" +"mapAdd" +"mapAll" +"mapApply" +"mapConcat" +"mapContains" +"mapContainsKeyLike" +"mapExists" +"mapExtractKeyLike" +"mapFilter" +"mapFromArrays" +"mapFromString" +"mapKeys" +"mapPartialReverseSort" +"mapPartialSort" +"mapPopulateSeries" +"mapReverseSort" +"mapSort" +"mapSubtract" +"mapUpdate" +"mapValues" +"match" +"materialize" +"max" +"max2" +"maxArgMax" +"maxArgMin" +"maxArray" +"maxDistinct" +"maxForEach" +"maxIf" +"maxIntersections" +"maxIntersectionsArgMax" +"maxIntersectionsArgMin" +"maxIntersectionsArray" +"maxIntersectionsDistinct" +"maxIntersectionsForEach" +"maxIntersectionsIf" +"maxIntersectionsMap" +"maxIntersectionsMerge" +"maxIntersectionsNull" +"maxIntersectionsOrDefault" +"maxIntersectionsOrNull" +"maxIntersectionsPosition" +"maxIntersectionsPositionArgMax" +"maxIntersectionsPositionArgMin" +"maxIntersectionsPositionArray" +"maxIntersectionsPositionDistinct" +"maxIntersectionsPositionForEach" +"maxIntersectionsPositionIf" +"maxIntersectionsPositionMap" +"maxIntersectionsPositionMerge" +"maxIntersectionsPositionNull" +"maxIntersectionsPositionOrDefault" +"maxIntersectionsPositionOrNull" +"maxIntersectionsPositionResample" +"maxIntersectionsPositionSimpleState" +"maxIntersectionsPositionState" +"maxIntersectionsResample" +"maxIntersectionsSimpleState" +"maxIntersectionsState" +"maxMap" +"maxMappedArrays" +"maxMappedArraysArgMax" +"maxMappedArraysArgMin" +"maxMappedArraysArray" +"maxMappedArraysDistinct" +"maxMappedArraysForEach" +"maxMappedArraysIf" +"maxMappedArraysMap" +"maxMappedArraysMerge" +"maxMappedArraysNull" +"maxMappedArraysOrDefault" +"maxMappedArraysOrNull" +"maxMappedArraysResample" +"maxMappedArraysSimpleState" +"maxMappedArraysState" +"maxMerge" +"maxNull" +"maxOrDefault" +"maxOrNull" +"maxResample" +"maxSimpleState" +"maxState" +"meanZTest" +"meanZTestArgMax" +"meanZTestArgMin" +"meanZTestArray" +"meanZTestDistinct" +"meanZTestForEach" +"meanZTestIf" +"meanZTestMap" +"meanZTestMerge" +"meanZTestNull" +"meanZTestOrDefault" +"meanZTestOrNull" +"meanZTestResample" +"meanZTestSimpleState" +"meanZTestState" +"median" +"medianArgMax" +"medianArgMin" +"medianArray" +"medianBFloat16" +"medianBFloat16ArgMax" +"medianBFloat16ArgMin" +"medianBFloat16Array" +"medianBFloat16Distinct" +"medianBFloat16ForEach" +"medianBFloat16If" +"medianBFloat16Map" +"medianBFloat16Merge" +"medianBFloat16Null" +"medianBFloat16OrDefault" +"medianBFloat16OrNull" +"medianBFloat16Resample" +"medianBFloat16SimpleState" +"medianBFloat16State" +"medianBFloat16Weighted" +"medianBFloat16WeightedArgMax" +"medianBFloat16WeightedArgMin" +"medianBFloat16WeightedArray" +"medianBFloat16WeightedDistinct" +"medianBFloat16WeightedForEach" +"medianBFloat16WeightedIf" +"medianBFloat16WeightedMap" +"medianBFloat16WeightedMerge" +"medianBFloat16WeightedNull" +"medianBFloat16WeightedOrDefault" +"medianBFloat16WeightedOrNull" +"medianBFloat16WeightedResample" +"medianBFloat16WeightedSimpleState" +"medianBFloat16WeightedState" +"medianDD" +"medianDDArgMax" +"medianDDArgMin" +"medianDDArray" +"medianDDDistinct" +"medianDDForEach" +"medianDDIf" +"medianDDMap" +"medianDDMerge" +"medianDDNull" +"medianDDOrDefault" +"medianDDOrNull" +"medianDDResample" +"medianDDSimpleState" +"medianDDState" +"medianDeterministic" +"medianDeterministicArgMax" +"medianDeterministicArgMin" +"medianDeterministicArray" +"medianDeterministicDistinct" +"medianDeterministicForEach" +"medianDeterministicIf" +"medianDeterministicMap" +"medianDeterministicMerge" +"medianDeterministicNull" +"medianDeterministicOrDefault" +"medianDeterministicOrNull" +"medianDeterministicResample" +"medianDeterministicSimpleState" +"medianDeterministicState" +"medianDistinct" +"medianExact" +"medianExactArgMax" +"medianExactArgMin" +"medianExactArray" +"medianExactDistinct" +"medianExactForEach" +"medianExactHigh" +"medianExactHighArgMax" +"medianExactHighArgMin" +"medianExactHighArray" +"medianExactHighDistinct" +"medianExactHighForEach" +"medianExactHighIf" +"medianExactHighMap" +"medianExactHighMerge" +"medianExactHighNull" +"medianExactHighOrDefault" +"medianExactHighOrNull" +"medianExactHighResample" +"medianExactHighSimpleState" +"medianExactHighState" +"medianExactIf" +"medianExactLow" +"medianExactLowArgMax" +"medianExactLowArgMin" +"medianExactLowArray" +"medianExactLowDistinct" +"medianExactLowForEach" +"medianExactLowIf" +"medianExactLowMap" +"medianExactLowMerge" +"medianExactLowNull" +"medianExactLowOrDefault" +"medianExactLowOrNull" +"medianExactLowResample" +"medianExactLowSimpleState" +"medianExactLowState" +"medianExactMap" +"medianExactMerge" +"medianExactNull" +"medianExactOrDefault" +"medianExactOrNull" +"medianExactResample" +"medianExactSimpleState" +"medianExactState" +"medianExactWeighted" +"medianExactWeightedArgMax" +"medianExactWeightedArgMin" +"medianExactWeightedArray" +"medianExactWeightedDistinct" +"medianExactWeightedForEach" +"medianExactWeightedIf" +"medianExactWeightedMap" +"medianExactWeightedMerge" +"medianExactWeightedNull" +"medianExactWeightedOrDefault" +"medianExactWeightedOrNull" +"medianExactWeightedResample" +"medianExactWeightedSimpleState" +"medianExactWeightedState" +"medianForEach" +"medianGK" +"medianGKArgMax" +"medianGKArgMin" +"medianGKArray" +"medianGKDistinct" +"medianGKForEach" +"medianGKIf" +"medianGKMap" +"medianGKMerge" +"medianGKNull" +"medianGKOrDefault" +"medianGKOrNull" +"medianGKResample" +"medianGKSimpleState" +"medianGKState" +"medianIf" +"medianInterpolatedWeighted" +"medianInterpolatedWeightedArgMax" +"medianInterpolatedWeightedArgMin" +"medianInterpolatedWeightedArray" +"medianInterpolatedWeightedDistinct" +"medianInterpolatedWeightedForEach" +"medianInterpolatedWeightedIf" +"medianInterpolatedWeightedMap" +"medianInterpolatedWeightedMerge" +"medianInterpolatedWeightedNull" +"medianInterpolatedWeightedOrDefault" +"medianInterpolatedWeightedOrNull" +"medianInterpolatedWeightedResample" +"medianInterpolatedWeightedSimpleState" +"medianInterpolatedWeightedState" +"medianMap" +"medianMerge" +"medianNull" +"medianOrDefault" +"medianOrNull" +"medianResample" +"medianSimpleState" +"medianState" +"medianTDigest" +"medianTDigestArgMax" +"medianTDigestArgMin" +"medianTDigestArray" +"medianTDigestDistinct" +"medianTDigestForEach" +"medianTDigestIf" +"medianTDigestMap" +"medianTDigestMerge" +"medianTDigestNull" +"medianTDigestOrDefault" +"medianTDigestOrNull" +"medianTDigestResample" +"medianTDigestSimpleState" +"medianTDigestState" +"medianTDigestWeighted" +"medianTDigestWeightedArgMax" +"medianTDigestWeightedArgMin" +"medianTDigestWeightedArray" +"medianTDigestWeightedDistinct" +"medianTDigestWeightedForEach" +"medianTDigestWeightedIf" +"medianTDigestWeightedMap" +"medianTDigestWeightedMerge" +"medianTDigestWeightedNull" +"medianTDigestWeightedOrDefault" +"medianTDigestWeightedOrNull" +"medianTDigestWeightedResample" +"medianTDigestWeightedSimpleState" +"medianTDigestWeightedState" +"medianTiming" +"medianTimingArgMax" +"medianTimingArgMin" +"medianTimingArray" +"medianTimingDistinct" +"medianTimingForEach" +"medianTimingIf" +"medianTimingMap" +"medianTimingMerge" +"medianTimingNull" +"medianTimingOrDefault" +"medianTimingOrNull" +"medianTimingResample" +"medianTimingSimpleState" +"medianTimingState" +"medianTimingWeighted" +"medianTimingWeightedArgMax" +"medianTimingWeightedArgMin" +"medianTimingWeightedArray" +"medianTimingWeightedDistinct" +"medianTimingWeightedForEach" +"medianTimingWeightedIf" +"medianTimingWeightedMap" +"medianTimingWeightedMerge" +"medianTimingWeightedNull" +"medianTimingWeightedOrDefault" +"medianTimingWeightedOrNull" +"medianTimingWeightedResample" +"medianTimingWeightedSimpleState" +"medianTimingWeightedState" +"metroHash64" +"mid" +"min" +"min2" +"minArgMax" +"minArgMin" +"minArray" +"minDistinct" +"minForEach" +"minIf" +"minMap" +"minMappedArrays" +"minMappedArraysArgMax" +"minMappedArraysArgMin" +"minMappedArraysArray" +"minMappedArraysDistinct" +"minMappedArraysForEach" +"minMappedArraysIf" +"minMappedArraysMap" +"minMappedArraysMerge" +"minMappedArraysNull" +"minMappedArraysOrDefault" +"minMappedArraysOrNull" +"minMappedArraysResample" +"minMappedArraysSimpleState" +"minMappedArraysState" +"minMerge" +"minNull" +"minOrDefault" +"minOrNull" +"minResample" +"minSampleSizeContinous" +"minSampleSizeContinuous" +"minSampleSizeConversion" +"minSimpleState" +"minState" +"minus" +"mismatches" +"mod" +"modulo" +"moduloLegacy" +"moduloOrZero" +"monthName" +"mortonDecode" +"mortonEncode" +"multiFuzzyMatchAllIndices" +"multiFuzzyMatchAny" +"multiFuzzyMatchAnyIndex" +"multiIf" +"multiMatchAllIndices" +"multiMatchAny" +"multiMatchAnyIndex" +"multiSearchAllPositions" +"multiSearchAllPositionsCaseInsensitive" +"multiSearchAllPositionsCaseInsensitiveUTF8" +"multiSearchAllPositionsUTF8" +"multiSearchAny" +"multiSearchAnyCaseInsensitive" +"multiSearchAnyCaseInsensitiveUTF8" +"multiSearchAnyUTF8" +"multiSearchFirstIndex" +"multiSearchFirstIndexCaseInsensitive" +"multiSearchFirstIndexCaseInsensitiveUTF8" +"multiSearchFirstIndexUTF8" +"multiSearchFirstPosition" +"multiSearchFirstPositionCaseInsensitive" +"multiSearchFirstPositionCaseInsensitiveUTF8" +"multiSearchFirstPositionUTF8" +"multiply" +"multiplyDecimal" +"murmurHash2_32" +"murmurHash2_64" +"murmurHash3_128" +"murmurHash3_32" +"murmurHash3_64" +"negate" +"neighbor" +"nested" +"netloc" +"ngramDistance" +"ngramDistanceCaseInsensitive" +"ngramDistanceCaseInsensitiveUTF8" +"ngramDistanceUTF8" +"ngramMinHash" +"ngramMinHashArg" +"ngramMinHashArgCaseInsensitive" +"ngramMinHashArgCaseInsensitiveUTF8" +"ngramMinHashArgUTF8" +"ngramMinHashCaseInsensitive" +"ngramMinHashCaseInsensitiveUTF8" +"ngramMinHashUTF8" +"ngramSearch" +"ngramSearchCaseInsensitive" +"ngramSearchCaseInsensitiveUTF8" +"ngramSearchUTF8" +"ngramSimHash" +"ngramSimHashCaseInsensitive" +"ngramSimHashCaseInsensitiveUTF8" +"ngramSimHashUTF8" +"ngrams" +"nonNegativeDerivative" +"nonNegativeDerivativeArgMax" +"nonNegativeDerivativeArgMin" +"nonNegativeDerivativeArray" +"nonNegativeDerivativeDistinct" +"nonNegativeDerivativeForEach" +"nonNegativeDerivativeIf" +"nonNegativeDerivativeMap" +"nonNegativeDerivativeMerge" +"nonNegativeDerivativeNull" +"nonNegativeDerivativeOrDefault" +"nonNegativeDerivativeOrNull" +"nonNegativeDerivativeResample" +"nonNegativeDerivativeSimpleState" +"nonNegativeDerivativeState" +"normL1" +"normL2" +"normL2Squared" +"normLinf" +"normLp" +"normalizeL1" +"normalizeL2" +"normalizeLinf" +"normalizeLp" +"normalizeQuery" +"normalizeQueryKeepNames" +"normalizeUTF8NFC" +"normalizeUTF8NFD" +"normalizeUTF8NFKC" +"normalizeUTF8NFKD" +"normalizedQueryHash" +"normalizedQueryHashKeepNames" +"not" +"notEmpty" +"notEquals" +"notILike" +"notIn" +"notInIgnoreSet" +"notLike" +"notNullIn" +"notNullInIgnoreSet" +"nothing" +"nothingArgMax" +"nothingArgMin" +"nothingArray" +"nothingDistinct" +"nothingForEach" +"nothingIf" +"nothingMap" +"nothingMerge" +"nothingNull" +"nothingNull" +"nothingNullArgMax" +"nothingNullArgMin" +"nothingNullArray" +"nothingNullDistinct" +"nothingNullForEach" +"nothingNullIf" +"nothingNullMap" +"nothingNullMerge" +"nothingNullNull" +"nothingNullOrDefault" +"nothingNullOrNull" +"nothingNullResample" +"nothingNullSimpleState" +"nothingNullState" +"nothingOrDefault" +"nothingOrNull" +"nothingResample" +"nothingSimpleState" +"nothingState" +"nothingUInt64" +"nothingUInt64ArgMax" +"nothingUInt64ArgMin" +"nothingUInt64Array" +"nothingUInt64Distinct" +"nothingUInt64ForEach" +"nothingUInt64If" +"nothingUInt64Map" +"nothingUInt64Merge" +"nothingUInt64Null" +"nothingUInt64OrDefault" +"nothingUInt64OrNull" +"nothingUInt64Resample" +"nothingUInt64SimpleState" +"nothingUInt64State" +"now" +"now64" +"nowInBlock" +"nth_value" +"nth_valueArgMax" +"nth_valueArgMin" +"nth_valueArray" +"nth_valueDistinct" +"nth_valueForEach" +"nth_valueIf" +"nth_valueMap" +"nth_valueMerge" +"nth_valueNull" +"nth_valueOrDefault" +"nth_valueOrNull" +"nth_valueResample" +"nth_valueSimpleState" +"nth_valueState" +"ntile" +"ntileArgMax" +"ntileArgMin" +"ntileArray" +"ntileDistinct" +"ntileForEach" +"ntileIf" +"ntileMap" +"ntileMerge" +"ntileNull" +"ntileOrDefault" +"ntileOrNull" +"ntileResample" +"ntileSimpleState" +"ntileState" +"nullIf" +"nullIn" +"nullInIgnoreSet" +"or" +"parseDateTime" +"parseDateTime32BestEffort" +"parseDateTime32BestEffortOrNull" +"parseDateTime32BestEffortOrZero" +"parseDateTime64BestEffort" +"parseDateTime64BestEffortOrNull" +"parseDateTime64BestEffortOrZero" +"parseDateTime64BestEffortUS" +"parseDateTime64BestEffortUSOrNull" +"parseDateTime64BestEffortUSOrZero" +"parseDateTimeBestEffort" +"parseDateTimeBestEffortOrNull" +"parseDateTimeBestEffortOrZero" +"parseDateTimeBestEffortUS" +"parseDateTimeBestEffortUSOrNull" +"parseDateTimeBestEffortUSOrZero" +"parseDateTimeInJodaSyntax" +"parseDateTimeInJodaSyntaxOrNull" +"parseDateTimeInJodaSyntaxOrZero" +"parseDateTimeOrNull" +"parseDateTimeOrZero" +"parseTimeDelta" +"partitionId" +"path" +"pathFull" +"pi" +"plus" +"pmod" +"pointInEllipses" +"pointInPolygon" +"polygonAreaCartesian" +"polygonAreaSpherical" +"polygonConvexHullCartesian" +"polygonPerimeterCartesian" +"polygonPerimeterSpherical" +"polygonsDistanceCartesian" +"polygonsDistanceSpherical" +"polygonsEqualsCartesian" +"polygonsIntersectionCartesian" +"polygonsIntersectionSpherical" +"polygonsSymDifferenceCartesian" +"polygonsSymDifferenceSpherical" +"polygonsUnionCartesian" +"polygonsUnionSpherical" +"polygonsWithinCartesian" +"polygonsWithinSpherical" +"port" +"portRFC" +"position" +"positionCaseInsensitive" +"positionCaseInsensitiveUTF8" +"positionUTF8" +"positiveModulo" +"positive_modulo" +"pow" +"power" +"proportionsZTest" +"protocol" +"punycodeDecode" +"punycodeEncode" "quantile" +"quantileArgMax" +"quantileArgMin" +"quantileArray" +"quantileBFloat16" +"quantileBFloat16ArgMax" +"quantileBFloat16ArgMin" +"quantileBFloat16Array" +"quantileBFloat16Distinct" +"quantileBFloat16ForEach" +"quantileBFloat16If" +"quantileBFloat16Map" +"quantileBFloat16Merge" +"quantileBFloat16Null" +"quantileBFloat16OrDefault" +"quantileBFloat16OrNull" +"quantileBFloat16Resample" +"quantileBFloat16SimpleState" +"quantileBFloat16State" +"quantileBFloat16Weighted" +"quantileBFloat16WeightedArgMax" +"quantileBFloat16WeightedArgMin" +"quantileBFloat16WeightedArray" +"quantileBFloat16WeightedDistinct" +"quantileBFloat16WeightedForEach" +"quantileBFloat16WeightedIf" +"quantileBFloat16WeightedMap" +"quantileBFloat16WeightedMerge" +"quantileBFloat16WeightedNull" +"quantileBFloat16WeightedOrDefault" +"quantileBFloat16WeightedOrNull" +"quantileBFloat16WeightedResample" +"quantileBFloat16WeightedSimpleState" +"quantileBFloat16WeightedState" +"quantileDD" +"quantileDDArgMax" +"quantileDDArgMin" +"quantileDDArray" +"quantileDDDistinct" +"quantileDDForEach" +"quantileDDIf" +"quantileDDMap" +"quantileDDMerge" +"quantileDDNull" +"quantileDDOrDefault" +"quantileDDOrNull" +"quantileDDResample" +"quantileDDSimpleState" +"quantileDDState" +"quantileDeterministic" +"quantileDeterministicArgMax" +"quantileDeterministicArgMin" +"quantileDeterministicArray" +"quantileDeterministicDistinct" +"quantileDeterministicForEach" +"quantileDeterministicIf" +"quantileDeterministicMap" +"quantileDeterministicMerge" +"quantileDeterministicNull" +"quantileDeterministicOrDefault" +"quantileDeterministicOrNull" +"quantileDeterministicResample" +"quantileDeterministicSimpleState" +"quantileDeterministicState" +"quantileDistinct" +"quantileExact" +"quantileExactArgMax" +"quantileExactArgMin" +"quantileExactArray" +"quantileExactDistinct" +"quantileExactExclusive" +"quantileExactExclusiveArgMax" +"quantileExactExclusiveArgMin" +"quantileExactExclusiveArray" +"quantileExactExclusiveDistinct" +"quantileExactExclusiveForEach" +"quantileExactExclusiveIf" +"quantileExactExclusiveMap" +"quantileExactExclusiveMerge" +"quantileExactExclusiveNull" +"quantileExactExclusiveOrDefault" +"quantileExactExclusiveOrNull" +"quantileExactExclusiveResample" +"quantileExactExclusiveSimpleState" +"quantileExactExclusiveState" +"quantileExactForEach" +"quantileExactHigh" +"quantileExactHighArgMax" +"quantileExactHighArgMin" +"quantileExactHighArray" +"quantileExactHighDistinct" +"quantileExactHighForEach" +"quantileExactHighIf" +"quantileExactHighMap" +"quantileExactHighMerge" +"quantileExactHighNull" +"quantileExactHighOrDefault" +"quantileExactHighOrNull" +"quantileExactHighResample" +"quantileExactHighSimpleState" +"quantileExactHighState" +"quantileExactIf" +"quantileExactInclusive" +"quantileExactInclusiveArgMax" +"quantileExactInclusiveArgMin" +"quantileExactInclusiveArray" +"quantileExactInclusiveDistinct" +"quantileExactInclusiveForEach" +"quantileExactInclusiveIf" +"quantileExactInclusiveMap" +"quantileExactInclusiveMerge" +"quantileExactInclusiveNull" +"quantileExactInclusiveOrDefault" +"quantileExactInclusiveOrNull" +"quantileExactInclusiveResample" +"quantileExactInclusiveSimpleState" +"quantileExactInclusiveState" +"quantileExactLow" +"quantileExactLowArgMax" +"quantileExactLowArgMin" +"quantileExactLowArray" +"quantileExactLowDistinct" +"quantileExactLowForEach" +"quantileExactLowIf" +"quantileExactLowMap" +"quantileExactLowMerge" +"quantileExactLowNull" +"quantileExactLowOrDefault" +"quantileExactLowOrNull" +"quantileExactLowResample" +"quantileExactLowSimpleState" +"quantileExactLowState" +"quantileExactMap" +"quantileExactMerge" +"quantileExactNull" +"quantileExactOrDefault" +"quantileExactOrNull" +"quantileExactResample" +"quantileExactSimpleState" +"quantileExactState" +"quantileExactWeighted" +"quantileExactWeightedArgMax" +"quantileExactWeightedArgMin" +"quantileExactWeightedArray" +"quantileExactWeightedDistinct" +"quantileExactWeightedForEach" +"quantileExactWeightedIf" +"quantileExactWeightedMap" +"quantileExactWeightedMerge" +"quantileExactWeightedNull" +"quantileExactWeightedOrDefault" +"quantileExactWeightedOrNull" +"quantileExactWeightedResample" +"quantileExactWeightedSimpleState" +"quantileExactWeightedState" +"quantileForEach" +"quantileGK" +"quantileGKArgMax" +"quantileGKArgMin" +"quantileGKArray" +"quantileGKDistinct" +"quantileGKForEach" +"quantileGKIf" +"quantileGKMap" +"quantileGKMerge" +"quantileGKNull" +"quantileGKOrDefault" +"quantileGKOrNull" +"quantileGKResample" +"quantileGKSimpleState" +"quantileGKState" +"quantileIf" +"quantileInterpolatedWeighted" +"quantileInterpolatedWeightedArgMax" +"quantileInterpolatedWeightedArgMin" +"quantileInterpolatedWeightedArray" +"quantileInterpolatedWeightedDistinct" +"quantileInterpolatedWeightedForEach" +"quantileInterpolatedWeightedIf" +"quantileInterpolatedWeightedMap" +"quantileInterpolatedWeightedMerge" +"quantileInterpolatedWeightedNull" +"quantileInterpolatedWeightedOrDefault" +"quantileInterpolatedWeightedOrNull" +"quantileInterpolatedWeightedResample" +"quantileInterpolatedWeightedSimpleState" +"quantileInterpolatedWeightedState" +"quantileMap" +"quantileMerge" +"quantileNull" +"quantileOrDefault" +"quantileOrNull" +"quantileResample" +"quantileSimpleState" +"quantileState" +"quantileTDigest" +"quantileTDigestArgMax" +"quantileTDigestArgMin" +"quantileTDigestArray" +"quantileTDigestDistinct" +"quantileTDigestForEach" +"quantileTDigestIf" +"quantileTDigestMap" +"quantileTDigestMerge" +"quantileTDigestNull" +"quantileTDigestOrDefault" +"quantileTDigestOrNull" +"quantileTDigestResample" +"quantileTDigestSimpleState" +"quantileTDigestState" +"quantileTDigestWeighted" +"quantileTDigestWeightedArgMax" +"quantileTDigestWeightedArgMin" +"quantileTDigestWeightedArray" +"quantileTDigestWeightedDistinct" +"quantileTDigestWeightedForEach" +"quantileTDigestWeightedIf" +"quantileTDigestWeightedMap" +"quantileTDigestWeightedMerge" +"quantileTDigestWeightedNull" +"quantileTDigestWeightedOrDefault" +"quantileTDigestWeightedOrNull" +"quantileTDigestWeightedResample" +"quantileTDigestWeightedSimpleState" +"quantileTDigestWeightedState" +"quantileTiming" +"quantileTimingArgMax" +"quantileTimingArgMin" +"quantileTimingArray" +"quantileTimingDistinct" +"quantileTimingForEach" +"quantileTimingIf" +"quantileTimingMap" +"quantileTimingMerge" +"quantileTimingNull" +"quantileTimingOrDefault" +"quantileTimingOrNull" +"quantileTimingResample" +"quantileTimingSimpleState" +"quantileTimingState" +"quantileTimingWeighted" +"quantileTimingWeightedArgMax" +"quantileTimingWeightedArgMin" +"quantileTimingWeightedArray" +"quantileTimingWeightedDistinct" +"quantileTimingWeightedForEach" +"quantileTimingWeightedIf" +"quantileTimingWeightedMap" +"quantileTimingWeightedMerge" +"quantileTimingWeightedNull" +"quantileTimingWeightedOrDefault" +"quantileTimingWeightedOrNull" +"quantileTimingWeightedResample" +"quantileTimingWeightedSimpleState" +"quantileTimingWeightedState" +"quantiles" +"quantilesArgMax" +"quantilesArgMin" +"quantilesArray" +"quantilesBFloat16" +"quantilesBFloat16ArgMax" +"quantilesBFloat16ArgMin" +"quantilesBFloat16Array" +"quantilesBFloat16Distinct" +"quantilesBFloat16ForEach" +"quantilesBFloat16If" +"quantilesBFloat16Map" +"quantilesBFloat16Merge" +"quantilesBFloat16Null" +"quantilesBFloat16OrDefault" +"quantilesBFloat16OrNull" +"quantilesBFloat16Resample" +"quantilesBFloat16SimpleState" +"quantilesBFloat16State" +"quantilesBFloat16Weighted" +"quantilesBFloat16WeightedArgMax" +"quantilesBFloat16WeightedArgMin" +"quantilesBFloat16WeightedArray" +"quantilesBFloat16WeightedDistinct" +"quantilesBFloat16WeightedForEach" +"quantilesBFloat16WeightedIf" +"quantilesBFloat16WeightedMap" +"quantilesBFloat16WeightedMerge" +"quantilesBFloat16WeightedNull" +"quantilesBFloat16WeightedOrDefault" +"quantilesBFloat16WeightedOrNull" +"quantilesBFloat16WeightedResample" +"quantilesBFloat16WeightedSimpleState" +"quantilesBFloat16WeightedState" +"quantilesDD" +"quantilesDDArgMax" +"quantilesDDArgMin" +"quantilesDDArray" +"quantilesDDDistinct" +"quantilesDDForEach" +"quantilesDDIf" +"quantilesDDMap" +"quantilesDDMerge" +"quantilesDDNull" +"quantilesDDOrDefault" +"quantilesDDOrNull" +"quantilesDDResample" +"quantilesDDSimpleState" +"quantilesDDState" +"quantilesDeterministic" +"quantilesDeterministicArgMax" +"quantilesDeterministicArgMin" +"quantilesDeterministicArray" +"quantilesDeterministicDistinct" +"quantilesDeterministicForEach" +"quantilesDeterministicIf" +"quantilesDeterministicMap" +"quantilesDeterministicMerge" +"quantilesDeterministicNull" +"quantilesDeterministicOrDefault" +"quantilesDeterministicOrNull" +"quantilesDeterministicResample" +"quantilesDeterministicSimpleState" +"quantilesDeterministicState" +"quantilesDistinct" +"quantilesExact" +"quantilesExactArgMax" +"quantilesExactArgMin" +"quantilesExactArray" +"quantilesExactDistinct" +"quantilesExactExclusive" +"quantilesExactExclusiveArgMax" +"quantilesExactExclusiveArgMin" +"quantilesExactExclusiveArray" +"quantilesExactExclusiveDistinct" +"quantilesExactExclusiveForEach" +"quantilesExactExclusiveIf" +"quantilesExactExclusiveMap" +"quantilesExactExclusiveMerge" +"quantilesExactExclusiveNull" +"quantilesExactExclusiveOrDefault" +"quantilesExactExclusiveOrNull" +"quantilesExactExclusiveResample" +"quantilesExactExclusiveSimpleState" +"quantilesExactExclusiveState" +"quantilesExactForEach" +"quantilesExactHigh" +"quantilesExactHighArgMax" +"quantilesExactHighArgMin" +"quantilesExactHighArray" +"quantilesExactHighDistinct" +"quantilesExactHighForEach" +"quantilesExactHighIf" +"quantilesExactHighMap" +"quantilesExactHighMerge" +"quantilesExactHighNull" +"quantilesExactHighOrDefault" +"quantilesExactHighOrNull" +"quantilesExactHighResample" +"quantilesExactHighSimpleState" +"quantilesExactHighState" +"quantilesExactIf" +"quantilesExactInclusive" +"quantilesExactInclusiveArgMax" +"quantilesExactInclusiveArgMin" +"quantilesExactInclusiveArray" +"quantilesExactInclusiveDistinct" +"quantilesExactInclusiveForEach" +"quantilesExactInclusiveIf" +"quantilesExactInclusiveMap" +"quantilesExactInclusiveMerge" +"quantilesExactInclusiveNull" +"quantilesExactInclusiveOrDefault" +"quantilesExactInclusiveOrNull" +"quantilesExactInclusiveResample" +"quantilesExactInclusiveSimpleState" +"quantilesExactInclusiveState" +"quantilesExactLow" +"quantilesExactLowArgMax" +"quantilesExactLowArgMin" +"quantilesExactLowArray" +"quantilesExactLowDistinct" +"quantilesExactLowForEach" +"quantilesExactLowIf" +"quantilesExactLowMap" +"quantilesExactLowMerge" +"quantilesExactLowNull" +"quantilesExactLowOrDefault" +"quantilesExactLowOrNull" +"quantilesExactLowResample" +"quantilesExactLowSimpleState" +"quantilesExactLowState" +"quantilesExactMap" +"quantilesExactMerge" +"quantilesExactNull" +"quantilesExactOrDefault" +"quantilesExactOrNull" +"quantilesExactResample" +"quantilesExactSimpleState" +"quantilesExactState" +"quantilesExactWeighted" +"quantilesExactWeightedArgMax" +"quantilesExactWeightedArgMin" +"quantilesExactWeightedArray" +"quantilesExactWeightedDistinct" +"quantilesExactWeightedForEach" +"quantilesExactWeightedIf" +"quantilesExactWeightedMap" +"quantilesExactWeightedMerge" +"quantilesExactWeightedNull" +"quantilesExactWeightedOrDefault" +"quantilesExactWeightedOrNull" +"quantilesExactWeightedResample" +"quantilesExactWeightedSimpleState" +"quantilesExactWeightedState" +"quantilesForEach" +"quantilesGK" +"quantilesGKArgMax" +"quantilesGKArgMin" +"quantilesGKArray" +"quantilesGKDistinct" +"quantilesGKForEach" +"quantilesGKIf" +"quantilesGKMap" +"quantilesGKMerge" +"quantilesGKNull" +"quantilesGKOrDefault" +"quantilesGKOrNull" +"quantilesGKResample" +"quantilesGKSimpleState" +"quantilesGKState" +"quantilesIf" +"quantilesInterpolatedWeighted" +"quantilesInterpolatedWeightedArgMax" +"quantilesInterpolatedWeightedArgMin" +"quantilesInterpolatedWeightedArray" +"quantilesInterpolatedWeightedDistinct" +"quantilesInterpolatedWeightedForEach" +"quantilesInterpolatedWeightedIf" +"quantilesInterpolatedWeightedMap" +"quantilesInterpolatedWeightedMerge" +"quantilesInterpolatedWeightedNull" +"quantilesInterpolatedWeightedOrDefault" +"quantilesInterpolatedWeightedOrNull" +"quantilesInterpolatedWeightedResample" +"quantilesInterpolatedWeightedSimpleState" +"quantilesInterpolatedWeightedState" +"quantilesMap" +"quantilesMerge" +"quantilesNull" +"quantilesOrDefault" +"quantilesOrNull" +"quantilesResample" +"quantilesSimpleState" +"quantilesState" +"quantilesTDigest" +"quantilesTDigestArgMax" +"quantilesTDigestArgMin" +"quantilesTDigestArray" +"quantilesTDigestDistinct" +"quantilesTDigestForEach" +"quantilesTDigestIf" +"quantilesTDigestMap" +"quantilesTDigestMerge" +"quantilesTDigestNull" +"quantilesTDigestOrDefault" +"quantilesTDigestOrNull" +"quantilesTDigestResample" +"quantilesTDigestSimpleState" +"quantilesTDigestState" +"quantilesTDigestWeighted" +"quantilesTDigestWeightedArgMax" +"quantilesTDigestWeightedArgMin" +"quantilesTDigestWeightedArray" +"quantilesTDigestWeightedDistinct" +"quantilesTDigestWeightedForEach" +"quantilesTDigestWeightedIf" +"quantilesTDigestWeightedMap" +"quantilesTDigestWeightedMerge" +"quantilesTDigestWeightedNull" +"quantilesTDigestWeightedOrDefault" +"quantilesTDigestWeightedOrNull" +"quantilesTDigestWeightedResample" +"quantilesTDigestWeightedSimpleState" +"quantilesTDigestWeightedState" +"quantilesTiming" +"quantilesTimingArgMax" +"quantilesTimingArgMin" +"quantilesTimingArray" +"quantilesTimingDistinct" +"quantilesTimingForEach" +"quantilesTimingIf" +"quantilesTimingMap" +"quantilesTimingMerge" +"quantilesTimingNull" +"quantilesTimingOrDefault" +"quantilesTimingOrNull" +"quantilesTimingResample" +"quantilesTimingSimpleState" +"quantilesTimingState" +"quantilesTimingWeighted" +"quantilesTimingWeightedArgMax" +"quantilesTimingWeightedArgMin" +"quantilesTimingWeightedArray" +"quantilesTimingWeightedDistinct" +"quantilesTimingWeightedForEach" +"quantilesTimingWeightedIf" +"quantilesTimingWeightedMap" +"quantilesTimingWeightedMerge" +"quantilesTimingWeightedNull" +"quantilesTimingWeightedOrDefault" +"quantilesTimingWeightedOrNull" +"quantilesTimingWeightedResample" +"quantilesTimingWeightedSimpleState" +"quantilesTimingWeightedState" +"queryID" +"queryString" +"queryStringAndFragment" +"query_id" +"radians" +"rand" +"rand32" +"rand64" +"randBernoulli" +"randBinomial" +"randCanonical" +"randChiSquared" +"randConstant" +"randExponential" +"randFisherF" +"randLogNormal" +"randNegativeBinomial" +"randNormal" +"randPoisson" +"randStudentT" +"randUniform" +"randomFixedString" +"randomPrintableASCII" +"randomString" +"randomStringUTF8" +"range" +"rank" +"rankArgMax" +"rankArgMin" +"rankArray" +"rankCorr" +"rankCorrArgMax" +"rankCorrArgMin" +"rankCorrArray" +"rankCorrDistinct" +"rankCorrForEach" +"rankCorrIf" +"rankCorrMap" +"rankCorrMerge" +"rankCorrNull" +"rankCorrOrDefault" +"rankCorrOrNull" +"rankCorrResample" +"rankCorrSimpleState" +"rankCorrState" +"rankDistinct" +"rankForEach" +"rankIf" +"rankMap" +"rankMerge" +"rankNull" +"rankOrDefault" +"rankOrNull" +"rankResample" +"rankSimpleState" +"rankState" +"readWKTMultiPolygon" +"readWKTPoint" +"readWKTPolygon" +"readWKTRing" +"regexpExtract" +"regexpQuoteMeta" +"regionHierarchy" +"regionIn" +"regionToArea" +"regionToCity" +"regionToContinent" +"regionToCountry" +"regionToDistrict" +"regionToName" +"regionToPopulation" +"regionToTopContinent" +"reinterpret" +"reinterpretAsDate" +"reinterpretAsDateTime" +"reinterpretAsFixedString" +"reinterpretAsFloat32" +"reinterpretAsFloat64" +"reinterpretAsInt128" +"reinterpretAsInt16" +"reinterpretAsInt256" +"reinterpretAsInt32" +"reinterpretAsInt64" +"reinterpretAsInt8" +"reinterpretAsString" +"reinterpretAsUInt128" +"reinterpretAsUInt16" +"reinterpretAsUInt256" +"reinterpretAsUInt32" +"reinterpretAsUInt64" +"reinterpretAsUInt8" +"reinterpretAsUUID" +"repeat" +"replace" +"replaceAll" +"replaceOne" +"replaceRegexpAll" +"replaceRegexpOne" +"replicate" +"retention" +"retentionArgMax" +"retentionArgMin" +"retentionArray" +"retentionDistinct" +"retentionForEach" +"retentionIf" +"retentionMap" +"retentionMerge" +"retentionNull" +"retentionOrDefault" +"retentionOrNull" +"retentionResample" +"retentionSimpleState" +"retentionState" +"reverse" +"reverseUTF8" +"revision" +"right" +"rightPad" +"rightPadUTF8" +"rightUTF8" +"round" +"roundAge" +"roundBankers" +"roundDown" +"roundDuration" +"roundToExp2" +"rowNumberInAllBlocks" +"rowNumberInBlock" +"row_number" +"row_numberArgMax" +"row_numberArgMin" +"row_numberArray" +"row_numberDistinct" +"row_numberForEach" +"row_numberIf" +"row_numberMap" +"row_numberMerge" +"row_numberNull" +"row_numberOrDefault" +"row_numberOrNull" +"row_numberResample" +"row_numberSimpleState" +"row_numberState" +"rpad" +"rtrim" +"runningAccumulate" +"runningConcurrency" +"runningDifference" +"runningDifferenceStartingWithFirstValue" +"s2CapContains" +"s2CapUnion" +"s2CellsIntersect" +"s2GetNeighbors" +"s2RectAdd" +"s2RectContains" +"s2RectIntersection" +"s2RectUnion" +"s2ToGeo" +"scalarProduct" +"sequenceCount" +"sequenceCountArgMax" +"sequenceCountArgMin" +"sequenceCountArray" +"sequenceCountDistinct" +"sequenceCountForEach" +"sequenceCountIf" +"sequenceCountMap" +"sequenceCountMerge" +"sequenceCountNull" +"sequenceCountOrDefault" +"sequenceCountOrNull" +"sequenceCountResample" +"sequenceCountSimpleState" +"sequenceCountState" +"sequenceMatch" +"sequenceMatchArgMax" +"sequenceMatchArgMin" +"sequenceMatchArray" +"sequenceMatchDistinct" +"sequenceMatchForEach" +"sequenceMatchIf" +"sequenceMatchMap" +"sequenceMatchMerge" +"sequenceMatchNull" +"sequenceMatchOrDefault" +"sequenceMatchOrNull" +"sequenceMatchResample" +"sequenceMatchSimpleState" +"sequenceMatchState" +"sequenceNextNode" +"sequenceNextNodeArgMax" +"sequenceNextNodeArgMin" +"sequenceNextNodeArray" +"sequenceNextNodeDistinct" +"sequenceNextNodeForEach" +"sequenceNextNodeIf" +"sequenceNextNodeMap" +"sequenceNextNodeMerge" +"sequenceNextNodeNull" +"sequenceNextNodeOrDefault" +"sequenceNextNodeOrNull" +"sequenceNextNodeResample" +"sequenceNextNodeSimpleState" +"sequenceNextNodeState" +"seriesDecomposeSTL" +"seriesOutliersDetectTukey" +"seriesPeriodDetectFFT" +"serverTimeZone" +"serverTimezone" +"serverUUID" +"shardCount" +"shardNum" +"showCertificate" +"sigmoid" +"sign" +"simpleJSONExtractBool" +"simpleJSONExtractFloat" +"simpleJSONExtractInt" +"simpleJSONExtractRaw" +"simpleJSONExtractString" +"simpleJSONExtractUInt" +"simpleJSONHas" +"simpleLinearRegression" +"simpleLinearRegressionArgMax" +"simpleLinearRegressionArgMin" +"simpleLinearRegressionArray" +"simpleLinearRegressionDistinct" +"simpleLinearRegressionForEach" +"simpleLinearRegressionIf" +"simpleLinearRegressionMap" +"simpleLinearRegressionMerge" +"simpleLinearRegressionNull" +"simpleLinearRegressionOrDefault" +"simpleLinearRegressionOrNull" +"simpleLinearRegressionResample" +"simpleLinearRegressionSimpleState" +"simpleLinearRegressionState" +"sin" +"singleValueOrNull" +"singleValueOrNullArgMax" +"singleValueOrNullArgMin" +"singleValueOrNullArray" +"singleValueOrNullDistinct" +"singleValueOrNullForEach" +"singleValueOrNullIf" +"singleValueOrNullMap" +"singleValueOrNullMerge" +"singleValueOrNullNull" +"singleValueOrNullOrDefault" +"singleValueOrNullOrNull" +"singleValueOrNullResample" +"singleValueOrNullSimpleState" +"singleValueOrNullState" +"sinh" +"sipHash128" +"sipHash128Keyed" +"sipHash128Reference" +"sipHash128ReferenceKeyed" +"sipHash64" +"sipHash64Keyed" +"skewPop" +"skewPopArgMax" +"skewPopArgMin" +"skewPopArray" +"skewPopDistinct" +"skewPopForEach" +"skewPopIf" +"skewPopMap" +"skewPopMerge" +"skewPopNull" +"skewPopOrDefault" +"skewPopOrNull" +"skewPopResample" +"skewPopSimpleState" +"skewPopState" +"skewSamp" +"skewSampArgMax" +"skewSampArgMin" +"skewSampArray" +"skewSampDistinct" +"skewSampForEach" +"skewSampIf" +"skewSampMap" +"skewSampMerge" +"skewSampNull" +"skewSampOrDefault" +"skewSampOrNull" +"skewSampResample" +"skewSampSimpleState" +"skewSampState" +"sleep" +"sleepEachRow" +"snowflakeToDateTime" +"snowflakeToDateTime64" +"soundex" +"space" +"sparkBar" +"sparkBarArgMax" +"sparkBarArgMin" +"sparkBarArray" +"sparkBarDistinct" +"sparkBarForEach" +"sparkBarIf" +"sparkBarMap" +"sparkBarMerge" +"sparkBarNull" +"sparkBarOrDefault" +"sparkBarOrNull" +"sparkBarResample" +"sparkBarSimpleState" +"sparkBarState" +"sparkbar" +"sparkbarArgMax" +"sparkbarArgMin" +"sparkbarArray" +"sparkbarDistinct" +"sparkbarForEach" +"sparkbarIf" +"sparkbarMap" +"sparkbarMerge" +"sparkbarNull" +"sparkbarOrDefault" +"sparkbarOrNull" +"sparkbarResample" +"sparkbarSimpleState" +"sparkbarState" +"splitByAlpha" +"splitByChar" +"splitByNonAlpha" +"splitByRegexp" +"splitByString" +"splitByWhitespace" +"sqid" +"sqidDecode" +"sqidEncode" +"sqrt" +"startsWith" +"startsWithUTF8" +"stddevPop" +"stddevPopArgMax" +"stddevPopArgMin" +"stddevPopArray" +"stddevPopDistinct" +"stddevPopForEach" +"stddevPopIf" +"stddevPopMap" +"stddevPopMerge" +"stddevPopNull" +"stddevPopOrDefault" +"stddevPopOrNull" +"stddevPopResample" +"stddevPopSimpleState" +"stddevPopStable" +"stddevPopStableArgMax" +"stddevPopStableArgMin" +"stddevPopStableArray" +"stddevPopStableDistinct" +"stddevPopStableForEach" +"stddevPopStableIf" +"stddevPopStableMap" +"stddevPopStableMerge" +"stddevPopStableNull" +"stddevPopStableOrDefault" +"stddevPopStableOrNull" +"stddevPopStableResample" +"stddevPopStableSimpleState" +"stddevPopStableState" +"stddevPopState" +"stddevSamp" +"stddevSampArgMax" +"stddevSampArgMin" +"stddevSampArray" +"stddevSampDistinct" +"stddevSampForEach" +"stddevSampIf" +"stddevSampMap" +"stddevSampMerge" +"stddevSampNull" +"stddevSampOrDefault" +"stddevSampOrNull" +"stddevSampResample" +"stddevSampSimpleState" +"stddevSampStable" +"stddevSampStableArgMax" +"stddevSampStableArgMin" +"stddevSampStableArray" +"stddevSampStableDistinct" +"stddevSampStableForEach" +"stddevSampStableIf" +"stddevSampStableMap" +"stddevSampStableMerge" +"stddevSampStableNull" +"stddevSampStableOrDefault" +"stddevSampStableOrNull" +"stddevSampStableResample" +"stddevSampStableSimpleState" +"stddevSampStableState" +"stddevSampState" +"stem" +"stochasticLinearRegression" +"stochasticLinearRegressionArgMax" +"stochasticLinearRegressionArgMin" +"stochasticLinearRegressionArray" +"stochasticLinearRegressionDistinct" +"stochasticLinearRegressionForEach" +"stochasticLinearRegressionIf" +"stochasticLinearRegressionMap" +"stochasticLinearRegressionMerge" +"stochasticLinearRegressionNull" +"stochasticLinearRegressionOrDefault" +"stochasticLinearRegressionOrNull" +"stochasticLinearRegressionResample" +"stochasticLinearRegressionSimpleState" +"stochasticLinearRegressionState" +"stochasticLogisticRegression" +"stochasticLogisticRegressionArgMax" +"stochasticLogisticRegressionArgMin" +"stochasticLogisticRegressionArray" +"stochasticLogisticRegressionDistinct" +"stochasticLogisticRegressionForEach" +"stochasticLogisticRegressionIf" +"stochasticLogisticRegressionMap" +"stochasticLogisticRegressionMerge" +"stochasticLogisticRegressionNull" +"stochasticLogisticRegressionOrDefault" +"stochasticLogisticRegressionOrNull" +"stochasticLogisticRegressionResample" +"stochasticLogisticRegressionSimpleState" +"stochasticLogisticRegressionState" +"str_to_date" +"str_to_map" +"stringJaccardIndex" +"stringJaccardIndexUTF8" +"stringToH3" +"structureToCapnProtoSchema" +"structureToProtobufSchema" +"studentTTest" +"studentTTestArgMax" +"studentTTestArgMin" +"studentTTestArray" +"studentTTestDistinct" +"studentTTestForEach" +"studentTTestIf" +"studentTTestMap" +"studentTTestMerge" +"studentTTestNull" +"studentTTestOrDefault" +"studentTTestOrNull" +"studentTTestResample" +"studentTTestSimpleState" +"studentTTestState" +"subBitmap" +"subDate" +"substr" +"substring" +"substringIndex" +"substringIndexUTF8" +"substringUTF8" +"subtractDays" +"subtractHours" +"subtractInterval" +"subtractMicroseconds" +"subtractMilliseconds" +"subtractMinutes" +"subtractMonths" +"subtractNanoseconds" +"subtractQuarters" +"subtractSeconds" +"subtractTupleOfIntervals" +"subtractWeeks" +"subtractYears" +"sum" +"sumArgMax" +"sumArgMin" +"sumArray" +"sumCount" +"sumCountArgMax" +"sumCountArgMin" +"sumCountArray" +"sumCountDistinct" +"sumCountForEach" +"sumCountIf" +"sumCountMap" +"sumCountMerge" +"sumCountNull" +"sumCountOrDefault" +"sumCountOrNull" +"sumCountResample" +"sumCountSimpleState" +"sumCountState" +"sumDistinct" +"sumForEach" +"sumIf" +"sumKahan" +"sumKahanArgMax" +"sumKahanArgMin" +"sumKahanArray" +"sumKahanDistinct" +"sumKahanForEach" +"sumKahanIf" +"sumKahanMap" +"sumKahanMerge" +"sumKahanNull" +"sumKahanOrDefault" +"sumKahanOrNull" +"sumKahanResample" +"sumKahanSimpleState" +"sumKahanState" +"sumMap" +"sumMapFiltered" +"sumMapFilteredArgMax" +"sumMapFilteredArgMin" +"sumMapFilteredArray" +"sumMapFilteredDistinct" +"sumMapFilteredForEach" +"sumMapFilteredIf" +"sumMapFilteredMap" +"sumMapFilteredMerge" +"sumMapFilteredNull" +"sumMapFilteredOrDefault" +"sumMapFilteredOrNull" +"sumMapFilteredResample" +"sumMapFilteredSimpleState" +"sumMapFilteredState" +"sumMapFilteredWithOverflow" +"sumMapFilteredWithOverflowArgMax" +"sumMapFilteredWithOverflowArgMin" +"sumMapFilteredWithOverflowArray" +"sumMapFilteredWithOverflowDistinct" +"sumMapFilteredWithOverflowForEach" +"sumMapFilteredWithOverflowIf" +"sumMapFilteredWithOverflowMap" +"sumMapFilteredWithOverflowMerge" +"sumMapFilteredWithOverflowNull" +"sumMapFilteredWithOverflowOrDefault" +"sumMapFilteredWithOverflowOrNull" +"sumMapFilteredWithOverflowResample" +"sumMapFilteredWithOverflowSimpleState" +"sumMapFilteredWithOverflowState" +"sumMapWithOverflow" +"sumMapWithOverflowArgMax" +"sumMapWithOverflowArgMin" +"sumMapWithOverflowArray" +"sumMapWithOverflowDistinct" +"sumMapWithOverflowForEach" +"sumMapWithOverflowIf" +"sumMapWithOverflowMap" +"sumMapWithOverflowMerge" +"sumMapWithOverflowNull" +"sumMapWithOverflowOrDefault" +"sumMapWithOverflowOrNull" +"sumMapWithOverflowResample" +"sumMapWithOverflowSimpleState" +"sumMapWithOverflowState" +"sumMappedArrays" +"sumMappedArraysArgMax" +"sumMappedArraysArgMin" +"sumMappedArraysArray" +"sumMappedArraysDistinct" +"sumMappedArraysForEach" +"sumMappedArraysIf" +"sumMappedArraysMap" +"sumMappedArraysMerge" +"sumMappedArraysNull" +"sumMappedArraysOrDefault" +"sumMappedArraysOrNull" +"sumMappedArraysResample" +"sumMappedArraysSimpleState" +"sumMappedArraysState" +"sumMerge" +"sumNull" +"sumOrDefault" +"sumOrNull" +"sumResample" +"sumSimpleState" +"sumState" +"sumWithOverflow" +"sumWithOverflowArgMax" +"sumWithOverflowArgMin" +"sumWithOverflowArray" +"sumWithOverflowDistinct" +"sumWithOverflowForEach" +"sumWithOverflowIf" +"sumWithOverflowMap" +"sumWithOverflowMerge" +"sumWithOverflowNull" +"sumWithOverflowOrDefault" +"sumWithOverflowOrNull" +"sumWithOverflowResample" +"sumWithOverflowSimpleState" +"sumWithOverflowState" +"svg" +"synonyms" +"tan" +"tanh" +"tcpPort" +"tgamma" +"theilsU" +"theilsUArgMax" +"theilsUArgMin" +"theilsUArray" +"theilsUDistinct" +"theilsUForEach" +"theilsUIf" +"theilsUMap" +"theilsUMerge" +"theilsUNull" +"theilsUOrDefault" +"theilsUOrNull" +"theilsUResample" +"theilsUSimpleState" +"theilsUState" +"throwIf" +"tid" +"timeDiff" +"timeSlot" +"timeSlots" +"timeZone" +"timeZoneOf" +"timeZoneOffset" +"timestamp" +"timestampDiff" +"timestamp_diff" +"timezone" +"timezoneOf" +"timezoneOffset" +"toBool" +"toColumnTypeName" +"toDate" +"toDate32" +"toDate32OrDefault" +"toDate32OrNull" +"toDate32OrZero" +"toDateOrDefault" +"toDateOrNull" +"toDateOrZero" +"toDateTime" +"toDateTime32" +"toDateTime64" +"toDateTime64OrDefault" +"toDateTime64OrNull" +"toDateTime64OrZero" +"toDateTimeOrDefault" +"toDateTimeOrNull" +"toDateTimeOrZero" +"toDayOfMonth" +"toDayOfWeek" +"toDayOfYear" +"toDaysSinceYearZero" +"toDecimal128" +"toDecimal128OrDefault" +"toDecimal128OrNull" +"toDecimal128OrZero" +"toDecimal256" +"toDecimal256OrDefault" +"toDecimal256OrNull" +"toDecimal256OrZero" +"toDecimal32" +"toDecimal32OrDefault" +"toDecimal32OrNull" +"toDecimal32OrZero" +"toDecimal64" +"toDecimal64OrDefault" +"toDecimal64OrNull" +"toDecimal64OrZero" +"toDecimalString" +"toFixedString" +"toFloat32" +"toFloat32OrDefault" +"toFloat32OrNull" +"toFloat32OrZero" +"toFloat64" +"toFloat64OrDefault" +"toFloat64OrNull" +"toFloat64OrZero" +"toHour" +"toIPv4" +"toIPv4OrDefault" +"toIPv4OrNull" +"toIPv4OrZero" +"toIPv6" +"toIPv6OrDefault" +"toIPv6OrNull" +"toIPv6OrZero" +"toISOWeek" +"toISOYear" +"toInt128" +"toInt128OrDefault" +"toInt128OrNull" +"toInt128OrZero" +"toInt16" +"toInt16OrDefault" +"toInt16OrNull" +"toInt16OrZero" +"toInt256" +"toInt256OrDefault" +"toInt256OrNull" +"toInt256OrZero" +"toInt32" +"toInt32OrDefault" +"toInt32OrNull" +"toInt32OrZero" +"toInt64" +"toInt64OrDefault" +"toInt64OrNull" +"toInt64OrZero" +"toInt8" +"toInt8OrDefault" +"toInt8OrNull" +"toInt8OrZero" +"toIntervalDay" +"toIntervalHour" +"toIntervalMicrosecond" +"toIntervalMillisecond" +"toIntervalMinute" +"toIntervalMonth" +"toIntervalNanosecond" +"toIntervalQuarter" +"toIntervalSecond" +"toIntervalWeek" +"toIntervalYear" +"toJSONString" +"toLastDayOfMonth" +"toLastDayOfWeek" +"toLowCardinality" +"toMillisecond" +"toMinute" +"toModifiedJulianDay" +"toModifiedJulianDayOrNull" +"toMonday" +"toMonth" +"toNullable" +"toQuarter" +"toRelativeDayNum" +"toRelativeHourNum" +"toRelativeMinuteNum" +"toRelativeMonthNum" +"toRelativeQuarterNum" +"toRelativeSecondNum" +"toRelativeWeekNum" +"toRelativeYearNum" +"toSecond" +"toStartOfDay" +"toStartOfFifteenMinutes" +"toStartOfFiveMinute" +"toStartOfFiveMinutes" +"toStartOfHour" +"toStartOfISOYear" +"toStartOfInterval" +"toStartOfMicrosecond" +"toStartOfMillisecond" +"toStartOfMinute" +"toStartOfMonth" +"toStartOfNanosecond" +"toStartOfQuarter" +"toStartOfSecond" +"toStartOfTenMinutes" +"toStartOfWeek" +"toStartOfYear" +"toString" +"toStringCutToZero" +"toTime" +"toTimeZone" +"toTimezone" +"toTypeName" +"toUInt128" +"toUInt128OrDefault" +"toUInt128OrNull" +"toUInt128OrZero" +"toUInt16" +"toUInt16OrDefault" +"toUInt16OrNull" +"toUInt16OrZero" +"toUInt256" +"toUInt256OrDefault" +"toUInt256OrNull" +"toUInt256OrZero" +"toUInt32" +"toUInt32OrDefault" +"toUInt32OrNull" +"toUInt32OrZero" +"toUInt64" +"toUInt64OrDefault" +"toUInt64OrNull" +"toUInt64OrZero" +"toUInt8" +"toUInt8OrDefault" +"toUInt8OrNull" +"toUInt8OrZero" +"toUTCTimestamp" +"toUUID" +"toUUIDOrDefault" +"toUUIDOrNull" +"toUUIDOrZero" +"toUnixTimestamp" +"toUnixTimestamp64Micro" +"toUnixTimestamp64Milli" +"toUnixTimestamp64Nano" +"toValidUTF8" +"toWeek" +"toYYYYMM" +"toYYYYMMDD" +"toYYYYMMDDhhmmss" +"toYear" +"toYearWeek" +"to_utc_timestamp" +"today" +"tokens" +"topK" +"topKArgMax" +"topKArgMin" +"topKArray" +"topKDistinct" +"topKForEach" +"topKIf" +"topKMap" +"topKMerge" +"topKNull" +"topKOrDefault" +"topKOrNull" +"topKResample" +"topKSimpleState" +"topKState" +"topKWeighted" +"topKWeightedArgMax" +"topKWeightedArgMin" +"topKWeightedArray" +"topKWeightedDistinct" +"topKWeightedForEach" +"topKWeightedIf" +"topKWeightedMap" +"topKWeightedMerge" +"topKWeightedNull" +"topKWeightedOrDefault" +"topKWeightedOrNull" +"topKWeightedResample" +"topKWeightedSimpleState" +"topKWeightedState" +"topLevelDomain" +"topLevelDomainRFC" +"transactionID" +"transactionLatestSnapshot" +"transactionOldestSnapshot" +"transform" +"translate" +"translateUTF8" +"trim" +"trimBoth" +"trimLeft" +"trimRight" +"trunc" +"truncate" +"tryBase58Decode" +"tryBase64Decode" +"tryDecrypt" +"tryIdnaEncode" +"tryPunycodeDecode" +"tumble" +"tumbleEnd" +"tumbleStart" +"tuple" +"tupleConcat" +"tupleDivide" +"tupleDivideByNumber" +"tupleElement" +"tupleHammingDistance" +"tupleIntDiv" +"tupleIntDivByNumber" +"tupleIntDivOrZero" +"tupleIntDivOrZeroByNumber" +"tupleMinus" +"tupleModulo" +"tupleModuloByNumber" +"tupleMultiply" +"tupleMultiplyByNumber" +"tupleNegate" +"tuplePlus" +"tupleToNameValuePairs" +"ucase" +"unbin" +"unhex" +"uniq" +"uniqArgMax" +"uniqArgMin" +"uniqArray" +"uniqCombined" +"uniqCombined64" +"uniqCombined64ArgMax" +"uniqCombined64ArgMin" +"uniqCombined64Array" +"uniqCombined64Distinct" +"uniqCombined64ForEach" +"uniqCombined64If" +"uniqCombined64Map" +"uniqCombined64Merge" +"uniqCombined64Null" +"uniqCombined64OrDefault" +"uniqCombined64OrNull" +"uniqCombined64Resample" +"uniqCombined64SimpleState" +"uniqCombined64State" +"uniqCombinedArgMax" +"uniqCombinedArgMin" +"uniqCombinedArray" +"uniqCombinedDistinct" +"uniqCombinedForEach" +"uniqCombinedIf" +"uniqCombinedMap" +"uniqCombinedMerge" +"uniqCombinedNull" +"uniqCombinedOrDefault" +"uniqCombinedOrNull" +"uniqCombinedResample" +"uniqCombinedSimpleState" +"uniqCombinedState" +"uniqDistinct" +"uniqExact" +"uniqExactArgMax" +"uniqExactArgMin" +"uniqExactArray" +"uniqExactDistinct" +"uniqExactForEach" +"uniqExactIf" +"uniqExactMap" +"uniqExactMerge" +"uniqExactNull" +"uniqExactOrDefault" +"uniqExactOrNull" +"uniqExactResample" +"uniqExactSimpleState" +"uniqExactState" +"uniqForEach" +"uniqHLL12" +"uniqHLL12ArgMax" +"uniqHLL12ArgMin" +"uniqHLL12Array" +"uniqHLL12Distinct" +"uniqHLL12ForEach" +"uniqHLL12If" +"uniqHLL12Map" +"uniqHLL12Merge" +"uniqHLL12Null" +"uniqHLL12OrDefault" +"uniqHLL12OrNull" +"uniqHLL12Resample" +"uniqHLL12SimpleState" +"uniqHLL12State" +"uniqIf" +"uniqMap" +"uniqMerge" +"uniqNull" +"uniqOrDefault" +"uniqOrNull" +"uniqResample" +"uniqSimpleState" +"uniqState" +"uniqTheta" +"uniqThetaArgMax" +"uniqThetaArgMin" +"uniqThetaArray" +"uniqThetaDistinct" +"uniqThetaForEach" +"uniqThetaIf" +"uniqThetaIntersect" +"uniqThetaMap" +"uniqThetaMerge" +"uniqThetaNot" +"uniqThetaNull" +"uniqThetaOrDefault" +"uniqThetaOrNull" +"uniqThetaResample" +"uniqThetaSimpleState" +"uniqThetaState" +"uniqThetaUnion" +"uniqUpTo" +"uniqUpToArgMax" +"uniqUpToArgMin" +"uniqUpToArray" +"uniqUpToDistinct" +"uniqUpToForEach" +"uniqUpToIf" +"uniqUpToMap" +"uniqUpToMerge" +"uniqUpToNull" +"uniqUpToOrDefault" +"uniqUpToOrNull" +"uniqUpToResample" +"uniqUpToSimpleState" +"uniqUpToState" +"upper" +"upperUTF8" +"uptime" +"user" +"validateNestedArraySizes" +"varPop" +"varPopArgMax" +"varPopArgMin" +"varPopArray" +"varPopDistinct" +"varPopForEach" +"varPopIf" +"varPopMap" +"varPopMerge" +"varPopNull" +"varPopOrDefault" +"varPopOrNull" +"varPopResample" +"varPopSimpleState" +"varPopStable" +"varPopStableArgMax" +"varPopStableArgMin" +"varPopStableArray" +"varPopStableDistinct" +"varPopStableForEach" +"varPopStableIf" +"varPopStableMap" +"varPopStableMerge" +"varPopStableNull" +"varPopStableOrDefault" +"varPopStableOrNull" +"varPopStableResample" +"varPopStableSimpleState" +"varPopStableState" +"varPopState" +"varSamp" +"varSampArgMax" +"varSampArgMin" +"varSampArray" +"varSampDistinct" +"varSampForEach" +"varSampIf" +"varSampMap" +"varSampMerge" +"varSampNull" +"varSampOrDefault" +"varSampOrNull" +"varSampResample" +"varSampSimpleState" +"varSampStable" +"varSampStableArgMax" +"varSampStableArgMin" +"varSampStableArray" +"varSampStableDistinct" +"varSampStableForEach" +"varSampStableIf" +"varSampStableMap" +"varSampStableMerge" +"varSampStableNull" +"varSampStableOrDefault" +"varSampStableOrNull" +"varSampStableResample" +"varSampStableSimpleState" +"varSampStableState" +"varSampState" +"variantElement" +"variantType" +"vectorDifference" +"vectorSum" +"version" +"visibleWidth" +"visitParamExtractBool" +"visitParamExtractFloat" +"visitParamExtractInt" +"visitParamExtractRaw" +"visitParamExtractString" +"visitParamExtractUInt" +"visitParamHas" +"week" +"welchTTest" +"welchTTestArgMax" +"welchTTestArgMin" +"welchTTestArray" +"welchTTestDistinct" +"welchTTestForEach" +"welchTTestIf" +"welchTTestMap" +"welchTTestMerge" +"welchTTestNull" +"welchTTestOrDefault" +"welchTTestOrNull" +"welchTTestResample" +"welchTTestSimpleState" +"welchTTestState" +"widthBucket" +"width_bucket" +"windowFunnel" +"windowFunnelArgMax" +"windowFunnelArgMin" +"windowFunnelArray" +"windowFunnelDistinct" +"windowFunnelForEach" +"windowFunnelIf" +"windowFunnelMap" +"windowFunnelMerge" +"windowFunnelNull" +"windowFunnelOrDefault" +"windowFunnelOrNull" +"windowFunnelResample" +"windowFunnelSimpleState" +"windowFunnelState" +"windowID" +"wkt" +"wordShingleMinHash" +"wordShingleMinHashArg" +"wordShingleMinHashArgCaseInsensitive" +"wordShingleMinHashArgCaseInsensitiveUTF8" +"wordShingleMinHashArgUTF8" +"wordShingleMinHashCaseInsensitive" +"wordShingleMinHashCaseInsensitiveUTF8" +"wordShingleMinHashUTF8" +"wordShingleSimHash" +"wordShingleSimHashCaseInsensitive" +"wordShingleSimHashCaseInsensitiveUTF8" +"wordShingleSimHashUTF8" +"wyHash64" +"xor" +"xxHash32" +"xxHash64" +"xxh3" +"yandexConsistentHash" +"yearweek" +"yesterday" +"zookeeperSessionUptime" diff --git a/tests/fuzz/dictionaries/key_words.dict b/tests/fuzz/dictionaries/key_words.dict index 0db2d159759..db517a2382c 100644 --- a/tests/fuzz/dictionaries/key_words.dict +++ b/tests/fuzz/dictionaries/key_words.dict @@ -1,183 +1,543 @@ -# Key words (based on keywords from antlr parser) - "ADD" +"ADD COLUMN" +"ADD CONSTRAINT" +"ADD INDEX" +"ADD PROJECTION" +"ADD STATISTIC" +"ADMIN OPTION FOR" "AFTER" +"ALGORITHM" "ALIAS" "ALL" +"ALLOWED_LATENESS" "ALTER" +"ALTER COLUMN" +"ALTER DATABASE" +"ALTER LIVE VIEW" +"ALTER POLICY" +"ALTER PROFILE" +"ALTER QUOTA" +"ALTER ROLE" +"ALTER ROW POLICY" +"ALTER SETTINGS PROFILE" +"ALTER TABLE" +"ALTER TEMPORARY TABLE" +"ALTER USER" "AND" +"AND STDOUT" "ANTI" "ANY" -"ARRAY" +"APPEND" +"APPLY" +"APPLY DELETED MASK" +"ARRAY JOIN" "AS" +"ASC" "ASCENDING" "ASOF" +"ASSUME" "AST" "ASYNC" "ATTACH" +"ATTACH PART" +"ATTACH PARTITION" +"ATTACH POLICY" +"ATTACH PROFILE" +"ATTACH QUOTA" +"ATTACH ROLE" +"ATTACH ROW POLICY" +"ATTACH SETTINGS PROFILE" +"ATTACH USER" +"AUTO_INCREMENT" +"AZURE" +"BACKUP" +"BCRYPT_HASH" +"BCRYPT_PASSWORD" +"BEGIN TRANSACTION" "BETWEEN" +"BIDIRECTIONAL" "BOTH" "BY" +"CASCADE" "CASE" "CAST" +"CHANGE" +"CHANGEABLE_IN_READONLY" +"CHANGED" +"CHAR" +"CHAR VARYING" +"CHARACTER" +"CHARACTER LARGE OBJECT" +"CHARACTER VARYING" "CHECK" -"CLEAR" +"CHECK ALL TABLES" +"CHECK TABLE" +"CLEANUP" +"CLEAR COLUMN" +"CLEAR INDEX" +"CLEAR PROJECTION" +"CLEAR STATISTIC" "CLUSTER" +"CLUSTERS" +"CN" "CODEC" "COLLATE" "COLUMN" +"COLUMNS" "COMMENT" +"COMMENT COLUMN" +"COMMIT" +"COMPRESSION" +"CONST" "CONSTRAINT" "CREATE" +"CREATE POLICY" +"CREATE PROFILE" +"CREATE QUOTA" +"CREATE ROLE" +"CREATE ROW POLICY" +"CREATE SETTINGS PROFILE" +"CREATE TABLE" +"CREATE TEMPORARY TABLE" +"CREATE USER" "CROSS" "CUBE" +"CURRENT GRANTS" +"CURRENT QUOTA" +"CURRENT ROLES" +"CURRENT ROW" +"CURRENT TRANSACTION" +"CURRENTUSER" +"CURRENT_USER" +"D" "DATABASE" "DATABASES" "DATE" "DAY" +"DAYS" +"DD" "DEDUPLICATE" "DEFAULT" -"DELAY" +"DEFAULT DATABASE" +"DEFAULT ROLE" +"DEFINER" "DELETE" +"DEPENDS ON" "DESC" "DESCENDING" "DESCRIBE" "DETACH" +"DETACH PART" +"DETACH PARTITION" "DICTIONARIES" "DICTIONARY" "DISK" "DISTINCT" -"DISTRIBUTED" +"DISTINCT ON" +"DIV" +"DOUBLE_SHA1_HASH" +"DOUBLE_SHA1_PASSWORD" "DROP" +"DROP COLUMN" +"DROP CONSTRAINT" +"DROP DEFAULT" +"DROP DETACHED PART" +"DROP DETACHED PARTITION" +"DROP INDEX" +"DROP PART" +"DROP PARTITION" +"DROP PROJECTION" +"DROP STATISTIC" +"DROP TABLE" +"DROP TEMPORARY TABLE" "ELSE" +"EMPTY" +"EMPTY AS" +"ENABLED ROLES" "END" +"ENFORCED" "ENGINE" +"EPHEMERAL" +"EPHEMERAL SEQUENTIAL" +"ESTIMATE" +"EVENT" "EVENTS" +"EVERY" +"EXCEPT" +"EXCEPT DATABASE" +"EXCEPT DATABASES" +"EXCEPT TABLE" +"EXCEPT TABLES" +"EXCHANGE DICTIONARIES" +"EXCHANGE TABLES" "EXISTS" "EXPLAIN" "EXPRESSION" -"EXTRACT" -"FETCHES" +"EXTENDED" +"EXTERNAL DDL FROM" +"FALSE" +"FETCH" +"FETCH PART" +"FETCH PARTITION" +"FIELDS" +"FILE" +"FILESYSTEM CACHE" +"FILESYSTEM CACHES" +"FILTER" "FINAL" "FIRST" -"FLUSH" +"FOLLOWING" "FOR" +"FOREIGN" +"FOREIGN KEY" +"FORGET PARTITION" "FORMAT" "FREEZE" "FROM" +"FROM INFILE" +"FROM SHARD" "FULL" +"FULLTEXT" "FUNCTION" "GLOBAL" +"GLOBAL IN" +"GLOBAL NOT IN" +"GRANT" +"GRANT OPTION FOR" +"GRANTEES" "GRANULARITY" -"GROUP" +"GROUP BY" +"GROUPING SETS" +"GROUPS" +"H" +"HASH" "HAVING" +"HDFS" +"HH" "HIERARCHICAL" +"HOST" "HOUR" +"HOURS" +"HTTP" "ID" -"IF" +"IDENTIFIED" +"IF EMPTY" +"IF EXISTS" +"IF NOT EXISTS" +"IGNORE NULLS" "ILIKE" "IN" +"IN PARTITION" "INDEX" -"INF" +"INDEXES" +"INDICES" +"INHERIT" "INJECTIVE" "INNER" -"INSERT" +"INSERT INTO" +"INTERPOLATE" +"INTERSECT" "INTERVAL" -"INTO" -"IS" +"INTO OUTFILE" +"INVISIBLE" +"INVOKER" +"IP" +"IS NOT DISTINCT FROM" +"IS NOT NULL" +"IS NULL" "IS_OBJECT_ID" "JOIN" +"KERBEROS" "KEY" +"KEY BY" +"KEYED BY" +"KEYS" "KILL" +"KIND" +"LARGE OBJECT" "LAST" "LAYOUT" +"LDAP" "LEADING" "LEFT" +"LEFT ARRAY JOIN" +"LESS THAN" +"LEVEL" "LIFETIME" +"LIGHTWEIGHT" "LIKE" "LIMIT" +"LINEAR" +"LIST" "LIVE" "LOCAL" -"LOGS" +"M" +"MATCH" "MATERIALIZE" +"MATERIALIZE COLUMN" +"MATERIALIZE INDEX" +"MATERIALIZE PROJECTION" +"MATERIALIZE STATISTIC" +"MATERIALIZE TTL" "MATERIALIZED" "MAX" +"MCS" +"MEMORY" "MERGES" +"MI" +"MICROSECOND" +"MICROSECONDS" +"MILLISECOND" +"MILLISECONDS" "MIN" "MINUTE" +"MINUTES" +"MM" +"MOD" "MODIFY" +"MODIFY COLUMN" +"MODIFY COMMENT" +"MODIFY ORDER BY" +"MODIFY QUERY" +"MODIFY REFRESH" +"MODIFY SAMPLE BY" +"MODIFY SETTING" +"MODIFY SQL SECURITY" +"MODIFY TTL" "MONTH" +"MONTHS" "MOVE" +"MOVE PART" +"MOVE PARTITION" +"MS" "MUTATION" -"NAN_SQL" -"NO" +"N" +"NAME" +"NAMED COLLECTION" +"NANOSECOND" +"NANOSECONDS" +"NEXT" +"NO ACTION" +"NO DELAY" +"NO LIMITS" +"NONE" "NOT" -"NULL_SQL" +"NOT BETWEEN" +"NOT IDENTIFIED" +"NOT ILIKE" +"NOT IN" +"NOT KEYED" +"NOT LIKE" +"NOT OVERRIDABLE" +"NO_PASSWORD" +"NS" +"NULL" "NULLS" "OFFSET" "ON" -"OPTIMIZE" +"ON DELETE" +"ON UPDATE" +"ON VOLUME" +"ONLY" +"OPTIMIZE TABLE" "OR" -"ORDER" +"OR REPLACE" +"ORDER BY" "OUTER" -"OUTFILE" +"OVER" +"OVERRIDABLE" +"PART" +"PARTIAL" "PARTITION" +"PARTITION BY" +"PARTITIONS" +"PART_MOVE_TO_SHARD" +"PASTE" +"PERIODIC REFRESH" +"PERMANENTLY" +"PERMISSIVE" +"PERSISTENT" +"PERSISTENT SEQUENTIAL" +"PIPELINE" +"PLAINTEXT_PASSWORD" +"PLAN" "POPULATE" +"PRECEDING" +"PRECISION" "PREWHERE" "PRIMARY" +"PRIMARY KEY" +"PROFILE" "PROJECTION" +"PULL" +"Protobuf" +"Q" +"QQ" "QUARTER" +"QUARTERS" +"QUERY" +"QUERY TREE" +"QUOTA" +"RANDOMIZE FOR" +"RANDOMIZED" "RANGE" -"RELOAD" +"READONLY" +"REALM" +"RECOMPRESS" +"REFERENCES" +"REFRESH" +"REGEXP" "REMOVE" +"REMOVE SAMPLE BY" +"REMOVE TTL" "RENAME" +"RENAME COLUMN" +"RENAME DATABASE" +"RENAME DICTIONARY" +"RENAME TABLE" +"RENAME TO" "REPLACE" -"REPLICA" -"REPLICATED" +"REPLACE PARTITION" +"RESET SETTING" +"RESPECT NULLS" +"RESTORE" +"RESTRICT" +"RESTRICTIVE" +"RESUME" +"REVOKE" "RIGHT" +"ROLLBACK" "ROLLUP" +"ROW" +"ROWS" +"S" +"S3" +"SALT" "SAMPLE" +"SAMPLE BY" +"SCHEME" "SECOND" +"SECONDS" "SELECT" "SEMI" -"SENDS" +"SERVER" "SET" +"SET DEFAULT" +"SET DEFAULT ROLE" +"SET FAKE TIME" +"SET NULL" +"SET ROLE" +"SET ROLE DEFAULT" +"SET TRANSACTION SNAPSHOT" "SETTINGS" +"SHA256_HASH" +"SHA256_PASSWORD" "SHOW" +"SHOW ACCESS" +"SHOW CREATE" +"SHOW ENGINES" +"SHOW FUNCTIONS" +"SHOW GRANTS" +"SHOW PRIVILEGES" +"SHOW PROCESSLIST" +"SHOW SETTING" +"SIGNED" +"SIMPLE" "SOURCE" -"START" -"STOP" -"SUBSTRING" +"SPATIAL" +"SQL SECURITY" +"SQL_TSI_DAY" +"SQL_TSI_HOUR" +"SQL_TSI_MICROSECOND" +"SQL_TSI_MILLISECOND" +"SQL_TSI_MINUTE" +"SQL_TSI_MONTH" +"SQL_TSI_NANOSECOND" +"SQL_TSI_QUARTER" +"SQL_TSI_SECOND" +"SQL_TSI_WEEK" +"SQL_TSI_YEAR" +"SS" +"SSH_KEY" +"SSL_CERTIFICATE" +"STATISTIC" +"STEP" +"STORAGE" +"STRICT" +"STRICTLY_ASCENDING" +"SUBPARTITION" +"SUBPARTITION BY" +"SUBPARTITIONS" +"SUSPEND" "SYNC" "SYNTAX" "SYSTEM" "TABLE" +"TABLE OVERRIDE" "TABLES" "TEMPORARY" +"TEMPORARY TABLE" "TEST" "THEN" -"TIES" -"TIMEOUT" "TIMESTAMP" "TO" +"TO DISK" +"TO INNER UUID" +"TO SHARD" +"TO TABLE" +"TO VOLUME" "TOP" "TOTALS" +"TRACKING ONLY" "TRAILING" -"TRIM" +"TRANSACTION" +"TRIGGER" +"TRUE" "TRUNCATE" "TTL" "TYPE" +"TYPEOF" +"UNBOUNDED" +"UNDROP" +"UNFREEZE" "UNION" +"UNIQUE" +"UNSET FAKE TIME" +"UNSIGNED" "UPDATE" +"URL" "USE" "USING" "UUID" +"VALID UNTIL" "VALUES" +"VARYING" "VIEW" -"VOLUME" +"VISIBLE" "WATCH" +"WATERMARK" "WEEK" +"WEEKS" "WHEN" "WHERE" +"WINDOW" "WITH" +"WITH ADMIN OPTION" +"WITH CHECK" +"WITH FILL" +"WITH GRANT OPTION" +"WITH NAME" +"WITH REPLACE OPTION" +"WITH TIES" +"WK" +"WRITABLE" +"WW" "YEAR" +"YEARS" +"YY" +"YYYY" +"ZKPATH" +"bagexpansion" +"base_backup" +"cluster_host_ids" +"with_itemindex"