Merge branch 'master' into issue_45590

This commit is contained in:
Nikolai Kochetov 2023-01-27 15:08:28 +01:00 committed by GitHub
commit 313947e0b0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
246 changed files with 3062 additions and 2351 deletions

View File

@ -234,6 +234,7 @@ function run_tests
--check-zookeeper-session
--order random
--print-time
--report-logs-stats
--jobs "${NPROC}"
)
time clickhouse-test "${test_opts[@]}" -- "$FASTTEST_FOCUS" 2>&1 \

View File

@ -215,10 +215,10 @@ The two-argument form of `toDayOfWeek()` enables you to specify whether the week
| Mode | First day of week | Range |
|------|-------------------|------------------------------------------------|
| 0 | Monday | 1-7, Monday = 1, Tuesday = 2, ..., Sunday = 7 |
| 1 | Monday | 0-6, Monday = 0, Tuesday = 1, ..., Sunday = 6 |
| 2 | Sunday | 0-6, Sunday = 0, Monday = 1, ..., Saturday = 6 |
| 3 | Sunday | 1-7, Sunday = 1, Monday = 2, ..., Saturday = 7 |
| 0 | Monday | 1-7: Monday = 1, Tuesday = 2, ..., Sunday = 7 |
| 1 | Monday | 0-6: Monday = 0, Tuesday = 1, ..., Sunday = 6 |
| 2 | Sunday | 0-6: Sunday = 0, Monday = 1, ..., Saturday = 6 |
| 3 | Sunday | 1-7: Sunday = 1, Monday = 2, ..., Saturday = 7 |
Alias: `DAYOFWEEK`.

View File

@ -277,7 +277,7 @@ private:
}
if (queries.empty())
throw Exception("Empty list of queries.", ErrorCodes::EMPTY_DATA_PASSED);
throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Empty list of queries.");
}
else
{

View File

@ -719,7 +719,7 @@ bool Client::processWithFuzzing(const String & full_query)
// uniformity.
// Surprisingly, this is a client exception, because we get the
// server exception w/o throwing (see onReceiveException()).
client_exception = std::make_unique<Exception>(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode());
client_exception = std::make_unique<Exception>(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode());
have_error = true;
}
@ -854,7 +854,7 @@ bool Client::processWithFuzzing(const String & full_query)
}
catch (...)
{
client_exception = std::make_unique<Exception>(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode());
client_exception = std::make_unique<Exception>(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode());
have_error = true;
}

View File

@ -165,9 +165,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
/// should throw exception early and make exception message more readable.
if (const auto * insert_query = res->as<ASTInsertQuery>(); insert_query && insert_query->data)
{
throw Exception(
"Can't format ASTInsertQuery with data, since data will be lost",
DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA);
throw Exception(DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA,
"Can't format ASTInsertQuery with data, since data will be lost");
}
if (!quiet)
{

View File

@ -196,7 +196,7 @@ void Keeper::createServer(const std::string & listen_host, const char * port_nam
}
else
{
throw Exception{message, ErrorCodes::NETWORK_ERROR};
throw Exception::createDeprecated(message, ErrorCodes::NETWORK_ERROR);
}
}
}
@ -375,7 +375,7 @@ try
if (effective_user_id == 0)
{
message += " Run under 'sudo -u " + data_owner + "'.";
throw Exception(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA);
throw Exception::createDeprecated(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA);
}
else
{

View File

@ -243,7 +243,6 @@ ColumnFloat64::MutablePtr CatBoostLibraryHandler::evalImpl(
const ColumnRawPtrs & columns,
bool cat_features_are_strings) const
{
std::string error_msg = "Error occurred while applying CatBoost model: ";
size_t column_size = columns.front()->size();
auto result = ColumnFloat64::create(column_size * tree_count);
@ -265,7 +264,8 @@ ColumnFloat64::MutablePtr CatBoostLibraryHandler::evalImpl(
result_buf, column_size * tree_count))
{
throw Exception(error_msg + api.GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL);
throw Exception(ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL,
"Error occurred while applying CatBoost model: {}", api.GetErrorString());
}
return result;
}
@ -288,7 +288,8 @@ ColumnFloat64::MutablePtr CatBoostLibraryHandler::evalImpl(
cat_features_buf, cat_features_count,
result_buf, column_size * tree_count))
{
throw Exception(error_msg + api.GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL);
throw Exception(ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL,
"Error occurred while applying CatBoost model: {}", api.GetErrorString());
}
}
else
@ -304,7 +305,8 @@ ColumnFloat64::MutablePtr CatBoostLibraryHandler::evalImpl(
cat_features_buf, cat_features_count,
result_buf, column_size * tree_count))
{
throw Exception(error_msg + api.GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL);
throw Exception(ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL,
"Error occurred while applying CatBoost model: {}", api.GetErrorString());
}
}

View File

@ -416,7 +416,7 @@ void Server::createServer(
}
else
{
throw Exception{message, ErrorCodes::NETWORK_ERROR};
throw Exception::createDeprecated(message, ErrorCodes::NETWORK_ERROR);
}
}
}
@ -946,7 +946,7 @@ try
if (effective_user_id == 0)
{
message += " Run under 'sudo -u " + data_owner + "'.";
throw Exception(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA);
throw Exception::createDeprecated(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA);
}
else
{

View File

@ -79,9 +79,7 @@ AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(std::string_vi
::DB::encodeSHA256(text, hash.data());
return hash;
#else
throw DB::Exception(
"SHA256 passwords support is disabled, because ClickHouse was built without SSL library",
DB::ErrorCodes::SUPPORT_IS_DISABLED);
throw DB::Exception(DB::ErrorCodes::SUPPORT_IS_DISABLED, "SHA256 passwords support is disabled, because ClickHouse was built without SSL library");
#endif
}

View File

@ -484,13 +484,15 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
return true;
};
auto access_denied = [&](const String & error_msg, int error_code [[maybe_unused]])
auto access_denied = [&]<typename... FmtArgs>(int error_code [[maybe_unused]],
FormatStringHelper<String, FmtArgs...> fmt_string [[maybe_unused]],
FmtArgs && ...fmt_args [[maybe_unused]])
{
if (trace_log)
LOG_TRACE(trace_log, "Access denied: {}{}", (AccessRightsElement{flags, args...}.toStringWithoutOptions()),
(grant_option ? " WITH GRANT OPTION" : ""));
if constexpr (throw_if_denied)
throw Exception(getUserName() + ": " + error_msg, error_code);
throw Exception(error_code, std::move(fmt_string), getUserName(), std::forward<FmtArgs>(fmt_args)...);
return false;
};
@ -519,18 +521,16 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
{
if (grant_option && acs->isGranted(flags, args...))
{
return access_denied(
"Not enough privileges. "
return access_denied(ErrorCodes::ACCESS_DENIED,
"{}: Not enough privileges. "
"The required privileges have been granted, but without grant option. "
"To execute this query it's necessary to have grant "
+ AccessRightsElement{flags, args...}.toStringWithoutOptions() + " WITH GRANT OPTION",
ErrorCodes::ACCESS_DENIED);
"To execute this query it's necessary to have grant {} WITH GRANT OPTION",
AccessRightsElement{flags, args...}.toStringWithoutOptions());
}
return access_denied(
"Not enough privileges. To execute this query it's necessary to have grant "
+ AccessRightsElement{flags, args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : ""),
ErrorCodes::ACCESS_DENIED);
return access_denied(ErrorCodes::ACCESS_DENIED,
"{}: Not enough privileges. To execute this query it's necessary to have grant {}",
AccessRightsElement{flags, args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : ""));
}
struct PrecalculatedFlags
@ -557,32 +557,34 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
if (params.readonly)
{
if constexpr (grant_option)
return access_denied("Cannot change grants in readonly mode.", ErrorCodes::READONLY);
return access_denied(ErrorCodes::READONLY, "{}: Cannot change grants in readonly mode.");
if ((flags & precalc.not_readonly_flags) ||
((params.readonly == 1) && (flags & precalc.not_readonly_1_flags)))
{
if (params.interface == ClientInfo::Interface::HTTP && params.http_method == ClientInfo::HTTPMethod::GET)
{
return access_denied(
"Cannot execute query in readonly mode. "
"For queries over HTTP, method GET implies readonly. You should use method POST for modifying queries",
ErrorCodes::READONLY);
return access_denied(ErrorCodes::READONLY,
"{}: Cannot execute query in readonly mode. "
"For queries over HTTP, method GET implies readonly. "
"You should use method POST for modifying queries");
}
else
return access_denied("Cannot execute query in readonly mode", ErrorCodes::READONLY);
return access_denied(ErrorCodes::READONLY, "{}: Cannot execute query in readonly mode");
}
}
if (!params.allow_ddl && !grant_option)
{
if (flags & precalc.ddl_flags)
return access_denied("Cannot execute query. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
return access_denied(ErrorCodes::QUERY_IS_PROHIBITED,
"Cannot execute query. DDL queries are prohibited for the user {}");
}
if (!params.allow_introspection && !grant_option)
{
if (flags & precalc.introspection_flags)
return access_denied("Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
return access_denied(ErrorCodes::FUNCTION_NOT_ALLOWED, "{}: Introspection functions are disabled, "
"because setting 'allow_introspection_functions' is set to 0");
}
return access_granted();
@ -679,11 +681,13 @@ void ContextAccess::checkGrantOption(const AccessRightsElements & elements) cons
template <bool throw_if_denied, typename Container, typename GetNameFunction>
bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const GetNameFunction & get_name_function) const
{
auto show_error = [this](const String & msg, int error_code [[maybe_unused]])
auto show_error = []<typename... FmtArgs>(int error_code [[maybe_unused]],
FormatStringHelper<FmtArgs...> fmt_string [[maybe_unused]],
FmtArgs && ...fmt_args [[maybe_unused]])
{
UNUSED(this);
if constexpr (throw_if_denied)
throw Exception(getUserName() + ": " + msg, error_code);
throw Exception(error_code, std::move(fmt_string), std::forward<FmtArgs>(fmt_args)...);
return false;
};
if (is_full_access)
@ -691,7 +695,7 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const
if (user_was_dropped)
{
show_error("User has been dropped", ErrorCodes::UNKNOWN_USER);
show_error(ErrorCodes::UNKNOWN_USER, "User has been dropped");
return false;
}
@ -716,14 +720,15 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const
role_name = "ID {" + toString(role_id) + "}";
if (info->enabled_roles.count(role_id))
show_error("Not enough privileges. "
"Role " + backQuote(*role_name) + " is granted, but without ADMIN option. "
"To execute this query it's necessary to have the role " + backQuoteIfNeed(*role_name) + " granted with ADMIN option.",
ErrorCodes::ACCESS_DENIED);
show_error(ErrorCodes::ACCESS_DENIED,
"Not enough privileges. "
"Role {} is granted, but without ADMIN option. "
"To execute this query it's necessary to have the role {} granted with ADMIN option.",
backQuote(*role_name), backQuoteIfNeed(*role_name));
else
show_error("Not enough privileges. "
"To execute this query it's necessary to have the role " + backQuoteIfNeed(*role_name) + " granted with ADMIN option.",
ErrorCodes::ACCESS_DENIED);
show_error(ErrorCodes::ACCESS_DENIED, "Not enough privileges. "
"To execute this query it's necessary to have the role {} granted with ADMIN option.",
backQuoteIfNeed(*role_name));
}
return false;

View File

@ -81,7 +81,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
{
ret = krb5_cc_resolve(k5.ctx, cache_name.c_str(), &k5.out_cc);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in resolving cache{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in resolving cache: {}", fmtError(ret));
LOG_TRACE(log,"Resolved cache");
}
else
@ -89,7 +89,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
// Resolve the default cache and get its type and default principal (if it is initialized).
ret = krb5_cc_default(k5.ctx, &defcache);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while getting default cache{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while getting default cache: {}", fmtError(ret));
LOG_TRACE(log,"Resolved default cache");
deftype = krb5_cc_get_type(k5.ctx, defcache);
if (krb5_cc_get_principal(k5.ctx, defcache, &defcache_princ) != 0)
@ -99,7 +99,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
// Use the specified principal name.
ret = krb5_parse_name_flags(k5.ctx, principal.c_str(), 0, &k5.me);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error when parsing principal name {}", principal + fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error when parsing principal name ({}): {}", principal, fmtError(ret));
// Cache related commands
if (k5.out_cc == nullptr && krb5_cc_support_switch(k5.ctx, deftype))
@ -107,7 +107,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
// Use an existing cache for the client principal if we can.
ret = krb5_cc_cache_match(k5.ctx, k5.me, &k5.out_cc);
if (ret && ret != KRB5_CC_NOTFOUND)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while searching for cache for {}", principal + fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while searching for cache for ({}): {}", principal, fmtError(ret));
if (0 == ret)
{
LOG_TRACE(log,"Using default cache: {}", krb5_cc_get_name(k5.ctx, k5.out_cc));
@ -118,7 +118,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
// Create a new cache to avoid overwriting the initialized default cache.
ret = krb5_cc_new_unique(k5.ctx, deftype, nullptr, &k5.out_cc);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while generating new cache{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while generating new cache: {}", fmtError(ret));
LOG_TRACE(log,"Using default cache: {}", krb5_cc_get_name(k5.ctx, k5.out_cc));
k5.switch_to_cache = 1;
}
@ -134,24 +134,24 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
ret = krb5_unparse_name(k5.ctx, k5.me, &k5.name);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error when unparsing name{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error when unparsing name: {}", fmtError(ret));
LOG_TRACE(log,"Using principal: {}", k5.name);
// Allocate a new initial credential options structure.
ret = krb5_get_init_creds_opt_alloc(k5.ctx, &options);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in options allocation{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in options allocation: {}", fmtError(ret));
// Resolve keytab
ret = krb5_kt_resolve(k5.ctx, keytab_file.c_str(), &keytab);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in resolving keytab {}{}", keytab_file, fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in resolving keytab ({}): {}", keytab_file, fmtError(ret));
LOG_TRACE(log,"Using keytab: {}", keytab_file);
// Set an output credential cache in initial credential options.
ret = krb5_get_init_creds_opt_set_out_ccache(k5.ctx, options, k5.out_cc);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in setting output credential cache{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in setting output credential cache: {}", fmtError(ret));
// Action: init or renew
LOG_TRACE(log,"Trying to renew credentials");
@ -165,7 +165,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
// Request KDC for an initial credentials using keytab.
ret = krb5_get_init_creds_keytab(k5.ctx, &my_creds, k5.me, keytab, 0, nullptr, options);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in getting initial credentials{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error in getting initial credentials: {}", fmtError(ret));
else
LOG_TRACE(log,"Got initial credentials");
}
@ -175,7 +175,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
// Initialize a credential cache. Destroy any existing contents of cache and initialize it for the default principal.
ret = krb5_cc_initialize(k5.ctx, k5.out_cc, k5.me);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error when initializing cache{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error when initializing cache: {}", fmtError(ret));
LOG_TRACE(log,"Initialized cache");
// Store credentials in a credential cache.
ret = krb5_cc_store_cred(k5.ctx, k5.out_cc, &my_creds);
@ -189,7 +189,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
// Make a credential cache the primary cache for its collection.
ret = krb5_cc_switch(k5.ctx, k5.out_cc);
if (ret)
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while switching to new cache{}", fmtError(ret));
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while switching to new cache: {}", fmtError(ret));
}
LOG_TRACE(log,"Authenticated to Kerberos v5");

View File

@ -205,7 +205,7 @@ void LDAPClient::handleError(int result_code, String text)
}
}
throw Exception(text, ErrorCodes::LDAP_ERROR);
throw Exception::createDeprecated(text, ErrorCodes::LDAP_ERROR);
}
}
@ -569,7 +569,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params)
message += matched_msg;
}
throw Exception(message, ErrorCodes::LDAP_ERROR);
throw Exception::createDeprecated(message, ErrorCodes::LDAP_ERROR);
}
break;

