mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge pull request #58198 from azat/exception-fmt
Fix all Exception with missing arguments
This commit is contained in:
commit
d524951416
@ -51,10 +51,10 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat
|
||||
void AggregateFunctionFactory::registerNullsActionTransformation(const String & source_ignores_nulls, const String & target_respect_nulls)
|
||||
{
|
||||
if (!aggregate_functions.contains(source_ignores_nulls))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "registerNullsActionTransformation: Source aggregation '{}' not found");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "registerNullsActionTransformation: Source aggregation '{}' not found", source_ignores_nulls);
|
||||
|
||||
if (!aggregate_functions.contains(target_respect_nulls))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "registerNullsActionTransformation: Target aggregation '{}' not found");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "registerNullsActionTransformation: Target aggregation '{}' not found", target_respect_nulls);
|
||||
|
||||
if (!respect_nulls.emplace(source_ignores_nulls, target_respect_nulls).second)
|
||||
throw Exception(
|
||||
|
@ -197,7 +197,7 @@ public:
|
||||
virtual void insertMergeResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const
|
||||
{
|
||||
if (isState())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} is marked as State but method insertMergeResultInto is not implemented");
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} is marked as State but method insertMergeResultInto is not implemented", getName());
|
||||
|
||||
insertResultInto(place, to, arena);
|
||||
}
|
||||
|
@ -536,7 +536,8 @@ CNF CNF::toCNF(const QueryTreeNodePtr & node, ContextPtr context, size_t max_gro
|
||||
if (!cnf)
|
||||
throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS,
|
||||
"Cannot convert expression '{}' to CNF, because it produces to many clauses."
|
||||
"Size of boolean formula in CNF can be exponential of size of source formula.");
|
||||
"Size of boolean formula in CNF can be exponential of size of source formula.",
|
||||
node->formatConvertedASTForErrorMessage());
|
||||
|
||||
return *cnf;
|
||||
}
|
||||
|
@ -146,7 +146,7 @@ UInt64 BackupReaderS3::getFileSize(const String & file_name)
|
||||
{
|
||||
auto objects = listObjects(*client, s3_uri, file_name);
|
||||
if (objects.empty())
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Object {} must exist");
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Object {} must exist", file_name);
|
||||
return objects[0].GetSize();
|
||||
}
|
||||
|
||||
@ -299,7 +299,7 @@ UInt64 BackupWriterS3::getFileSize(const String & file_name)
|
||||
{
|
||||
auto objects = listObjects(*client, s3_uri, file_name);
|
||||
if (objects.empty())
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Object {} must exist");
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Object {} must exist", file_name);
|
||||
return objects[0].GetSize();
|
||||
}
|
||||
|
||||
|
@ -201,7 +201,7 @@ void LocalConnection::sendQuery(
|
||||
catch (...)
|
||||
{
|
||||
state->io.onException();
|
||||
state->exception = std::make_unique<Exception>(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception");
|
||||
state->exception = std::make_unique<Exception>(Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception"));
|
||||
}
|
||||
}
|
||||
|
||||
@ -311,7 +311,7 @@ bool LocalConnection::poll(size_t)
|
||||
catch (...)
|
||||
{
|
||||
state->io.onException();
|
||||
state->exception = std::make_unique<Exception>(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception");
|
||||
state->exception = std::make_unique<Exception>(Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception"));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -93,15 +93,6 @@ public:
|
||||
return Exception(msg, code, remote_);
|
||||
}
|
||||
|
||||
/// Message must be a compile-time constant
|
||||
template <typename T>
|
||||
requires std::is_convertible_v<T, String>
|
||||
Exception(int code, T && message) : Exception(message, code)
|
||||
{
|
||||
capture_thread_frame_pointers = thread_frame_pointers;
|
||||
message_format_string = tryGetStaticFormatString(message);
|
||||
}
|
||||
|
||||
/// 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); }
|
||||
|
@ -120,7 +120,7 @@ String Macros::expand(const String & s,
|
||||
auto uuid = ServerUUID::get();
|
||||
if (UUIDHelpers::Nil == uuid)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Macro {server_uuid} expanded to zero, which means the UUID is not initialized (most likely it's not a server application)");
|
||||
"Macro {{server_uuid}} expanded to zero, which means the UUID is not initialized (most likely it's not a server application)");
|
||||
res += toString(uuid);
|
||||
info.expanded_other = true;
|
||||
}
|
||||
|
@ -471,7 +471,7 @@ private:
|
||||
/// Message must be a compile-time constant
|
||||
template <typename T>
|
||||
requires std::is_convertible_v<T, String>
|
||||
Exception(T && message, const Error code_) : DB::Exception(DB::ErrorCodes::KEEPER_EXCEPTION, std::forward<T>(message)), code(code_)
|
||||
Exception(T && message, const Error code_) : DB::Exception(std::forward<T>(message), DB::ErrorCodes::KEEPER_EXCEPTION, /* remote_= */ false), code(code_)
|
||||
{
|
||||
incrementErrorMetrics(code);
|
||||
}
|
||||
|
@ -914,7 +914,7 @@ void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::strin
|
||||
{
|
||||
auto ephemerals_it = ephemerals.find(session_id);
|
||||
if (ephemerals_it == ephemerals.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path", session_id);
|
||||
|
||||
ephemerals_it->second.erase(path);
|
||||
if (ephemerals_it->second.empty())
|
||||
|
@ -255,7 +255,7 @@ DatabaseS3::Configuration DatabaseS3::parseArguments(ASTs engine_args, ContextPt
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_);
|
||||
|
||||
if (engine_args.size() > 3)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message.c_str());
|
||||
throw Exception::createRuntime(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message.c_str());
|
||||
|
||||
if (engine_args.empty())
|
||||
return result;
|
||||
@ -269,7 +269,7 @@ DatabaseS3::Configuration DatabaseS3::parseArguments(ASTs engine_args, ContextPt
|
||||
if (boost::iequals(second_arg, "NOSIGN"))
|
||||
result.no_sign_request = true;
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, error_message.c_str());
|
||||
throw Exception::createRuntime(ErrorCodes::BAD_ARGUMENTS, error_message.c_str());
|
||||
}
|
||||
|
||||
// url, access_key_id, secret_access_key
|
||||
@ -279,7 +279,7 @@ DatabaseS3::Configuration DatabaseS3::parseArguments(ASTs engine_args, ContextPt
|
||||
auto secret_key = checkAndGetLiteralArgument<String>(engine_args[2], "secret_access_key");
|
||||
|
||||
if (key_id.empty() || secret_key.empty() || boost::iequals(key_id, "NOSIGN"))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, error_message.c_str());
|
||||
throw Exception::createRuntime(ErrorCodes::BAD_ARGUMENTS, error_message.c_str());
|
||||
|
||||
result.access_key_id = key_id;
|
||||
result.secret_access_key = secret_key;
|
||||
|
@ -37,7 +37,7 @@ DictionaryTypedSpecialAttribute makeDictionaryTypedSpecialAttribute(
|
||||
auto expression = config.getString(config_prefix + ".expression", "");
|
||||
|
||||
if (name.empty() && !expression.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty");
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty", config_prefix);
|
||||
|
||||
const auto type_name = config.getString(config_prefix + ".type", default_type);
|
||||
return DictionaryTypedSpecialAttribute{std::move(name), std::move(expression), DataTypeFactory::instance().get(type_name)};
|
||||
|
@ -683,7 +683,7 @@ void RangeHashedDictionary<dictionary_key_type>::loadData()
|
||||
|
||||
if (configuration.require_nonempty && 0 == element_count)
|
||||
throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY,
|
||||
"{}: dictionary source is empty and 'require_nonempty' property is set.");
|
||||
"{}: dictionary source is empty and 'require_nonempty' property is set.", getFullName());
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
|
@ -654,7 +654,7 @@ private:
|
||||
|
||||
if (tuple_size < 1)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Tuple second argument of function {} must contain multiple constant string columns");
|
||||
"Tuple second argument of function {} must contain multiple constant string columns", getName());
|
||||
|
||||
for (size_t i = 0; i < tuple_col.tupleSize(); ++i)
|
||||
{
|
||||
|
@ -125,7 +125,7 @@ void GregorianDate::init(ReadBuffer & in)
|
||||
assertEOF(in);
|
||||
|
||||
if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > monthLength(is_leap_year(year_), month_))
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date, out of range (year: {}, month: {}, day_of_month: {}).");
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date, out of range (year: {}, month: {}, day_of_month: {}).", year_, month_, day_of_month_);
|
||||
}
|
||||
|
||||
bool GregorianDate::tryInit(ReadBuffer & in)
|
||||
|
@ -4539,7 +4539,7 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w
|
||||
if (!storage_id)
|
||||
{
|
||||
if (exception)
|
||||
exception->emplace(ErrorCodes::UNKNOWN_TABLE, "Both table name and UUID are empty");
|
||||
exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Both table name and UUID are empty"));
|
||||
return storage_id;
|
||||
}
|
||||
|
||||
@ -4600,7 +4600,7 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w
|
||||
if (current_database.empty())
|
||||
{
|
||||
if (exception)
|
||||
exception->emplace(ErrorCodes::UNKNOWN_DATABASE, "Default database is not selected");
|
||||
exception->emplace(Exception(ErrorCodes::UNKNOWN_DATABASE, "Default database is not selected"));
|
||||
return StorageID::createEmpty();
|
||||
}
|
||||
storage_id.database_name = current_database;
|
||||
|
@ -331,7 +331,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
if (!table_id)
|
||||
{
|
||||
if (exception)
|
||||
exception->emplace(ErrorCodes::UNKNOWN_TABLE, "Cannot find table: StorageID is empty");
|
||||
exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot find table: StorageID is empty"));
|
||||
return {};
|
||||
}
|
||||
|
||||
|
@ -1730,7 +1730,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, Cont
|
||||
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"Seems like cluster is configured for cross-replication, "
|
||||
"but zookeeper_path for ReplicatedMergeTree is not specified or contains {uuid} macro. "
|
||||
"but zookeeper_path for ReplicatedMergeTree is not specified or contains {{uuid}} macro. "
|
||||
"It's not supported for cross replication, because tables must have different UUIDs. "
|
||||
"Please specify unique zookeeper_path explicitly.");
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
@ -239,7 +240,8 @@ CNFQuery TreeCNFConverter::toCNF(
|
||||
if (!cnf)
|
||||
throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS,
|
||||
"Cannot convert expression '{}' to CNF, because it produces to many clauses."
|
||||
"Size of boolean formula in CNF can be exponential of size of source formula.");
|
||||
"Size of boolean formula in CNF can be exponential of size of source formula.",
|
||||
queryToString(query));
|
||||
|
||||
return *cnf;
|
||||
}
|
||||
|
@ -63,7 +63,7 @@ public:
|
||||
void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier)
|
||||
{
|
||||
if (hasColumn(column.name))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists", column.name);
|
||||
|
||||
addColumnImpl(column, column_identifier);
|
||||
}
|
||||
|
@ -1289,7 +1289,7 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node)
|
||||
case avro::Type::AVRO_MAP:
|
||||
return std::make_shared<DataTypeMap>(avroNodeToDataType(node->leafAt(0)), avroNodeToDataType(node->leafAt(1)));
|
||||
default:
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro column {} is not supported for inserting.");
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro column {} is not supported for inserting.", nodeName(node));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -91,7 +91,7 @@ void JSONAsStringRowInputFormat::readJSONObject(IColumn & column)
|
||||
bool quotes = false;
|
||||
|
||||
if (*buf->position() != '{')
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "JSON object must begin with '{'.");
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "JSON object must begin with '{{'.");
|
||||
|
||||
++buf->position();
|
||||
++balance;
|
||||
|
@ -657,7 +657,7 @@ void TCPHandler::runImpl()
|
||||
catch (...)
|
||||
{
|
||||
state.io.onException();
|
||||
exception = std::make_unique<DB::Exception>(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception");
|
||||
exception = std::make_unique<DB::Exception>(Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception"));
|
||||
}
|
||||
|
||||
try
|
||||
|
@ -212,7 +212,7 @@ void MergeTreeIndexAggregatorUSearch<Metric>::update(const Block & block, size_t
|
||||
{
|
||||
auto rc = index->add(static_cast<uint32_t>(index->size()), &column_array_data_float_data[column_array_offsets[current_row - 1]]);
|
||||
if (!rc)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release());
|
||||
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, rc.error.release());
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddCount);
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members);
|
||||
@ -243,7 +243,7 @@ void MergeTreeIndexAggregatorUSearch<Metric>::update(const Block & block, size_t
|
||||
{
|
||||
auto rc = index->add(static_cast<uint32_t>(index->size()), item.data());
|
||||
if (!rc)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release());
|
||||
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, rc.error.release());
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddCount);
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members);
|
||||
|
Loading…
Reference in New Issue
Block a user