diff --git a/docs/en/sql-reference/functions/conditional-functions.md b/docs/en/sql-reference/functions/conditional-functions.md index eb4e98961f1..564186fd8db 100644 --- a/docs/en/sql-reference/functions/conditional-functions.md +++ b/docs/en/sql-reference/functions/conditional-functions.md @@ -234,3 +234,34 @@ SELECT least(toDateTime32(now() + toIntervalDay(1)), toDateTime64(now(), 3)) :::note The type returned is a DateTime64 as the DataTime32 must be promoted to 64 bit for the comparison. ::: + +## clamp + +Constrain the return value between A and B. + +**Syntax** + +``` sql +clamp(value, min, max) +``` + +**Arguments** + +- `value` – Input value. +- `min` – Limit the lower bound. +- `max` – Limit the upper bound. + +**Returned values** + +If the value is less than the minimum value, return the minimum value; if it is greater than the maximum value, return the maximum value; otherwise, return the current value. + +Examples: + +```sql +SELECT clamp(1, 2, 3) result, toTypeName(result) type; +``` +```response +┌─result─┬─type────┐ +│ 2 │ Float64 │ +└────────┴─────────┘ +``` \ No newline at end of file diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 8d14b7eeb0d..8abffcfc8ee 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -201,8 +201,11 @@ public: void convertToNullable() override { - chassert(kind == FunctionKind::ORDINARY); - wrap_with_nullable = true; + /// Ignore other function kinds. + /// We might try to convert aggregate/window function for invalid query + /// before the validation happened. + if (kind == FunctionKind::ORDINARY) + wrap_with_nullable = true; } void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index e20ff4ab909..b6837c86615 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -475,7 +475,7 @@ struct TableExpressionData class ExpressionsStack { public: - void pushNode(const QueryTreeNodePtr & node) + void push(const QueryTreeNodePtr & node) { if (node->hasAlias()) { @@ -492,7 +492,7 @@ public: expressions.emplace_back(node); } - void popNode() + void pop() { const auto & top_expression = expressions.back(); const auto & top_expression_alias = top_expression->getAlias(); @@ -730,6 +730,8 @@ struct IdentifierResolveScope join_use_nulls = context->getSettingsRef().join_use_nulls; else if (parent_scope) join_use_nulls = parent_scope->join_use_nulls; + + alias_name_to_expression_node = &alias_name_to_expression_node_before_group_by; } QueryTreeNodePtr scope_node; @@ -745,7 +747,10 @@ struct IdentifierResolveScope std::unordered_map expression_argument_name_to_node; /// Alias name to query expression node - std::unordered_map alias_name_to_expression_node; + std::unordered_map alias_name_to_expression_node_before_group_by; + std::unordered_map alias_name_to_expression_node_after_group_by; + + std::unordered_map * alias_name_to_expression_node = nullptr; /// Alias name to lambda node std::unordered_map alias_name_to_lambda_node; @@ -878,6 +883,22 @@ struct IdentifierResolveScope return it->second; } + void pushExpressionNode(const QueryTreeNodePtr & node) + { + bool had_aggregate_function = expressions_in_resolve_process_stack.hasAggregateFunction(); + expressions_in_resolve_process_stack.push(node); + if (group_by_use_nulls && had_aggregate_function != expressions_in_resolve_process_stack.hasAggregateFunction()) + alias_name_to_expression_node = &alias_name_to_expression_node_before_group_by; + } + + void popExpressionNode() + { + bool had_aggregate_function = expressions_in_resolve_process_stack.hasAggregateFunction(); + expressions_in_resolve_process_stack.pop(); + if (group_by_use_nulls && had_aggregate_function != expressions_in_resolve_process_stack.hasAggregateFunction()) + alias_name_to_expression_node = &alias_name_to_expression_node_after_group_by; + } + /// Dump identifier resolve scope [[maybe_unused]] void dump(WriteBuffer & buffer) const { @@ -894,8 +915,8 @@ struct IdentifierResolveScope for (const auto & [alias_name, node] : expression_argument_name_to_node) buffer << "Alias name " << alias_name << " node " << node->formatASTForErrorMessage() << '\n'; - buffer << "Alias name to expression node table size " << alias_name_to_expression_node.size() << '\n'; - for (const auto & [alias_name, node] : alias_name_to_expression_node) + buffer << "Alias name to expression node table size " << alias_name_to_expression_node->size() << '\n'; + for (const auto & [alias_name, node] : *alias_name_to_expression_node) buffer << "Alias name " << alias_name << " expression node " << node->dumpTree() << '\n'; buffer << "Alias name to function node table size " << alias_name_to_lambda_node.size() << '\n'; @@ -1023,7 +1044,7 @@ private: if (is_lambda_node) { - if (scope.alias_name_to_expression_node.contains(alias)) + if (scope.alias_name_to_expression_node->contains(alias)) scope.nodes_with_duplicated_aliases.insert(node); auto [_, inserted] = scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); @@ -1036,7 +1057,7 @@ private: if (scope.alias_name_to_lambda_node.contains(alias)) scope.nodes_with_duplicated_aliases.insert(node); - auto [_, inserted] = scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); + auto [_, inserted] = scope.alias_name_to_expression_node->insert(std::make_pair(alias, node)); if (!inserted) scope.nodes_with_duplicated_aliases.insert(node); @@ -1838,7 +1859,7 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection( if (allow_expression_identifiers) { - for (const auto & [name, expression] : scope.alias_name_to_expression_node) + for (const auto & [name, expression] : *scope.alias_name_to_expression_node) { assert(expression); auto expression_identifier = Identifier(name); @@ -1868,7 +1889,7 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection( { if (allow_function_identifiers) { - for (const auto & [name, _] : scope.alias_name_to_expression_node) + for (const auto & [name, _] : *scope.alias_name_to_expression_node) valid_identifiers_result.insert(Identifier(name)); } @@ -2768,7 +2789,7 @@ bool QueryAnalyzer::tryBindIdentifierToAliases(const IdentifierLookup & identifi auto get_alias_name_to_node_map = [&]() -> const std::unordered_map & { if (identifier_lookup.isExpressionLookup()) - return scope.alias_name_to_expression_node; + return *scope.alias_name_to_expression_node; else if (identifier_lookup.isFunctionLookup()) return scope.alias_name_to_lambda_node; @@ -2830,7 +2851,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier auto get_alias_name_to_node_map = [&]() -> std::unordered_map & { if (identifier_lookup.isExpressionLookup()) - return scope.alias_name_to_expression_node; + return *scope.alias_name_to_expression_node; else if (identifier_lookup.isFunctionLookup()) return scope.alias_name_to_lambda_node; @@ -2868,7 +2889,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier /// Resolve expression if necessary if (node_type == QueryTreeNodeType::IDENTIFIER) { - scope.expressions_in_resolve_process_stack.pushNode(it->second); + scope.pushExpressionNode(it->second); auto & alias_identifier_node = it->second->as(); auto identifier = alias_identifier_node.getIdentifier(); @@ -2899,9 +2920,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier if (identifier_lookup.isExpressionLookup()) scope.alias_name_to_lambda_node.erase(identifier_bind_part); else if (identifier_lookup.isFunctionLookup()) - scope.alias_name_to_expression_node.erase(identifier_bind_part); + scope.alias_name_to_expression_node->erase(identifier_bind_part); - scope.expressions_in_resolve_process_stack.popNode(); + scope.popExpressionNode(); } else if (node_type == QueryTreeNodeType::FUNCTION) { @@ -4098,8 +4119,8 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook * SELECT id FROM ( SELECT ... ) AS subquery ARRAY JOIN [0] AS id INNER JOIN second_table USING (id) * In the example, identifier `id` should be resolved into one from USING (id) column. */ - auto alias_it = scope.alias_name_to_expression_node.find(identifier_lookup.identifier.getFullName()); - if (alias_it != scope.alias_name_to_expression_node.end() && alias_it->second->getNodeType() == QueryTreeNodeType::COLUMN) + auto alias_it = scope.alias_name_to_expression_node->find(identifier_lookup.identifier.getFullName()); + if (alias_it != scope.alias_name_to_expression_node->end() && alias_it->second->getNodeType() == QueryTreeNodeType::COLUMN) { const auto & column_node = alias_it->second->as(); if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::ARRAY_JOIN) @@ -5203,10 +5224,14 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod for (size_t i = 0; i < lambda_arguments_nodes_size; ++i) { auto & lambda_argument_node = lambda_arguments_nodes[i]; - auto & lambda_argument_node_typed = lambda_argument_node->as(); - const auto & lambda_argument_name = lambda_argument_node_typed.getIdentifier().getFullName(); + const auto * lambda_argument_identifier = lambda_argument_node->as(); + const auto * lambda_argument_column = lambda_argument_node->as(); + if (!lambda_argument_identifier && !lambda_argument_column) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected IDENTIFIER or COLUMN as lambda argument, got {}", lambda_node->dumpTree()); + const auto & lambda_argument_name = lambda_argument_identifier ? lambda_argument_identifier->getIdentifier().getFullName() + : lambda_argument_column->getColumnName(); - bool has_expression_node = scope.alias_name_to_expression_node.contains(lambda_argument_name); + bool has_expression_node = scope.alias_name_to_expression_node->contains(lambda_argument_name); bool has_alias_node = scope.alias_name_to_lambda_node.contains(lambda_argument_name); if (has_expression_node || has_alias_node) @@ -5214,7 +5239,7 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod throw Exception(ErrorCodes::BAD_ARGUMENTS, "Alias name '{}' inside lambda {} cannot have same name as lambda argument. In scope {}", lambda_argument_name, - lambda_argument_node_typed.formatASTForErrorMessage(), + lambda_argument_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } @@ -6233,8 +6258,8 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id * * To resolve b we need to resolve a. */ - auto it = scope.alias_name_to_expression_node.find(node_alias); - if (it != scope.alias_name_to_expression_node.end()) + auto it = scope.alias_name_to_expression_node->find(node_alias); + if (it != scope.alias_name_to_expression_node->end()) node = it->second; if (allow_lambda_expression) @@ -6245,7 +6270,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id } } - scope.expressions_in_resolve_process_stack.pushNode(node); + scope.pushExpressionNode(node); auto node_type = node->getNodeType(); @@ -6274,7 +6299,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id resolved_identifier_node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::FUNCTION}, scope).resolved_identifier; if (resolved_identifier_node && !node_alias.empty()) - scope.alias_name_to_expression_node.erase(node_alias); + scope.alias_name_to_expression_node->erase(node_alias); } if (!resolved_identifier_node && allow_table_expression) @@ -6490,13 +6515,23 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id validateTreeSize(node, scope.context->getSettingsRef().max_expanded_ast_elements, node_to_tree_size); - if (!scope.expressions_in_resolve_process_stack.hasAggregateFunction()) + /// Lambda can be inside the aggregate function, so we should check parent scopes. + /// Most likely only the root scope can have an arrgegate function, but let's check all just in case. + bool in_aggregate_function_scope = false; + for (const auto * scope_ptr = &scope; scope_ptr; scope_ptr = scope_ptr->parent_scope) + in_aggregate_function_scope = in_aggregate_function_scope || scope_ptr->expressions_in_resolve_process_stack.hasAggregateFunction(); + + if (!in_aggregate_function_scope) { - auto it = scope.nullable_group_by_keys.find(node); - if (it != scope.nullable_group_by_keys.end()) + for (const auto * scope_ptr = &scope; scope_ptr; scope_ptr = scope_ptr->parent_scope) { - node = it->node->clone(); - node->convertToNullable(); + auto it = scope_ptr->nullable_group_by_keys.find(node); + if (it != scope_ptr->nullable_group_by_keys.end()) + { + node = it->node->clone(); + node->convertToNullable(); + break; + } } } @@ -6505,8 +6540,8 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id */ if (!node_alias.empty() && use_alias_table && !scope.group_by_use_nulls) { - auto it = scope.alias_name_to_expression_node.find(node_alias); - if (it != scope.alias_name_to_expression_node.end()) + auto it = scope.alias_name_to_expression_node->find(node_alias); + if (it != scope.alias_name_to_expression_node->end()) it->second = node; if (allow_lambda_expression) @@ -6519,7 +6554,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id resolved_expressions.emplace(node, result_projection_names); - scope.expressions_in_resolve_process_stack.popNode(); + scope.popExpressionNode(); bool expression_was_root = scope.expressions_in_resolve_process_stack.empty(); if (expression_was_root) scope.non_cached_identifier_lookups_during_expression_resolve.clear(); @@ -6863,11 +6898,11 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod */ resolve_settings.allow_to_resolve_subquery_during_identifier_resolution = false; - scope.expressions_in_resolve_process_stack.pushNode(current_join_tree_node); + scope.pushExpressionNode(current_join_tree_node); auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, resolve_settings); - scope.expressions_in_resolve_process_stack.popNode(); + scope.popExpressionNode(); bool expression_was_root = scope.expressions_in_resolve_process_stack.empty(); if (expression_was_root) scope.non_cached_identifier_lookups_during_expression_resolve.clear(); @@ -7453,7 +7488,7 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif for (auto & array_join_expression : array_join_nodes) { auto array_join_expression_alias = array_join_expression->getAlias(); - if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node.contains(array_join_expression_alias)) + if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node->contains(array_join_expression_alias)) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "ARRAY JOIN expression {} with duplicate alias {}. In scope {}", array_join_expression->formatASTForErrorMessage(), @@ -7547,8 +7582,8 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif array_join_nodes = std::move(array_join_column_expressions); for (auto & array_join_column_expression : array_join_nodes) { - auto it = scope.alias_name_to_expression_node.find(array_join_column_expression->getAlias()); - if (it != scope.alias_name_to_expression_node.end()) + auto it = scope.alias_name_to_expression_node->find(array_join_column_expression->getAlias()); + if (it != scope.alias_name_to_expression_node->end()) { auto & array_join_column_expression_typed = array_join_column_expression->as(); auto array_join_column = std::make_shared(array_join_column_expression_typed.getColumn(), @@ -8077,8 +8112,10 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Clone is needed cause aliases share subtrees. /// If not clone, the same (shared) subtree could be resolved again with different (Nullable) type /// See 03023_group_by_use_nulls_analyzer_crashes - for (auto & [_, node] : scope.alias_name_to_expression_node) - node = node->clone(); + for (auto & [key, node] : scope.alias_name_to_expression_node_before_group_by) + scope.alias_name_to_expression_node_after_group_by[key] = node->clone(); + + scope.alias_name_to_expression_node = &scope.alias_name_to_expression_node_after_group_by; } if (query_node_typed.hasHaving()) @@ -8158,8 +8195,8 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier bool has_node_in_alias_table = false; - auto it = scope.alias_name_to_expression_node.find(node_alias); - if (it != scope.alias_name_to_expression_node.end()) + auto it = scope.alias_name_to_expression_node->find(node_alias); + if (it != scope.alias_name_to_expression_node->end()) { has_node_in_alias_table = true; @@ -8218,7 +8255,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Remove aliases from expression and lambda nodes - for (auto & [_, node] : scope.alias_name_to_expression_node) + for (auto & [_, node] : *scope.alias_name_to_expression_node) node->removeAlias(); for (auto & [_, node] : scope.alias_name_to_lambda_node) diff --git a/src/Common/PageCache.cpp b/src/Common/PageCache.cpp index d4598d4683b..56bd8c1a339 100644 --- a/src/Common/PageCache.cpp +++ b/src/Common/PageCache.cpp @@ -198,12 +198,18 @@ size_t PageCache::getPinnedSize() const PageCache::MemoryStats PageCache::getResidentSetSize() const { MemoryStats stats; + #ifdef OS_LINUX if (use_madv_free) { std::unordered_set cache_mmap_addrs; { std::lock_guard lock(global_mutex); + + /// Don't spend time on reading smaps if page cache is not used. + if (mmaps.empty()) + return stats; + for (const auto & m : mmaps) cache_mmap_addrs.insert(reinterpret_cast(m.ptr)); } @@ -258,7 +264,7 @@ PageCache::MemoryStats PageCache::getResidentSetSize() const UInt64 addr = unhexUInt(s.c_str()); current_range_is_cache = cache_mmap_addrs.contains(addr); } - else if (s == "Rss:" || s == "LazyFree") + else if (s == "Rss:" || s == "LazyFree:") { skip_whitespace(); size_t val; diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index be1f222dc96..b8f5c000e75 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -1,9 +1,13 @@ -#include #include +#include +#include #include #include +#if USE_MULTITARGET_CODE +#include +#endif namespace DB { @@ -215,5 +219,71 @@ size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, s return computeWidthImpl(data, size, prefix, limit); } + +DECLARE_DEFAULT_CODE( +bool isAllASCII(const UInt8 * data, size_t size) +{ + UInt8 mask = 0; + for (size_t i = 0; i < size; ++i) + mask |= data[i]; + + return !(mask & 0x80); +}) + +DECLARE_SSE42_SPECIFIC_CODE( +/// Copy from https://github.com/lemire/fastvalidate-utf-8/blob/master/include/simdasciicheck.h +bool isAllASCII(const UInt8 * data, size_t size) +{ + __m128i masks = _mm_setzero_si128(); + + size_t i = 0; + for (; i + 16 <= size; i += 16) + { + __m128i bytes = _mm_loadu_si128(reinterpret_cast(data + i)); + masks = _mm_or_si128(masks, bytes); + } + int mask = _mm_movemask_epi8(masks); + + UInt8 tail_mask = 0; + for (; i < size; i++) + tail_mask |= data[i]; + + mask |= (tail_mask & 0x80); + return !mask; +}) + +DECLARE_AVX2_SPECIFIC_CODE( +bool isAllASCII(const UInt8 * data, size_t size) +{ + __m256i masks = _mm256_setzero_si256(); + + size_t i = 0; + for (; i + 32 <= size; i += 32) + { + __m256i bytes = _mm256_loadu_si256(reinterpret_cast(data + i)); + masks = _mm256_or_si256(masks, bytes); + } + int mask = _mm256_movemask_epi8(masks); + + UInt8 tail_mask = 0; + for (; i < size; i++) + tail_mask |= data[i]; + + mask |= (tail_mask & 0x80); + return !mask; +}) + +bool isAllASCII(const UInt8* data, size_t size) +{ +#if USE_MULTITARGET_CODE + if (isArchSupported(TargetArch::AVX2)) + return TargetSpecific::AVX2::isAllASCII(data, size); + if (isArchSupported(TargetArch::SSE42)) + return TargetSpecific::SSE42::isAllASCII(data, size); +#endif + return TargetSpecific::Default::isAllASCII(data, size); +} + + } } diff --git a/src/Common/UTF8Helpers.h b/src/Common/UTF8Helpers.h index a4dd88921b7..933b62c7b63 100644 --- a/src/Common/UTF8Helpers.h +++ b/src/Common/UTF8Helpers.h @@ -136,7 +136,10 @@ size_t computeWidth(const UInt8 * data, size_t size, size_t prefix = 0) noexcept */ size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept; -} +/// If all the characters in the string are ASCII, return true. +bool isAllASCII(const UInt8* data, size_t size); + +} } diff --git a/src/Functions/GatherUtils/Sinks.h b/src/Functions/GatherUtils/Sinks.h index a8054da1159..2aa7c147136 100644 --- a/src/Functions/GatherUtils/Sinks.h +++ b/src/Functions/GatherUtils/Sinks.h @@ -48,7 +48,7 @@ struct NumericArraySink : public ArraySinkImpl> NumericArraySink(IColumn & elements_, ColumnArray::Offsets & offsets_, size_t column_size) : elements(assert_cast(elements_).getData()), offsets(offsets_) { - offsets.resize(column_size); + offsets.resize_exact(column_size); } void next() @@ -69,7 +69,7 @@ struct NumericArraySink : public ArraySinkImpl> void reserve(size_t num_elements) { - elements.reserve(num_elements); + elements.reserve_exact(num_elements); } }; @@ -85,7 +85,7 @@ struct StringSink StringSink(ColumnString & col, size_t column_size) : elements(col.getChars()), offsets(col.getOffsets()) { - offsets.resize(column_size); + offsets.resize_exact(column_size); } void ALWAYS_INLINE next() @@ -108,7 +108,7 @@ struct StringSink void reserve(size_t num_elements) { - elements.reserve(num_elements); + elements.reserve_exact(num_elements); } }; @@ -125,7 +125,7 @@ struct FixedStringSink FixedStringSink(ColumnFixedString & col, size_t column_size) : elements(col.getChars()), string_size(col.getN()), total_rows(column_size) { - elements.resize(column_size * string_size); + elements.resize_exact(column_size * string_size); } void next() @@ -146,7 +146,7 @@ struct FixedStringSink void reserve(size_t num_elements) { - elements.reserve(num_elements); + elements.reserve_exact(num_elements); } }; @@ -165,7 +165,7 @@ struct GenericArraySink : public ArraySinkImpl GenericArraySink(IColumn & elements_, ColumnArray::Offsets & offsets_, size_t column_size) : elements(elements_), offsets(offsets_) { - offsets.resize(column_size); + offsets.resize_exact(column_size); } void next() @@ -210,7 +210,7 @@ struct NullableArraySink : public ArraySink void reserve(size_t num_elements) { ArraySink::reserve(num_elements); - null_map.reserve(num_elements); + null_map.reserve_exact(num_elements); } }; diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index 4e3009a695d..e5e3451fe4c 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -323,6 +323,8 @@ struct StringSource return {&elements[prev_offset], length + elem_size > offset ? std::min(elem_size, length + elem_size - offset) : 0}; return {&elements[prev_offset + elem_size - offset], std::min(length, offset)}; } + + const ColumnString::Chars & getElements() const { return elements; } }; /// Treats Enum values as Strings, modeled after StringSource @@ -517,11 +519,12 @@ struct FixedStringSource const UInt8 * pos; const UInt8 * end; size_t string_size; + const typename ColumnString::Chars & elements; + size_t row_num = 0; size_t column_size = 0; - explicit FixedStringSource(const ColumnFixedString & col) - : string_size(col.getN()) + explicit FixedStringSource(const ColumnFixedString & col) : string_size(col.getN()), elements(col.getChars()) { const auto & chars = col.getChars(); pos = chars.data(); @@ -592,6 +595,8 @@ struct FixedStringSource return {pos, length + string_size > offset ? std::min(string_size, length + string_size - offset) : 0}; return {pos + string_size - offset, std::min(length, offset)}; } + + const ColumnString::Chars & getElements() const { return elements; } }; diff --git a/src/Functions/LowerUpperImpl.h b/src/Functions/LowerUpperImpl.h index f093e00f7ab..72b3ce1ca34 100644 --- a/src/Functions/LowerUpperImpl.h +++ b/src/Functions/LowerUpperImpl.h @@ -13,14 +13,14 @@ struct LowerUpperImpl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - res_data.resize(data.size()); + res_data.resize_exact(data.size()); res_offsets.assign(offsets); array(data.data(), data.data() + data.size(), res_data.data()); } static void vectorFixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data) { - res_data.resize(data.size()); + res_data.resize_exact(data.size()); array(data.data(), data.data() + data.size(), res_data.data()); } diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 7ca98166576..bb794a0f8ed 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -1,8 +1,9 @@ #pragma once #include +#include +#include #include #include -#include #ifdef __SSE2__ #include @@ -92,7 +93,15 @@ struct LowerUpperUTF8Impl { if (data.empty()) return; - res_data.resize(data.size()); + + bool all_ascii = UTF8::isAllASCII(data.data(), data.size()); + if (all_ascii) + { + LowerUpperImpl::vector(data, offsets, res_data, res_offsets); + return; + } + + res_data.resize_exact(data.size()); res_offsets.assign(offsets); array(data.data(), data.data() + data.size(), offsets, res_data.data()); } diff --git a/src/Functions/clamp.cpp b/src/Functions/clamp.cpp new file mode 100644 index 00000000000..bb347a575e4 --- /dev/null +++ b/src/Functions/clamp.cpp @@ -0,0 +1,69 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + + +class FunctionClamp : public IFunction +{ + +public: + static constexpr auto name = "clamp"; + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 3; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + DataTypePtr getReturnTypeImpl(const DataTypes & types) const override + { + if (types.size() != 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires 3 arguments", getName()); + + return getLeastSupertype(types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + size_t arg_size = arguments.size(); + Columns converted_columns(arg_size); + for (size_t arg = 0; arg < arg_size; ++arg) + converted_columns[arg] = castColumn(arguments[arg], result_type)->convertToFullColumnIfConst(); + + auto result_column = result_type->createColumn(); + for (size_t row_num = 0; row_num < input_rows_count; ++row_num) + { + if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[2], 1) > 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The minimum value cannot be greater than the maximum value for function {}", getName()); + + size_t best_arg = 0; + if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) > 0) + best_arg = 1; + else if (converted_columns[2]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) < 0) + best_arg = 2; + + result_column->insertFrom(*converted_columns[best_arg], row_num); + } + + return result_column; + } + +}; + +REGISTER_FUNCTION(Clamp) +{ + factory.registerFunction(); +} +} diff --git a/src/Functions/padString.cpp b/src/Functions/padString.cpp index ccef87d83e7..0922e0ddb8a 100644 --- a/src/Functions/padString.cpp +++ b/src/Functions/padString.cpp @@ -210,19 +210,18 @@ namespace pad_string = column_pad_const->getValue(); } - PaddingChars padding_chars{pad_string}; auto col_res = ColumnString::create(); StringSink res_sink{*col_res, input_rows_count}; if (const ColumnString * col = checkAndGetColumn(column_string.get())) - executeForSource(StringSource{*col}, column_length, padding_chars, res_sink); + executeForSource(StringSource{*col}, column_length, pad_string, res_sink); else if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_string.get())) - executeForSource(FixedStringSource{*col_fixed}, column_length, padding_chars, res_sink); + executeForSource(FixedStringSource{*col_fixed}, column_length, pad_string, res_sink); else if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) - executeForSource(ConstSource{*col_const}, column_length, padding_chars, res_sink); + executeForSource(ConstSource{*col_const}, column_length, pad_string, res_sink); else if (const ColumnConst * col_const_fixed = checkAndGetColumnConst(column_string.get())) - executeForSource(ConstSource{*col_const_fixed}, column_length, padding_chars, res_sink); + executeForSource(ConstSource{*col_const_fixed}, column_length, pad_string, res_sink); else throw Exception( ErrorCodes::ILLEGAL_COLUMN, @@ -235,23 +234,40 @@ namespace private: template - void executeForSource( - SourceStrings && strings, - const ColumnPtr & column_length, - const PaddingChars & padding_chars, - StringSink & res_sink) const + void executeForSource(SourceStrings && strings, const ColumnPtr & column_length, const String & pad_string, StringSink & res_sink) const { - if (const auto * col_const = checkAndGetColumn(column_length.get())) - executeForSourceAndLength(std::forward(strings), ConstSource{*col_const}, padding_chars, res_sink); + const auto & chars = strings.getElements(); + bool all_ascii = UTF8::isAllASCII(reinterpret_cast(pad_string.data()), pad_string.size()) + && UTF8::isAllASCII(chars.data(), chars.size()); + bool is_actually_utf8 = is_utf8 && !all_ascii; + + if (!is_actually_utf8) + { + PaddingChars padding_chars{pad_string}; + if (const auto * col_const = checkAndGetColumn(column_length.get())) + executeForSourceAndLength( + std::forward(strings), ConstSource{*col_const}, padding_chars, res_sink); + else + executeForSourceAndLength( + std::forward(strings), GenericValueSource{*column_length}, padding_chars, res_sink); + } else - executeForSourceAndLength(std::forward(strings), GenericValueSource{*column_length}, padding_chars, res_sink); + { + PaddingChars padding_chars{pad_string}; + if (const auto * col_const = checkAndGetColumn(column_length.get())) + executeForSourceAndLength( + std::forward(strings), ConstSource{*col_const}, padding_chars, res_sink); + else + executeForSourceAndLength( + std::forward(strings), GenericValueSource{*column_length}, padding_chars, res_sink); + } } - template + template void executeForSourceAndLength( SourceStrings && strings, SourceLengths && lengths, - const PaddingChars & padding_chars, + const PaddingChars & padding_chars, StringSink & res_sink) const { bool is_const_new_length = lengths.isConst(); @@ -263,7 +279,7 @@ namespace for (; !res_sink.isEnd(); res_sink.next(), strings.next(), lengths.next()) { auto str = strings.getWhole(); - ssize_t current_length = getLengthOfSlice(str); + ssize_t current_length = getLengthOfSlice(str); if (!res_sink.rowNum() || !is_const_new_length) { @@ -293,7 +309,7 @@ namespace } else if (new_length < current_length) { - str = removeSuffixFromSlice(str, current_length - new_length); + str = removeSuffixFromSlice(str, current_length - new_length); writeSlice(str, res_sink); } else if (new_length > current_length) diff --git a/src/Functions/reverse.cpp b/src/Functions/reverse.cpp index 32b998523c7..39608b77997 100644 --- a/src/Functions/reverse.cpp +++ b/src/Functions/reverse.cpp @@ -1,10 +1,10 @@ #include -#include #include #include #include #include #include +#include "reverse.h" namespace DB @@ -17,42 +17,6 @@ namespace ErrorCodes namespace { - -/** Reverse the string as a sequence of bytes. - */ -struct ReverseImpl -{ - static void vector(const ColumnString::Chars & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) - { - res_data.resize(data.size()); - res_offsets.assign(offsets); - size_t size = offsets.size(); - - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - for (size_t j = prev_offset; j < offsets[i] - 1; ++j) - res_data[j] = data[offsets[i] + prev_offset - 2 - j]; - res_data[offsets[i] - 1] = 0; - prev_offset = offsets[i]; - } - } - - static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data) - { - res_data.resize(data.size()); - size_t size = data.size() / n; - - for (size_t i = 0; i < size; ++i) - for (size_t j = i * n; j < (i + 1) * n; ++j) - res_data[j] = data[(i * 2 + 1) * n - j - 1]; - } -}; - - class FunctionReverse : public IFunction { public: diff --git a/src/Functions/reverse.h b/src/Functions/reverse.h new file mode 100644 index 00000000000..5f999af4297 --- /dev/null +++ b/src/Functions/reverse.h @@ -0,0 +1,42 @@ +#pragma once + +#include + +namespace DB +{ + +/** Reverse the string as a sequence of bytes. + */ +struct ReverseImpl +{ + static void vector(const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize_exact(data.size()); + res_offsets.assign(offsets); + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + for (size_t j = prev_offset; j < offsets[i] - 1; ++j) + res_data[j] = data[offsets[i] + prev_offset - 2 - j]; + res_data[offsets[i] - 1] = 0; + prev_offset = offsets[i]; + } + } + + static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data) + { + res_data.resize_exact(data.size()); + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + for (size_t j = i * n; j < (i + 1) * n; ++j) + res_data[j] = data[(i * 2 + 1) * n - j - 1]; + } +}; + +} diff --git a/src/Functions/reverseUTF8.cpp b/src/Functions/reverseUTF8.cpp index 8a76af05d86..4ea861919a1 100644 --- a/src/Functions/reverseUTF8.cpp +++ b/src/Functions/reverseUTF8.cpp @@ -1,7 +1,9 @@ -#include #include +#include #include #include +#include +#include "reverse.h" namespace DB @@ -25,10 +27,18 @@ struct ReverseUTF8Impl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { + bool all_ascii = UTF8::isAllASCII(data.data(), data.size()); + if (all_ascii) + { + ReverseImpl::vector(data, offsets, res_data, res_offsets); + return; + } + res_data.resize(data.size()); res_offsets.assign(offsets); size_t size = offsets.size(); + ColumnString::Offset prev_offset = 0; for (size_t i = 0; i < size; ++i) { diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index e809914f5f0..122f83d758b 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -148,9 +148,23 @@ public: if constexpr (is_utf8) { if (const ColumnString * col = checkAndGetColumn(column_string.get())) - return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, UTF8StringSource(*col), input_rows_count); + { + bool all_ascii = UTF8::isAllASCII(col->getChars().data(), col->getChars().size()); + if (all_ascii) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, StringSource(*col), input_rows_count); + else + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, UTF8StringSource(*col), input_rows_count); + } + if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) - return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + { + StringRef str_ref = col_const->getDataAt(0); + bool all_ascii = UTF8::isAllASCII(reinterpret_cast(str_ref.data), str_ref.size); + if (all_ascii) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + else + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + } throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); } else diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 5f3f054b624..74474cb4b23 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -129,8 +129,10 @@ namespace res_data.reserve(str_column->getChars().size() / 2); res_offsets.reserve(rows); + bool all_ascii = UTF8::isAllASCII(str_column->getChars().data(), str_column->getChars().size()) + && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); + = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); for (size_t i = 0; i < rows; ++i) { @@ -140,10 +142,12 @@ namespace StringRef res_ref; if constexpr (!is_utf8) res_ref = substringIndex(str_ref, delim[0], count); + else if (all_ascii) + res_ref = substringIndex(str_ref, delim[0], count); else res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); - appendToResultColumn(res_ref, res_data, res_offsets); + appendToResultColumn(res_ref, res_data, res_offsets); } } @@ -158,8 +162,10 @@ namespace res_data.reserve(str_column->getChars().size() / 2); res_offsets.reserve(rows); + bool all_ascii = UTF8::isAllASCII(str_column->getChars().data(), str_column->getChars().size()) + && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); + = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); for (size_t i = 0; i < rows; ++i) { @@ -168,10 +174,12 @@ namespace StringRef res_ref; if constexpr (!is_utf8) res_ref = substringIndex(str_ref, delim[0], count); + else if (all_ascii) + res_ref = substringIndex(str_ref, delim[0], count); else res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); - appendToResultColumn(res_ref, res_data, res_offsets); + appendToResultColumn(res_ref, res_data, res_offsets); } } @@ -186,8 +194,10 @@ namespace res_data.reserve(str.size() * rows / 2); res_offsets.reserve(rows); + bool all_ascii = UTF8::isAllASCII(reinterpret_cast(str.data()), str.size()) + && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); + = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); StringRef str_ref{str.data(), str.size()}; for (size_t i = 0; i < rows; ++i) @@ -197,18 +207,26 @@ namespace StringRef res_ref; if constexpr (!is_utf8) res_ref = substringIndex(str_ref, delim[0], count); + else if (all_ascii) + res_ref = substringIndex(str_ref, delim[0], count); else res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); - appendToResultColumn(res_ref, res_data, res_offsets); + appendToResultColumn(res_ref, res_data, res_offsets); } } + template static void appendToResultColumn(const StringRef & res_ref, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { size_t res_offset = res_data.size(); res_data.resize(res_offset + res_ref.size + 1); - memcpy(&res_data[res_offset], res_ref.data, res_ref.size); + + if constexpr (padded) + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], res_ref.data, res_ref.size); + else + memcpy(&res_data[res_offset], res_ref.data, res_ref.size); + res_offset += res_ref.size; res_data[res_offset] = 0; ++res_offset; diff --git a/src/Functions/trim.cpp b/src/Functions/trim.cpp index dd51c606ff7..1f0011b8e99 100644 --- a/src/Functions/trim.cpp +++ b/src/Functions/trim.cpp @@ -46,8 +46,8 @@ public: ColumnString::Offsets & res_offsets) { size_t size = offsets.size(); - res_offsets.resize(size); - res_data.reserve(data.size()); + res_offsets.resize_exact(size); + res_data.reserve_exact(data.size()); size_t prev_offset = 0; size_t res_offset = 0; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 813546aa052..8823af55936 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -191,10 +191,14 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons result = sendRequest(attempt, range_begin, range_begin + n - 1); std::istream & istr = result->GetBody(); - copyFromIStreamWithProgressCallback(istr, to, n, progress_callback, &bytes_copied); + bool cancelled = false; + copyFromIStreamWithProgressCallback(istr, to, n, progress_callback, &bytes_copied, &cancelled); ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, bytes_copied); + if (cancelled) + return initial_n - n + bytes_copied; + if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); diff --git a/tests/performance/ascii.xml b/tests/performance/ascii.xml new file mode 100644 index 00000000000..83440437d2c --- /dev/null +++ b/tests/performance/ascii.xml @@ -0,0 +1,22 @@ + + select substringUTF8(materialize('hello world'), 2, 5) from numbers(10000000) + select substringUTF8(materialize('hello 世界'), 2, 5) from numbers(10000000) + + select substringIndexUTF8(materialize('www.clickhouse.com'), '.', 2) from numbers(10000000) + select substringIndexUTF8(materialize('官网www.clickhouse.com'), '.', 2) from numbers(10000000) + + select reverseUTF8(materialize('hello world')) from numbers(10000000) + select reverseUTF8(materialize('hello 世界')) from numbers(10000000) + + select lowerUTF8(materialize('hello world')) from numbers(10000000) + select lowerUTF8(materialize('hello 世界')) from numbers(10000000) + + select upperUTF8(materialize('hello world')) from numbers(10000000) + select upperUTF8(materialize('hello 世界')) from numbers(10000000) + + select leftPadUTF8(materialize('hello '), 10, ',') from numbers(10000000) + select leftPadUTF8(materialize('hello '), 10, '世界') from numbers(10000000) + + select rightPadUTF8(materialize('hello '), 10, ',') from numbers(10000000) + select rightPadUTF8(materialize('hello '), 10, '世界') from numbers(10000000) + diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 3ddf165dec0..a152066a460 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -201,6 +201,7 @@ cbrt ceil char cityHash64 +clamp coalesce concat concatAssumeInjective diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference index 17a17484a0c..02ea01eb2e6 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference @@ -66,3 +66,61 @@ a a a a a a +0 0 +0 \N +1 2 +1 \N +2 4 +2 \N +\N 0 +\N 2 +\N 4 +\N \N +0 0 nan +2 4 nan +1 2 nan +2 \N nan +0 \N nan +1 \N nan +\N 2 nan +\N 0 nan +\N 4 nan +\N \N nan +[] +['.'] +['.','.'] +['.','.','.'] +['.','.','.','.'] +['.','.','.','.','.'] +['.','.','.','.','.','.'] +['.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.','.'] +[] +[] +[] +[] +[] +[] +[] +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +10 diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql index 68710137542..b8c173520a9 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql @@ -21,3 +21,39 @@ SELECT tuple(number + 1) AS x FROM numbers(10) GROUP BY number + 1, toString(x) SELECT tuple(tuple(number)) AS x FROM numbers(10) WHERE toString(toUUID(tuple(number), NULL), x) GROUP BY number, (toString(x), number) WITH CUBE SETTINGS group_by_use_nulls = 1 FORMAT Null; SELECT materialize('a'), 'a' AS key GROUP BY key WITH CUBE WITH TOTALS SETTINGS group_by_use_nulls = 1; + +EXPLAIN QUERY TREE +SELECT a, b +FROM numbers(3) +GROUP BY number as a, (number + number) as b WITH CUBE +ORDER BY a, b format Null; + +SELECT a, b +FROM numbers(3) +GROUP BY number as a, (number + number) as b WITH CUBE +ORDER BY a, b; + +SELECT + a, + b, + cramersVBiasCorrected(a, b) +FROM numbers(3) +GROUP BY + number AS a, + number + number AS b + WITH CUBE +SETTINGS group_by_use_nulls = 1; + +SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_use_nulls=1; + +SELECT count('Lambda as function parameter') AS c FROM (SELECT ignore(ignore('Lambda as function parameter', 28, 28, 28, 28, 28, 28), 28), materialize('Lambda as function parameter'), 28, 28, 'world', 5 FROM system.numbers WHERE ignore(materialize('Lambda as function parameter'), materialize(toLowCardinality(28)), 28, 28, 28, 28, toUInt128(28)) LIMIT 2) GROUP BY GROUPING SETS ((toLowCardinality(0)), (toLowCardinality(toNullable(28))), (1)) HAVING nullIf(c, 10) < 50 ORDER BY c ASC NULLS FIRST settings group_by_use_nulls=1; -- { serverError ILLEGAL_AGGREGATION } + +SELECT arraySplit(x -> 0, []) WHERE materialize(1) GROUP BY (0, ignore('a')) WITH ROLLUP SETTINGS group_by_use_nulls = 1; + +SELECT arraySplit(x -> toUInt8(number), []) from numbers(1) GROUP BY toUInt8(number) WITH ROLLUP SETTINGS group_by_use_nulls = 1; + +SELECT arraySplit(number -> toUInt8(number), []) from numbers(1) GROUP BY toUInt8(number) WITH ROLLUP SETTINGS group_by_use_nulls = 1; + +SELECT count(arraySplit(number -> toUInt8(number), [arraySplit(x -> toUInt8(number), [])])) FROM numbers(10) GROUP BY number, [number] WITH ROLLUP settings group_by_use_nulls=1; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} + +SELECT count(arraySplit(x -> toUInt8(number), [])) FROM numbers(10) GROUP BY number, [number] WITH ROLLUP settings group_by_use_nulls=1; \ No newline at end of file diff --git a/tests/queries/0_stateless/03036_clamp.reference b/tests/queries/0_stateless/03036_clamp.reference new file mode 100644 index 00000000000..b866caf2261 --- /dev/null +++ b/tests/queries/0_stateless/03036_clamp.reference @@ -0,0 +1,14 @@ +10 +20 +15 +b +0 +['hello'] +-1 +234 +\N +\N +5 +0 +1 +2 diff --git a/tests/queries/0_stateless/03036_clamp.sql b/tests/queries/0_stateless/03036_clamp.sql new file mode 100644 index 00000000000..9973265c13b --- /dev/null +++ b/tests/queries/0_stateless/03036_clamp.sql @@ -0,0 +1,15 @@ +SELECT clamp(1, 10, 20); +SELECT clamp(30, 10, 20); +SELECT clamp(15, 10, 20); +SELECT clamp('a', 'b', 'c'); +SELECT clamp(today(), yesterday() - 10, yesterday() + 10) - today(); +SELECT clamp([], ['hello'], ['world']); +SELECT clamp(-1., -1000., 18446744073709551615.); +SELECT clamp(toNullable(123), 234, 456); +select clamp(1, null, 5); +select clamp(1, 6, null); +select clamp(1, 5, nan); +select clamp(toInt64(number), toInt64(number-1), toInt64(number+1)) from numbers(3); +select clamp(number, number-1, number+1) from numbers(3); -- { serverError NO_COMMON_TYPE } +select clamp(1, 3, 2); -- { serverError BAD_ARGUMENTS } +select clamp(1, data[1], data[2])from (select arrayJoin([[1, 2], [2,3], [3,2], [4, 4]]) as data); -- { serverError BAD_ARGUMENTS }