View File

@ -266,7 +266,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
if (!explain.empty())
{
if (reaction == THROW_ON_VIOLATION)
throw Exception(explain, code);
throw Exception::createDeprecated(explain, code);
else
return false;
}

View File

@ -106,7 +106,7 @@ public:
default:
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Map key type " + key_type->getName() + " is not is not supported by combinator " + getName());
"Map key type {} is not is not supported by combinator {}", key_type->getName(), getName());
}
}
else

View File

@ -66,13 +66,13 @@ public:
, kind(kind_)
{
if (!isNativeNumber(arguments[0]))
throw Exception{getName() + ": first argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: first argument must be represented by integer", getName());
if (!isNativeNumber(arguments[1]))
throw Exception{getName() + ": second argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: second argument must be represented by integer", getName());
if (!arguments[0]->equals(*arguments[1]))
throw Exception{getName() + ": arguments must have the same type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: arguments must have the same type", getName());
}
String getName() const override

View File

@ -88,9 +88,9 @@ createAggregateFunctionSequenceNode(const std::string & name, const DataTypes &
name, toString(min_required_args + 1));
if (argument_types.size() > max_events_size + min_required_args)
throw Exception(fmt::format(
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.",
name, max_events_size + min_required_args, max_events_size), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
name, max_events_size + min_required_args, max_events_size);
if (const auto * cond_arg = argument_types[2].get(); cond_arg && !isUInt8(cond_arg))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of aggregate function {}, "
@ -100,9 +100,8 @@ createAggregateFunctionSequenceNode(const std::string & name, const DataTypes &
{
const auto * cond_arg = argument_types[i].get();
if (!isUInt8(cond_arg))
throw Exception(fmt::format(
"Illegal type '{}' of {} argument of aggregate function '{}', must be UInt8", cond_arg->getName(), i + 1, name),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type '{}' of {} argument of aggregate function '{}', must be UInt8", cond_arg->getName(), i + 1, name);
}
if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String)

View File

@ -235,7 +235,7 @@ private:
if (skip_degree_ == skip_degree)
return;
if (skip_degree_ > detail::MAX_SKIP_DEGREE)
throw DB::Exception{"skip_degree exceeds maximum value", DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED};
throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, "skip_degree exceeds maximum value");
skip_degree = skip_degree_;
if (skip_degree == detail::MAX_SKIP_DEGREE)
skip_mask = static_cast<UInt32>(-1);

View File

@ -1,14 +1,15 @@
#pragma once
#include <optional>
#include <utility>
#include <Common/SettingsChanges.h>
#include <base/scope_guard.h>
#include <Common/Exception.h>
#include <Core/Settings.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/UnionNode.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -89,4 +90,134 @@ private:
template <typename Derived>
using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor<Derived, true /*const_visitor*/>;
/** Same as InDepthQueryTreeVisitor and additionally keeps track of current scope context.
* This can be useful if your visitor has special logic that depends on current scope context.
*/
template <typename Derived, bool const_visitor = false>
class InDepthQueryTreeVisitorWithContext
{
public:
using VisitQueryTreeNodeType = std::conditional_t<const_visitor, const QueryTreeNodePtr, QueryTreeNodePtr>;
explicit InDepthQueryTreeVisitorWithContext(ContextPtr context)
: current_context(std::move(context))
{}
/// Return true if visitor should traverse tree top to bottom, false otherwise
bool shouldTraverseTopToBottom() const
{
return true;
}
/// Return true if visitor should visit child, false otherwise
bool needChildVisit(VisitQueryTreeNodeType & parent [[maybe_unused]], VisitQueryTreeNodeType & child [[maybe_unused]])
{
return true;
}
const ContextPtr & getContext() const
{
return current_context;
}
const Settings & getSettings() const
{
return current_context->getSettingsRef();
}
void visit(VisitQueryTreeNodeType & query_tree_node)
{
auto current_scope_context_ptr = current_context;
SCOPE_EXIT(
current_context = std::move(current_scope_context_ptr);
);
if (auto * query_node = query_tree_node->template as<QueryNode>())
current_context = query_node->getContext();
else if (auto * union_node = query_tree_node->template as<UnionNode>())
current_context = union_node->getContext();
bool traverse_top_to_bottom = getDerived().shouldTraverseTopToBottom();
if (!traverse_top_to_bottom)
visitChildren(query_tree_node);
getDerived().visitImpl(query_tree_node);
if (traverse_top_to_bottom)
visitChildren(query_tree_node);
}
private:
Derived & getDerived()
{
return *static_cast<Derived *>(this);
}
const Derived & getDerived() const
{
return *static_cast<Derived *>(this);
}
void visitChildren(VisitQueryTreeNodeType & expression)
{
for (auto & child : expression->getChildren())
{
if (!child)
continue;
bool need_visit_child = getDerived().needChildVisit(expression, child);
if (need_visit_child)
visit(child);
}
}
ContextPtr current_context;
};
template <typename Derived>
using ConstInDepthQueryTreeVisitorWithContext = InDepthQueryTreeVisitorWithContext<Derived, true /*const_visitor*/>;
/** Visitor that use another visitor to visit node only if condition for visiting node is true.
* For example, your visitor need to visit only query tree nodes or union nodes.
*
* Condition interface:
* struct Condition
* {
* bool operator()(VisitQueryTreeNodeType & node)
* {
* return shouldNestedVisitorVisitNode(node);
* }
* }
*/
template <typename Visitor, typename Condition, bool const_visitor = false>
class InDepthQueryTreeConditionalVisitor : public InDepthQueryTreeVisitor<InDepthQueryTreeConditionalVisitor<Visitor, Condition, const_visitor>, const_visitor>
{
public:
using Base = InDepthQueryTreeVisitor<InDepthQueryTreeConditionalVisitor<Visitor, Condition, const_visitor>, const_visitor>;
using VisitQueryTreeNodeType = typename Base::VisitQueryTreeNodeType;
explicit InDepthQueryTreeConditionalVisitor(Visitor & visitor_, Condition & condition_)
: visitor(visitor_)
, condition(condition_)
{
}
bool shouldTraverseTopToBottom() const
{
return visitor.shouldTraverseTopToBottom();
}
void visitImpl(VisitQueryTreeNodeType & query_tree_node)
{
if (condition(query_tree_node))
visitor.visit(query_tree_node);
}
Visitor & visitor;
Condition & condition;
};
template <typename Visitor, typename Condition>
using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor<Visitor, Condition, true /*const_visitor*/>;
}

View File

@ -45,12 +45,11 @@ Field zeroField(const Field & value)
* TODO: Support `groupBitAnd`, `groupBitOr`, `groupBitXor` functions.
* TODO: Support rewrite `f((2 * n) * n)` into '2 * f(n * n)'.
*/
class AggregateFunctionsArithmericOperationsVisitor : public InDepthQueryTreeVisitor<AggregateFunctionsArithmericOperationsVisitor>
class AggregateFunctionsArithmericOperationsVisitor : public InDepthQueryTreeVisitorWithContext<AggregateFunctionsArithmericOperationsVisitor>
{
public:
explicit AggregateFunctionsArithmericOperationsVisitor(ContextPtr context_)
: context(std::move(context_))
{}
using Base = InDepthQueryTreeVisitorWithContext<AggregateFunctionsArithmericOperationsVisitor>;
using Base::Base;
/// Traverse tree bottom to top
static bool shouldTraverseTopToBottom()
@ -60,6 +59,9 @@ public:
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions)
return;
auto * aggregate_function_node = node->as<FunctionNode>();
if (!aggregate_function_node || !aggregate_function_node->isAggregateFunction())
return;
@ -175,7 +177,7 @@ private:
inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const
{
auto function = FunctionFactory::instance().get(function_name, context);
auto function = FunctionFactory::instance().get(function_name, getContext());
function_node.resolveAsFunction(function->build(function_node.getArgumentColumns()));
}
@ -191,8 +193,6 @@ private:
function_node.resolveAsAggregateFunction(std::move(aggregate_function));
}
ContextPtr context;
};
}

View File

@ -1,17 +1,23 @@
#include <Analyzer/Passes/ConvertOrLikeChainPass.h>
#include <memory>
#include <unordered_map>
#include <vector>
#include <Analyzer/Passes/ConvertOrLikeChainPass.h>
#include <Core/Field.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/likePatternToRegexp.h>
#include <Interpreters/Context.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/UnionNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/HashUtils.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Core/Field.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/likePatternToRegexp.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -19,36 +25,28 @@ namespace DB
namespace
{
class ConvertOrLikeChainVisitor : public InDepthQueryTreeVisitor<ConvertOrLikeChainVisitor>
class ConvertOrLikeChainVisitor : public InDepthQueryTreeVisitorWithContext<ConvertOrLikeChainVisitor>
{
using FunctionNodes = std::vector<std::shared_ptr<FunctionNode>>;
const FunctionOverloadResolverPtr match_function_ref;
const FunctionOverloadResolverPtr or_function_resolver;
public:
using Base = InDepthQueryTreeVisitorWithContext<ConvertOrLikeChainVisitor>;
using Base::Base;
explicit ConvertOrLikeChainVisitor(ContextPtr context)
: InDepthQueryTreeVisitor<ConvertOrLikeChainVisitor>()
, match_function_ref(FunctionFactory::instance().get("multiMatchAny", context))
, or_function_resolver(FunctionFactory::instance().get("or", context))
explicit ConvertOrLikeChainVisitor(FunctionOverloadResolverPtr or_function_resolver_,
FunctionOverloadResolverPtr match_function_resolver_,
ContextPtr context)
: Base(std::move(context))
, or_function_resolver(std::move(or_function_resolver_))
, match_function_resolver(std::move(match_function_resolver_))
{}
static bool needChildVisit(VisitQueryTreeNodeType & parent, VisitQueryTreeNodeType &)
bool needChildVisit(VisitQueryTreeNodeType &, VisitQueryTreeNodeType &)
{
ContextPtr context;
if (auto * query = parent->as<QueryNode>())
context = query->getContext();
else if (auto * union_node = parent->as<UnionNode>())
context = union_node->getContext();
if (context)
{
const auto & settings = context->getSettingsRef();
return settings.optimize_or_like_chain
&& settings.allow_hyperscan
&& settings.max_hyperscan_regexp_length == 0
&& settings.max_hyperscan_regexp_total_length == 0;
}
return true;
const auto & settings = getSettings();
return settings.optimize_or_like_chain
&& settings.allow_hyperscan
&& settings.max_hyperscan_regexp_length == 0
&& settings.max_hyperscan_regexp_total_length == 0;
}
void visitImpl(QueryTreeNodePtr & node)
@ -61,27 +59,28 @@ public:
QueryTreeNodePtrWithHashMap<Array> node_to_patterns;
FunctionNodes match_functions;
for (auto & arg : function_node->getArguments())
{
unique_elems.push_back(arg);
auto * arg_func = arg->as<FunctionNode>();
if (!arg_func)
for (auto & argument : function_node->getArguments())
{
unique_elems.push_back(argument);
auto * argument_function = argument->as<FunctionNode>();
if (!argument_function)
continue;
const bool is_like = arg_func->getFunctionName() == "like";
const bool is_ilike = arg_func->getFunctionName() == "ilike";
const bool is_like = argument_function->getFunctionName() == "like";
const bool is_ilike = argument_function->getFunctionName() == "ilike";
/// Not {i}like -> bail out.
if (!is_like && !is_ilike)
continue;
const auto & like_arguments = arg_func->getArguments().getNodes();
const auto & like_arguments = argument_function->getArguments().getNodes();
if (like_arguments.size() != 2)
continue;
auto identifier = like_arguments[0];
auto * pattern = like_arguments[1]->as<ConstantNode>();
const auto & like_first_argument = like_arguments[0];
const auto * pattern = like_arguments[1]->as<ConstantNode>();
if (!pattern || !isString(pattern->getResultType()))
continue;
@ -91,17 +90,20 @@ public:
regexp = "(?i)" + regexp;
unique_elems.pop_back();
auto it = node_to_patterns.find(identifier);
auto it = node_to_patterns.find(like_first_argument);
if (it == node_to_patterns.end())
{
it = node_to_patterns.insert({identifier, Array{}}).first;
it = node_to_patterns.insert({like_first_argument, Array{}}).first;
/// The second argument will be added when all patterns are known.
auto match_function = std::make_shared<FunctionNode>("multiMatchAny");
match_function->getArguments().getNodes().push_back(identifier);
match_function->getArguments().getNodes().push_back(like_first_argument);
match_functions.push_back(match_function);
unique_elems.push_back(std::move(match_function));
}
it->second.push_back(regexp);
}
@ -111,23 +113,29 @@ public:
auto & arguments = match_function->getArguments().getNodes();
auto & patterns = node_to_patterns.at(arguments[0]);
arguments.push_back(std::make_shared<ConstantNode>(Field{std::move(patterns)}));
match_function->resolveAsFunction(match_function_ref);
match_function->resolveAsFunction(match_function_resolver);
}
/// OR must have at least two arguments.
if (unique_elems.size() == 1)
unique_elems.push_back(std::make_shared<ConstantNode>(false));
unique_elems.push_back(std::make_shared<ConstantNode>(static_cast<UInt8>(0)));
function_node->getArguments().getNodes() = std::move(unique_elems);
function_node->resolveAsFunction(or_function_resolver);
}
private:
using FunctionNodes = std::vector<std::shared_ptr<FunctionNode>>;
const FunctionOverloadResolverPtr or_function_resolver;
const FunctionOverloadResolverPtr match_function_resolver;
};
}
void ConvertOrLikeChainPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
void ConvertOrLikeChainPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
ConvertOrLikeChainVisitor visitor(context);
auto or_function_resolver = FunctionFactory::instance().get("or", context);
auto match_function_resolver = FunctionFactory::instance().get("multiMatchAny", context);
ConvertOrLikeChainVisitor visitor(std::move(or_function_resolver), std::move(match_function_resolver), std::move(context));
visitor.visit(query_tree_node);
}

