Merge branch 'master' into more-consistent-datetime64-parsing

This commit is contained in:
Alexey Milovidov 2021-02-21 23:24:07 +03:00
commit f7b7561552
46 changed files with 404 additions and 105 deletions

View File

@ -6,6 +6,25 @@
namespace common
{
/// Multiply and ignore overflow.
template <typename T1, typename T2>
inline auto NO_SANITIZE_UNDEFINED mulIgnoreOverflow(T1 x, T2 y)
{
return x * y;
}
template <typename T1, typename T2>
inline auto NO_SANITIZE_UNDEFINED addIgnoreOverflow(T1 x, T2 y)
{
return x + y;
}
template <typename T1, typename T2>
inline auto NO_SANITIZE_UNDEFINED subIgnoreOverflow(T1 x, T2 y)
{
return x - y;
}
template <typename T>
inline bool addOverflow(T x, T y, T & res)
{
@ -35,14 +54,14 @@ namespace common
{
static constexpr __int128 min_int128 = minInt128();
static constexpr __int128 max_int128 = maxInt128();
res = x + y;
res = addIgnoreOverflow(x, y);
return (y > 0 && x > max_int128 - y) || (y < 0 && x < min_int128 - y);
}
template <>
inline bool addOverflow(wInt256 x, wInt256 y, wInt256 & res)
{
res = x + y;
res = addIgnoreOverflow(x, y);
return (y > 0 && x > std::numeric_limits<wInt256>::max() - y) ||
(y < 0 && x < std::numeric_limits<wInt256>::min() - y);
}
@ -50,7 +69,7 @@ namespace common
template <>
inline bool addOverflow(wUInt256 x, wUInt256 y, wUInt256 & res)
{
res = x + y;
res = addIgnoreOverflow(x, y);
return x > std::numeric_limits<wUInt256>::max() - y;
}
@ -83,14 +102,14 @@ namespace common
{
static constexpr __int128 min_int128 = minInt128();
static constexpr __int128 max_int128 = maxInt128();
res = x - y;
res = subIgnoreOverflow(x, y);
return (y < 0 && x > max_int128 + y) || (y > 0 && x < min_int128 + y);
}
template <>
inline bool subOverflow(wInt256 x, wInt256 y, wInt256 & res)
{
res = x - y;
res = subIgnoreOverflow(x, y);
return (y < 0 && x > std::numeric_limits<wInt256>::max() + y) ||
(y > 0 && x < std::numeric_limits<wInt256>::min() + y);
}
@ -98,7 +117,7 @@ namespace common
template <>
inline bool subOverflow(wUInt256 x, wUInt256 y, wUInt256 & res)
{
res = x - y;
res = subIgnoreOverflow(x, y);
return x < y;
}
@ -129,40 +148,33 @@ namespace common
template <>
inline bool mulOverflow(__int128 x, __int128 y, __int128 & res)
{
res = static_cast<unsigned __int128>(x) * static_cast<unsigned __int128>(y); /// Avoid signed integer overflow.
res = mulIgnoreOverflow(x, y);
if (!x || !y)
return false;
unsigned __int128 a = (x > 0) ? x : -x;
unsigned __int128 b = (y > 0) ? y : -y;
return (a * b) / b != a;
return mulIgnoreOverflow(a, b) / b != a;
}
template <>
inline bool mulOverflow(wInt256 x, wInt256 y, wInt256 & res)
{
res = x * y;
res = mulIgnoreOverflow(x, y);
if (!x || !y)
return false;
wInt256 a = (x > 0) ? x : -x;
wInt256 b = (y > 0) ? y : -y;
return (a * b) / b != a;
return mulIgnoreOverflow(a, b) / b != a;
}
template <>
inline bool mulOverflow(wUInt256 x, wUInt256 y, wUInt256 & res)
{
res = x * y;
res = mulIgnoreOverflow(x, y);
if (!x || !y)
return false;
return (x * y) / y != x;
}
/// Multiply and ignore overflow.
template <typename T1, typename T2>
inline auto NO_SANITIZE_UNDEFINED mulIgnoreOverflow(T1 x, T2 y)
{
return x * y;
return res / y != x;
}
}

View File

@ -6,5 +6,9 @@ services:
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3308:3306
command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency
- 3308:3306
command: --server_id=100 --log-bin='mysql-bin-1.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3

View File

@ -6,5 +6,10 @@ services:
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 33308:3306
command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency
- 33308:3306
command: --server_id=100 --log-bin='mysql-bin-1.log'
--default_authentication_plugin='mysql_native_password'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3

View File

@ -97,6 +97,7 @@ function configure
rm -r right/db ||:
rm -r db0/preprocessed_configs ||:
rm -r db0/{data,metadata}/system ||:
rm db0/status ||:
cp -al db0/ left/db/
cp -al db0/ right/db/
}

