Merge branch 'ClickHouse:master' into external_cross_join

This commit is contained in:
p1rattttt 2024-05-14 11:55:13 +03:00 committed by GitHub
commit 937107fb6b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
25 changed files with 575 additions and 128 deletions

View File

@ -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 │
└────────┴─────────┘
```

View File

@ -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;

View File

@ -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<std::string, QueryTreeNodePtr> expression_argument_name_to_node;
/// Alias name to query expression node
std::unordered_map<std::string, QueryTreeNodePtr> alias_name_to_expression_node;
std::unordered_map<std::string, QueryTreeNodePtr> alias_name_to_expression_node_before_group_by;
std::unordered_map<std::string, QueryTreeNodePtr> alias_name_to_expression_node_after_group_by;
std::unordered_map<std::string, QueryTreeNodePtr> * alias_name_to_expression_node = nullptr;
/// Alias name to lambda node
std::unordered_map<std::string, QueryTreeNodePtr> 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<std::string, QueryTreeNodePtr> &
{
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<std::string, QueryTreeNodePtr> &
{
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<IdentifierNode &>();
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<ColumnNode &>();
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<IdentifierNode &>();
const auto & lambda_argument_name = lambda_argument_node_typed.getIdentifier().getFullName();
const auto * lambda_argument_identifier = lambda_argument_node->as<IdentifierNode>();
const auto * lambda_argument_column = lambda_argument_node->as<ColumnNode>();
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<ColumnNode &>();
auto array_join_column = std::make_shared<ColumnNode>(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)

View File

@ -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<UInt64> 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<UInt64>(m.ptr));
}
@ -258,7 +264,7 @@ PageCache::MemoryStats PageCache::getResidentSetSize() const
UInt64 addr = unhexUInt<UInt64>(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;

View File

@ -1,9 +1,13 @@
#include <Common/UTF8Helpers.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/TargetSpecific.h>
#include <Common/UTF8Helpers.h>
#include <widechar_width.h>
#include <bit>
#if USE_MULTITARGET_CODE
#include <immintrin.h>
#endif
namespace DB
{
@ -215,5 +219,71 @@ size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, s
return computeWidthImpl<BytesBeforeLimit>(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<const __m128i *>(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<const __m256i *>(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);
}
}
}

View File

@ -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);
}
}

View File

@ -48,7 +48,7 @@ struct NumericArraySink : public ArraySinkImpl<NumericArraySink<T>>
NumericArraySink(IColumn & elements_, ColumnArray::Offsets & offsets_, size_t column_size)
: elements(assert_cast<ColVecType&>(elements_).getData()), offsets(offsets_)
{
offsets.resize(column_size);
offsets.resize_exact(column_size);
}
void next()
@ -69,7 +69,7 @@ struct NumericArraySink : public ArraySinkImpl<NumericArraySink<T>>
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>
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);
}
};

View File

@ -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; }
};

View File

@ -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());
}

View File

@ -1,8 +1,9 @@
#pragma once
#include <Columns/ColumnString.h>
#include <Functions/LowerUpperImpl.h>
#include <base/defines.h>
#include <Poco/UTF8Encoding.h>
#include <Common/UTF8Helpers.h>
#include <base/defines.h>
#ifdef __SSE2__
#include <emmintrin.h>
@ -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<not_case_lower_bound, not_case_upper_bound>::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());
}

69
src/Functions/clamp.cpp Normal file
View File

@ -0,0 +1,69 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/getLeastSupertype.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/Context.h>
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<FunctionClamp>(); }
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<FunctionClamp>();
}
}

View File

@ -210,19 +210,18 @@ namespace
pad_string = column_pad_const->getValue<String>();
}
PaddingChars<is_utf8> padding_chars{pad_string};
auto col_res = ColumnString::create();
StringSink res_sink{*col_res, input_rows_count};
if (const ColumnString * col = checkAndGetColumn<ColumnString>(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<ColumnFixedString>(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<ColumnString>(column_string.get()))
executeForSource(ConstSource<StringSource>{*col_const}, column_length, padding_chars, res_sink);
executeForSource(ConstSource<StringSource>{*col_const}, column_length, pad_string, res_sink);
else if (const ColumnConst * col_const_fixed = checkAndGetColumnConst<ColumnFixedString>(column_string.get()))
executeForSource(ConstSource<FixedStringSource>{*col_const_fixed}, column_length, padding_chars, res_sink);
executeForSource(ConstSource<FixedStringSource>{*col_const_fixed}, column_length, pad_string, res_sink);
else
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
@ -235,23 +234,40 @@ namespace
private:
template <typename SourceStrings>
void executeForSource(
SourceStrings && strings,
const ColumnPtr & column_length,
const PaddingChars<is_utf8> & 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<ColumnConst>(column_length.get()))
executeForSourceAndLength(std::forward<SourceStrings>(strings), ConstSource<GenericValueSource>{*col_const}, padding_chars, res_sink);
const auto & chars = strings.getElements();
bool all_ascii = UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(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<false> padding_chars{pad_string};
if (const auto * col_const = checkAndGetColumn<ColumnConst>(column_length.get()))
executeForSourceAndLength<false>(
std::forward<SourceStrings>(strings), ConstSource<GenericValueSource>{*col_const}, padding_chars, res_sink);
else
executeForSourceAndLength<false>(
std::forward<SourceStrings>(strings), GenericValueSource{*column_length}, padding_chars, res_sink);
}
else
executeForSourceAndLength(std::forward<SourceStrings>(strings), GenericValueSource{*column_length}, padding_chars, res_sink);
{
PaddingChars<true> padding_chars{pad_string};
if (const auto * col_const = checkAndGetColumn<ColumnConst>(column_length.get()))
executeForSourceAndLength<true>(
std::forward<SourceStrings>(strings), ConstSource<GenericValueSource>{*col_const}, padding_chars, res_sink);
else
executeForSourceAndLength<true>(
std::forward<SourceStrings>(strings), GenericValueSource{*column_length}, padding_chars, res_sink);
}
}
template <typename SourceStrings, typename SourceLengths>
template <bool is_actually_utf8, typename SourceStrings, typename SourceLengths>
void executeForSourceAndLength(
SourceStrings && strings,
SourceLengths && lengths,
const PaddingChars<is_utf8> & padding_chars,
const PaddingChars<is_actually_utf8> & 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<is_utf8>(str);
ssize_t current_length = getLengthOfSlice<is_actually_utf8>(str);
if (!res_sink.rowNum() || !is_const_new_length)
{
@ -293,7 +309,7 @@ namespace
}
else if (new_length < current_length)
{
str = removeSuffixFromSlice<is_utf8>(str, current_length - new_length);
str = removeSuffixFromSlice<is_actually_utf8>(str, current_length - new_length);
writeSlice(str, res_sink);
}
else if (new_length > current_length)

View File

@ -1,10 +1,10 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnArray.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <base/map.h>
#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:

42
src/Functions/reverse.h Normal file
View File

@ -0,0 +1,42 @@
#pragma once
#include <Columns/ColumnString.h>
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];
}
};
}

View File

@ -1,7 +1,9 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringToString.h>
#include <Common/UTF8Helpers.h>
#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)
{

View File

@ -148,9 +148,23 @@ public:
if constexpr (is_utf8)
{
if (const ColumnString * col = checkAndGetColumn<ColumnString>(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<ColumnString>(column_string.get()))
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<UTF8StringSource>(*col_const), input_rows_count);
{
StringRef str_ref = col_const->getDataAt(0);
bool all_ascii = UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(str_ref.data), str_ref.size);
if (all_ascii)
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<StringSource>(*col_const), input_rows_count);
else
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<UTF8StringSource>(*col_const), input_rows_count);
}
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName());
}
else

View File

@ -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<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !is_utf8 ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
= !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(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<true>(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<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !is_utf8 ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
= !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(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<true>(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<const UInt8 *>(str.data()), str.size())
&& UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !is_utf8 ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
= !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(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<false>(res_ref, res_data, res_offsets);
}
}
template <bool padded>
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;

View File

@ -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;

View File

@ -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);

View File

@ -0,0 +1,22 @@
<test>
<query>select substringUTF8(materialize('hello world'), 2, 5) from numbers(10000000)</query>
<query>select substringUTF8(materialize('hello 世界'), 2, 5) from numbers(10000000)</query>
<query>select substringIndexUTF8(materialize('www.clickhouse.com'), '.', 2) from numbers(10000000)</query>
<query>select substringIndexUTF8(materialize('官网www.clickhouse.com'), '.', 2) from numbers(10000000)</query>
<query>select reverseUTF8(materialize('hello world')) from numbers(10000000)</query>
<query>select reverseUTF8(materialize('hello 世界')) from numbers(10000000)</query>
<query>select lowerUTF8(materialize('hello world')) from numbers(10000000)</query>
<query>select lowerUTF8(materialize('hello 世界')) from numbers(10000000)</query>
<query>select upperUTF8(materialize('hello world')) from numbers(10000000)</query>
<query>select upperUTF8(materialize('hello 世界')) from numbers(10000000)</query>
<query>select leftPadUTF8(materialize('hello '), 10, ',') from numbers(10000000)</query>
<query>select leftPadUTF8(materialize('hello '), 10, '世界') from numbers(10000000)</query>
<query>select rightPadUTF8(materialize('hello '), 10, ',') from numbers(10000000)</query>
<query>select rightPadUTF8(materialize('hello '), 10, '世界') from numbers(10000000)</query>
</test>

View File

@ -201,6 +201,7 @@ cbrt
ceil
char
cityHash64
clamp
coalesce
concat
concatAssumeInjective

View File

@ -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

View File

@ -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;

View File

@ -0,0 +1,14 @@
10
20
15
b
0
['hello']
-1
234
\N
\N
5
0
1
2

View File

@ -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 }