View File

@ -16,11 +16,17 @@ namespace DB
namespace
{
class CountDistinctVisitor : public InDepthQueryTreeVisitor<CountDistinctVisitor>
class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContext<CountDistinctVisitor>
{
public:
static void visitImpl(QueryTreeNodePtr & node)
using Base = InDepthQueryTreeVisitorWithContext<CountDistinctVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().count_distinct_optimization)
return;
auto * query_node = node->as<QueryNode>();
/// Check that query has only SELECT clause
@ -78,9 +84,9 @@ public:
}
void CountDistinctPass::run(QueryTreeNodePtr query_tree_node, ContextPtr)
void CountDistinctPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
CountDistinctVisitor visitor;
CountDistinctVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}

View File

@ -16,12 +16,11 @@ namespace DB
namespace
{
class CustomizeFunctionsVisitor : public InDepthQueryTreeVisitor<CustomizeFunctionsVisitor>
class CustomizeFunctionsVisitor : public InDepthQueryTreeVisitorWithContext<CustomizeFunctionsVisitor>
{
public:
explicit CustomizeFunctionsVisitor(ContextPtr & context_)
: context(context_)
{}
using Base = InDepthQueryTreeVisitorWithContext<CustomizeFunctionsVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node) const
{
@ -29,7 +28,7 @@ public:
if (!function_node)
return;
const auto & settings = context->getSettingsRef();
const auto & settings = getSettings();
/// After successful function replacement function name and function name lowercase must be recalculated
auto function_name = function_node->getFunctionName();
@ -154,19 +153,16 @@ public:
inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const
{
auto function = FunctionFactory::instance().get(function_name, context);
auto function = FunctionFactory::instance().get(function_name, getContext());
function_node.resolveAsFunction(function->build(function_node.getArgumentColumns()));
}
private:
ContextPtr & context;
};
}
void CustomizeFunctionsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
CustomizeFunctionsVisitor visitor(context);
CustomizeFunctionsVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}

View File

@ -22,15 +22,17 @@ namespace DB
namespace
{
class FunctionToSubcolumnsVisitor : public InDepthQueryTreeVisitor<FunctionToSubcolumnsVisitor>
class FunctionToSubcolumnsVisitor : public InDepthQueryTreeVisitorWithContext<FunctionToSubcolumnsVisitor>
{
public:
explicit FunctionToSubcolumnsVisitor(ContextPtr & context_)
: context(context_)
{}
using Base = InDepthQueryTreeVisitorWithContext<FunctionToSubcolumnsVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node) const
{
if (!getSettings().optimize_functions_to_subcolumns)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node)
return;
@ -192,11 +194,9 @@ public:
private:
inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const
{
auto function = FunctionFactory::instance().get(function_name, context);
auto function = FunctionFactory::instance().get(function_name, getContext());
function_node.resolveAsFunction(function->build(function_node.getArgumentColumns()));
}
ContextPtr & context;
};
}

View File

@ -5,7 +5,7 @@
namespace DB
{
/** Transform functions to subcolumns.
/** Transform functions to subcolumns. Enabled using setting optimize_functions_to_subcolumns.
* It can help to reduce amount of read data.
*
* Example: SELECT tupleElement(column, subcolumn) FROM test_table;

View File

@ -26,16 +26,22 @@ namespace ErrorCodes
namespace
{
class FuseFunctionsVisitor : public InDepthQueryTreeVisitor<FuseFunctionsVisitor>
class FuseFunctionsVisitor : public InDepthQueryTreeVisitorWithContext<FuseFunctionsVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<FuseFunctionsVisitor>;
using Base::Base;
explicit FuseFunctionsVisitor(const std::unordered_set<String> names_to_collect_)
: names_to_collect(names_to_collect_)
explicit FuseFunctionsVisitor(const std::unordered_set<String> names_to_collect_, ContextPtr context)
: Base(std::move(context))
, names_to_collect(names_to_collect_)
{}
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_syntax_fuse_functions)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node || !function_node->isAggregateFunction() || !names_to_collect.contains(function_node->getFunctionName()))
return;
@ -201,7 +207,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector<QueryTreeNodePtr *> & nodes
void tryFuseSumCountAvg(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
FuseFunctionsVisitor visitor({"sum", "count", "avg"});
FuseFunctionsVisitor visitor({"sum", "count", "avg"}, context);
visitor.visit(query_tree_node);
for (auto & [argument, nodes] : visitor.argument_to_functions_mapping)
@ -220,7 +226,7 @@ void tryFuseSumCountAvg(QueryTreeNodePtr query_tree_node, ContextPtr context)
void tryFuseQuantiles(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
FuseFunctionsVisitor visitor_quantile({"quantile"});
FuseFunctionsVisitor visitor_quantile({"quantile"}, context);
visitor_quantile.visit(query_tree_node);
for (auto & [argument, nodes_set] : visitor_quantile.argument_to_functions_mapping)

View File

@ -12,15 +12,22 @@ namespace DB
namespace
{
class IfChainToMultiIfPassVisitor : public InDepthQueryTreeVisitor<IfChainToMultiIfPassVisitor>
class IfChainToMultiIfPassVisitor : public InDepthQueryTreeVisitorWithContext<IfChainToMultiIfPassVisitor>
{
public:
explicit IfChainToMultiIfPassVisitor(FunctionOverloadResolverPtr multi_if_function_ptr_)
: multi_if_function_ptr(std::move(multi_if_function_ptr_))
using Base = InDepthQueryTreeVisitorWithContext<IfChainToMultiIfPassVisitor>;
using Base::Base;
explicit IfChainToMultiIfPassVisitor(FunctionOverloadResolverPtr multi_if_function_ptr_, ContextPtr context)
: Base(std::move(context))
, multi_if_function_ptr(std::move(multi_if_function_ptr_))
{}
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_if_chain_to_multiif)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node || function_node->getFunctionName() != "if" || function_node->getArguments().getNodes().size() != 3)
return;
@ -68,7 +75,8 @@ private:
void IfChainToMultiIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
IfChainToMultiIfPassVisitor visitor(FunctionFactory::instance().get("multiIf", context));
auto multi_if_function_ptr = FunctionFactory::instance().get("multiIf", context);
IfChainToMultiIfPassVisitor visitor(std::move(multi_if_function_ptr), std::move(context));
visitor.visit(query_tree_node);
}

View File

@ -107,21 +107,24 @@ void wrapIntoToString(FunctionNode & function_node, QueryTreeNodePtr arg, Contex
assert(isString(function_node.getResultType()));
}
class ConvertStringsToEnumVisitor : public InDepthQueryTreeVisitor<ConvertStringsToEnumVisitor>
class ConvertStringsToEnumVisitor : public InDepthQueryTreeVisitorWithContext<ConvertStringsToEnumVisitor>
{
public:
explicit ConvertStringsToEnumVisitor(ContextPtr context_)
: context(std::move(context_))
{
}
using Base = InDepthQueryTreeVisitorWithContext<ConvertStringsToEnumVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_if_transform_strings_to_enum)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node)
return;
const auto & context = getContext();
/// to preserve return type (String) of the current function_node, we wrap the newly
/// generated function nodes into toString
@ -198,16 +201,13 @@ public:
return;
}
}
private:
ContextPtr context;
};
}
void IfTransformStringsToEnumPass::run(QueryTreeNodePtr query, ContextPtr context)
{
ConvertStringsToEnumVisitor visitor(context);
ConvertStringsToEnumVisitor visitor(std::move(context));
visitor.visit(query);
}

View File

@ -10,15 +10,22 @@ namespace DB
namespace
{
class MultiIfToIfVisitor : public InDepthQueryTreeVisitor<MultiIfToIfVisitor>
class MultiIfToIfVisitor : public InDepthQueryTreeVisitorWithContext<MultiIfToIfVisitor>
{
public:
explicit MultiIfToIfVisitor(FunctionOverloadResolverPtr if_function_ptr_)
: if_function_ptr(if_function_ptr_)
using Base = InDepthQueryTreeVisitorWithContext<MultiIfToIfVisitor>;
using Base::Base;
explicit MultiIfToIfVisitor(FunctionOverloadResolverPtr if_function_ptr_, ContextPtr context)
: Base(std::move(context))
, if_function_ptr(std::move(if_function_ptr_))
{}
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_multiif_to_if)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node || function_node->getFunctionName() != "multiIf")
return;
@ -38,7 +45,8 @@ private:
void MultiIfToIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
MultiIfToIfVisitor visitor(FunctionFactory::instance().get("if", context));
auto if_function_ptr = FunctionFactory::instance().get("if", context);
MultiIfToIfVisitor visitor(std::move(if_function_ptr), std::move(context));
visitor.visit(query_tree_node);
}

View File

@ -14,12 +14,17 @@ namespace DB
namespace
{
class NormalizeCountVariantsVisitor : public InDepthQueryTreeVisitor<NormalizeCountVariantsVisitor>
class NormalizeCountVariantsVisitor : public InDepthQueryTreeVisitorWithContext<NormalizeCountVariantsVisitor>
{
public:
explicit NormalizeCountVariantsVisitor(ContextPtr context_) : context(std::move(context_)) {}
using Base = InDepthQueryTreeVisitorWithContext<NormalizeCountVariantsVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_normalize_count_variants)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node || !function_node->isAggregateFunction() || (function_node->getFunctionName() != "count" && function_node->getFunctionName() != "sum"))
return;
@ -42,15 +47,13 @@ public:
else if (function_node->getFunctionName() == "sum" &&
first_argument_constant_literal.getType() == Field::Types::UInt64 &&
first_argument_constant_literal.get<UInt64>() == 1 &&
!context->getSettingsRef().aggregate_functions_null_for_empty)
!getSettings().aggregate_functions_null_for_empty)
{
resolveAsCountAggregateFunction(*function_node);
function_node->getArguments().getNodes().clear();
}
}
private:
ContextPtr context;
static inline void resolveAsCountAggregateFunction(FunctionNode & function_node)
{
AggregateFunctionProperties properties;

View File

@ -1,26 +1,33 @@
#include <Analyzer/Passes/OptimizeGroupByFunctionKeysPass.h>
#include <algorithm>
#include <queue>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/HashUtils.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/QueryNode.h>
#include <algorithm>
#include <queue>
namespace DB
{
class OptimizeGroupByFunctionKeysVisitor : public InDepthQueryTreeVisitor<OptimizeGroupByFunctionKeysVisitor>
class OptimizeGroupByFunctionKeysVisitor : public InDepthQueryTreeVisitorWithContext<OptimizeGroupByFunctionKeysVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<OptimizeGroupByFunctionKeysVisitor>;
using Base::Base;
static bool needChildVisit(QueryTreeNodePtr & /*parent*/, QueryTreeNodePtr & child)
{
return !child->as<FunctionNode>();
}
static void visitImpl(QueryTreeNodePtr & node)
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_group_by_function_keys)
return;
auto * query = node->as<QueryNode>();
if (!query)
return;
@ -41,11 +48,10 @@ public:
optimizeGroupingSet(group_by);
}
private:
struct NodeWithInfo
{
QueryTreeNodePtr node;
bool parents_are_only_deterministic;
bool parents_are_only_deterministic = false;
};
static bool canBeEliminated(QueryTreeNodePtr & node, const QueryTreeNodePtrWithHashSet & group_by_keys)
@ -64,7 +70,7 @@ private:
// TODO: Also process CONSTANT here. We can simplify GROUP BY x, x + 1 to GROUP BY x.
while (!candidates.empty())
{
auto [candidate, deterministic_context] = candidates.back();
auto [candidate, parents_are_only_deterministic] = candidates.back();
candidates.pop_back();
bool found = group_by_keys.contains(candidate);
@ -80,7 +86,7 @@ private:
if (!found)
{
bool is_deterministic_function = deterministic_context && function->getFunction()->isDeterministicInScopeOfQuery();
bool is_deterministic_function = parents_are_only_deterministic && function->getFunction()->isDeterministicInScopeOfQuery();
for (auto it = arguments.rbegin(); it != arguments.rend(); ++it)
candidates.push_back({ *it, is_deterministic_function });
}
@ -91,7 +97,7 @@ private:
return false;
break;
case QueryTreeNodeType::CONSTANT:
if (!deterministic_context)
if (!parents_are_only_deterministic)
return false;
break;
default:
@ -117,9 +123,10 @@ private:
}
};
void OptimizeGroupByFunctionKeysPass::run(QueryTreeNodePtr query_tree_node, ContextPtr /*context*/)
void OptimizeGroupByFunctionKeysPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
OptimizeGroupByFunctionKeysVisitor().visit(query_tree_node);
OptimizeGroupByFunctionKeysVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}
}