View File

@ -23,7 +23,7 @@ LowCardinality(data_type)
Эффективность использования типа данных `LowCarditality` зависит от разнообразия данных. Если словарь содержит менее 10 000 различных значений, ClickHouse в основном показывает более высокую эффективность чтения и хранения данных. Если же словарь содержит более 100 000 различных значений, ClickHouse может работать хуже, чем при использовании обычных типов данных.
При работе со строками, использование `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность.
При работе со строками используйте `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность.
## Пример

View File

@ -30,6 +30,10 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
const String & getAggregateFunctionCanonicalNameIfAny(const String & name)
{
return AggregateFunctionFactory::instance().getCanonicalNameIfAny(name);
}
void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, CaseSensitiveness case_sensitiveness)
{
@ -41,10 +45,14 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat
throw Exception("AggregateFunctionFactory: the aggregate function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
if (case_sensitiveness == CaseInsensitive
&& !case_insensitive_aggregate_functions.emplace(Poco::toLower(name), creator_with_properties).second)
throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
if (case_sensitiveness == CaseInsensitive)
{
auto key = Poco::toLower(name);
if (!case_insensitive_aggregate_functions.emplace(key, creator_with_properties).second)
throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
case_insensitive_name_mapping[key] = name;
}
}
static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types)

View File

@ -35,6 +35,8 @@ protected:
return name;
}
std::unordered_map<String, String> case_insensitive_name_mapping;
public:
/// For compatibility with SQL, it's possible to specify that certain function name is case insensitive.
enum CaseSensitiveness
@ -68,9 +70,12 @@ public:
factory_name + ": the alias name '" + alias_name + "' is already registered as real name", ErrorCodes::LOGICAL_ERROR);
if (case_sensitiveness == CaseInsensitive)
{
if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_dict_name).second)
throw Exception(
factory_name + ": case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
case_insensitive_name_mapping[alias_name_lowercase] = real_name;
}
if (!aliases.emplace(alias_name, real_dict_name).second)
throw Exception(factory_name + ": alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
@ -111,6 +116,15 @@ public:
return getMap().count(name) || getCaseInsensitiveMap().count(name) || isAlias(name);
}
/// Return the canonical name (the name used in registration) if it's different from `name`.
const String & getCanonicalNameIfAny(const String & name) const
{
auto it = case_insensitive_name_mapping.find(Poco::toLower(name));
if (it != case_insensitive_name_mapping.end())
return it->second;
return name;
}
virtual ~IFactoryWithAliases() override {}
private:

View File

@ -383,6 +383,7 @@ class IColumn;
M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \
M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \
M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \
M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \
M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \
M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \

View File

