mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge branch 'ClickHouse:master' into external_cross_join
This commit is contained in:
commit
937107fb6b
@ -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 │
|
||||
└────────┴─────────┘
|
||||
```
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
|
@ -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
69
src/Functions/clamp.cpp
Normal 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>();
|
||||
}
|
||||
}
|
@ -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)
|
||||
|
@ -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
42
src/Functions/reverse.h
Normal 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];
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -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)
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
|
22
tests/performance/ascii.xml
Normal file
22
tests/performance/ascii.xml
Normal 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>
|
@ -201,6 +201,7 @@ cbrt
|
||||
ceil
|
||||
char
|
||||
cityHash64
|
||||
clamp
|
||||
coalesce
|
||||
concat
|
||||
concatAssumeInjective
|
||||
|
@ -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
|
||||
|
@ -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;
|
14
tests/queries/0_stateless/03036_clamp.reference
Normal file
14
tests/queries/0_stateless/03036_clamp.reference
Normal file
@ -0,0 +1,14 @@
|
||||
10
|
||||
20
|
||||
15
|
||||
b
|
||||
0
|
||||
['hello']
|
||||
-1
|
||||
234
|
||||
\N
|
||||
\N
|
||||
5
|
||||
0
|
||||
1
|
||||
2
|
15
tests/queries/0_stateless/03036_clamp.sql
Normal file
15
tests/queries/0_stateless/03036_clamp.sql
Normal 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 }
|
Loading…
Reference in New Issue
Block a user