View File

@ -1,11 +1,13 @@
#include <Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.h>
#include <Functions/IFunction.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/HashUtils.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/SortNode.h>
#include <Functions/IFunction.h>
namespace DB
{
@ -13,9 +15,12 @@ namespace DB
namespace
{
class OptimizeRedundantFunctionsInOrderByVisitor : public InDepthQueryTreeVisitor<OptimizeRedundantFunctionsInOrderByVisitor>
class OptimizeRedundantFunctionsInOrderByVisitor : public InDepthQueryTreeVisitorWithContext<OptimizeRedundantFunctionsInOrderByVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<OptimizeRedundantFunctionsInOrderByVisitor>;
using Base::Base;
static bool needChildVisit(QueryTreeNodePtr & node, QueryTreeNodePtr & /*parent*/)
{
if (node->as<FunctionNode>())
@ -25,6 +30,9 @@ public:
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_redundant_functions_in_order_by)
return;
auto * query = node->as<QueryNode>();
if (!query)
return;
@ -116,9 +124,10 @@ private:
}
void OptimizeRedundantFunctionsInOrderByPass::run(QueryTreeNodePtr query_tree_node, ContextPtr /*context*/)
void OptimizeRedundantFunctionsInOrderByPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
OptimizeRedundantFunctionsInOrderByVisitor().visit(query_tree_node);
OptimizeRedundantFunctionsInOrderByVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}
}

View File

@ -1943,7 +1943,7 @@ void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & ta
if (!table_node && !table_function_node && !query_node && !union_node)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected table expression. Expected table, table function, query or union node. Actual {}",
"Unexpected table expression. Expected table, table function, query or union node. Table node: {}, scope node: {}",
table_expression_node->formatASTForErrorMessage(),
scope.scope_node->formatASTForErrorMessage());
@ -4366,12 +4366,9 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
{
if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name))
{
std::string error_message = fmt::format("Aggregate function with name '{}' does not exists. In scope {}",
function_name,
scope.scope_node->formatASTForErrorMessage());
AggregateFunctionFactory::instance().appendHintsMessage(error_message, function_name);
throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, error_message);
throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, "Aggregate function with name '{}' does not exists. In scope {}{}",
function_name, scope.scope_node->formatASTForErrorMessage(),
getHintsErrorMessageSuffix(AggregateFunctionFactory::instance().getHints(function_name)));
}
if (!function_lambda_arguments_indexes.empty())
@ -5726,7 +5723,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node,
case QueryTreeNodeType::IDENTIFIER:
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Identifiers in FROM section must be already resolved. In scope {}",
"Identifiers in FROM section must be already resolved. Node {}, scope {}",
join_tree_node->formatASTForErrorMessage(),
scope.scope_node->formatASTForErrorMessage());
}

View File

@ -20,15 +20,17 @@ namespace DB
namespace
{
class SumIfToCountIfVisitor : public InDepthQueryTreeVisitor<SumIfToCountIfVisitor>
class SumIfToCountIfVisitor : public InDepthQueryTreeVisitorWithContext<SumIfToCountIfVisitor>
{
public:
explicit SumIfToCountIfVisitor(ContextPtr & context_)
: context(context_)
{}
using Base = InDepthQueryTreeVisitorWithContext<SumIfToCountIfVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_rewrite_sum_if_to_count_if)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node || !function_node->isAggregateFunction())
return;
@ -56,7 +58,7 @@ public:
if (!isInt64OrUInt64FieldType(constant_value_literal.getType()))
return;
if (constant_value_literal.get<UInt64>() != 1 || context->getSettingsRef().aggregate_functions_null_for_empty)
if (constant_value_literal.get<UInt64>() != 1 || getSettings().aggregate_functions_null_for_empty)
return;
function_node_arguments_nodes[0] = std::move(function_node_arguments_nodes[1]);
@ -122,7 +124,7 @@ public:
auto & not_function_arguments = not_function->getArguments().getNodes();
not_function_arguments.push_back(nested_if_function_arguments_nodes[0]);
not_function->resolveAsFunction(FunctionFactory::instance().get("not", context)->build(not_function->getArgumentColumns()));
not_function->resolveAsFunction(FunctionFactory::instance().get("not", getContext())->build(not_function->getArgumentColumns()));
function_node_arguments_nodes[0] = std::move(not_function);
function_node_arguments_nodes.resize(1);
@ -143,8 +145,6 @@ private:
function_node.resolveAsAggregateFunction(std::move(aggregate_function));
}
ContextPtr & context;
};
}

View File

@ -25,11 +25,17 @@ bool isUniqFunction(const String & function_name)
function_name == "uniqTheta";
}
class UniqInjectiveFunctionsEliminationVisitor : public InDepthQueryTreeVisitor<UniqInjectiveFunctionsEliminationVisitor>
class UniqInjectiveFunctionsEliminationVisitor : public InDepthQueryTreeVisitorWithContext<UniqInjectiveFunctionsEliminationVisitor>
{
public:
static void visitImpl(QueryTreeNodePtr & node)
using Base = InDepthQueryTreeVisitorWithContext<UniqInjectiveFunctionsEliminationVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_injective_functions_inside_uniq)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node || !function_node->isAggregateFunction() || !isUniqFunction(function_node->getFunctionName()))
return;
@ -81,9 +87,9 @@ public:
}
void UniqInjectiveFunctionsEliminationPass::run(QueryTreeNodePtr query_tree_node, ContextPtr)
void UniqInjectiveFunctionsEliminationPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
UniqInjectiveFunctionsEliminationVisitor visitor;
UniqInjectiveFunctionsEliminationVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}

View File

@ -1,5 +1,7 @@
#include <Analyzer/QueryNode.h>
#include <fmt/core.h>
#include <Common/SipHash.h>
#include <Common/FieldVisitorToString.h>
@ -17,7 +19,6 @@
#include <Parsers/ASTSetQuery.h>
#include <Analyzer/Utils.h>
#include <fmt/core.h>
namespace DB
{
@ -36,7 +37,7 @@ QueryNode::QueryNode(ContextMutablePtr context_, SettingsChanges settings_change
}
QueryNode::QueryNode(ContextMutablePtr context_)
: QueryNode(context_, {} /*settings_changes*/)
: QueryNode(std::move(context_), {} /*settings_changes*/)
{}
void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const
@ -185,10 +186,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
{
buffer << '\n' << std::string(indent + 2, ' ') << "SETTINGS";
for (const auto & change : settings_changes)
{
buffer << fmt::format(" {}={}", change.name, toString(change.value));
}
buffer << '\n';
}
}

View File

@ -1,6 +1,7 @@
#include <memory>
#include <Analyzer/QueryTreePassManager.h>
#include <memory>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
@ -133,7 +134,6 @@ private:
* TODO: Support setting optimize_aggregators_of_group_by_keys.
* TODO: Support setting optimize_duplicate_order_by_and_distinct.
* TODO: Support setting optimize_monotonous_functions_in_order_by.
* TODO: Support settings.optimize_or_like_chain.
* TODO: Add optimizations based on function semantics. Example: SELECT * FROM test_table WHERE id != id. (id is not nullable column).
*/
@ -210,53 +210,31 @@ void QueryTreePassManager::dump(WriteBuffer & buffer, size_t up_to_pass_index)
void addQueryTreePasses(QueryTreePassManager & manager)
{
auto context = manager.getContext();
const auto & settings = context->getSettingsRef();
manager.addPass(std::make_unique<QueryAnalysisPass>());
manager.addPass(std::make_unique<FunctionToSubcolumnsPass>());
if (settings.optimize_functions_to_subcolumns)
manager.addPass(std::make_unique<FunctionToSubcolumnsPass>());
if (settings.count_distinct_optimization)
manager.addPass(std::make_unique<CountDistinctPass>());
if (settings.optimize_rewrite_sum_if_to_count_if)
manager.addPass(std::make_unique<SumIfToCountIfPass>());
if (settings.optimize_normalize_count_variants)
manager.addPass(std::make_unique<NormalizeCountVariantsPass>());
manager.addPass(std::make_unique<CountDistinctPass>());
manager.addPass(std::make_unique<SumIfToCountIfPass>());
manager.addPass(std::make_unique<NormalizeCountVariantsPass>());
manager.addPass(std::make_unique<CustomizeFunctionsPass>());
if (settings.optimize_arithmetic_operations_in_aggregate_functions)
manager.addPass(std::make_unique<AggregateFunctionsArithmericOperationsPass>());
if (settings.optimize_injective_functions_inside_uniq)
manager.addPass(std::make_unique<UniqInjectiveFunctionsEliminationPass>());
if (settings.optimize_group_by_function_keys)
manager.addPass(std::make_unique<OptimizeGroupByFunctionKeysPass>());
if (settings.optimize_multiif_to_if)
manager.addPass(std::make_unique<MultiIfToIfPass>());
manager.addPass(std::make_unique<AggregateFunctionsArithmericOperationsPass>());
manager.addPass(std::make_unique<UniqInjectiveFunctionsEliminationPass>());
manager.addPass(std::make_unique<OptimizeGroupByFunctionKeysPass>());
manager.addPass(std::make_unique<MultiIfToIfPass>());
manager.addPass(std::make_unique<IfConstantConditionPass>());
manager.addPass(std::make_unique<IfChainToMultiIfPass>());
if (settings.optimize_if_chain_to_multiif)
manager.addPass(std::make_unique<IfChainToMultiIfPass>());
if (settings.optimize_redundant_functions_in_order_by)
manager.addPass(std::make_unique<OptimizeRedundantFunctionsInOrderByPass>());
manager.addPass(std::make_unique<OptimizeRedundantFunctionsInOrderByPass>());
manager.addPass(std::make_unique<OrderByTupleEliminationPass>());
manager.addPass(std::make_unique<OrderByLimitByDuplicateEliminationPass>());
if (settings.optimize_syntax_fuse_functions)
manager.addPass(std::make_unique<FuseFunctionsPass>());
manager.addPass(std::make_unique<FuseFunctionsPass>());
if (settings.optimize_if_transform_strings_to_enum)
manager.addPass(std::make_unique<IfTransformStringsToEnumPass>());
manager.addPass(std::make_unique<IfTransformStringsToEnumPass>());
manager.addPass(std::make_unique<ConvertOrLikeChainPass>());

View File

@ -79,7 +79,7 @@ namespace
request.SetMaxKeys(1);
auto outcome = client.ListObjects(request);
if (!outcome.IsSuccess())
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
return outcome.GetResult().GetContents();
}
@ -233,7 +233,7 @@ void BackupWriterS3::removeFile(const String & file_name)
request.SetKey(fs::path(s3_uri.key) / file_name);
auto outcome = client->DeleteObject(request);
if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType()))
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
}
void BackupWriterS3::removeFiles(const Strings & file_names)
@ -291,7 +291,7 @@ void BackupWriterS3::removeFilesBatch(const Strings & file_names)
auto outcome = client->DeleteObjects(request);
if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType()))
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
}
}

View File

@ -93,7 +93,7 @@ namespace
catch (...)
{
if (coordination)
coordination->setError(current_host, Exception{getCurrentExceptionCode(), getCurrentExceptionMessage(true, true)});
coordination->setError(current_host, Exception(getCurrentExceptionMessageAndPattern(true, true), getCurrentExceptionCode()));
}
}

View File