@ -21,6 +21,10 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
const String & getFunctionCanonicalNameIfAny(const String & name)
{
return FunctionFactory::instance().getCanonicalNameIfAny(name);
}
void FunctionFactory::registerFunction(const
std::string & name,
@ -36,10 +40,13 @@ void FunctionFactory::registerFunction(const
throw Exception("FunctionFactory: the function name '" + name + "' is already registered as alias",
ErrorCodes::LOGICAL_ERROR);
if (case_sensitiveness == CaseInsensitive
&& !case_insensitive_functions.emplace(function_name_lowercase, creator).second)
throw Exception("FunctionFactory: the case insensitive function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
if (case_sensitiveness == CaseInsensitive)
{
if (!case_insensitive_functions.emplace(function_name_lowercase, creator).second)
throw Exception("FunctionFactory: the case insensitive function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
case_insensitive_name_mapping[function_name_lowercase] = name;
}
}

View File

@ -1313,7 +1313,7 @@ public:
else if constexpr (std::is_same_v<Name, NameToDecimal256>)
return createDecimalMaxPrecision<Decimal256>(scale);
throw Exception("Something wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected branch in code of conversion function: it is a bug.", ErrorCodes::LOGICAL_ERROR);
}
else
{
@ -1337,7 +1337,7 @@ public:
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0));
else if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected branch in code of conversion function: it is a bug.", ErrorCodes::LOGICAL_ERROR);
else
return std::make_shared<ToDataType>();
}

View File