@ -451,7 +451,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query)
catch (const Exception &)
{
/// Catch client errors like NO_ROW_DELIMITER
throw LocalFormatError(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode());
throw LocalFormatError(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode());
}
/// Received data block is immediately displayed to the user.
@ -629,7 +629,7 @@ try
}
catch (...)
{
throw LocalFormatError(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode());
throw LocalFormatError(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode());
}
@ -1897,7 +1897,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
{
// Surprisingly, this is a client error. A server error would
// have been reported without throwing (see onReceiveSeverException()).
client_exception = std::make_unique<Exception>(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode());
client_exception = std::make_unique<Exception>(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode());
have_error = true;
}

View File

@ -187,7 +187,7 @@ void LocalConnection::sendQuery(
catch (...)
{
state->io.onException();
state->exception = std::make_unique<Exception>("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
state->exception = std::make_unique<Exception>(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception");
}
}
@ -291,7 +291,7 @@ bool LocalConnection::poll(size_t)
catch (...)
{
state->io.onException();
state->exception = std::make_unique<Exception>("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
state->exception = std::make_unique<Exception>(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception");
}
}

View File

@ -83,7 +83,7 @@ template <is_decimal T>
UInt64 ColumnDecimal<T>::get64([[maybe_unused]] size_t n) const
{
if constexpr (sizeof(T) > sizeof(UInt64))
throw Exception(String("Method get64 is not supported for ") + getFamilyName(), ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method get64 is not supported for {}", getFamilyName());
else
return static_cast<NativeT>(data[n]);
}

View File

@ -35,7 +35,7 @@ ColumnNullable::ColumnNullable(MutableColumnPtr && nested_column_, MutableColumn
nested_column = getNestedColumn().convertToFullColumnIfConst();
if (!getNestedColumn().canBeInsideNullable())
throw Exception{getNestedColumn().getName() + " cannot be inside Nullable column", ErrorCodes::ILLEGAL_COLUMN};
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "{} cannot be inside Nullable column", getNestedColumn().getName());
if (isColumnConst(*null_map))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnNullable cannot have constant null map");

View File

@ -207,8 +207,9 @@ private:
if (size >= MMAP_THRESHOLD)
{
if (alignment > mmap_min_alignment)
throw DB::Exception(fmt::format("Too large alignment {}: more than page size when allocating {}.",
ReadableSize(alignment), ReadableSize(size)), DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS,
"Too large alignment {}: more than page size when allocating {}.",
ReadableSize(alignment), ReadableSize(size));
buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE,
mmap_flags, -1, 0);

View File

@ -140,7 +140,7 @@ void CancelToken::raise()
{
std::unique_lock lock(signal_mutex);
if (exception_code != 0)
throw DB::Exception(
throw DB::Exception::createRuntime(
std::exchange(exception_code, 0),
std::exchange(exception_message, {}));
else

View File

@ -88,7 +88,7 @@ public:
{
/// A more understandable error message.
if (e.code() == DB::ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == DB::ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throw DB::ParsingException("File " + path + " is empty. You must fill it manually with appropriate value.", e.code());
throw DB::ParsingException(e.code(), "File {} is empty. You must fill it manually with appropriate value.", path);
else
throw;
}

View File

@ -252,7 +252,7 @@ uint64_t readOffset(std::string_view & sp, bool is64_bit)
// Read "len" bytes
std::string_view readBytes(std::string_view & sp, uint64_t len)
{
SAFE_CHECK(len <= sp.size(), "invalid string length: " + std::to_string(len) + " vs. " + std::to_string(sp.size()));
SAFE_CHECK(len <= sp.size(), "invalid string length: {} vs. {}", len, sp.size());
std::string_view ret(sp.data(), len);
sp.remove_prefix(len);
return ret;
@ -953,7 +953,7 @@ bool Dwarf::findDebugInfoOffset(uintptr_t address, std::string_view aranges, uin
Dwarf::Die Dwarf::getDieAtOffset(const CompilationUnit & cu, uint64_t offset) const
{
SAFE_CHECK(offset < info_.size(), fmt::format("unexpected offset {}, info size {}", offset, info_.size()));
SAFE_CHECK(offset < info_.size(), "unexpected offset {}, info size {}", offset, info_.size());
Die die;
std::string_view sp{info_.data() + offset, cu.offset + cu.size - offset};
die.offset = offset;

View File

@ -188,7 +188,7 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string
{
PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
if (!start_of_message.empty())
message.message = fmt::format("{}: {}", start_of_message, message.message);
message.text = fmt::format("{}: {}", start_of_message, message.text);
LOG_ERROR(logger, message);
}
@ -339,7 +339,7 @@ std::string getExtraExceptionInfo(const std::exception & e)
std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/)
{
return getCurrentExceptionMessageAndPattern(with_stacktrace, check_embedded_stacktrace, with_extra_info).message;
return getCurrentExceptionMessageAndPattern(with_stacktrace, check_embedded_stacktrace, with_extra_info).text;
}
PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/)
@ -481,7 +481,7 @@ void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::str
std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
{
return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).message;
return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).text;
}
PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
@ -577,10 +577,6 @@ ParsingException::ParsingException(const std::string & msg, int code)
: Exception(msg, code)
{
}
ParsingException::ParsingException(int code, const std::string & message)
: Exception(message, code)
{
}
/// We use additional field formatted_message_ to make this method const.
std::string ParsingException::displayText() const

View File

@ -16,26 +16,6 @@
namespace Poco { class Logger; }
/// Extract format string from a string literal and constructs consteval fmt::format_string
template <typename... Args>
struct FormatStringHelperImpl
{
std::string_view message_format_string;
fmt::format_string<Args...> fmt_str;
template<typename T>
consteval FormatStringHelperImpl(T && str) : message_format_string(tryGetStaticFormatString(str)), fmt_str(std::forward<T>(str)) {}
template<typename T>
FormatStringHelperImpl(fmt::basic_runtime<T> && str) : message_format_string(), fmt_str(std::forward<fmt::basic_runtime<T>>(str)) {}
PreformattedMessage format(Args && ...args) const
{
return PreformattedMessage{fmt::format(fmt_str, std::forward<Args...>(args)...), message_format_string};
}
};
template <typename... Args>
using FormatStringHelper = FormatStringHelperImpl<std::type_identity_t<Args>...>;
namespace DB
{
@ -48,6 +28,17 @@ public:
Exception() = default;
Exception(const PreformattedMessage & msg, int code): Exception(msg.text, code)
{
message_format_string = msg.format_string;
}
Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.text), code)
{
message_format_string = msg.format_string;
}
protected:
// used to remove the sensitive information from exceptions if query_masking_rules is configured
struct MessageMasked
{
@ -62,11 +53,16 @@ public:
// delegating constructor to mask sensitive information from the message
Exception(const std::string & msg, int code, bool remote_ = false): Exception(MessageMasked(msg), code, remote_) {}
Exception(std::string && msg, int code, bool remote_ = false): Exception(MessageMasked(std::move(msg)), code, remote_) {}
Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.message), code)
public:
/// This creator is for exceptions that should format a message using fmt::format from the variadic ctor Exception(code, fmt, ...),
/// but were not rewritten yet. It will be removed.
static Exception createDeprecated(const std::string & msg, int code, bool remote_ = false)
{
message_format_string = msg.format_string;
return Exception(msg, code, remote_);
}
/// Message must be a compile-time constant
template<typename T, typename = std::enable_if_t<std::is_convertible_v<T, String>>>
Exception(int code, T && message)
: Exception(message, code)
@ -74,9 +70,11 @@ public:
message_format_string = tryGetStaticFormatString(message);
}
template<> Exception(int code, const String & message) : Exception(message, code) {}
template<> Exception(int code, String & message) : Exception(message, code) {}
template<> Exception(int code, String && message) : Exception(std::move(message), code) {}
/// These creators are for messages that were received by network or generated by a third-party library in runtime.
/// Please use a constructor for all other cases.
static Exception createRuntime(int code, const String & message) { return Exception(message, code); }
static Exception createRuntime(int code, String & message) { return Exception(message, code); }
static Exception createRuntime(int code, String && message) { return Exception(std::move(message), code); }
// Format message with fmt::format, like the logging functions.
template <typename... Args>
@ -167,11 +165,9 @@ private:
/// more convenient calculation of problem line number.
class ParsingException : public Exception
{
ParsingException(const std::string & msg, int code);
public:
ParsingException();
ParsingException(const std::string & msg, int code);
ParsingException(int code, const std::string & message);
ParsingException(int code, std::string && message) : Exception(message, code) {}
// Format message with fmt::format, like the logging functions.
template <typename... Args>

View File

@ -0,0 +1,7 @@
#include <Common/LoggingFormatStringHelpers.h>
[[noreturn]] void functionThatFailsCompilationOfConstevalFunctions(const char * error)
{
throw std::runtime_error(error);
}

View File

@ -2,24 +2,70 @@
#include <base/defines.h>
#include <fmt/format.h>
struct PreformattedMessage;
consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs);
template <typename T> constexpr std::string_view tryGetStaticFormatString(T && x);
/// Extract format string from a string literal and constructs consteval fmt::format_string
template <typename... Args>
struct FormatStringHelperImpl
{
std::string_view message_format_string;
fmt::format_string<Args...> fmt_str;
template<typename T>
consteval FormatStringHelperImpl(T && str) : message_format_string(tryGetStaticFormatString(str)), fmt_str(std::forward<T>(str))
{
formatStringCheckArgsNumImpl(message_format_string, sizeof...(Args));
}
template<typename T>
FormatStringHelperImpl(fmt::basic_runtime<T> && str) : message_format_string(), fmt_str(std::forward<fmt::basic_runtime<T>>(str)) {}
PreformattedMessage format(Args && ...args) const;
};
template <typename... Args>
using FormatStringHelper = FormatStringHelperImpl<std::type_identity_t<Args>...>;
/// Saves a format string for already formatted message
struct PreformattedMessage
{
String message;
std::string text;
std::string_view format_string;
operator const String & () const { return message; }
operator String () && { return std::move(message); }
template <typename... Args>
static PreformattedMessage create(FormatStringHelper<Args...> fmt, Args &&... args);
operator const std::string & () const { return text; }
operator std::string () && { return std::move(text); }
operator fmt::format_string<> () const { UNREACHABLE(); }
};
template <typename... Args>
PreformattedMessage FormatStringHelperImpl<Args...>::format(Args && ...args) const
{
return PreformattedMessage{fmt::format(fmt_str, std::forward<Args>(args)...), message_format_string};
}
template <typename... Args>
PreformattedMessage PreformattedMessage::create(FormatStringHelper<Args...> fmt, Args && ...args)
{
return fmt.format(std::forward<Args>(args)...);
}
template<typename T> struct is_fmt_runtime : std::false_type {};
template<typename T> struct is_fmt_runtime<fmt::basic_runtime<T>> : std::true_type {};
template <typename T> constexpr std::string_view tryGetStaticFormatString(T && x)
{
/// Failure of this asserting indicates that something went wrong during type deduction.
/// For example, a string literal was implicitly converted to std::string. It should not happen.
/// Format string for an exception or log message must be a string literal (compile-time constant).
/// Failure of this assertion may indicate one of the following issues:
/// - A message was already formatted into std::string before passing to Exception(...) or LOG_XXXXX(...).
/// Please use variadic constructor of Exception.
/// Consider using PreformattedMessage or LogToStr if you want to avoid double formatting and/or copy-paste.
/// - A string literal was converted to std::string (or const char *).
/// - Use Exception::createRuntime or fmt::runtime if there's no format string
/// and a message is generated in runtime by a third-party library
/// or deserialized from somewhere.
static_assert(!std::is_same_v<std::string, std::decay_t<T>>);
if constexpr (is_fmt_runtime<std::decay_t<T>>::value)
@ -53,3 +99,60 @@ template <typename T, typename... Ts> constexpr auto firstArg(T && x, Ts &&...)
/// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor
template <typename T, typename... Ts> constexpr auto firstArg(fmt::basic_runtime<T> && data, Ts &&...) { return data.str.data(); }
consteval ssize_t formatStringCountArgsNum(const char * const str, size_t len)
{
/// It does not count named args, but we don't use them
size_t cnt = 0;
size_t i = 0;
while (i + 1 < len)
{
if (str[i] == '{' && str[i + 1] == '}')
{
i += 2;
cnt += 1;
}
else if (str[i] == '{')
{
/// Ignore checks for complex formatting like "{:.3f}"
return -1;
}
else
{
i += 1;
}
}
return cnt;
}
[[noreturn]] void functionThatFailsCompilationOfConstevalFunctions(const char * error);
/// fmt::format checks that there are enough arguments, but ignores extra arguments (e.g. fmt::format("{}", 1, 2) compiles)
/// This function will fail to compile if the number of "{}" substitutions does not exactly match
consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs)
{
if (str.empty())
return;
ssize_t cnt = formatStringCountArgsNum(str.data(), str.size());
if (0 <= cnt && cnt != nargs)
functionThatFailsCompilationOfConstevalFunctions("unexpected number of arguments in a format string");
}
template <typename... Args>
struct CheckArgsNumHelperImpl
{
template<typename T>
consteval CheckArgsNumHelperImpl(T && str)
{
formatStringCheckArgsNumImpl(tryGetStaticFormatString(str), sizeof...(Args));
}
/// No checks for fmt::runtime and PreformattedMessage
template<typename T> CheckArgsNumHelperImpl(fmt::basic_runtime<T> &&) {}
template<> CheckArgsNumHelperImpl(PreformattedMessage &) {}
template<> CheckArgsNumHelperImpl(const PreformattedMessage &) {}
template<> CheckArgsNumHelperImpl(PreformattedMessage &&) {}
};
template <typename... Args> using CheckArgsNumHelper = CheckArgsNumHelperImpl<std::type_identity_t<Args>...>;
template <typename... Args> void formatStringCheckArgsNum(CheckArgsNumHelper<Args...>, Args &&...) {}

View File

@ -187,7 +187,7 @@ public:
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Value with key `{}` is used twice in the SET query",
"Value with key `{}` is used twice in the SET query (collection name: {})",
name, query.collection_name);
}
}

View File

@ -211,9 +211,8 @@ PoolWithFailoverBase<TNestedPool>::get(size_t max_ignored_errors, bool fallback_
max_ignored_errors, fallback_to_stale_replicas,
try_get_entry, get_priority);
if (results.empty() || results[0].entry.isNull())
throw DB::Exception(
"PoolWithFailoverBase::getMany() returned less than min_entries entries.",
DB::ErrorCodes::LOGICAL_ERROR);
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR,
"PoolWithFailoverBase::getMany() returned less than min_entries entries.");
return results[0].entry;
}
@ -320,10 +319,8 @@ PoolWithFailoverBase<TNestedPool>::getMany(
try_results.resize(up_to_date_count);
}
else
throw DB::Exception(
"Could not find enough connections to up-to-date replicas. Got: " + std::to_string(up_to_date_count)
+ ", needed: " + std::to_string(min_entries),
DB::ErrorCodes::ALL_REPLICAS_ARE_STALE);
throw DB::Exception(DB::ErrorCodes::ALL_REPLICAS_ARE_STALE,
"Could not find enough connections to up-to-date replicas. Got: {}, needed: {}", up_to_date_count, max_entries);
return try_results;
}

View File

@ -62,10 +62,10 @@ public:
, replacement(replacement_string)
{
if (!regexp.ok())
throw DB::Exception(
"SensitiveDataMasker: cannot compile re2: " + regexp_string_ + ", error: " + regexp.error()
+ ". Look at https://github.com/google/re2/wiki/Syntax for reference.",
DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
throw DB::Exception(DB::ErrorCodes::CANNOT_COMPILE_REGEXP,
"SensitiveDataMasker: cannot compile re2: {}, error: {}. "
"Look at https://github.com/google/re2/wiki/Syntax for reference.",
regexp_string_, regexp.error());
}
uint64_t apply(std::string & data) const

View File

@ -100,7 +100,7 @@ size_t TLDListsHolder::parseAndAddTldList(const std::string & name, const std::s
tld_list_tmp.emplace(line, TLDType::TLD_REGULAR);
}
if (!in.eof())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not all list had been read", name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not all list had been read: {}", name);
TLDList tld_list(tld_list_tmp.size());
for (const auto & [host, type] : tld_list_tmp)

View File

@ -58,7 +58,7 @@ UInt64 Throttler::add(size_t amount)
}
if (limit && count_value > limit)
throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED);
throw Exception::createDeprecated(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED);
/// Wait unless there is positive amount of tokens - throttling
Int64 sleep_time = 0;

View File

@ -41,7 +41,7 @@ To assert_cast(From && from)
}
catch (const std::exception & e)
{
throw DB::Exception(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
}
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Bad cast from type {} to {}",

View File

@ -57,6 +57,7 @@ namespace
if (_is_clients_log || _logger->is((PRIORITY))) \
{ \
std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \
formatStringCheckArgsNum(__VA_ARGS__); \
if (auto _channel = _logger->getChannel()) \
{ \
std::string file_function; \

View File

@ -171,9 +171,8 @@ TEST(Common, RWLockDeadlock)
auto holder2 = lock2->getLock(RWLockImpl::Read, "q1", std::chrono::milliseconds(100));
if (!holder2)
{
throw Exception(
"Locking attempt timed out! Possible deadlock avoided. Client should retry.",
ErrorCodes::DEADLOCK_AVOIDED);
throw Exception(ErrorCodes::DEADLOCK_AVOIDED,
"Locking attempt timed out! Possible deadlock avoided. Client should retry.");
}
}
catch (const Exception & e)
@ -202,9 +201,8 @@ TEST(Common, RWLockDeadlock)
auto holder1 = lock1->getLock(RWLockImpl::Read, "q3", std::chrono::milliseconds(100));
if (!holder1)
{
throw Exception(
"Locking attempt timed out! Possible deadlock avoided. Client should retry.",
ErrorCodes::DEADLOCK_AVOIDED);
throw Exception(ErrorCodes::DEADLOCK_AVOIDED,
"Locking attempt timed out! Possible deadlock avoided. Client should retry.");
}
}
catch (const Exception & e)

View File

@ -37,7 +37,7 @@ To typeid_cast(From & from)
}
catch (const std::exception & e)
{
throw DB::Exception(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
}
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Bad cast from type {} to {}",
@ -58,7 +58,7 @@ To typeid_cast(From * from)
}
catch (const std::exception & e)
{
throw DB::Exception(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
}
}
@ -93,6 +93,6 @@ To typeid_cast(const std::shared_ptr<From> & from)
}
catch (const std::exception & e)
{
throw DB::Exception(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -86,7 +86,7 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
{
message << ". The mismatch is caused by single bit flip in data block at byte " << (bit_pos / 8) << ", bit " << (bit_pos % 8) << ". "
<< message_hardware_failure;
throw Exception(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
}
flip_bit(tmp_data, bit_pos); /// Restore
@ -101,10 +101,10 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
{
message << ". The mismatch is caused by single bit flip in checksum. "
<< message_hardware_failure;
throw Exception(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
}
throw Exception(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
}
static void readHeaderAndGetCodecAndSize(

View File

@ -141,7 +141,7 @@ size_t encrypt(std::string_view plaintext, char * ciphertext_and_tag, Encryption
reinterpret_cast<const uint8_t*>(key.data()), key.size(),
tag_size, nullptr);
if (!ok_init)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
throw Exception::createDeprecated(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
/// encrypt data using context and given nonce.
size_t out_len;
@ -152,7 +152,7 @@ size_t encrypt(std::string_view plaintext, char * ciphertext_and_tag, Encryption
reinterpret_cast<const uint8_t *>(plaintext.data()), plaintext.size(),
nullptr, 0);
if (!ok_open)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
throw Exception::createDeprecated(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
return out_len;
}
@ -171,7 +171,7 @@ size_t decrypt(std::string_view ciphertext, char * plaintext, EncryptionMethod m
reinterpret_cast<const uint8_t*>(key.data()), key.size(),
tag_size, nullptr);
if (!ok_init)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
throw Exception::createDeprecated(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
/// decrypt data using given nonce
size_t out_len;
@ -182,7 +182,7 @@ size_t decrypt(std::string_view ciphertext, char * plaintext, EncryptionMethod m
reinterpret_cast<const uint8_t *>(ciphertext.data()), ciphertext.size(),
nullptr, 0);
if (!ok_open)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
throw Exception::createDeprecated(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
return out_len;
}

View File

@ -29,11 +29,13 @@ namespace ErrorCodes
extern const int AMBIGUOUS_COLUMN_NAME;
}
template <typename ReturnType>
static ReturnType onError(const std::string & message [[maybe_unused]], int code [[maybe_unused]])
template <typename ReturnType, typename... FmtArgs>
static ReturnType onError(int code [[maybe_unused]],
FormatStringHelper<FmtArgs...> fmt_string [[maybe_unused]],
FmtArgs && ...fmt_args [[maybe_unused]])
{
if constexpr (std::is_same_v<ReturnType, void>)
throw Exception(message, code);
throw Exception(code, std::move(fmt_string), std::forward<FmtArgs>(fmt_args)...);
else
return false;
}
@ -44,13 +46,13 @@ static ReturnType checkColumnStructure(const ColumnWithTypeAndName & actual, con
std::string_view context_description, bool allow_materialize, int code)
{
if (actual.name != expected.name)
return onError<ReturnType>("Block structure mismatch in " + std::string(context_description) + " stream: different names of columns:\n"
+ actual.dumpStructure() + "\n" + expected.dumpStructure(), code);
return onError<ReturnType>(code, "Block structure mismatch in {} stream: different names of columns:\n{}\n{}",
context_description, actual.dumpStructure(), expected.dumpStructure());
if ((actual.type && !expected.type) || (!actual.type && expected.type)
|| (actual.type && expected.type && !actual.type->equals(*expected.type)))
return onError<ReturnType>("Block structure mismatch in " + std::string(context_description) + " stream: different types:\n"
+ actual.dumpStructure() + "\n" + expected.dumpStructure(), code);
return onError<ReturnType>(code, "Block structure mismatch in {} stream: different types:\n{}\n{}",
context_description, actual.dumpStructure(), expected.dumpStructure());
if (!actual.column || !expected.column)
return ReturnType(true);
@ -74,22 +76,18 @@ static ReturnType checkColumnStructure(const ColumnWithTypeAndName & actual, con
if (actual_column_maybe_agg && expected_column_maybe_agg)
{
if (!actual_column_maybe_agg->getAggregateFunction()->haveSameStateRepresentation(*expected_column_maybe_agg->getAggregateFunction()))
return onError<ReturnType>(
fmt::format(
return onError<ReturnType>(code,
"Block structure mismatch in {} stream: different columns:\n{}\n{}",
context_description,
actual.dumpStructure(),
expected.dumpStructure()),
code);
expected.dumpStructure());
}
else if (actual_column->getName() != expected.column->getName())
return onError<ReturnType>(
fmt::format(
return onError<ReturnType>(code,
"Block structure mismatch in {} stream: different columns:\n{}\n{}",
context_description,
actual.dumpStructure(),
expected.dumpStructure()),
code);
expected.dumpStructure());
if (isColumnConst(*actual.column) && isColumnConst(*expected.column)
&& !actual.column->empty() && !expected.column->empty()) /// don't check values in empty columns
@ -98,14 +96,12 @@ static ReturnType checkColumnStructure(const ColumnWithTypeAndName & actual, con
Field expected_value = assert_cast<const ColumnConst &>(*expected.column).getField();
if (actual_value != expected_value)
return onError<ReturnType>(
fmt::format(
return onError<ReturnType>(code,
"Block structure mismatch in {} stream: different values of constants in column '{}': actual: {}, expected: {}",
context_description,
actual.name,
applyVisitor(FieldVisitorToString(), actual_value),
applyVisitor(FieldVisitorToString(), expected_value)),
code);
applyVisitor(FieldVisitorToString(), expected_value));
}
return ReturnType(true);
@ -117,8 +113,8 @@ static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, std:
{
size_t columns = rhs.columns();
if (lhs.columns() != columns)
return onError<ReturnType>("Block structure mismatch in " + std::string(context_description) + " stream: different number of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
return onError<ReturnType>(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch in {} stream: different number of columns:\n{}\n{}",
context_description, lhs.dumpStructure(), rhs.dumpStructure());
for (size_t i = 0; i < columns; ++i)
{

View File

@ -95,7 +95,7 @@ void MySQLClient::handshake()
packet_endpoint->resetSequenceId();
if (packet_response.getType() == PACKET_ERR)
throw Exception(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
else if (packet_response.getType() == PACKET_AUTH_SWITCH)
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Access denied for user {}", user);
}
@ -110,7 +110,7 @@ void MySQLClient::writeCommand(char command, String query)
switch (packet_response.getType())
{
case PACKET_ERR:
throw Exception(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
case PACKET_OK:
break;
default:
@ -128,7 +128,7 @@ void MySQLClient::registerSlaveOnMaster(UInt32 slave_id)
packet_endpoint->receivePacket(packet_response);
packet_endpoint->resetSequenceId();
if (packet_response.getType() == PACKET_ERR)
throw Exception(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
void MySQLClient::ping()

View File

@ -111,7 +111,7 @@ namespace MySQLReplication
else if (query.starts_with("XA"))
{
if (query.starts_with("XA ROLLBACK"))
throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::LOGICAL_ERROR);
throw ReplicationError(ErrorCodes::LOGICAL_ERROR, "ParseQueryEvent: Unsupported query event: {}", query);
typ = QUERY_EVENT_XA;
if (!query.starts_with("XA COMMIT"))
transaction_complete = false;
@ -247,7 +247,7 @@ namespace MySQLReplication
break;
}
default:
throw ReplicationError("ParseMetaData: Unhandled data type:" + std::to_string(typ), ErrorCodes::UNKNOWN_EXCEPTION);
throw ReplicationError(ErrorCodes::UNKNOWN_EXCEPTION, "ParseMetaData: Unhandled data type: {}", std::to_string(typ));
}
}
}
@ -770,8 +770,8 @@ namespace MySQLReplication
break;
}
default:
throw ReplicationError(
"ParseRow: Unhandled MySQL field type:" + std::to_string(field_type), ErrorCodes::UNKNOWN_EXCEPTION);
throw ReplicationError(ErrorCodes::UNKNOWN_EXCEPTION,
"ParseRow: Unhandled MySQL field type: {}", std::to_string(field_type));
}
}
null_index++;
@ -873,7 +873,7 @@ namespace MySQLReplication
break;
}
default:
throw ReplicationError("Position update with unsupported event", ErrorCodes::LOGICAL_ERROR);
throw ReplicationError(ErrorCodes::LOGICAL_ERROR, "Position update with unsupported event");
}
}
@ -901,11 +901,11 @@ namespace MySQLReplication
switch (header)
{
case PACKET_EOF:
throw ReplicationError("Master maybe lost", ErrorCodes::CANNOT_READ_ALL_DATA);
throw ReplicationError(ErrorCodes::CANNOT_READ_ALL_DATA, "Master maybe lost");
case PACKET_ERR:
ERRPacket err;
err.readPayloadWithUnpacked(payload);
throw ReplicationError(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION);
throw ReplicationError::createDeprecated(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION);
}
// skip the generic response packets header flag.
payload.ignore(1);

View File

@ -74,7 +74,7 @@ ConnectionHolderPtr PoolWithFailover::get()
if (replicas_with_priority.empty())
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "No address specified");
DB::WriteBufferFromOwnString error_message;
PreformattedMessage error_message;
for (size_t try_idx = 0; try_idx < max_tries; ++try_idx)
{
for (auto & priority : replicas_with_priority)
@ -107,7 +107,7 @@ ConnectionHolderPtr PoolWithFailover::get()
catch (const pqxx::broken_connection & pqxx_error)
{
LOG_ERROR(log, "Connection error: {}", pqxx_error.what());
error_message << fmt::format(
error_message = PreformattedMessage::create(
"Try {}. Connection to {} failed with error: {}\n",
try_idx + 1, DB::backQuote(replica.connection_info.host_port), pqxx_error.what());
@ -131,7 +131,7 @@ ConnectionHolderPtr PoolWithFailover::get()
}
}
throw DB::Exception(DB::ErrorCodes::POSTGRESQL_CONNECTION_FAILURE, error_message.str());
throw DB::Exception(error_message, DB::ErrorCodes::POSTGRESQL_CONNECTION_FAILURE);
}
}

View File

@ -402,7 +402,7 @@ void SettingFieldEnum<EnumT, Traits>::readBinary(ReadBuffer & in)
auto it = map.find(value); \
if (it != map.end()) \
return it->second; \
throw Exception( \
throw Exception::createDeprecated( \
"Unexpected value of " #NEW_NAME ":" + std::to_string(std::underlying_type<EnumType>::type(value)), \
ERROR_CODE_FOR_UNEXPECTED_NAME); \
} \
@ -428,7 +428,7 @@ void SettingFieldEnum<EnumT, Traits>::readBinary(ReadBuffer & in)
msg += "'" + String{name} + "'"; \
} \
msg += "]"; \
throw Exception(msg, ERROR_CODE_FOR_UNEXPECTED_NAME); \
throw Exception::createDeprecated(msg, ERROR_CODE_FOR_UNEXPECTED_NAME); \
}
// Mostly like SettingFieldEnum, but can have multiple enum values (or none) set at once.