@ -8,7 +8,7 @@ namespace DB
void registerFunctionsRound(FunctionFactory & factory)
{
factory.registerFunction<FunctionRound>("round", FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionRoundBankers>("roundBankers", FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionRoundBankers>("roundBankers", FunctionFactory::CaseSensitive);
factory.registerFunction<FunctionFloor>("floor", FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCeil>("ceil", FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionTrunc>("trunc", FunctionFactory::CaseInsensitive);

View File

@ -18,7 +18,7 @@ namespace DB
void registerFunctionExtractAllGroupsVertical(FunctionFactory & factory)
{
factory.registerFunction<FunctionExtractAllGroups<VerticalImpl>>();
factory.registerAlias("extractAllGroups", VerticalImpl::Name, FunctionFactory::CaseInsensitive);
factory.registerAlias("extractAllGroups", VerticalImpl::Name, FunctionFactory::CaseSensitive);
}
}

View File

@ -1,4 +1,6 @@
#include <mutex>
#include <ext/bit_cast.h>
#include <Common/FieldVisitors.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnString.h>
@ -13,6 +15,7 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/getLeastSupertype.h>
#include <Interpreters/convertFieldToType.h>
namespace DB
@ -491,7 +494,7 @@ private:
dst.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
const auto * it = table.find(ext::bit_cast<UInt64>(src[i]));
if (it)
memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); /// little endian.
else
@ -507,7 +510,7 @@ private:
dst.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
const auto * it = table.find(ext::bit_cast<UInt64>(src[i]));
if (it)
memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); /// little endian.
else
@ -523,7 +526,7 @@ private:
dst.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
const auto * it = table.find(ext::bit_cast<UInt64>(src[i]));
if (it)
memcpy(&dst[i], &it->getMapped(), sizeof(dst[i]));
else
@ -541,7 +544,7 @@ private:
ColumnString::Offset current_dst_offset = 0;
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
const auto * it = table.find(ext::bit_cast<UInt64>(src[i]));
StringRef ref = it ? it->getMapped() : dst_default;
dst_data.resize(current_dst_offset + ref.size);
memcpy(&dst_data[current_dst_offset], ref.data, ref.size);
@ -562,7 +565,8 @@ private:
ColumnString::Offset current_dst_default_offset = 0;
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
Field key = src[i];
const auto * it = table.find(key.reinterpret<UInt64>());
StringRef ref;
if (it)
@ -778,50 +782,66 @@ private:
/// Note: Doesn't check the duplicates in the `from` array.
if (from[0].getType() != Field::Types::String && to[0].getType() != Field::Types::String)
const IDataType & from_type = *arguments[0].type;
if (from[0].getType() != Field::Types::String)
{
cache.table_num_to_num = std::make_unique<Cache::NumToNum>();
auto & table = *cache.table_num_to_num;
for (size_t i = 0; i < size; ++i)
if (to[0].getType() != Field::Types::String)
{
// Field may be of Float type, but for the purpose of bitwise
// equality we can treat them as UInt64, hence the reinterpret().
table[from[i].reinterpret<UInt64>()] = (*used_to)[i].reinterpret<UInt64>();
cache.table_num_to_num = std::make_unique<Cache::NumToNum>();
auto & table = *cache.table_num_to_num;
for (size_t i = 0; i < size; ++i)
{
Field key = convertFieldToType(from[i], from_type);
if (key.isNull())
continue;
// Field may be of Float type, but for the purpose of bitwise
// equality we can treat them as UInt64, hence the reinterpret().
table[key.reinterpret<UInt64>()] = (*used_to)[i].reinterpret<UInt64>();
}
}
else
{
cache.table_num_to_string = std::make_unique<Cache::NumToString>();
auto & table = *cache.table_num_to_string;
for (size_t i = 0; i < size; ++i)
{
Field key = convertFieldToType(from[i], from_type);
if (key.isNull())
continue;
const String & str_to = to[i].get<const String &>();
StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[key.reinterpret<UInt64>()] = ref;
}
}
}
else if (from[0].getType() != Field::Types::String && to[0].getType() == Field::Types::String)
else
{
cache.table_num_to_string = std::make_unique<Cache::NumToString>();
auto & table = *cache.table_num_to_string;
for (size_t i = 0; i < size; ++i)
if (to[0].getType() != Field::Types::String)
{
const String & str_to = to[i].get<const String &>();
StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[from[i].reinterpret<UInt64>()] = ref;
cache.table_string_to_num = std::make_unique<Cache::StringToNum>();
auto & table = *cache.table_string_to_num;
for (size_t i = 0; i < size; ++i)
{
const String & str_from = from[i].get<const String &>();
StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
table[ref] = (*used_to)[i].reinterpret<UInt64>();
}
}
}
else if (from[0].getType() == Field::Types::String && to[0].getType() != Field::Types::String)
{
cache.table_string_to_num = std::make_unique<Cache::StringToNum>();
auto & table = *cache.table_string_to_num;
for (size_t i = 0; i < size; ++i)
else
{
const String & str_from = from[i].get<const String &>();
StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
table[ref] = (*used_to)[i].reinterpret<UInt64>();
}
}
else if (from[0].getType() == Field::Types::String && to[0].getType() == Field::Types::String)
{
cache.table_string_to_string = std::make_unique<Cache::StringToString>();
auto & table = *cache.table_string_to_string;
for (size_t i = 0; i < size; ++i)
{
const String & str_from = from[i].get<const String &>();
const String & str_to = to[i].get<const String &>();
StringRef ref_from{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
StringRef ref_to{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[ref_from] = ref_to;
cache.table_string_to_string = std::make_unique<Cache::StringToString>();
auto & table = *cache.table_string_to_string;
for (size_t i = 0; i < size; ++i)
{
const String & str_from = from[i].get<const String &>();
const String & str_to = to[i].get<const String &>();
StringRef ref_from{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
StringRef ref_to{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[ref_from] = ref_to;
}
}
}

View File

@ -0,0 +1,45 @@
#include <Interpreters/FunctionNameNormalizer.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTCreateQuery.h>
namespace DB
{
const String & getFunctionCanonicalNameIfAny(const String & name);
const String & getAggregateFunctionCanonicalNameIfAny(const String & name);
void FunctionNameNormalizer::visit(IAST * ast)
{
if (!ast)
return;
// Normalize only selected children. Avoid normalizing engine clause because some engine might
// have the same name as function, e.g. Log.
if (auto * node_storage = ast->as<ASTStorage>())
{
visit(node_storage->partition_by);
visit(node_storage->primary_key);
visit(node_storage->order_by);
visit(node_storage->sample_by);
visit(node_storage->ttl_table);
return;
}
// Normalize only selected children. Avoid normalizing type clause because some type might
// have the same name as function, e.g. Date.
if (auto * node_decl = ast->as<ASTColumnDeclaration>())
{
visit(node_decl->default_expression.get());
visit(node_decl->ttl.get());
return;
}
if (auto * node_func = ast->as<ASTFunction>())
node_func->name = getAggregateFunctionCanonicalNameIfAny(getFunctionCanonicalNameIfAny(node_func->name));
for (auto & child : ast->children)
visit(child.get());
}
}

View File

@ -0,0 +1,14 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
struct FunctionNameNormalizer
{
static void visit(IAST *);
};
}

View File

@ -58,6 +58,7 @@
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/addTypeConversionToAST.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <common/logger_useful.h>
@ -1176,6 +1177,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, cons
BlockIO InterpreterCreateQuery::execute()
{
FunctionNameNormalizer().visit(query_ptr.get());
auto & create = query_ptr->as<ASTCreateQuery &>();
if (!create.cluster.empty())
{

View File

@ -442,10 +442,10 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
auto type_literal = std::make_shared<ASTLiteral>(columns_desc.getPhysical(column).type->getName());
const auto & update_expr = kv.second;
auto updated_column = makeASTFunction("cast",
auto updated_column = makeASTFunction("CAST",
makeASTFunction("if",
getPartitionAndPredicateExpressionForMutationCommand(command),
makeASTFunction("cast",
makeASTFunction("CAST",
update_expr->clone(),
type_literal),
std::make_shared<ASTIdentifier>(column)),

View File

@ -29,7 +29,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v
/// cast of numeric constant in condition to UInt8
if (const auto * function = condition->as<ASTFunction>())
{
if (function->name == "cast")
if (function->name == "CAST")
{
if (const auto * expr_list = function->arguments->as<ASTExpressionList>())
{

View File

@ -8,6 +8,7 @@
#include <Interpreters/ArrayJoinedColumnsVisitor.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/Context.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/MarkTableIdentifiersVisitor.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
@ -934,6 +935,10 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings &
MarkTableIdentifiersVisitor::Data identifiers_data{aliases};
MarkTableIdentifiersVisitor(identifiers_data).visit(query);
/// Rewrite function names to their canonical ones.
if (settings.normalize_function_names)
FunctionNameNormalizer().visit(query.get());
/// Common subexpression elimination. Rewrite rules.
QueryNormalizer::Data normalizer_data(aliases, settings);
QueryNormalizer(normalizer_data).visit(query);

View File

@ -20,7 +20,7 @@ namespace ErrorCodes
ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name)
{
auto func = makeASTFunction("cast", ast, std::make_shared<ASTLiteral>(type_name));
auto func = makeASTFunction("CAST", ast, std::make_shared<ASTLiteral>(type_name));
if (ASTWithAlias * ast_with_alias = dynamic_cast<ASTWithAlias *>(ast.get()))
{

View File

@ -15,6 +15,7 @@
#include <Parsers/ExpressionElementParsers.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/typeid_cast.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Poco/Util/AbstractConfiguration.h>
@ -35,6 +36,10 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
auto ast = node->clone();
ReplaceQueryParameterVisitor param_visitor(context.getQueryParameters());
param_visitor.visit(ast);
if (context.getSettingsRef().normalize_function_names)
FunctionNameNormalizer().visit(ast.get());
String name = ast->getColumnName();
auto syntax_result = TreeRewriter(context).analyze(ast, source_columns);
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions();

View File

@ -43,7 +43,7 @@ void addDefaultRequiredExpressionsRecursively(const Block & block, const String
RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr);
NameSet required_columns_names = columns_context.requiredColumns();
auto cast_func = makeASTFunction("cast", column_default_expr, std::make_shared<ASTLiteral>(columns.get(required_column).type->getName()));
auto cast_func = makeASTFunction("CAST", column_default_expr, std::make_shared<ASTLiteral>(columns.get(required_column).type->getName()));
default_expr_list_accum->children.emplace_back(setAlias(cast_func, required_column));
added_columns.emplace(required_column);
@ -79,7 +79,7 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi
continue;
auto cast_func = makeASTFunction(
"cast", std::make_shared<ASTIdentifier>(required_column.name), std::make_shared<ASTLiteral>(required_column.type->getName()));
"CAST", std::make_shared<ASTIdentifier>(required_column.name), std::make_shared<ASTLiteral>(required_column.type->getName()));
conversion_expr_list->children.emplace_back(setAlias(cast_func, required_column.name));

View File

@ -59,6 +59,7 @@ SRCS(
ExternalModelsLoader.cpp
ExtractExpressionInfoVisitor.cpp
FillingRow.cpp
FunctionNameNormalizer.cpp
HashJoin.cpp
IExternalLoadable.cpp
IInterpreter.cpp

View File

@ -864,7 +864,7 @@ bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
expr_list_args->children.push_back(std::move(type_literal));
auto func_node = std::make_shared<ASTFunction>();
func_node->name = "cast";
func_node->name = "CAST";
func_node->arguments = std::move(expr_list_args);
func_node->children.push_back(func_node->arguments);

View File

@ -626,7 +626,7 @@ void ConstantExpressionTemplate::TemplateStructure::addNodesToCastResult(const I
expr = makeASTFunction("assumeNotNull", std::move(expr));
}
expr = makeASTFunction("cast", std::move(expr), std::make_shared<ASTLiteral>(result_column_type.getName()));
expr = makeASTFunction("CAST", std::move(expr), std::make_shared<ASTLiteral>(result_column_type.getName()));
if (null_as_default)
{

View File

@ -1133,6 +1133,14 @@ void TCPHandler::receiveQuery()
}
query_context->applySettingsChanges(settings_changes);
/// Disable function name normalization when it's a secondary query, because queries are either
/// already normalized on initiator node, or not normalized and should remain unnormalized for
/// compatibility.
if (client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
{
query_context->setSetting("normalize_function_names", Field(0));
}
// Use the received query id, or generate a random default. It is convenient
// to also generate the default OpenTelemetry trace id at the same time, and
// set the trace parent.

View File

@ -16,6 +16,7 @@
#include <Common/PipeFDs.h>
#include <Common/CurrentThread.h>
#include <common/getThreadId.h>
#include <common/logger_useful.h>
namespace DB
@ -150,6 +151,7 @@ namespace
StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_)
: IStorageSystemOneBlock<StorageSystemStackTrace>(table_id_)
, log(&Poco::Logger::get("StorageSystemStackTrace"))
{
notification_pipe.open();
@ -229,6 +231,8 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte
}
else
{
LOG_DEBUG(log, "Cannot obtain a stack trace for thread {}", tid);
/// Cannot obtain a stack trace. But create a record in result nevertheless.
res_columns[0]->insert(tid);

View File

@ -6,6 +6,10 @@
#include <ext/shared_ptr_helper.h>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace Poco
{
class Logger;
}
namespace DB
{
@ -30,6 +34,8 @@ protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;
mutable std::mutex mutex;
Poco::Logger * log;
};
}

View File

@ -44,8 +44,8 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
f.write("=".join([var, value]) + "\n")
return full_path
def run_and_check(args, env=None, shell=False):
res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell)
def run_and_check(args, env=None, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE):
res = subprocess.run(args, stdout=stdout, stderr=stderr, env=env, shell=shell)
if res.returncode != 0:
# check_call(...) from subprocess does not print stderr, so we do it manually
print('Stderr:\n{}\n'.format(res.stderr.decode('utf-8')))

View File

@ -37,6 +37,12 @@ class MySQLNodeInstance:
self.docker_compose = docker_compose
self.project_name = project_name
self.base_dir = p.dirname(__file__)
self.instances_dir = p.join(self.base_dir, '_instances_mysql')
if not os.path.exists(self.instances_dir):
os.mkdir(self.instances_dir)
self.docker_logs_path = p.join(self.instances_dir, 'docker_mysql.log')
def alloc_connection(self):
if self.mysql_connection is None:
@ -71,10 +77,28 @@ class MySQLNodeInstance:
cursor.execute(executio_query)
return cursor.fetchall()
def start_and_wait(self):
run_and_check(['docker-compose',
'-p', cluster.project_name,
'-f', self.docker_compose,
'up', '--no-recreate', '-d',
])
self.wait_mysql_to_start(120)
def close(self):
if self.mysql_connection is not None:
self.mysql_connection.close()
with open(self.docker_logs_path, "w+") as f:
try:
run_and_check([
'docker-compose',
'-p', cluster.project_name,
'-f', self.docker_compose, 'logs',
], stdout=f)
except Exception as e:
print("Unable to get logs from docker mysql.")
def wait_mysql_to_start(self, timeout=60):
start = time.time()
while time.time() - start < timeout:
@ -95,9 +119,7 @@ def started_mysql_5_7():
mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, docker_compose)
try:
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d'])
mysql_node.wait_mysql_to_start(120)
mysql_node.start_and_wait()
yield mysql_node
finally:
mysql_node.close()
@ -111,9 +133,7 @@ def started_mysql_8_0():
mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, docker_compose)
try:
run_and_check(
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d'])
mysql_node.wait_mysql_to_start(120)
mysql_node.start_and_wait()
yield mysql_node
finally:
mysql_node.close()

View File

@ -217,7 +217,7 @@ def test_mysql_replacement_query(mysql_client, server_address):
--password=123 -e "select database();"
'''.format(host=server_address, port=server_port), demux=True)
assert code == 0
assert stdout.decode() == 'database()\ndefault\n'
assert stdout.decode() == 'DATABASE()\ndefault\n'
code, (stdout, stderr) = mysql_client.exec_run('''
mysql --protocol tcp -h {host} -P {port} default -u default

View File

@ -114,7 +114,7 @@ FROM
(
SELECT
1 AS id,
identity(cast(1, \'UInt8\')) AS subquery
identity(CAST(1, \'UInt8\')) AS subquery
WHERE subquery = 1
)
WHERE subquery = 1

View File

@ -10,11 +10,11 @@ hello
CREATE TABLE default.cast
(
`x` UInt8,
`e` Enum8('hello' = 1, 'world' = 2) DEFAULT cast(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')
`e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')
)
ENGINE = MergeTree
ORDER BY e
SETTINGS index_granularity = 8192
x UInt8
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT cast(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
1 hello

View File

@ -1,12 +1,12 @@
CREATE TABLE default.cast1
(
`x` UInt8,
`e` Enum8('hello' = 1, 'world' = 2) DEFAULT cast(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')
`e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')
)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1')
ORDER BY e
SETTINGS index_granularity = 8192
x UInt8
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT cast(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
1 hello
1 hello

View File

@ -1,3 +1,5 @@
SET database_atomic_wait_for_drop_and_detach_synchronously=1;
DROP TABLE IF EXISTS cast1;
DROP TABLE IF EXISTS cast2;

View File

@ -2,7 +2,7 @@ SELECT 1
WHERE 0
SELECT 1
SELECT 1
WHERE (1 IN (0, 2)) AND (2 = (identity(cast(2, \'UInt8\')) AS subquery))
WHERE (1 IN (0, 2)) AND (2 = (identity(CAST(2, \'UInt8\')) AS subquery))
SELECT 1
WHERE 1 IN (
(

View File

@ -5,7 +5,7 @@ SELECT (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n) FO
1,10
EXPLAIN SYNTAX SELECT (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n);
SELECT
identity(cast(0, \'UInt64\')) AS n,
identity(CAST(0, \'UInt64\')) AS n,
toUInt64(10 / n)
SELECT * FROM (WITH (SELECT * FROM system.numbers LIMIT 1 OFFSET 1) AS n, toUInt64(10 / n) as q SELECT * FROM system.one WHERE q > 0);
0

View File

@ -11,7 +11,7 @@ arraySort(used_table_functions)
['numbers']
arraySort(used_functions)
['addDays','array','arrayFlatten','cast','modulo','plus','substring','toDate','toDayOfYear','toTypeName','toWeek']
['CAST','addDays','array','arrayFlatten','modulo','plus','substring','toDate','toDayOfYear','toTypeName','toWeek']
arraySort(used_data_type_families)
['Array','Int32','Nullable','String']

View File

@ -1,3 +1,5 @@
SET database_atomic_wait_for_drop_and_detach_synchronously=1;
SELECT uniqArray([1, 1, 2]),
SUBSTRING('Hello, world', 7, 5),
flatten([[[BIT_AND(123)]], [[mod(3, 2)], [CAST('1' AS INTEGER)]]]),

View File

@ -0,0 +1,30 @@
-
Hello
-
World
-
-
-
-
-
-
-
-
Hello
-
World
-
-
-
-
-
-
-
Hello
-
World
-
-
-
-
-

View File

@ -0,0 +1,3 @@
SELECT transform(number / 2, [0.5, 1.5], ['Hello', 'World'], '-') FROM numbers(10);
SELECT transform(number / 2, [1.0, 2.0], ['Hello', 'World'], '-') FROM numbers(10);
SELECT transform(number / 2, [1, 2], ['Hello', 'World'], '-') FROM numbers(10);

View File

@ -0,0 +1,66 @@
SELECT
CAST(1, 'INT'),
ceil(1),
ceil(1),
char(49),
CHAR_LENGTH('1'),
CHARACTER_LENGTH('1'),
coalesce(1),
concat('1', '1'),
corr(1, 1),
cos(1),
count(),
covarPop(1, 1),
covarSamp(1, 1),
DATABASE(),
dateDiff('DAY', toDate('2020-10-24'), toDate('2019-10-24')),
exp(1),
arrayFlatten([[1]]),
floor(1),
FQDN(),
greatest(1),
1,
ifNull(1, 1),
lower('A'),
least(1),
length('1'),
log(1),
position('1', '1'),
log(1),
log10(1),
log2(1),
lower('A'),
max(1),
substring('123', 1, 1),
min(1),
1 % 1,
NOT 1,
now(),
now64(),
nullIf(1, 1),
pi(),
position('123', '2'),
pow(1, 1),
pow(1, 1),
rand(),
replaceAll('1', '1', '2'),
reverse('123'),
round(1),
sin(1),
sqrt(1),
stddevPop(1),
stddevSamp(1),
substring('123', 2),
substring('123', 2),
count(),
tan(1),
tanh(1),
trunc(1),
trunc(1),
upper('A'),
upper('A'),
currentUser(),
varPop(1),
varSamp(1),
toWeek(toDate('2020-10-24')),
toYearWeek(toDate('2020-10-24'))

View File

@ -0,0 +1 @@
EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH('1'), CHARACTER_LENGTH('1'), COALESCE(1), CONCAT('1', '1'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), DATEDIFF('DAY', toDate('2020-10-24'), toDate('2019-10-24')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE('A'), LEAST(1), LENGTH('1'), LN(1), LOCATE('1', '1'), LOG(1), LOG10(1), LOG2(1), LOWER('A'), MAX(1), MID('123', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION('123', '2'), POW(1, 1), POWER(1, 1), RAND(), REPLACE('1', '1', '2'), REVERSE('123'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR('123', 2), SUBSTRING('123', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE('A'), UPPER('A'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate('2020-10-24')), YEARWEEK(toDate('2020-10-24')) format TSVRaw;

View File

@ -0,0 +1,11 @@
CREATE TEMPORARY TABLE decimal
(
f dec(38, 38)
);
INSERT INTO decimal VALUES (0);
INSERT INTO decimal VALUES (0.42);
INSERT INTO decimal VALUES (-0.42);
SELECT f + 1048575, f - 21, f - 84, f * 21, f * -21, f / 21, f / 84 FROM decimal WHERE f > 0; -- { serverError 407 }
SELECT f + -2, f - 21, f - 84, f * 21, f * -21, f / 9223372036854775807, f / 84 FROM decimal WHERE f > 0; -- { serverError 407 }

View File

@ -520,6 +520,7 @@
"00626_replace_partition_from_table",
"00626_replace_partition_from_table_zookeeper",
"00633_materialized_view_and_too_many_parts_zookeeper",
"00643_cast_zookeeper",
"00652_mergetree_mutations",
"00652_replicated_mutations_zookeeper",
"00682_empty_parts_merge",
@ -733,10 +734,11 @@
"01602_show_create_view",
"01603_rename_overwrite_bug",
"01646_system_restart_replicas_smoke", // system restart replicas is a global query
"01676_dictget_in_default_expression",
"01715_background_checker_blather_zookeeper",
"01700_system_zookeeper_path_in",
"01656_test_query_log_factories_info",
"01669_columns_declaration_serde",
"01676_dictget_in_default_expression",
"01700_system_zookeeper_path_in",
"01715_background_checker_blather_zookeeper",
"attach",
"ddl_dictionaries",
"dictionary",