View File

@ -933,7 +933,7 @@ void BaseDaemon::handleSignal(int signal_id)
onInterruptSignals(signal_id);
}
else
throw DB::Exception(std::string("Unsupported signal: ") + strsignal(signal_id), 0); // NOLINT(concurrency-mt-unsafe) // it is not thread-safe but ok in this context
throw DB::Exception::createDeprecated(std::string("Unsupported signal: ") + strsignal(signal_id), 0); // NOLINT(concurrency-mt-unsafe) // it is not thread-safe but ok in this context
}
void BaseDaemon::onInterruptSignals(int signal_id)

View File

@ -189,7 +189,8 @@ template <template <typename> typename DecimalType>
inline DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
{
if (precision_value < DecimalUtils::min_precision || precision_value > DecimalUtils::max_precision<Decimal256>)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Wrong precision");
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Wrong precision: it must be between {} and {}, got {}",
DecimalUtils::min_precision, DecimalUtils::max_precision<Decimal256>, precision_value);
if (static_cast<UInt64>(scale_value) > precision_value)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Negative scales and scales larger than precision are not supported");

View File

@ -57,7 +57,7 @@ static DataTypePtr create(const ASTPtr & arguments)
{
if (func->name != "Nullable" || func->arguments->children.size() != 1)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
"Expected 'Nullable(<schema_name>)' as parameter for type Object", func->name);
"Expected 'Nullable(<schema_name>)' as parameter for type Object (function: {})", func->name);
schema_argument = func->arguments->children[0];
is_nullable = true;

View File

@ -53,10 +53,10 @@ static std::optional<Exception> checkTupleNames(const Strings & names)
for (const auto & name : names)
{
if (name.empty())
return Exception("Names of tuple elements cannot be empty", ErrorCodes::BAD_ARGUMENTS);
return Exception(ErrorCodes::BAD_ARGUMENTS, "Names of tuple elements cannot be empty");
if (!names_set.insert(name).second)
return Exception("Names of tuple elements must be unique", ErrorCodes::DUPLICATE_COLUMN);
return Exception(ErrorCodes::DUPLICATE_COLUMN, "Names of tuple elements must be unique");
}
return {};

View File

@ -373,8 +373,8 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams(
/// Check consistency between offsets and elements subcolumns.
/// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER.
if (!nested_column->empty() && nested_column->size() != last_offset)
throw ParsingException("Cannot read all array values: read just " + toString(nested_column->size()) + " of " + toString(last_offset),
ErrorCodes::CANNOT_READ_ALL_DATA);
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all array values: read just {} of {}",
toString(nested_column->size()), toString(last_offset));
column = std::move(mutable_column);
}

View File

@ -360,19 +360,20 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col
/// or if someone uses tab or LF in TSV null_representation.
/// In the first case we cannot continue reading anyway. The second case seems to be unlikely.
if (null_representation.find('\t') != std::string::npos || null_representation.find('\n') != std::string::npos)
throw DB::ParsingException("TSV custom null representation containing '\\t' or '\\n' may not work correctly "
"for large input.", ErrorCodes::CANNOT_READ_ALL_DATA);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation "
"containing '\\t' or '\\n' may not work correctly for large input.");
WriteBufferFromOwnString parsed_value;
if constexpr (escaped)
nested_serialization->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings);
else
nested_serialization->serializeTextRaw(nested_column, nested_column.size() - 1, parsed_value, settings);
throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable"
+ " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos)
+ "\", which was deserialized as \""
+ parsed_value.str() + "\". It seems that input data is ill-formatted.",
ErrorCodes::CANNOT_READ_ALL_DATA);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable"
" at position {}: got \"{}\", which was deserialized as \"{}\". "
"It seems that input data is ill-formatted.",
std::string(pos, buf.buffer().end()),
std::string(istr.position(), std::min(size_t(10), istr.available())),
istr.count(), std::string(pos, buf.position() - pos), parsed_value.str());
};
return safeDeserialize<ReturnType>(column, *nested_serialization, check_for_null, deserialize_nested);
@ -584,16 +585,17 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB
/// In the first case we cannot continue reading anyway. The second case seems to be unlikely.
if (null_representation.find(settings.csv.delimiter) != std::string::npos || null_representation.find('\r') != std::string::npos
|| null_representation.find('\n') != std::string::npos)
throw DB::ParsingException("CSV custom null representation containing format_csv_delimiter, '\\r' or '\\n' may not work correctly "
"for large input.", ErrorCodes::CANNOT_READ_ALL_DATA);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing "
"format_csv_delimiter, '\\r' or '\\n' may not work correctly for large input.");
WriteBufferFromOwnString parsed_value;
nested_serialization->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings);
throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable"
+ " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos)
+ "\", which was deserialized as \""
+ parsed_value.str() + "\". It seems that input data is ill-formatted.",
ErrorCodes::CANNOT_READ_ALL_DATA);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable"
" at position {}: got \"{}\", which was deserialized as \"{}\". "
"It seems that input data is ill-formatted.",
std::string(pos, buf.buffer().end()),
std::string(istr.position(), std::min(size_t(10), istr.available())),
istr.count(), std::string(pos, buf.position() - pos), parsed_value.str());
};
return safeDeserialize<ReturnType>(column, *nested_serialization, check_for_null, deserialize_nested);

View File

@ -231,7 +231,7 @@ void SerializationObject<Parser>::deserializeBinaryBulkStatePrefix(
auto kind = magic_enum::enum_cast<BinarySerializationKind>(kind_raw);
if (!kind)
throw Exception(ErrorCodes::INCORRECT_DATA,
"Unknown binary serialization kind of Object: " + std::to_string(kind_raw));
"Unknown binary serialization kind of Object: {}", std::to_string(kind_raw));
auto state_object = std::make_shared<DeserializeStateObject>();
state_object->kind = *kind;
@ -255,7 +255,7 @@ void SerializationObject<Parser>::deserializeBinaryBulkStatePrefix(
else
{
throw Exception(ErrorCodes::INCORRECT_DATA,
"Unknown binary serialization kind of Object: " + std::to_string(kind_raw));
"Unknown binary serialization kind of Object: {}", std::to_string(kind_raw));
}
settings.path.push_back(Substream::ObjectData);

View File

@ -40,7 +40,6 @@ template <typename DataTypes>
String getExceptionMessagePrefix(const DataTypes & types)
{
WriteBufferFromOwnString res;
res << "There is no supertype for types ";
bool first = true;
for (const auto & type : types)
@ -65,9 +64,9 @@ DataTypePtr throwOrReturn(const DataTypes & types, std::string_view message_suff
return nullptr;
if (message_suffix.empty())
throw Exception(error_code, getExceptionMessagePrefix(types));
throw Exception(error_code, "There is no supertype for types {}", getExceptionMessagePrefix(types));
throw Exception(error_code, "{} {}", getExceptionMessagePrefix(types), message_suffix);
throw Exception(error_code, "There is no supertype for types {} {}", getExceptionMessagePrefix(types), message_suffix);
}
template <LeastSupertypeOnError on_error>

View File

@ -49,7 +49,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
auto get_nothing_or_throw = [throw_if_result_is_nothing, & types](const std::string & reason)
{
if (throw_if_result_is_nothing)
throw Exception(getExceptionMessagePrefix(types) + reason, ErrorCodes::NO_COMMON_TYPE);
throw Exception::createDeprecated(getExceptionMessagePrefix(types) + reason, ErrorCodes::NO_COMMON_TYPE);
return std::make_shared<DataTypeNothing>();
};

View File

@ -47,10 +47,10 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume
else
{
if (argument && argument->value.getType() != field_type)
throw Exception(getExceptionMessage(fmt::format(" has wrong type: {}", argument->value.getTypeName()),
throw Exception::createDeprecated(getExceptionMessage(fmt::format(" has wrong type: {}", argument->value.getTypeName()),
argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else
throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type),
throw Exception::createDeprecated(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
}
@ -67,7 +67,7 @@ static DataTypePtr create(const ASTPtr & arguments)
const auto timezone = getArgument<String, ArgumentKind::Optional>(arguments, !!scale, "timezone", "DateTime");
if (!scale && !timezone)
throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64),
throw Exception::createDeprecated(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// If scale is defined, the data type is DateTime when scale = 0 otherwise the data type is DateTime64

View File

@ -41,10 +41,9 @@ namespace
}
catch (Exception & e)
{
throw Exception(
fmt::format("Error while loading dictionary '{}.{}': {}",
database_name, load_result.name, e.displayText()),
e.code());
throw Exception(e.code(),
"Error while loading dictionary '{}.{}': {}",
database_name, load_result.name, e.displayText());
}
}
}
@ -118,7 +117,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co
/* hilite = */ false, "", /* allow_multi_statements = */ false, 0, settings.max_parser_depth);
if (!ast && throw_on_error)
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
throw Exception::createDeprecated(error_message, ErrorCodes::SYNTAX_ERROR);
return ast;
}

View File

@ -252,13 +252,11 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
{
auto print_create_ast = create.clone();
print_create_ast->as<ASTCreateQuery>()->attach = false;
throw Exception(
fmt::format(
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"The MaterializedMySQL database engine no longer supports Ordinary databases. To re-create the database, delete "
"the old one by executing \"rm -rf {}{{,.sql}}\", then re-create the database with the following query: {}",
metadata_path,
queryToString(print_create_ast)),
ErrorCodes::NOT_IMPLEMENTED);
queryToString(print_create_ast));
}
return std::make_shared<DatabaseMaterializedMySQL>(

View File

@ -676,7 +676,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(
"in file " + metadata_file_path, /* allow_multi_statements = */ false, 0, settings.max_parser_depth);
if (!ast && throw_on_error)
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
throw Exception::createDeprecated(error_message, ErrorCodes::SYNTAX_ERROR);
else if (!ast)
return nullptr;

View File

@ -136,7 +136,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context
if (local_tables_cache.find(table_name) == local_tables_cache.end())
{
if (throw_on_error)
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {}.{} doesn't exist.", database_name_in_mysql, table_name);
return nullptr;
}
@ -180,7 +180,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam
fetchTablesIntoLocalCache(getContext());
if (local_tables_cache.find(table_name) == local_tables_cache.end())
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {}.{} doesn't exist.", database_name_in_mysql, table_name);
return time_t(local_tables_cache[table_name].first);
}

View File

@ -147,7 +147,7 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S
first = false;
}
throw Exception(error_message.str(), ErrorCodes::ILLEGAL_MYSQL_VARIABLE);
throw Exception::createDeprecated(error_message.str(), ErrorCodes::ILLEGAL_MYSQL_VARIABLE);
}
}

View File

@ -214,12 +214,12 @@ void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & t
if (!checkPostgresTable(table_name))
throw Exception(ErrorCodes::UNKNOWN_TABLE,
"Cannot attach PostgreSQL table {} because it does not exist in PostgreSQL",
"Cannot attach PostgreSQL table {} because it does not exist in PostgreSQL (database: {})",
getTableNameForLogs(table_name), database_name);
if (!detached_or_dropped.contains(table_name))
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS,
"Cannot attach PostgreSQL table {} because it already exists",
"Cannot attach PostgreSQL table {} because it already exists (database: {})",
getTableNameForLogs(table_name), database_name);
if (cache_tables)

View File

@ -19,7 +19,7 @@ static std::mutex init_sqlite_db_mutex;
void processSQLiteError(const String & message, bool throw_on_error)
{
if (throw_on_error)
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, message);
throw Exception::createDeprecated(message, ErrorCodes::PATH_ACCESS_DENIED);
else
LOG_ERROR(&Poco::Logger::get("SQLiteEngine"), fmt::runtime(message));
}

View File

@ -54,9 +54,9 @@ void RegionsHierarchy::reload()
if (region_entry.id > max_region_id)
{
if (region_entry.id > max_size)
throw DB::Exception(
"Region id is too large: " + DB::toString(region_entry.id) + ", should be not more than " + DB::toString(max_size),
DB::ErrorCodes::INCORRECT_DATA);
throw DB::Exception(DB::ErrorCodes::INCORRECT_DATA,
"Region id is too large: {}, should be not more than {}",
DB::toString(region_entry.id), DB::toString(max_size));
max_region_id = region_entry.id;

View File

@ -84,9 +84,9 @@ void RegionsNames::reload()
max_region_id = name_entry.id;
if (name_entry.id > max_size)
throw DB::Exception(
"Region id is too large: " + DB::toString(name_entry.id) + ", should be not more than " + DB::toString(max_size),
DB::ErrorCodes::INCORRECT_DATA);
throw DB::Exception(DB::ErrorCodes::INCORRECT_DATA,
"Region id is too large: {}, should be not more than {}",
DB::toString(name_entry.id), DB::toString(max_size));
}
while (name_entry.id >= new_names_refs.size())

View File

@ -61,9 +61,8 @@ static void loadDiskLocalConfig(const String & name,
if (name == "default")
{
if (!path.empty())
throw Exception(
"\"default\" disk path should be provided in <path> not it <storage_configuration>",
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"\"default\" disk path should be provided in <path> not it <storage_configuration>");
path = context->getPath();
}
else

View File

@ -487,3 +487,13 @@ inline String directoryPath(const String & path)
}
}
template <>
struct fmt::formatter<fs::path> : fmt::formatter<std::string>
{
template <typename FormatCtx>
auto format(const fs::path & path, FormatCtx & ctx) const
{
return fmt::formatter<std::string>::format(path.string(), ctx);
}
};

View File

@ -110,7 +110,7 @@ void HDFSObjectStorage::removeObject(const StoredObject & object)
/// Add path from root to file name
int res = hdfsDelete(hdfs_fs.get(), path.substr(begin_of_path).c_str(), 0);
if (res == -1)
throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: " + path);
throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: {}", path);
}

View File

@ -44,11 +44,10 @@ namespace JSONUtils
{
const auto current_object_size = memory.size() + static_cast<size_t>(pos - in.position());
if (min_bytes != 0 && current_object_size > 10 * min_bytes)
throw ParsingException(
"Size of JSON object is extremely large. Expected not greater than " + std::to_string(min_bytes)
+ " bytes, but current is " + std::to_string(current_object_size)
+ " bytes per row. Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, most likely JSON is malformed",
ErrorCodes::INCORRECT_DATA);
throw ParsingException(ErrorCodes::INCORRECT_DATA,
"Size of JSON object is extremely large. Expected not greater than {} bytes, but current is {} bytes per row. "
"Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, "
"most likely JSON is malformed", min_bytes, current_object_size);
if (quotes)
{

View File

@ -103,7 +103,7 @@ Block NativeReader::read()
if (istr.eof())
{
if (use_index)
throw ParsingException("Input doesn't contain all data for index.", ErrorCodes::CANNOT_READ_ALL_DATA);
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Input doesn't contain all data for index.");
return res;
}

View File

@ -429,7 +429,7 @@ void ProtobufReader::ignoreGroup()
[[noreturn]] void ProtobufReader::throwUnknownFormat() const
{
throw Exception(
throw Exception::createDeprecated(
std::string("Protobuf messages are corrupted or don't match the provided schema.")
+ (root_message_has_length_delimiter
? " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint."

View File

@ -22,7 +22,7 @@ namespace OpenSSLDetails
void onError(std::string error_message)
{
error_message += ". OpenSSL error code: " + std::to_string(ERR_get_error());
throw DB::Exception(error_message, DB::ErrorCodes::OPENSSL_ERROR);
throw DB::Exception::createDeprecated(error_message, DB::ErrorCodes::OPENSSL_ERROR);
}
StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, StringRef key, std::array<char, EVP_MAX_KEY_LENGTH> & folded_key)

View File

@ -60,8 +60,7 @@ struct KeyHolder
inline StringRef setKey(size_t cipher_key_size, StringRef key) const
{
if (key.size != cipher_key_size)
throw DB::Exception(fmt::format("Invalid key size: {} expected {}", key.size, cipher_key_size),
DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid key size: {} expected {}", key.size, cipher_key_size);
return key;
}
@ -73,8 +72,7 @@ struct KeyHolder<CipherMode::MySQLCompatibility>
inline StringRef setKey(size_t cipher_key_size, StringRef key)
{
if (key.size < cipher_key_size)
throw DB::Exception(fmt::format("Invalid key size: {} expected {}", key.size, cipher_key_size),
DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid key size: {} expected {}", key.size, cipher_key_size);
// MySQL does something fancy with the keys that are too long,
// ruining compatibility with OpenSSL and not improving security.
@ -129,8 +127,7 @@ inline void validateIV(StringRef iv_value, const size_t cipher_iv_size)
// In MySQL mode we don't care if IV is longer than expected, only if shorter.
if ((mode == CipherMode::MySQLCompatibility && iv_value.size != 0 && iv_value.size < cipher_iv_size)
|| (mode == CipherMode::OpenSSLCompatibility && iv_value.size != 0 && iv_value.size != cipher_iv_size))
throw DB::Exception(fmt::format("Invalid IV size: {} expected {}", iv_value.size, cipher_iv_size),
DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid IV size: {} expected {}", iv_value.size, cipher_iv_size);
}
}
@ -600,7 +597,7 @@ private:
if (input_value.size < tag_size)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted data is too short: only {} bytes, "
"should contain at least {} bytes of a tag.",
input_value.size, block_size, tag_size);
input_value.size, tag_size);
input_value.size -= tag_size;
}

View File

@ -3181,8 +3181,8 @@ private:
const auto * from_type = checkAndGetDataType<DataTypeTuple>(from_type_untyped.get());
if (!from_type)
throw Exception{"CAST AS Tuple can only be performed between tuple types or from String.\nLeft type: "
+ from_type_untyped->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Tuple can only be performed between tuple types or from String.\n"
"Left type: {}, right type: {}", from_type_untyped->getName(), to_type->getName());
const auto & from_element_types = from_type->getElements();
const auto & to_element_types = to_type->getElements();
@ -3223,8 +3223,9 @@ private:
else
{
if (from_element_types.size() != to_element_types.size())
throw Exception{"CAST AS Tuple can only be performed between tuple types with the same number of elements or from String.\n"
"Left type: " + from_type->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Tuple can only be performed between tuple types "
"with the same number of elements or from String.\nLeft type: {}, right type: {}",
from_type->getName(), to_type->getName());
element_wrappers = getElementWrappers(from_element_types, to_element_types);
to_reverse_index.reserve(to_element_types.size());
@ -3337,8 +3338,8 @@ private:
if (const auto * from_tuple = checkAndGetDataType<DataTypeTuple>(from_type_untyped.get()))
{
if (from_tuple->getElements().size() != 2)
throw Exception{"CAST AS Map from tuple requeires 2 elements.\n"
"Left type: " + from_tuple->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Map from tuple requeires 2 elements. "
"Left type: {}, right type: {}", from_tuple->getName(), to_type->getName());
DataTypes from_kv_types;
const auto & to_kv_types = to_type->getKeyValueTypes();
@ -3359,8 +3360,8 @@ private:
{
const auto * nested_tuple = typeid_cast<const DataTypeTuple *>(from_array->getNestedType().get());
if (!nested_tuple || nested_tuple->getElements().size() != 2)
throw Exception{"CAST AS Map from array requeires nested tuple of 2 elements.\n"
"Left type: " + from_array->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Map from array requeires nested tuple of 2 elements. "
"Left type: {}, right type: {}", from_array->getName(), to_type->getName());
return createArrayToMapWrrapper(nested_tuple->getElements(), to_type->getKeyValueTypes());
}
@ -3370,8 +3371,8 @@ private:
}
else
{
throw Exception{"Unsupported types to CAST AS Map\n"
"Left type: " + from_type_untyped->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Unsupported types to CAST AS Map. "
"Left type: {}, right type: {}", from_type_untyped->getName(), to_type->getName());
}
}

View File

@ -182,7 +182,7 @@ public:
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of first argument of function, expected a string",
"Illegal type {} of first argument of function {}, expected a string",
arguments[0]->getName(),
getName());
@ -416,7 +416,7 @@ public:
if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64)))
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of fourth argument of function must be convertible to Int64.",
"Illegal type {} of fourth argument of function {} must be convertible to Int64.",
range_col_type->getName(),
getName());

View File

@ -265,9 +265,8 @@ struct TernaryValueBuilderImpl<>
{
[[noreturn]] static void build(const IColumn * x, UInt8 * /* nullable_ternary_column_data */)
{
throw Exception(
std::string("Unknown numeric column of type: ") + demangle(typeid(*x).name()),
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unknown numeric column of type: {}", demangle(typeid(*x).name()));
}
};
@ -440,13 +439,13 @@ struct TypedExecutorInvoker<Op>
template <typename T, typename Result>
static void apply(const ColumnVector<T> &, const IColumn & y, Result &)
{
throw Exception(std::string("Unknown numeric column y of type: ") + demangle(typeid(y).name()), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown numeric column y of type: {}", demangle(typeid(y).name()));
}
template <typename Result>
static void apply(const IColumn & x, const IColumn &, Result &)
{
throw Exception(std::string("Unknown numeric column x of type: ") + demangle(typeid(x).name()), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown numeric column x of type: {}", demangle(typeid(x).name()));
}
};

View File

@ -278,9 +278,8 @@ namespace DB
{
if (year_ < 0 || year_ > 9999)
{
throw Exception(
"Impossible to stringify: year too big or small: " + DB::toString(year_),
ErrorCodes::CANNOT_FORMAT_DATETIME);
throw Exception(ErrorCodes::CANNOT_FORMAT_DATETIME,
"Impossible to stringify: year too big or small: {}", DB::toString(year_));
}
else
{
@ -372,9 +371,7 @@ namespace DB
, day_of_month_(day_of_month)
{
if (month < 1 || month > 12)
throw Exception(
"Invalid month: " + DB::toString(month),
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid month: {}", DB::toString(month));
/* We can't validate day_of_month here, because we don't know if
* it's a leap year. */
}
@ -382,10 +379,8 @@ namespace DB
inline MonthDay::MonthDay(bool is_leap_year, uint16_t day_of_year)
{
if (day_of_year < 1 || day_of_year > (is_leap_year ? 366 : 365))
throw Exception(
std::string("Invalid day of year: ") +
(is_leap_year ? "leap, " : "non-leap, ") + DB::toString(day_of_year),
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year: {}{}",
(is_leap_year ? "leap, " : "non-leap, "), DB::toString(day_of_year));
month_ = 1;
uint16_t d = day_of_year;
@ -404,11 +399,8 @@ namespace DB
{
if (day_of_month_ < 1 || day_of_month_ > gd::monthLength(is_leap_year, month_))
{
throw Exception(
std::string("Invalid day of month: ") +
(is_leap_year ? "leap, " : "non-leap, ") + DB::toString(month_) +
"-" + DB::toString(day_of_month_),
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month: {}{}-{}",
(is_leap_year ? "leap, " : "non-leap, "), DB::toString(month_), DB::toString(day_of_month_));
}
const auto k = month_ <= 2 ? 0 : is_leap_year ? -1 :-2;
return (367 * month_ - 362) / 12 + k + day_of_month_;

View File

@ -600,13 +600,13 @@ struct CallPointInPolygon<>
template <typename T, typename PointInPolygonImpl>
static ColumnPtr call(const ColumnVector<T> &, const IColumn & y, PointInPolygonImpl &&)
{
throw Exception(std::string("Unknown numeric column type: ") + demangle(typeid(y).name()), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown numeric column type: {}", demangle(typeid(y).name()));
}
template <typename PointInPolygonImpl>
static ColumnPtr call(const IColumn & x, const IColumn &, PointInPolygonImpl &&)
{
throw Exception(std::string("Unknown numeric column type: ") + demangle(typeid(x).name()), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown numeric column type: {}", demangle(typeid(x).name()));
}
};

View File

@ -240,7 +240,7 @@ inline Regexps constructRegexps(const std::vector<String> & str_patterns, [[mayb
CompilerErrorPtr error(compile_error);
if (error->expression < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, String(error->message));
throw Exception::createRuntime(ErrorCodes::LOGICAL_ERROR, String(error->message));
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Pattern '{}' failed with error '{}'", str_patterns[error->expression], String(error->message));
}

View File

@ -367,7 +367,7 @@ struct ArrayAggregateImpl
executeType<Decimal128>(mapped, offsets, res))
return res;
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arraySum: {}" + mapped->getName());
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arraySum: {}", mapped->getName());
}
};

Some files were not shown because too many files have changed in this diff Show More