Better formatting for exception messages (#45449)

* save format string for NetException

* format exceptions

* format exceptions 2

* format exceptions 3

* format exceptions 4

* format exceptions 5

* format exceptions 6

* fix

* format exceptions 7

* format exceptions 8

* Update MergeTreeIndexGin.cpp

* Update AggregateFunctionMap.cpp

* Update AggregateFunctionMap.cpp

* fix
This commit is contained in:
Alexander Tokmakov 2023-01-24 00:13:58 +03:00 committed by GitHub
parent 62a8de34cc
commit 70d1adfe4b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1161 changed files with 6811 additions and 6892 deletions

View File

@ -968,7 +968,7 @@ void Client::processOptions(const OptionsDescription & options_description,
if (external_tables.back().file == "-")
++number_of_external_tables_with_stdin_source;
if (number_of_external_tables_with_stdin_source > 1)
throw Exception("Two or more external tables has stdin (-) set as --file field", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Two or more external tables has stdin (-) set as --file field");
}
catch (const Exception & e)
{
@ -1021,7 +1021,7 @@ void Client::processOptions(const OptionsDescription & options_description,
}
if (options.count("config-file") && options.count("config"))
throw Exception("Two or more configuration files referenced in arguments", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Two or more configuration files referenced in arguments");
if (options.count("config"))
config().setString("config-file", options["config"].as<std::string>());
@ -1212,14 +1212,14 @@ void Client::readArguments(
/// param_name value
++arg_num;
if (arg_num >= argc)
throw Exception("Parameter requires value", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter requires value");
arg = argv[arg_num];
query_parameters.emplace(String(param_continuation), String(arg));
}
else
{
if (equal_pos == 0)
throw Exception("Parameter name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter name cannot be empty");
/// param_name=value
query_parameters.emplace(param_continuation.substr(0, equal_pos), param_continuation.substr(equal_pos + 1));
@ -1233,7 +1233,7 @@ void Client::readArguments(
{
++arg_num;
if (arg_num >= argc)
throw Exception("Host argument requires value", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Host argument requires value");
arg = argv[arg_num];
host_arg = "--host=";
host_arg.append(arg);
@ -1265,7 +1265,7 @@ void Client::readArguments(
port_arg.push_back('=');
++arg_num;
if (arg_num >= argc)
throw Exception("Port argument requires value", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Port argument requires value");
arg = argv[arg_num];
port_arg.append(arg);
}

View File

@ -46,7 +46,7 @@ void checkAndWriteHeader(DB::ReadBuffer & in, DB::WriteBuffer & out)
UInt32 size_compressed = unalignedLoad<UInt32>(&header[1]);
if (size_compressed > DBMS_MAX_COMPRESSED_SIZE)
throw DB::Exception("Too large size_compressed. Most likely corrupted data.", DB::ErrorCodes::TOO_LARGE_SIZE_COMPRESSED);
throw DB::Exception(DB::ErrorCodes::TOO_LARGE_SIZE_COMPRESSED, "Too large size_compressed. Most likely corrupted data.");
UInt32 size_decompressed = unalignedLoad<UInt32>(&header[5]);
@ -113,10 +113,10 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
codecs = options["codec"].as<std::vector<std::string>>();
if ((use_lz4hc || use_zstd || use_deflate_qpl || use_none) && !codecs.empty())
throw Exception("Wrong options, codec flags like --zstd and --codec options are mutually exclusive", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong options, codec flags like --zstd and --codec options are mutually exclusive");
if (!codecs.empty() && options.count("level"))
throw Exception("Wrong options, --level is not compatible with --codec list", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong options, --level is not compatible with --codec list");
std::string method_family = "LZ4";

View File

@ -77,7 +77,7 @@ decltype(auto) ClusterCopier::retry(T && func, UInt64 max_tries)
std::exception_ptr exception;
if (max_tries == 0)
throw Exception("Cannot perform zero retries", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform zero retries");
for (UInt64 try_number = 1; try_number <= max_tries; ++try_number)
{
@ -123,7 +123,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
}
catch (Exception & e)
{
throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} has incorrect format. {}", partition_text_quoted, e.displayText());
}
};
@ -325,8 +325,8 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts)
if (!table_is_done)
{
throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution",
ErrorCodes::UNFINISHED);
throw Exception(ErrorCodes::UNFINISHED, "Too many tries to process table {}. Abort remaining execution",
task_table.table_id);
}
}
}
@ -666,7 +666,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
}
if (inject_fault)
throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED);
throw Exception(ErrorCodes::UNFINISHED, "Copy fault injection is activated");
}
/// Create node to signal that we finished moving
@ -753,7 +753,7 @@ std::shared_ptr<ASTCreateQuery> rewriteCreateQueryStorage(const ASTPtr & create_
auto res = std::make_shared<ASTCreateQuery>(create);
if (create.storage == nullptr || new_storage_ast == nullptr)
throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Storage is not specified");
res->setDatabase(new_table.first);
res->setTable(new_table.second);
@ -775,7 +775,7 @@ bool ClusterCopier::tryDropPartitionPiece(
const CleanStateClock & clean_state_clock)
{
if (is_safe_mode)
throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP PARTITION is prohibited in safe mode");
TaskTable & task_table = task_partition.task_shard.task_table;
ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number];
@ -944,7 +944,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
for (const String & partition_name : task_table.ordered_partition_names)
{
if (!task_table.cluster_partitions.contains(partition_name))
throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "There are no expected partition {}. It is a bug", partition_name);
ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name];
@ -1006,7 +1006,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
/// Previously when we discovered that shard does not contain current partition, we skipped it.
/// At this moment partition have to be present.
if (it_shard_partition == shard->partition_tasks.end())
throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "There are no such partition in a shard. This is a bug.");
auto & partition = it_shard_partition->second;
expected_shards.emplace_back(shard);
@ -1587,7 +1587,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
auto cancel_check = [&] ()
{
if (zookeeper->expired())
throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
throw Exception(ErrorCodes::UNFINISHED, "ZooKeeper session is expired, cancel INSERT SELECT");
if (!future_is_dirty_checker.valid())
future_is_dirty_checker = zookeeper->asyncExists(piece_is_dirty_flag_path);
@ -1603,7 +1603,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
LogicalClock dirt_discovery_epoch (status.stat.mzxid);
if (dirt_discovery_epoch == clean_state_clock.discovery_zxid)
return false;
throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
throw Exception(ErrorCodes::UNFINISHED, "Partition is dirty, cancel INSERT SELECT");
}
}
@ -1646,7 +1646,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
future_is_dirty_checker.get();
if (inject_fault)
throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED);
throw Exception(ErrorCodes::UNFINISHED, "Copy fault injection is activated");
}
catch (...)
{

View File

@ -90,9 +90,7 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
if (!endsWith(engine.name, "MergeTree"))
{
throw Exception(
"Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported engine was specified in {}, only *MergeTree engines are supported", storage_str);
}
if (isExtendedDefinitionStorage(storage_ast))
@ -109,14 +107,13 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
size_t min_args = is_replicated ? 3 : 1;
if (!engine.arguments)
throw Exception("Expected arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected arguments in {}", storage_str);
ASTPtr arguments_ast = engine.arguments->clone();
ASTs & arguments = arguments_ast->children;
if (arguments.size() < min_args)
throw Exception("Expected at least " + toString(min_args) + " arguments in " + storage_str,
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected at least {} arguments in {}", min_args, storage_str);
ASTPtr & month_arg = is_replicated ? arguments[2] : arguments[1];
return makeASTFunction("toYYYYMM", month_arg->clone());
@ -132,14 +129,12 @@ ASTPtr extractPrimaryKey(const ASTPtr & storage_ast)
if (!endsWith(engine.name, "MergeTree"))
{
throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported engine was specified in {}, only *MergeTree engines are supported", storage_str);
}
if (!isExtendedDefinitionStorage(storage_ast))
{
throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Is not extended deginition storage {} Will be fixed later.", storage_str);
}
if (storage.primary_key)
@ -158,20 +153,18 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast)
if (!endsWith(engine.name, "MergeTree"))
{
throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported engine was specified in {}, only *MergeTree engines are supported", storage_str);
}
if (!isExtendedDefinitionStorage(storage_ast))
{
throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Is not extended deginition storage {} Will be fixed later.", storage_str);
}
if (storage.order_by)
return storage.order_by->clone();
throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORDER BY cannot be empty");
}
/// Wraps only identifiers with backticks.
@ -191,7 +184,7 @@ std::string wrapIdentifiersWithBackticks(const ASTPtr & root)
return boost::algorithm::join(function_arguments, ", ");
}
throw Exception("Primary key could be represented only as columns or functions from columns.", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key could be represented only as columns or functions from columns.");
}
@ -210,9 +203,9 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast)
size_t sorting_key_size = sorting_key_expr_list->children.size();
if (primary_key_size > sorting_key_size)
throw Exception("Primary key must be a prefix of the sorting key, but its length: "
+ toString(primary_key_size) + " is greater than the sorting key length: " + toString(sorting_key_size),
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key must be a prefix of the sorting key, but its length: "
"{} is greater than the sorting key length: {}",
primary_key_size, sorting_key_size);
Names primary_key_columns;
NameSet primary_key_columns_set;
@ -228,12 +221,12 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast)
{
String pk_column = primary_key_expr_list->children[i]->getColumnName();
if (pk_column != sorting_key_column)
throw Exception("Primary key must be a prefix of the sorting key, but the column in the position "
+ toString(i) + " is " + sorting_key_column +", not " + pk_column,
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Primary key must be a prefix of the sorting key, "
"but the column in the position {} is {}, not {}", i, sorting_key_column, pk_column);
if (!primary_key_columns_set.emplace(pk_column).second)
throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key contains duplicate columns");
primary_key_columns.push_back(wrapIdentifiersWithBackticks(primary_key_expr_list->children[i]));
}
@ -250,9 +243,7 @@ bool isReplicatedTableEngine(const ASTPtr & storage_ast)
if (!endsWith(engine.name, "MergeTree"))
{
String storage_str = queryToString(storage_ast);
throw Exception(
"Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported engine was specified in {}, only *MergeTree engines are supported", storage_str);
}
return startsWith(engine.name, "Replicated");

View File

@ -119,7 +119,7 @@ struct TaskStateWithOwner
rb >> state >> "\n" >> escape >> res.owner;
if (state >= static_cast<int>(TaskState::Unknown))
throw Exception("Unknown state " + data, ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown state {}", data);
res.state = static_cast<TaskState>(state);
return res;

View File

@ -19,7 +19,7 @@ void DB::TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration & config
clusters_prefix = prefix + "remote_servers";
if (!config.has(clusters_prefix))
throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "You should specify list of clusters in {}", clusters_prefix);
Poco::Util::AbstractConfiguration::Keys tables_keys;
config.keys(prefix + "tables", tables_keys);

View File

@ -102,7 +102,7 @@ TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfigurati
for (const String &key : keys)
{
if (!startsWith(key, "partition"))
throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown key {} in {}", key, enabled_partitions_prefix);
enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key));
}
@ -213,8 +213,7 @@ ClusterPartition & TaskTable::getClusterPartition(const String & partition_name)
{
auto it = cluster_partitions.find(partition_name);
if (it == cluster_partitions.end())
throw Exception("There are no cluster partition " + partition_name + " in " + table_id,
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "There are no cluster partition {} in {}", partition_name, table_id);
return it->second;
}

View File

@ -44,7 +44,7 @@ public:
if (command_arguments.size() != 2)
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
String disk_name_from = config.getString("diskFrom", config.getString("disk", "default"));

View File

@ -33,7 +33,7 @@ public:
if (command_arguments.size() != 2)
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
String disk_name = config.getString("disk", "default");

View File

@ -40,7 +40,7 @@ public:
if (command_arguments.size() != 1)
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
String disk_name = config.getString("disk", "default");

View File

@ -32,7 +32,7 @@ public:
if (!command_arguments.empty())
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
for (const auto & [disk_name, _] : global_context->getDisksMap())

View File

@ -41,7 +41,7 @@ public:
if (command_arguments.size() != 1)
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
String disk_name = config.getString("disk", "default");

View File

@ -32,7 +32,7 @@ public:
if (command_arguments.size() != 2)
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
String disk_name = config.getString("disk", "default");

View File

@ -43,7 +43,7 @@ public:
if (command_arguments.size() != 1)
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
String disk_name = config.getString("disk", "default");

View File

@ -32,7 +32,7 @@ public:
if (command_arguments.size() != 1)
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
String disk_name = config.getString("disk", "default");

View File

@ -44,7 +44,7 @@ public:
if (command_arguments.size() != 1)
{
printHelpMessage();
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
String disk_name = config.getString("disk", "default");

View File

@ -119,7 +119,7 @@ void DisksApp::init(std::vector<String> & common_arguments)
{
std::cerr << "Unknown command name: " << command_name << "\n";
printHelpMessage(options_description);
throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments");
}
processOptions();

View File

@ -1160,7 +1160,7 @@ void processLog(const Options & options)
/// Will run multiple processes in parallel
size_t num_threads = options.threads;
if (num_threads == 0)
throw Exception("num-threads cannot be zero", ErrorCodes::INCORRECT_DATA);
throw Exception(ErrorCodes::INCORRECT_DATA, "num-threads cannot be zero");
std::vector<std::unique_ptr<ShellCommand>> show_commands(num_threads);
for (size_t i = 0; i < num_commits && i < num_threads; ++i)

View File

@ -484,8 +484,7 @@ try
config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), true), server_pool, socket));
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.");
#endif
});

View File

@ -169,7 +169,7 @@ std::vector<PODArray<char>> placeStringColumns(const ColumnRawPtrs & columns, si
else if (const auto * column_fixed_string = typeid_cast<const ColumnFixedString *>(column))
data.push_back(placeFixedStringColumn(*column_fixed_string, buffer + i, size));
else
throw Exception("Cannot place string column.", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot place string column.");
}
return data;

View File

@ -32,7 +32,7 @@ ExternalDictionaryLibraryHandler::ExternalDictionaryLibraryHandler(
if (lib_new)
lib_data = lib_new(&settings_holder->strings, ExternalDictionaryLibraryAPI::log);
else
throw Exception("Method extDict_libNew failed", ErrorCodes::EXTERNAL_LIBRARY_ERROR);
throw Exception(ErrorCodes::EXTERNAL_LIBRARY_ERROR, "Method extDict_libNew failed");
}
@ -173,22 +173,21 @@ Block ExternalDictionaryLibraryHandler::loadKeys(const Columns & key_columns)
Block ExternalDictionaryLibraryHandler::dataToBlock(ExternalDictionaryLibraryAPI::RawClickHouseLibraryTable data)
{
if (!data)
throw Exception("LibraryDictionarySource: No data returned", ErrorCodes::EXTERNAL_LIBRARY_ERROR);
throw Exception(ErrorCodes::EXTERNAL_LIBRARY_ERROR, "LibraryDictionarySource: No data returned");
const auto * columns_received = static_cast<const ExternalDictionaryLibraryAPI::Table *>(data);
if (columns_received->error_code)
throw Exception(
"LibraryDictionarySource: Returned error: " + std::to_string(columns_received->error_code) + " " + (columns_received->error_string ? columns_received->error_string : ""),
ErrorCodes::EXTERNAL_LIBRARY_ERROR);
throw Exception(ErrorCodes::EXTERNAL_LIBRARY_ERROR, "LibraryDictionarySource: Returned error: {} {}",
std::to_string(columns_received->error_code), (columns_received->error_string ? columns_received->error_string : ""));
MutableColumns columns = sample_block.cloneEmptyColumns();
for (size_t col_n = 0; col_n < columns_received->size; ++col_n)
{
if (columns.size() != columns_received->data[col_n].size)
throw Exception(
"LibraryDictionarySource: Returned unexpected number of columns: " + std::to_string(columns_received->data[col_n].size) + ", must be " + std::to_string(columns.size()),
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "LibraryDictionarySource: "
"Returned unexpected number of columns: {}, must be {}",
columns_received->data[col_n].size, columns.size());
for (size_t row_n = 0; row_n < columns_received->data[col_n].size; ++row_n)
{

View File

@ -359,7 +359,7 @@ void LocalServer::setupUsers()
if (users_config)
global_context->setUsersConfig(users_config);
else
throw Exception("Can't load config for users", ErrorCodes::CANNOT_LOAD_CONFIG);
throw Exception(ErrorCodes::CANNOT_LOAD_CONFIG, "Can't load config for users");
}
void LocalServer::connect()
@ -489,7 +489,7 @@ void LocalServer::processConfig()
if (is_interactive && !delayed_interactive)
{
if (config().has("query") && config().has("queries-file"))
throw Exception("Specify either `query` or `queries-file` option", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specify either `query` or `queries-file` option");
if (config().has("multiquery"))
is_multiquery = true;

View File

@ -880,7 +880,7 @@ public:
}
if (!it)
throw Exception("Logical error in markov model", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error in markov model");
size_t offset_from_begin_of_string = pos - data;
size_t determinator_sliding_window_size = params.determinator_sliding_window_size;
@ -1139,7 +1139,7 @@ public:
if (const auto * type = typeid_cast<const DataTypeNullable *>(&data_type))
return std::make_unique<NullableModel>(get(*type->getNestedType(), seed, markov_model_params));
throw Exception("Unsupported data type", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported data type");
}
};
@ -1384,7 +1384,7 @@ try
UInt8 version = 0;
readBinary(version, model_in);
if (version != 0)
throw Exception("Unknown version of the model file", ErrorCodes::UNKNOWN_FORMAT_VERSION);
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown version of the model file");
readBinary(source_rows, model_in);
@ -1392,14 +1392,14 @@ try
size_t header_size = 0;
readBinary(header_size, model_in);
if (header_size != data_types.size())
throw Exception("The saved model was created for different number of columns", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "The saved model was created for different number of columns");
for (size_t i = 0; i < header_size; ++i)
{
String type;
readBinary(type, model_in);
if (type != data_types[i])
throw Exception("The saved model was created for different types of columns", ErrorCodes::TYPE_MISMATCH);
throw Exception(ErrorCodes::TYPE_MISMATCH, "The saved model was created for different types of columns");
}
obfuscator.deserialize(model_in);

View File

@ -181,7 +181,7 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ
}
if (columns.empty())
throw Exception("Columns definition was not returned", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns definition was not returned");
WriteBufferFromHTTPServerResponse out(
response,

View File

@ -163,7 +163,7 @@ void ODBCSource::insertValue(
break;
}
default:
throw Exception("Unsupported value type", ErrorCodes::UNKNOWN_TYPE);
throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unsupported value type");
}
}

View File

@ -151,7 +151,7 @@ public:
auto connection_available = pool->tryBorrowObject(connection, []() { return nullptr; }, ODBC_POOL_WAIT_TIMEOUT);
if (!connection_available)
throw Exception("Unable to fetch connection within the timeout", ErrorCodes::NO_FREE_CONNECTION);
throw Exception(ErrorCodes::NO_FREE_CONNECTION, "Unable to fetch connection within the timeout");
try
{

View File

@ -44,7 +44,8 @@ IdentifierQuotingStyle getQuotingStyle(nanodbc::ConnectionHolderPtr connection)
else if (identifier_quote[0] == '"')
return IdentifierQuotingStyle::DoubleQuotes;
else
throw Exception("Can not map quote identifier '" + identifier_quote + "' to IdentifierQuotingStyle value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Can not map quote identifier '{}' to IdentifierQuotingStyle value", identifier_quote);
}
}

View File

@ -38,10 +38,10 @@ std::string validateODBCConnectionString(const std::string & connection_string)
static constexpr size_t MAX_CONNECTION_STRING_SIZE = 1000;
if (connection_string.empty())
throw Exception("ODBC connection string cannot be empty", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string cannot be empty");
if (connection_string.size() >= MAX_CONNECTION_STRING_SIZE)
throw Exception("ODBC connection string is too long", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string is too long");
const char * pos = connection_string.data();
const char * end = pos + connection_string.size();
@ -51,7 +51,7 @@ std::string validateODBCConnectionString(const std::string & connection_string)
while (pos < end && isWhitespaceASCII(*pos))
{
if (*pos != ' ')
throw Exception("ODBC connection string parameter contains unusual whitespace character", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string parameter contains unusual whitespace character");
++pos;
}
};
@ -63,7 +63,8 @@ std::string validateODBCConnectionString(const std::string & connection_string)
if (pos < end && isValidIdentifierBegin(*pos))
++pos;
else
throw Exception("ODBC connection string parameter name doesn't begin with valid identifier character", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING,
"ODBC connection string parameter name doesn't begin with valid identifier character");
/// Additionally allow dash and dot symbols in names.
/// Strictly speaking, the name with that characters should be escaped.
@ -83,7 +84,8 @@ std::string validateODBCConnectionString(const std::string & connection_string)
{
signed char c = *pos;
if (c < 32 || strchr("[]{}(),;?*=!@'\"", c) != nullptr)
throw Exception("ODBC connection string parameter value is unescaped and contains illegal character", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING,
"ODBC connection string parameter value is unescaped and contains illegal character");
++pos;
}
@ -97,7 +99,7 @@ std::string validateODBCConnectionString(const std::string & connection_string)
if (pos < end && *pos == '{')
++pos;
else
throw Exception("ODBC connection string parameter value doesn't begin with opening curly brace", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string parameter value doesn't begin with opening curly brace");
while (pos < end)
{
@ -109,13 +111,13 @@ std::string validateODBCConnectionString(const std::string & connection_string)
}
if (*pos == 0)
throw Exception("ODBC connection string parameter value contains ASCII NUL character", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string parameter value contains ASCII NUL character");
res += *pos;
++pos;
}
throw Exception("ODBC connection string parameter is escaped but there is no closing curly brace", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string parameter is escaped but there is no closing curly brace");
};
auto read_value = [&]
@ -139,25 +141,25 @@ std::string validateODBCConnectionString(const std::string & connection_string)
Poco::toUpperInPlace(name);
if (name == "FILEDSN" || name == "SAVEFILE" || name == "DRIVER")
throw Exception("ODBC connection string has forbidden parameter", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string has forbidden parameter");
if (pos >= end)
throw Exception("ODBC connection string parameter doesn't have value", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string parameter doesn't have value");
if (*pos == '=')
++pos;
else
throw Exception("ODBC connection string parameter doesn't have value", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string parameter doesn't have value");
skip_whitespaces();
std::string value = read_value();
skip_whitespaces();
if (name.size() > MAX_ELEMENT_SIZE || value.size() > MAX_ELEMENT_SIZE)
throw Exception("ODBC connection string has too long keyword or value", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string has too long keyword or value");
if (!parameters.emplace(name, value).second)
throw Exception("Duplicate parameter found in ODBC connection string", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "Duplicate parameter found in ODBC connection string");
if (pos >= end)
break;
@ -165,7 +167,7 @@ std::string validateODBCConnectionString(const std::string & connection_string)
if (*pos == ';')
++pos;
else
throw Exception("Unexpected character found after parameter value in ODBC connection string", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "Unexpected character found after parameter value in ODBC connection string");
}
/// Reconstruct the connection string.
@ -173,12 +175,12 @@ std::string validateODBCConnectionString(const std::string & connection_string)
auto it = parameters.find("DSN");
if (parameters.end() == it)
throw Exception("DSN parameter is mandatory for ODBC connection string", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "DSN parameter is mandatory for ODBC connection string");
std::string dsn = it->second;
if (dsn.empty())
throw Exception("DSN parameter cannot be empty in ODBC connection string", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "DSN parameter cannot be empty in ODBC connection string");
parameters.erase(it);
@ -241,7 +243,7 @@ std::string validateODBCConnectionString(const std::string & connection_string)
write_element(elem.first, elem.second);
if (reconstructed_connection_string.size() >= MAX_CONNECTION_STRING_SIZE)
throw Exception("ODBC connection string is too long", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
throw Exception(ErrorCodes::BAD_ODBC_CONNECTION_STRING, "ODBC connection string is too long");
return reconstructed_connection_string;
}

View File

@ -257,7 +257,7 @@ static std::string getCanonicalPath(std::string && path)
{
Poco::trimInPlace(path);
if (path.empty())
throw Exception("path configuration parameter is empty", ErrorCodes::INVALID_CONFIG_PARAMETER);
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "path configuration parameter is empty");
if (path.back() != '/')
path += '/';
return std::move(path);
@ -1116,7 +1116,7 @@ try
#endif
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Both http and https interserver ports are specified");
static const auto interserver_tags =
{
@ -1141,7 +1141,7 @@ try
int port = parse<int>(port_str);
if (port < 0 || port > 0xFFFF)
throw Exception("Out of range '" + String(port_tag) + "': " + toString(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Out of range '{}': {}", String(port_tag), port);
global_context->setInterserverIOAddress(this_host, port);
global_context->setInterserverScheme(scheme);
@ -1419,8 +1419,7 @@ try
global_context->getSettingsRef().send_timeout.totalSeconds(), true), server_pool, socket));
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.");
#endif
});
}
@ -1749,14 +1748,15 @@ try
std::lock_guard lock(servers_lock);
createServers(config(), listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers);
if (servers.empty())
throw Exception(
"No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG,
"No servers started (add valid listen_host and 'tcp_port' or 'http_port' "
"to configuration file.)");
}
if (servers.empty())
throw Exception("No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG,
"No servers started (add valid listen_host and 'tcp_port' or 'http_port' "
"to configuration file.)");
#if USE_SSL
CertificateReloader::instance().tryLoad(config());
@ -1816,7 +1816,7 @@ try
String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/");
int pool_size = config().getInt("distributed_ddl.pool_size", 1);
if (pool_size < 1)
throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "distributed_ddl.pool_size should be greater then 0");
global_context->setDDLWorker(std::make_unique<DDLWorker>(pool_size, ddl_zookeeper_path, global_context, &config(),
"distributed_ddl", "DDLWorker",
&CurrentMetrics::MaxDDLEntryID, &CurrentMetrics::MaxPushedDDLEntryID));
@ -1945,8 +1945,7 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
#if USE_SSL
return TCPServerConnectionFactory::Ptr(new TLSHandlerFactory(*this, conf_name));
#else
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.");
#endif
if (type == "proxy1")
@ -2113,8 +2112,7 @@ void Server::createServers(
httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
#else
UNUSED(port);
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS protocol is disabled because Poco library was built without NetSSL support.");
#endif
});
@ -2176,8 +2174,7 @@ void Server::createServers(
new Poco::Net::TCPServerParams));
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.");
#endif
});
@ -2282,8 +2279,7 @@ void Server::createServers(
http_params));
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.");
#endif
});
}

View File

@ -65,7 +65,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid)
}
if (gid == 0 && getgid() != 0)
throw Exception("Group has id 0, but dropping privileges to gid 0 does not make sense", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Group has id 0, but dropping privileges to gid 0 does not make sense");
if (0 != setgid(gid))
throwFromErrno(fmt::format("Cannot do 'setgid' to user ({})", arg_gid), ErrorCodes::SYSTEM_ERROR);
@ -90,7 +90,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid)
}
if (uid == 0 && getuid() != 0)
throw Exception("User has id 0, but dropping privileges to uid 0 does not make sense", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "User has id 0, but dropping privileges to uid 0 does not make sense");
if (0 != setuid(uid))
throwFromErrno(fmt::format("Cannot do 'setuid' to user ({})", arg_uid), ErrorCodes::SYSTEM_ERROR);

View File

@ -289,7 +289,7 @@ namespace
}
default:
throw Exception("Unknown type: " + toString(entity_type), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type: {}", toString(entity_type));
}
}
return res;

View File

@ -126,10 +126,10 @@ public:
std::lock_guard lock{mutex};
if (!registered_prefixes.empty())
{
throw Exception(
"Setting " + String{setting_name} + " is neither a builtin setting nor started with the prefix '"
+ boost::algorithm::join(registered_prefixes, "' or '") + "' registered for user-defined settings",
ErrorCodes::UNKNOWN_SETTING);
throw Exception(ErrorCodes::UNKNOWN_SETTING,
"Setting {} is neither a builtin setting nor started with the prefix '{}"
"' registered for user-defined settings",
String{setting_name}, boost::algorithm::join(registered_prefixes, "' or '"));
}
else
BaseSettingsHelpers::throwSettingNotFound(setting_name);
@ -450,7 +450,7 @@ void AccessControl::addStoragesFromUserDirectoriesConfig(
addReplicatedStorage(name, zookeeper_path, get_zookeeper_function, allow_backup);
}
else
throw Exception("Unknown storage type '" + type + "' at " + prefix + " in config", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown storage type '{}' at {} in config", type, prefix);
}
}

View File

@ -80,53 +80,53 @@ AccessEntityPtr deserializeAccessEntityImpl(const String & definition)
if (auto * create_user_query = query->as<ASTCreateUserQuery>())
{
if (res)
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION, "Two access entities attached in the same file");
res = user = std::make_unique<User>();
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query, /* allow_no_password = */ true, /* allow_plaintext_password = */ true);
}
else if (auto * create_role_query = query->as<ASTCreateRoleQuery>())
{
if (res)
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION, "Two access entities attached in the same file");
res = role = std::make_unique<Role>();
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
}
else if (auto * create_policy_query = query->as<ASTCreateRowPolicyQuery>())
{
if (res)
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION, "Two access entities attached in the same file");
res = policy = std::make_unique<RowPolicy>();
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
}
else if (auto * create_quota_query = query->as<ASTCreateQuotaQuery>())
{
if (res)
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION, "Two access entities attached in the same file");
res = quota = std::make_unique<Quota>();
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
}
else if (auto * create_profile_query = query->as<ASTCreateSettingsProfileQuery>())
{
if (res)
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION, "Two access entities attached in the same file");
res = profile = std::make_unique<SettingsProfile>();
InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query);
}
else if (auto * grant_query = query->as<ASTGrantQuery>())
{
if (!user && !role)
throw Exception("A user or role should be attached before grant", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION, "A user or role should be attached before grant");
if (user)
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
else
InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query);
}
else
throw Exception("No interpreter found for query " + query->getID(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION, "No interpreter found for query {}", query->getID());
}
if (!res)
throw Exception("No access entities attached", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION, "No access entities attached");
return res;
}

View File

@ -797,7 +797,7 @@ template <bool with_grant_option>
void AccessRights::grantImpl(const AccessRightsElement & element)
{
if (element.is_partial_revoke)
throw Exception("A partial revoke should be revoked, not granted", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "A partial revoke should be revoked, not granted");
if constexpr (with_grant_option)
{
grantImplHelper<true>(element);

View File

@ -176,7 +176,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const
if ([[maybe_unused]] const auto * always_allow_credentials = typeid_cast<const AlwaysAllowCredentials *>(&credentials))
return true;
throw Exception("areCredentialsValid(): authentication type " + toString(auth_data.getType()) + " not supported", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "areCredentialsValid(): authentication type {} not supported", toString(auth_data.getType()));
}
}

View File

@ -83,7 +83,7 @@ const AccessEntityTypeInfo & AccessEntityTypeInfo::get(AccessEntityType type_)
}
case AccessEntityType::MAX: break;
}
throw Exception("Unknown type: " + std::to_string(static_cast<size_t>(type_)), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type: {}", static_cast<size_t>(type_));
}
AccessEntityType AccessEntityTypeInfo::parseType(const String & name_)

View File

@ -44,7 +44,7 @@ namespace
boost::to_upper(uppercased_keyword);
it = keyword_to_flags_map.find(uppercased_keyword);
if (it == keyword_to_flags_map.end())
throw Exception("Unknown access type: " + String(keyword), ErrorCodes::UNKNOWN_ACCESS_TYPE);
throw Exception(ErrorCodes::UNKNOWN_ACCESS_TYPE, "Unknown access type: {}", String(keyword));
}
return it->second;
}
@ -179,7 +179,7 @@ namespace
else
{
if (nodes.contains(keyword))
throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "{} declared twice", keyword);
node = std::make_unique<Node>(keyword, node_type);
nodes[node->keyword] = node.get();
}
@ -225,9 +225,9 @@ namespace
# undef MAKE_ACCESS_FLAGS_NODE
if (!owned_nodes.contains("NONE"))
throw Exception("'NONE' not declared", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "'NONE' not declared");
if (!owned_nodes.contains("ALL"))
throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "'ALL' not declared");
all_node = std::move(owned_nodes["ALL"]);
none_node = std::move(owned_nodes["NONE"]);
@ -238,9 +238,9 @@ namespace
{
const auto & unused_node = *(owned_nodes.begin()->second);
if (unused_node.node_type == UNKNOWN)
throw Exception("Parent group '" + unused_node.keyword + "' not found", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent group '{}' not found", unused_node.keyword);
else
throw Exception("Access type '" + unused_node.keyword + "' should have parent group", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Access type '{}' should have parent group", unused_node.keyword);
}
}

View File

@ -67,7 +67,7 @@ const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType ty
case AuthenticationType::MAX:
break;
}
throw Exception("Unknown authentication type: " + std::to_string(static_cast<int>(type_)), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown authentication type: {}", static_cast<int>(type_));
}
@ -119,19 +119,19 @@ void AuthenticationData::setPassword(const String & password_)
case AuthenticationType::LDAP:
case AuthenticationType::KERBEROS:
case AuthenticationType::SSL_CERTIFICATE:
throw Exception("Cannot specify password for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify password for authentication type {}", toString(type));
case AuthenticationType::MAX:
break;
}
throw Exception("setPassword(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "setPassword(): authentication type {} not supported", toString(type));
}
String AuthenticationData::getPassword() const
{
if (type != AuthenticationType::PLAINTEXT_PASSWORD)
throw Exception("Cannot decode the password", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot decode the password");
return String(password_hash.data(), password_hash.data() + password_hash.size());
}
@ -157,7 +157,7 @@ void AuthenticationData::setPasswordHashHex(const String & hash)
String AuthenticationData::getPasswordHashHex() const
{
if (type == AuthenticationType::LDAP || type == AuthenticationType::KERBEROS || type == AuthenticationType::SSL_CERTIFICATE)
throw Exception("Cannot get password hex hash for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get password hex hash for authentication type {}", toString(type));
String hex;
hex.resize(password_hash.size() * 2);
@ -179,10 +179,9 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash)
case AuthenticationType::SHA256_PASSWORD:
{
if (hash.size() != 32)
throw Exception(
"Password hash for the 'SHA256_PASSWORD' authentication type has length " + std::to_string(hash.size())
+ " but must be exactly 32 bytes.",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Password hash for the 'SHA256_PASSWORD' authentication type has length {} "
"but must be exactly 32 bytes.", hash.size());
password_hash = hash;
return;
}
@ -190,10 +189,9 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash)
case AuthenticationType::DOUBLE_SHA1_PASSWORD:
{
if (hash.size() != 20)
throw Exception(
"Password hash for the 'DOUBLE_SHA1_PASSWORD' authentication type has length " + std::to_string(hash.size())
+ " but must be exactly 20 bytes.",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Password hash for the 'DOUBLE_SHA1_PASSWORD' authentication type has length {} "
"but must be exactly 20 bytes.", hash.size());
password_hash = hash;
return;
}
@ -202,18 +200,18 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash)
case AuthenticationType::LDAP:
case AuthenticationType::KERBEROS:
case AuthenticationType::SSL_CERTIFICATE:
throw Exception("Cannot specify password binary hash for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify password binary hash for authentication type {}", toString(type));
case AuthenticationType::MAX:
break;
}
throw Exception("setPasswordHashBinary(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "setPasswordHashBinary(): authentication type {} not supported", toString(type));
}
void AuthenticationData::setSalt(String salt_)
{
if (type != AuthenticationType::SHA256_PASSWORD)
throw Exception("setSalt(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "setSalt(): authentication type {} not supported", toString(type));
salt = std::move(salt_);
}
@ -225,7 +223,7 @@ String AuthenticationData::getSalt() const
void AuthenticationData::setSSLCertificateCommonNames(boost::container::flat_set<String> common_names_)
{
if (common_names_.empty())
throw Exception("The 'SSL CERTIFICATE' authentication type requires a non-empty list of common names.", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The 'SSL CERTIFICATE' authentication type requires a non-empty list of common names.");
ssl_certificate_common_names = std::move(common_names_);
}

View File

@ -113,7 +113,7 @@ const QuotaTypeInfo & QuotaTypeInfo::get(QuotaType type)
}
case QuotaType::MAX: break;
}
throw Exception("Unexpected quota type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected quota type: {}", static_cast<int>(type));
}
String toString(QuotaKeyType type)
@ -187,7 +187,7 @@ const QuotaKeyTypeInfo & QuotaKeyTypeInfo::get(QuotaKeyType type)
}
case QuotaKeyType::MAX: break;
}
throw Exception("Unexpected quota key type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected quota key type: {}", static_cast<int>(type));
}
}

View File

@ -75,7 +75,7 @@ const RowPolicyFilterTypeInfo & RowPolicyFilterTypeInfo::get(RowPolicyFilterType
#endif
case RowPolicyFilterType::MAX: break;
}
throw Exception("Unknown type: " + std::to_string(static_cast<size_t>(type_)), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type: {}", static_cast<size_t>(type_));
}
}

View File

@ -470,7 +470,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
/// If the current user has been dropped we always throw an exception (even if `throw_if_denied` is false)
/// because dropping of the current user is considered as a situation which is exceptional enough to stop
/// query execution.
throw Exception(getUserName() + ": User has been dropped", ErrorCodes::UNKNOWN_USER);
throw Exception(ErrorCodes::UNKNOWN_USER, "{}: User has been dropped", getUserName());
}
if (is_full_access)
@ -790,7 +790,7 @@ void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccess
auto current_user = getUser();
if (!current_user->grantees.match(grantee_id))
throw Exception(grantee.formatTypeWithName() + " is not allowed as grantee", ErrorCodes::ACCESS_DENIED);
throw Exception(ErrorCodes::ACCESS_DENIED, "{} is not allowed as grantee", grantee.formatTypeWithName());
}
void ContextAccess::checkGranteesAreAllowed(const std::vector<UUID> & grantee_ids) const

View File

@ -29,7 +29,7 @@ bool Credentials::isReady() const
void Credentials::throwNotReady()
{
throw Exception("Credentials are not ready", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Credentials are not ready");
}
AlwaysAllowCredentials::AlwaysAllowCredentials()

View File

@ -172,7 +172,8 @@ DiskAccessStorage::DiskAccessStorage(const String & storage_name_, const String
std::filesystem::create_directories(directory_path, create_dir_error_code);
if (!std::filesystem::exists(directory_path) || !std::filesystem::is_directory(directory_path) || create_dir_error_code)
throw Exception("Couldn't create directory " + directory_path + " reason: '" + create_dir_error_code.message() + "'", ErrorCodes::DIRECTORY_DOESNT_EXIST);
throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "Couldn't create directory {} reason: '{}'",
directory_path, create_dir_error_code.message());
bool should_rebuild_lists = std::filesystem::exists(getNeedRebuildListsMarkFilePath(directory_path));
if (!should_rebuild_lists)
@ -722,7 +723,7 @@ void DiskAccessStorage::deleteAccessEntityOnDisk(const UUID & id) const
{
auto file_path = getEntityFilePath(directory_path, id);
if (!std::filesystem::remove(file_path))
throw Exception("Couldn't delete " + file_path, ErrorCodes::FILE_DOESNT_EXIST);
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Couldn't delete {}", file_path);
}

View File

@ -29,11 +29,11 @@ struct EnabledQuota::Impl
std::chrono::system_clock::time_point end_of_interval)
{
const auto & type_info = QuotaTypeInfo::get(quota_type);
throw Exception(
"Quota for user " + backQuote(user_name) + " for " + to_string(duration) + " has been exceeded: "
+ type_info.valueToStringWithName(used) + "/" + type_info.valueToString(max) + ". "
+ "Interval will end at " + to_string(end_of_interval) + ". " + "Name of quota template: " + backQuote(quota_name),
ErrorCodes::QUOTA_EXCEEDED);
throw Exception(ErrorCodes::QUOTA_EXCEEDED, "Quota for user {} for {} has been exceeded: {}/{}. "
"Interval will end at {}. Name of quota template: {}",
backQuote(user_name), to_string(duration),
type_info.valueToStringWithName(used),
type_info.valueToString(max), to_string(end_of_interval), backQuote(quota_name));
}

View File

@ -47,15 +47,16 @@ void parseLDAPSearchParams(LDAPClient::SearchParams & params, const Poco::Util::
else if (scope == "subtree") params.scope = LDAPClient::SearchParams::Scope::SUBTREE;
else if (scope == "children") params.scope = LDAPClient::SearchParams::Scope::CHILDREN;
else
throw Exception("Invalid value for 'scope' field of LDAP search parameters in '" + prefix +
"' section, must be one of 'base', 'one_level', 'subtree', or 'children'", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Invalid value for 'scope' field of LDAP search parameters "
"in '{}' section, must be one of 'base', 'one_level', 'subtree', or 'children'", prefix);
}
}
void parseLDAPServer(LDAPClient::Params & params, const Poco::Util::AbstractConfiguration & config, const String & name)
{
if (name.empty())
throw Exception("LDAP server name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "LDAP server name cannot be empty");
const String ldap_server_config = "ldap_servers." + name;
@ -77,17 +78,17 @@ void parseLDAPServer(LDAPClient::Params & params, const Poco::Util::AbstractConf
const bool has_search_limit = config.has(ldap_server_config + ".search_limit");
if (!has_host)
throw Exception("Missing 'host' entry", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Missing 'host' entry");
params.host = config.getString(ldap_server_config + ".host");
if (params.host.empty())
throw Exception("Empty 'host' entry", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty 'host' entry");
if (has_bind_dn)
{
if (has_auth_dn_prefix || has_auth_dn_suffix)
throw Exception("Deprecated 'auth_dn_prefix' and 'auth_dn_suffix' entries cannot be used with 'bind_dn' entry", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Deprecated 'auth_dn_prefix' and 'auth_dn_suffix' entries cannot be used with 'bind_dn' entry");
params.bind_dn = config.getString(ldap_server_config + ".bind_dn");
}
@ -141,7 +142,9 @@ void parseLDAPServer(LDAPClient::Params & params, const Poco::Util::AbstractConf
else if (tls_minimum_protocol_version_lc_str == "tls1.2")
params.tls_minimum_protocol_version = LDAPClient::Params::TLSProtocolVersion::TLS1_2; //-V1048
else
throw Exception("Bad value for 'tls_minimum_protocol_version' entry, allowed values are: 'ssl2', 'ssl3', 'tls1.0', 'tls1.1', 'tls1.2'", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Bad value for 'tls_minimum_protocol_version' entry, allowed values are: "
"'ssl2', 'ssl3', 'tls1.0', 'tls1.1', 'tls1.2'");
}
if (has_tls_require_cert)
@ -158,7 +161,9 @@ void parseLDAPServer(LDAPClient::Params & params, const Poco::Util::AbstractConf
else if (tls_require_cert_lc_str == "demand")
params.tls_require_cert = LDAPClient::Params::TLSRequireCert::DEMAND; //-V1048
else
throw Exception("Bad value for 'tls_require_cert' entry, allowed values are: 'never', 'allow', 'try', 'demand'", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Bad value for 'tls_require_cert' entry, allowed values are: "
"'never', 'allow', 'try', 'demand'");
}
if (has_tls_cert_file)
@ -180,7 +185,7 @@ void parseLDAPServer(LDAPClient::Params & params, const Poco::Util::AbstractConf
{
UInt32 port = config.getUInt(ldap_server_config + ".port");
if (port > 65535)
throw Exception("Bad value for 'port' entry", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad value for 'port' entry");
params.port = port;
}
@ -212,13 +217,13 @@ void parseKerberosParams(GSSAcceptorContext::Params & params, const Poco::Util::
}
if (reealm_key_count > 0 && principal_keys_count > 0)
throw Exception("Realm and principal name cannot be specified simultaneously", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Realm and principal name cannot be specified simultaneously");
if (reealm_key_count > 1)
throw Exception("Multiple realm sections are not allowed", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple realm sections are not allowed");
if (principal_keys_count > 1)
throw Exception("Multiple principal sections are not allowed", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple principal sections are not allowed");
params.realm = config.getString("kerberos.realm", "");
params.principal = config.getString("kerberos.principal", "");
@ -274,10 +279,10 @@ void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfigur
}
if (ldap_servers_key_count > 1)
throw Exception("Multiple ldap_servers sections are not allowed", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple ldap_servers sections are not allowed");
if (kerberos_keys_count > 1)
throw Exception("Multiple kerberos sections are not allowed", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple kerberos sections are not allowed");
Poco::Util::AbstractConfiguration::Keys ldap_server_names;
config.keys("ldap_servers", ldap_server_names);
@ -291,7 +296,7 @@ void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfigur
ldap_server_name.resize(bracket_pos);
if (ldap_client_params_blueprint.contains(ldap_server_name))
throw Exception("Multiple LDAP servers with the same name are not allowed", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple LDAP servers with the same name are not allowed");
LDAPClient::Params ldap_client_params_tmp;
parseLDAPServer(ldap_client_params_tmp, config, ldap_server_name);
@ -346,7 +351,7 @@ bool ExternalAuthenticators::checkLDAPCredentials(const String & server, const B
// Retrieve the server parameters.
const auto pit = ldap_client_params_blueprint.find(server);
if (pit == ldap_client_params_blueprint.end())
throw Exception("LDAP server '" + server + "' is not configured", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "LDAP server '{}' is not configured", server);
params = pit->second;
params->user = credentials.getUserName();
@ -461,7 +466,7 @@ bool ExternalAuthenticators::checkKerberosCredentials(const String & realm, cons
std::scoped_lock lock(mutex);
if (!kerberos_params.has_value())
throw Exception("Kerberos is not enabled", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Kerberos is not enabled");
if (!credentials.isReady())
return false;
@ -480,7 +485,7 @@ GSSAcceptorContext::Params ExternalAuthenticators::getKerberosParams() const
std::scoped_lock lock(mutex);
if (!kerberos_params.has_value())
throw Exception("Kerberos is not enabled", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Kerberos is not enabled");
return kerberos_params.value();
}

View File

@ -265,16 +265,16 @@ void GSSAcceptorContext::initHandles()
if (!params.keytab.empty())
{
if (!std::filesystem::exists(params.keytab))
throw Exception("Keytab file not found", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Keytab file not found");
if (krb5_gss_register_acceptor_identity(params.keytab.c_str()))
throw Exception("Failed to register keytab file", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to register keytab file");
}
if (!params.principal.empty())
{
if (!params.realm.empty())
throw Exception("Realm and principal name cannot be specified simultaneously", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Realm and principal name cannot be specified simultaneously");
gss_buffer_desc acceptor_name_buf;
acceptor_name_buf.length = params.principal.size();
@ -305,7 +305,7 @@ void GSSAcceptorContext::initHandles()
if (GSS_ERROR(major_status))
{
const auto messages = extractStatusMessages(major_status, minor_status, GSS_C_NO_OID);
throw Exception("gss_import_name() failed" + (messages.empty() ? "" : ": " + messages), ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR, "gss_import_name() failed{}", (messages.empty() ? "" : ": " + messages));
}
minor_status = 0;
@ -323,7 +323,7 @@ void GSSAcceptorContext::initHandles()
if (GSS_ERROR(major_status))
{
const auto messages = extractStatusMessages(major_status, minor_status, GSS_C_NO_OID);
throw Exception("gss_acquire_cred() failed" + (messages.empty() ? "" : ": " + messages), ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR, "gss_acquire_cred() failed{}", (messages.empty() ? "" : ": " + messages));
}
}
}
@ -387,21 +387,26 @@ String GSSAcceptorContext::processToken(const String & input_token, Poco::Logger
if (major_status == GSS_S_COMPLETE)
{
if (!params.mechanism.empty() && !equalMechanisms(params.mechanism, mech_type))
throw Exception("gss_accept_sec_context() succeeded, but: the authentication mechanism is not what was expected", ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR,
"gss_accept_sec_context() succeeded, but: "
"the authentication mechanism is not what was expected");
if (flags & GSS_C_ANON_FLAG)
throw Exception("gss_accept_sec_context() succeeded, but: the initiator does not wish to be authenticated", ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR, "gss_accept_sec_context() succeeded, but: the initiator does not wish to be authenticated");
std::tie(user_name, realm) = extractNameAndRealm(initiator_name);
if (user_name.empty())
throw Exception("gss_accept_sec_context() succeeded, but: the initiator name cannot be extracted", ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR, "gss_accept_sec_context() succeeded, but: the initiator name cannot be extracted");
if (realm.empty())
throw Exception("gss_accept_sec_context() succeeded, but: the initiator realm cannot be extracted", ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR, "gss_accept_sec_context() succeeded, but: the initiator realm cannot be extracted");
if (!params.realm.empty() && params.realm != realm)
throw Exception("gss_accept_sec_context() succeeded, but: the initiator realm is not what was expected (expected: " + params.realm + ", actual: " + realm + ")", ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR,
"gss_accept_sec_context() succeeded, but: "
"the initiator realm is not what was expected (expected: {}, actual: {})",
params.realm, realm);
output_token = bufferToString(output_token_buf);
@ -420,7 +425,7 @@ String GSSAcceptorContext::processToken(const String & input_token, Poco::Logger
else
{
const auto messages = extractStatusMessages(major_status, minor_status, mech_type);
throw Exception("gss_accept_sec_context() failed" + (messages.empty() ? "" : ": " + messages), ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR, "gss_accept_sec_context() failed{}", (messages.empty() ? "" : ": " + messages));
}
}
catch (...)
@ -452,7 +457,7 @@ void GSSAcceptorContext::initHandles()
String GSSAcceptorContext::processToken(const String &, Poco::Logger *)
{
throw Exception("ClickHouse was built without GSS-API/Kerberos support", ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME);
throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without GSS-API/Kerberos support");
}
#endif // USE_KRB5

View File

@ -630,79 +630,70 @@ Poco::Logger * IAccessStorage::getLogger() const
void IAccessStorage::throwNotFound(const UUID & id) const
{
throw Exception(outputID(id) + " not found in " + getStorageName(), ErrorCodes::ACCESS_ENTITY_NOT_FOUND);
throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "{} not found in {}", outputID(id), getStorageName());
}
void IAccessStorage::throwNotFound(AccessEntityType type, const String & name) const
{
int error_code = AccessEntityTypeInfo::get(type).not_found_error_code;
throw Exception("There is no " + formatEntityTypeWithName(type, name) + " in " + getStorageName(), error_code);
throw Exception(error_code, "There is no {} in {}", formatEntityTypeWithName(type, name), getStorageName());
}
void IAccessStorage::throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type)
{
throw Exception(
outputID(id) + ": " + formatEntityTypeWithName(type, name) + " expected to be of type " + toString(required_type),
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: {} expected to be of type {}", outputID(id),
formatEntityTypeWithName(type, name), toString(required_type));
}
void IAccessStorage::throwIDCollisionCannotInsert(const UUID & id, AccessEntityType type, const String & name, AccessEntityType existing_type, const String & existing_name) const
{
throw Exception(
formatEntityTypeWithName(type, name) + ": cannot insert because the " + outputID(id) + " is already used by "
+ formatEntityTypeWithName(existing_type, existing_name) + " in " + getStorageName(),
ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS);
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "{}: "
"cannot insert because the {} is already used by {} in {}", formatEntityTypeWithName(type, name),
outputID(id), formatEntityTypeWithName(existing_type, existing_name), getStorageName());
}
void IAccessStorage::throwNameCollisionCannotInsert(AccessEntityType type, const String & name) const
{
throw Exception(
formatEntityTypeWithName(type, name) + ": cannot insert because " + formatEntityTypeWithName(type, name) + " already exists in "
+ getStorageName(),
ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS);
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "{}: cannot insert because {} already exists in {}",
formatEntityTypeWithName(type, name), formatEntityTypeWithName(type, name), getStorageName());
}
void IAccessStorage::throwNameCollisionCannotRename(AccessEntityType type, const String & old_name, const String & new_name) const
{
throw Exception(
formatEntityTypeWithName(type, old_name) + ": cannot rename to " + backQuote(new_name) + " because "
+ formatEntityTypeWithName(type, new_name) + " already exists in " + getStorageName(),
ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS);
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "{}: cannot rename to {} because {} already exists in {}",
formatEntityTypeWithName(type, old_name), backQuote(new_name), formatEntityTypeWithName(type, new_name), getStorageName());
}
void IAccessStorage::throwReadonlyCannotInsert(AccessEntityType type, const String & name) const
{
throw Exception(
"Cannot insert " + formatEntityTypeWithName(type, name) + " to " + getStorageName() + " because this storage is readonly",
ErrorCodes::ACCESS_STORAGE_READONLY);
throw Exception(ErrorCodes::ACCESS_STORAGE_READONLY, "Cannot insert {} to {} because this storage is readonly",
formatEntityTypeWithName(type, name), getStorageName());
}
void IAccessStorage::throwReadonlyCannotUpdate(AccessEntityType type, const String & name) const
{
throw Exception(
"Cannot update " + formatEntityTypeWithName(type, name) + " in " + getStorageName() + " because this storage is readonly",
ErrorCodes::ACCESS_STORAGE_READONLY);
throw Exception(ErrorCodes::ACCESS_STORAGE_READONLY, "Cannot update {} in {} because this storage is readonly",
formatEntityTypeWithName(type, name), getStorageName());
}
void IAccessStorage::throwReadonlyCannotRemove(AccessEntityType type, const String & name) const
{
throw Exception(
"Cannot remove " + formatEntityTypeWithName(type, name) + " from " + getStorageName() + " because this storage is readonly",
ErrorCodes::ACCESS_STORAGE_READONLY);
throw Exception(ErrorCodes::ACCESS_STORAGE_READONLY, "Cannot remove {} from {} because this storage is readonly",
formatEntityTypeWithName(type, name), getStorageName());
}
void IAccessStorage::throwAddressNotAllowed(const Poco::Net::IPAddress & address)
{
throw Exception("Connections from " + address.toString() + " are not allowed", ErrorCodes::IP_ADDRESS_NOT_ALLOWED);
throw Exception(ErrorCodes::IP_ADDRESS_NOT_ALLOWED, "Connections from {} are not allowed", address.toString());
}
void IAccessStorage::throwAuthenticationTypeNotAllowed(AuthenticationType auth_type)
@ -715,7 +706,7 @@ void IAccessStorage::throwAuthenticationTypeNotAllowed(AuthenticationType auth_t
void IAccessStorage::throwInvalidCredentials()
{
throw Exception("Invalid credentials", ErrorCodes::WRONG_PASSWORD);
throw Exception(ErrorCodes::WRONG_PASSWORD, "Invalid credentials");
}
void IAccessStorage::throwBackupNotAllowed() const

View File

@ -71,7 +71,7 @@ void KerberosInit::init(const String & keytab_file, const String & principal, co
const char *deftype = nullptr;
if (!std::filesystem::exists(keytab_file))
throw Exception("Keytab file does not exist", ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR, "Keytab file does not exist");
ret = krb5_init_context(&k5.ctx);
if (ret)
@ -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("Error in resolving cache" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error while getting default cache" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error when parsing principal name " + principal + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error while searching for cache for " + principal + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error while generating new cache" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error when unparsing name" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error in options allocation" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error in resolving keytab "+keytab_file + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error in setting output credential cache" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error in getting initial credentials" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error when initializing cache" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
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("Error while switching to new cache" + fmtError(ret), ErrorCodes::KERBEROS_ERROR);
throw Exception(ErrorCodes::KERBEROS_ERROR, "Error while switching to new cache{}", fmtError(ret));
}
LOG_TRACE(log,"Authenticated to Kerberos v5");

View File

@ -53,11 +53,11 @@ void LDAPAccessStorage::setConfiguration(const Poco::Util::AbstractConfiguration
const bool has_role_mapping = config.has(prefix_str + "role_mapping");
if (!has_server)
throw Exception("Missing 'server' field for LDAP user directory", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Missing 'server' field for LDAP user directory");
const auto ldap_server_name_cfg = config.getString(prefix_str + "server");
if (ldap_server_name_cfg.empty())
throw Exception("Empty 'server' field for LDAP user directory", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty 'server' field for LDAP user directory");
std::set<String> common_roles_cfg;
if (has_roles)
@ -321,7 +321,7 @@ std::set<String> LDAPAccessStorage::mapExternalRolesNoLock(const LDAPClient::Sea
std::set<String> role_names;
if (external_roles.size() != role_search_params.size())
throw Exception("Unable to map external roles", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unable to map external roles");
for (std::size_t i = 0; i < external_roles.size(); ++i)
{

View File

@ -226,13 +226,13 @@ bool LDAPClient::openConnection()
auto * uri = ldap_url_desc2str(&url);
if (!uri)
throw Exception("ldap_url_desc2str() failed", ErrorCodes::LDAP_ERROR);
throw Exception(ErrorCodes::LDAP_ERROR, "ldap_url_desc2str() failed");
SCOPE_EXIT({ ldap_memfree(uri); });
handleError(ldap_initialize(&handle, uri));
if (!handle)
throw Exception("ldap_initialize() failed", ErrorCodes::LDAP_ERROR);
throw Exception(ErrorCodes::LDAP_ERROR, "ldap_initialize() failed");
}
{
@ -371,10 +371,10 @@ bool LDAPClient::openConnection()
const auto user_dn_search_results = search(*params.user_dn_detection);
if (user_dn_search_results.empty())
throw Exception("Failed to detect user DN: empty search results", ErrorCodes::LDAP_ERROR);
throw Exception(ErrorCodes::LDAP_ERROR, "Failed to detect user DN: empty search results");
if (user_dn_search_results.size() > 1)
throw Exception("Failed to detect user DN: more than one entry in the search results", ErrorCodes::LDAP_ERROR);
throw Exception(ErrorCodes::LDAP_ERROR, "Failed to detect user DN: more than one entry in the search results");
final_user_dn = *user_dn_search_results.begin();
}
@ -383,7 +383,7 @@ bool LDAPClient::openConnection()
}
default:
throw Exception("Unknown SASL mechanism", ErrorCodes::LDAP_ERROR);
throw Exception(ErrorCodes::LDAP_ERROR, "Unknown SASL mechanism");
}
}
@ -576,7 +576,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params)
}
case -1:
throw Exception("Failed to process LDAP search message", ErrorCodes::LDAP_ERROR);
throw Exception(ErrorCodes::LDAP_ERROR, "Failed to process LDAP search message");
}
}
@ -586,10 +586,10 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params)
bool LDAPSimpleAuthClient::authenticate(const RoleSearchParamsList * role_search_params, SearchResultsList * role_search_results)
{
if (params.user.empty())
throw Exception("LDAP authentication of a user with empty name is not allowed", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "LDAP authentication of a user with empty name is not allowed");
if (!role_search_params != !role_search_results)
throw Exception("Cannot return LDAP search results", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot return LDAP search results");
// Silently reject authentication attempt if the password is empty as if it didn't match.
if (params.password.empty())
@ -628,12 +628,12 @@ bool LDAPSimpleAuthClient::authenticate(const RoleSearchParamsList * role_search
void LDAPClient::handleError(const int, String)
{
throw Exception("ClickHouse was built without LDAP support", ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME);
throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without LDAP support");
}
bool LDAPClient::openConnection()
{
throw Exception("ClickHouse was built without LDAP support", ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME);
throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without LDAP support");
}
void LDAPClient::closeConnection() noexcept
@ -642,12 +642,12 @@ void LDAPClient::closeConnection() noexcept
LDAPClient::SearchResults LDAPClient::search(const SearchParams &)
{
throw Exception("ClickHouse was built without LDAP support", ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME);
throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without LDAP support");
}
bool LDAPSimpleAuthClient::authenticate(const RoleSearchParamsList *, SearchResultsList *)
{
throw Exception("ClickHouse was built without LDAP support", ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME);
throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without LDAP support");
}
#endif // USE_LDAP

View File

@ -316,10 +316,8 @@ bool MultipleAccessStorage::updateImpl(const UUID & id, const UpdateFunc & updat
break;
if (storage->find(new_entity->getType(), new_entity->getName()))
{
throw Exception(
old_entity->formatTypeWithName() + ": cannot rename to " + backQuote(new_entity->getName()) + " because "
+ new_entity->formatTypeWithName() + " already exists in " + storage->getStorageName(),
ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS);
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "{}: cannot rename to {} because {} already exists in {}",
old_entity->formatTypeWithName(), backQuote(new_entity->getName()), new_entity->formatTypeWithName(), storage->getStorageName());
}
}
}

View File

@ -55,9 +55,8 @@ String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const
{
if (!params.client_key.empty())
return params.client_key;
throw Exception(
"Quota " + quota->getName() + " (for user " + params.user_name + ") requires a client supplied key.",
ErrorCodes::QUOTA_REQUIRES_CLIENT_KEY);
throw Exception(ErrorCodes::QUOTA_REQUIRES_CLIENT_KEY, "Quota {} (for user {}) requires a client supplied key.",
quota->getName(), params.user_name);
}
case QuotaKeyType::CLIENT_KEY_OR_USER_NAME:
{
@ -73,7 +72,7 @@ String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const
}
case QuotaKeyType::MAX: break;
}
throw Exception("Unexpected quota key type: " + std::to_string(static_cast<int>(quota->key_type)), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected quota key type: {}", static_cast<int>(quota->key_type));
}

View File

@ -51,7 +51,7 @@ ReplicatedAccessStorage::ReplicatedAccessStorage(
, backup_allowed(allow_backup_)
{
if (zookeeper_path.empty())
throw Exception("ZooKeeper path must be non-empty", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty");
if (zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
@ -458,7 +458,7 @@ zkutil::ZooKeeperPtr ReplicatedAccessStorage::getZooKeeperNoLock()
{
auto zookeeper = get_zookeeper();
if (!zookeeper)
throw Exception("Can't have Replicated access without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
throw Exception(ErrorCodes::NO_ZOOKEEPER, "Can't have Replicated access without ZooKeeper");
/// It's possible that we connected to different [Zoo]Keeper instance
/// so we may read a bit stale state.

View File

@ -254,7 +254,7 @@ bool RolesOrUsersSet::match(const UUID & user_id, const boost::container::flat_s
std::vector<UUID> RolesOrUsersSet::getMatchingIDs() const
{
if (all)
throw Exception("getAllMatchingIDs() can't get ALL ids without access_control", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "getAllMatchingIDs() can't get ALL ids without access_control");
std::vector<UUID> res;
boost::range::set_difference(ids, except_ids, std::back_inserter(res));
return res;

View File

@ -45,7 +45,7 @@ void RowPolicy::setFullName(const RowPolicyName & full_name_)
void RowPolicy::setName(const String &)
{
throw Exception("RowPolicy::setName() is not implemented", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RowPolicy::setName() is not implemented");
}

View File

@ -124,7 +124,7 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting
if (new_value != old_value)
{
if (old_value == SettingConstraintWritability::CONST)
throw Exception("Setting " + element.setting_name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Setting {} should not be changed", element.setting_name);
}
}
}
@ -274,7 +274,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
if (constraint.writability == SettingConstraintWritability::CONST)
{
if (reaction == THROW_ON_VIOLATION)
throw Exception("Setting " + setting_name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Setting {} should not be changed", setting_name);
else
return false;
}
@ -285,7 +285,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
if (!min_value.isNull() && !max_value.isNull() && less_or_cannot_compare(max_value, min_value))
{
if (reaction == THROW_ON_VIOLATION)
throw Exception("Setting " + setting_name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Setting {} should not be changed", setting_name);
else
return false;
}
@ -294,9 +294,8 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
{
if (reaction == THROW_ON_VIOLATION)
{
throw Exception(
"Setting " + setting_name + " shouldn't be less than " + applyVisitor(FieldVisitorToString(), min_value),
ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Setting {} shouldn't be less than {}",
setting_name, applyVisitor(FieldVisitorToString(), min_value));
}
else
change.value = min_value;
@ -306,9 +305,8 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
{
if (reaction == THROW_ON_VIOLATION)
{
throw Exception(
"Setting " + setting_name + " shouldn't be greater than " + applyVisitor(FieldVisitorToString(), max_value),
ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Setting {} shouldn't be greater than {}",
setting_name, applyVisitor(FieldVisitorToString(), max_value));
}
else
change.value = max_value;

View File

@ -58,7 +58,9 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A
access_control->checkSettingNameIsAllowed(setting_name);
/// Check if a CHANGEABLE_IN_READONLY is allowed.
if (ast.writability == SettingConstraintWritability::CHANGEABLE_IN_READONLY && !access_control->doesSettingsConstraintsReplacePrevious())
throw Exception("CHANGEABLE_IN_READONLY for " + setting_name + " is not allowed unless settings_constraints_replace_previous is enabled", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"CHANGEABLE_IN_READONLY for {} "
"is not allowed unless settings_constraints_replace_previous is enabled", setting_name);
}
value = ast.value;

View File

@ -94,7 +94,7 @@ void SettingsProfilesCache::setDefaultProfileName(const String & default_profile
auto it = profiles_by_name.find(default_profile_name);
if (it == profiles_by_name.end())
throw Exception("Settings profile " + backQuote(default_profile_name) + " not found", ErrorCodes::THERE_IS_NO_PROFILE);
throw Exception(ErrorCodes::THERE_IS_NO_PROFILE, "Settings profile {} not found", backQuote(default_profile_name));
default_profile_id = it->second;
}

View File

@ -67,11 +67,15 @@ namespace
size_t num_password_fields = has_no_password + has_password_plaintext + has_password_sha256_hex + has_password_double_sha1_hex + has_ldap + has_kerberos + has_certificates;
if (num_password_fields > 1)
throw Exception("More than one field of 'password', 'password_sha256_hex', 'password_double_sha1_hex', 'no_password', 'ldap', 'kerberos', 'ssl_certificates' are used to specify authentication info for user " + user_name + ". Must be only one of them.",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "More than one field of 'password', 'password_sha256_hex', "
"'password_double_sha1_hex', 'no_password', 'ldap', 'kerberos', 'ssl_certificates' "
"are used to specify authentication info for user {}. "
"Must be only one of them.", user_name);
if (num_password_fields < 1)
throw Exception("Either 'password' or 'password_sha256_hex' or 'password_double_sha1_hex' or 'no_password' or 'ldap' or 'kerberos' or 'ssl_certificates' must be specified for user " + user_name + ".", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Either 'password' or 'password_sha256_hex' "
"or 'password_double_sha1_hex' or 'no_password' or 'ldap' or 'kerberos' "
"or 'ssl_certificates' must be specified for user {}.", user_name);
if (has_password_plaintext)
{
@ -92,11 +96,11 @@ namespace
{
bool has_ldap_server = config.has(user_config + ".ldap.server");
if (!has_ldap_server)
throw Exception("Missing mandatory 'server' in 'ldap', with LDAP server name, for user " + user_name + ".", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Missing mandatory 'server' in 'ldap', with LDAP server name, for user {}.", user_name);
const auto ldap_server_name = config.getString(user_config + ".ldap.server");
if (ldap_server_name.empty())
throw Exception("LDAP server name cannot be empty for user " + user_name + ".", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "LDAP server name cannot be empty for user {}.", user_name);
user->auth_data = AuthenticationData{AuthenticationType::LDAP};
user->auth_data.setLDAPServerName(ldap_server_name);
@ -124,7 +128,7 @@ namespace
common_names.insert(std::move(value));
}
else
throw Exception("Unknown certificate pattern type: " + key, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown certificate pattern type: {}", key);
}
user->auth_data.setSSLCertificateCommonNames(std::move(common_names));
}
@ -167,7 +171,7 @@ namespace
else if (key.starts_with("host"))
user->allowed_client_hosts.addName(value);
else
throw Exception("Unknown address pattern type: " + key, ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE);
throw Exception(ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE, "Unknown address pattern type: {}", key);
}
}
@ -466,13 +470,15 @@ namespace
if (access_control.doesSettingsConstraintsReplacePrevious())
profile_element.writability = SettingConstraintWritability::CHANGEABLE_IN_READONLY;
else
throw Exception("Setting changeable_in_readonly for " + setting_name + " is not allowed unless settings_constraints_replace_previous is enabled", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting changeable_in_readonly for {} is not allowed "
"unless settings_constraints_replace_previous is enabled", setting_name);
}
else
throw Exception("Setting " + constraint_type + " value for " + setting_name + " isn't supported", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting {} value for {} isn't supported", constraint_type, setting_name);
}
if (writability_count > 1)
throw Exception("Not more than one constraint writability specifier (const/readonly/changeable_in_readonly) is allowed for " + setting_name, ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not more than one constraint writability specifier "
"(const/readonly/changeable_in_readonly) is allowed for {}", setting_name);
profile_elements.push_back(std::move(profile_element));
}

View File

@ -68,7 +68,7 @@ public:
void create(AggregateDataPtr __restrict place) const override
{
if (std::uniform_real_distribution<>(0.0, 1.0)(thread_local_rng) <= throw_probability)
throw Exception("Aggregate function " + getName() + " has thrown exception successfully", ErrorCodes::AGGREGATE_FUNCTION_THROW);
throw Exception(ErrorCodes::AGGREGATE_FUNCTION_THROW, "Aggregate function {} has thrown exception successfully", getName());
new (place) Data;
}
@ -116,7 +116,7 @@ void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory)
if (parameters.size() == 1)
throw_probability = parameters[0].safeGet<Float64>();
else if (parameters.size() > 1)
throw Exception("Aggregate function " + name + " cannot have more than one parameter", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} cannot have more than one parameter", name);
return std::make_shared<AggregateFunctionThrow>(argument_types, parameters, throw_probability);
});

View File

@ -78,7 +78,7 @@ public:
{
auto f_stat = data(place).getFStatistic();
if (std::isinf(f_stat) || isNaN(f_stat) || f_stat < 0)
throw Exception("F statistic is not defined or infinite for these arguments", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "F statistic is not defined or infinite for these arguments");
auto p_value = data(place).getPValue(f_stat);

View File

@ -52,8 +52,9 @@ public:
, serialization_val(type_val->getDefaultSerialization())
{
if (!type_val->isComparable())
throw Exception("Illegal type " + type_val->getName() + " of second argument of aggregate function " + getName()
+ " because the values of that data type are not comparable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of "
"aggregate function {} because the values of that data type are not comparable",
type_val->getName(), getName());
}
String getName() const override

View File

@ -26,7 +26,7 @@ public:
DataTypes transformArguments(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception("-Array aggregate functions require at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "-Array aggregate functions require at least one argument");
DataTypes nested_arguments;
for (const auto & type : arguments)
@ -34,8 +34,8 @@ public:
if (const DataTypeArray * array = typeid_cast<const DataTypeArray *>(type.get()))
nested_arguments.push_back(array->getNestedType());
else
throw Exception("Illegal type " + type->getName() + " of argument"
" for aggregate function with " + getName() + " suffix. Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument"
" for aggregate function with {} suffix. Must be array.", type->getName(), getName());
}
return nested_arguments;

View File

@ -36,7 +36,7 @@ public:
assert(parameters == nested_func->getParameters());
for (const auto & type : arguments)
if (!isArray(type))
throw Exception("All arguments for aggregate function " + getName() + " must be arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "All arguments for aggregate function {} must be arrays", getName());
}
String getName() const override
@ -129,7 +129,7 @@ public:
const IColumn::Offsets & ith_offsets = ith_column.getOffsets();
if (ith_offsets[row_num] != end || (row_num != 0 && ith_offsets[row_num - 1] != begin))
throw Exception("Arrays passed to " + getName() + " aggregate function have different sizes", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH, "Arrays passed to {} aggregate function have different sizes", getName());
}
for (size_t i = begin; i < end; ++i)

View File

@ -29,8 +29,8 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const
const DataTypePtr& data_type = argument_types[0];
if (!allowType(data_type))
throw Exception("Illegal type " + data_type->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
data_type->getName(), name);
AggregateFunctionPtr res;

View File

@ -71,11 +71,9 @@ createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & a
const auto data_type_weight = static_cast<const DataTypePtr>(argument_types[1]);
if (!allowTypes(data_type, data_type_weight))
throw Exception(
"Types " + data_type->getName() +
" and " + data_type_weight->getName() +
" are non-conforming as arguments for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Types {} and {} are non-conforming as arguments for aggregate function {}",
data_type->getName(), data_type_weight->getName(), name);
AggregateFunctionPtr ptr;

View File

@ -23,14 +23,15 @@ AggregateFunctionPtr createAggregateFunctionBitwise(const std::string & name, co
assertUnary(name, argument_types);
if (!argument_types[0]->canBeUsedInBitOperations())
throw Exception("The type " + argument_types[0]->getName() + " of argument for aggregate function " + name
+ " is illegal, because it cannot be used in bitwise operations",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The type {} of argument for aggregate function {} "
"is illegal, because it cannot be used in bitwise operations",
argument_types[0]->getName(), name);
AggregateFunctionPtr res(createWithUnsignedIntegerType<AggregateFunctionBitwise, Data>(*argument_types[0], argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument for aggregate function {}", argument_types[0]->getName(), name);
return res;
}

View File

@ -21,8 +21,9 @@ AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const
assertBinary(name, argument_types);
if (argument_types.size() < 2)
throw Exception("Aggregate function " + name + " requires at least two arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Aggregate function {} requires at least two arguments",
name);
return std::make_shared<AggregateFunctionBoundingRatio>(argument_types);
}

View File

@ -118,7 +118,9 @@ public:
const auto * y_arg = arguments.at(1).get();
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal types of arguments of aggregate function {}, must have number representation.", getName());
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Illegal types of arguments of aggregate function {}, must have number representation.",
getName());
}
bool allocatesMemoryInArena() const override { return false; }

View File

@ -163,16 +163,14 @@ AggregateFunctionPtr createAggregateFunctionCategoricalIV(
assertNoParameters(name, params);
if (arguments.size() < 2)
throw Exception(
"Aggregate function " + name + " requires two or more arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires two or more arguments",
name);
for (const auto & argument : arguments)
{
if (!WhichDataType(argument).isUInt8())
throw Exception(
"All the arguments of aggregate function " + name + " should be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "All the arguments of aggregate function {} should be UInt8",
name);
}
return std::make_shared<AggregateFunctionCategoricalIV>(arguments, params);

View File

@ -219,7 +219,7 @@ public:
: IAggregateFunctionDataHelper<AggregateFunctionCountData, AggregateFunctionCountNotNullUnary>({argument}, params, createResultType())
{
if (!argument->isNullable())
throw Exception("Logical error: not Nullable data type passed to AggregateFunctionCountNotNullUnary", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: not Nullable data type passed to AggregateFunctionCountNotNullUnary");
}
String getName() const override { return "count"; }

View File

@ -27,8 +27,8 @@ AggregateFunctionPtr createAggregateFunctionDeltaSum(
assertNoParameters(name, params);
if (arguments.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of arguments for aggregate function {}", name);
const DataTypePtr & data_type = arguments[0];
@ -36,8 +36,8 @@ AggregateFunctionPtr createAggregateFunctionDeltaSum(
return AggregateFunctionPtr(createWithNumericType<AggregationFunctionDeltaSum>(
*data_type, arguments, params));
else
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
arguments[0]->getName(), name);
}
}

View File

@ -26,16 +26,16 @@ AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
assertNoParameters(name, params);
if (arguments.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of arguments for aggregate function {}", name);
if (!isInteger(arguments[0]) && !isFloat(arguments[0]) && !isDate(arguments[0]) && !isDateTime(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " +
name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}, "
"must be Int, Float, Date, DateTime", arguments[0]->getName(), name);
if (!isInteger(arguments[1]) && !isFloat(arguments[1]) && !isDate(arguments[1]) && !isDateTime(arguments[1]))
throw Exception("Illegal type " + arguments[1]->getName() + " of argument for aggregate function " +
name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}, "
"must be Int, Float, Date, DateTime", arguments[1]->getName(), name);
return AggregateFunctionPtr(createWithTwoNumericOrDateTypes<AggregationFunctionDeltaSumTimestamp>(
*arguments[0], *arguments[1], arguments, params));

View File

@ -24,8 +24,8 @@ public:
DataTypes transformArguments(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception("Incorrect number of arguments for aggregate function with " + getName() + " suffix",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of arguments for aggregate function with {} suffix", getName());
return arguments;
}

View File

@ -21,8 +21,8 @@ AggregateFunctionPtr createAggregateFunctionEntropy(
{
assertNoParameters(name, parameters);
if (argument_types.empty())
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of arguments for aggregate function {}", name);
size_t num_args = argument_types.size();
if (num_args == 1)

View File

@ -32,8 +32,8 @@ public:
: IAggregateFunctionDataHelper<ExponentiallySmoothedAverage, AggregateFunctionExponentialMovingAverage>(argument_types_, params, createResultType())
{
if (params.size() != 1)
throw Exception{"Aggregate function " + getName() + " requires exactly one parameter: half decay time.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires exactly one parameter: "
"half decay time.", getName());
half_decay = applyVisitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
}

View File

@ -40,19 +40,19 @@ const String & getAggregateFunctionCanonicalNameIfAny(const String & name)
void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, CaseSensitiveness case_sensitiveness)
{
if (creator_with_properties.creator == nullptr)
throw Exception("AggregateFunctionFactory: the aggregate function " + name + " has been provided "
" a null constructor", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: "
"the aggregate function {} has been provided a null constructor", name);
if (!aggregate_functions.emplace(name, creator_with_properties).second)
throw Exception("AggregateFunctionFactory: the aggregate function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: the aggregate function name '{}' is not unique",
name);
if (case_sensitiveness == CaseInsensitive)
{
auto key = Poco::toLower(name);
if (!case_insensitive_aggregate_functions.emplace(key, creator_with_properties).second)
throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: "
"the case insensitive aggregate function name '{}' is not unique", name);
case_insensitive_name_mapping[key] = name;
}
}
@ -82,8 +82,8 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
{
AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null");
if (!combinator)
throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.",
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: cannot find aggregate function combinator "
"to apply a function to Nullable arguments.");
DataTypes nested_types = combinator->transformArguments(types_without_low_cardinality);
Array nested_parameters = combinator->transformParameters(parameters);
@ -106,7 +106,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
auto with_original_arguments = getImpl(name, types_without_low_cardinality, parameters, out_properties, false);
if (!with_original_arguments)
throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: AggregateFunctionFactory returned nullptr");
return with_original_arguments;
}

View File

@ -27,8 +27,8 @@ public:
if (const DataTypeArray * array = typeid_cast<const DataTypeArray *>(type.get()))
nested_arguments.push_back(array->getNestedType());
else
throw Exception("Illegal type " + type->getName() + " of argument"
" for aggregate function with " + getName() + " suffix. Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for "
"aggregate function with {} suffix. Must be array.", type->getName(), getName());
}
return nested_arguments;

View File

@ -113,11 +113,11 @@ public:
nested_size_of_data = nested_func->sizeOfData();
if (arguments.empty())
throw Exception("Aggregate function " + getName() + " require at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require at least one argument", getName());
for (const auto & type : arguments)
if (!isArray(type))
throw Exception("All arguments for aggregate function " + getName() + " must be arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "All arguments for aggregate function {} must be arrays", getName());
}
String getName() const override
@ -197,7 +197,7 @@ public:
const IColumn::Offsets & ith_offsets = ith_column.getOffsets();
if (ith_offsets[row_num] != end || (row_num != 0 && ith_offsets[row_num - 1] != begin))
throw Exception("Arrays passed to " + getName() + " aggregate function have different sizes", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH, "Arrays passed to {} aggregate function have different sizes", getName());
}
AggregateFunctionForEachData & state = ensureAggregateData(place, end - begin, *arena);

View File

@ -60,23 +60,23 @@ AggregateFunctionPtr createAggregateFunctionGroupArray(
{
auto type = parameters[0].getType();
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
limit_size = true;
max_elems = parameters[0].get<UInt64>();
}
else
throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 0 or 1",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of parameters for aggregate function {}, should be 0 or 1", name);
if (!limit_size)
{
if (Tlast)
throw Exception("groupArrayLast make sense only with max_elems (groupArrayLast(max_elems)())", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "groupArrayLast make sense only with max_elems (groupArrayLast(max_elems)())");
return createAggregateFunctionGroupArrayImpl<GroupArrayTrait</* Thas_limit= */ false, Tlast, /* Tsampler= */ Sampler::NONE>>(argument_types[0], parameters);
}
else
@ -89,18 +89,18 @@ AggregateFunctionPtr createAggregateFunctionGroupArraySample(
assertUnary(name, argument_types);
if (parameters.size() != 1 && parameters.size() != 2)
throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 1 or 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of parameters for aggregate function {}, should be 1 or 2", name);
auto get_parameter = [&](size_t i)
{
auto type = parameters[i].getType();
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[i].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[i].get<UInt64>() == 0))
throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
return parameters[i].get<UInt64>();
};

View File

@ -282,10 +282,10 @@ public:
readVarUInt(size, buf);
if (unlikely(size > AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE))
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size");
if (limit_num_elems && unlikely(size > max_elems))
throw Exception("Too large array size, it should not exceed " + toString(max_elems), ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size, it should not exceed {}", max_elems);
auto & value = this->data(place).value;
@ -613,10 +613,10 @@ public:
return;
if (unlikely(elems > AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE))
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size");
if (limit_num_elems && unlikely(elems > max_elems))
throw Exception("Too large array size, it should not exceed " + toString(max_elems), ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size, it should not exceed {}", max_elems);
auto & value = data(place).value;

View File

@ -22,7 +22,7 @@ AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(
assertBinary(name, argument_types);
if (argument_types.size() != 2)
throw Exception("Aggregate function groupArrayInsertAt requires two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function groupArrayInsertAt requires two arguments.");
return std::make_shared<AggregateFunctionGroupArrayInsertAtGeneric>(argument_types, parameters);
}

View File

@ -71,7 +71,7 @@ public:
if (!params.empty())
{
if (params.size() > 2)
throw Exception("Aggregate function " + getName() + " requires at most two parameters.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at most two parameters.", getName());
default_value = params[0];
@ -79,12 +79,12 @@ public:
{
length_to_resize = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[1]);
if (length_to_resize > AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE)
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size");
}
}
if (!isUnsignedInteger(arguments[1]))
throw Exception("Second argument of aggregate function " + getName() + " must be unsigned integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument of aggregate function {} must be unsigned integer.", getName());
if (default_value.isNull())
default_value = type->getDefault();
@ -92,8 +92,9 @@ public:
{
Field converted = convertFieldToType(default_value, *type);
if (converted.isNull())
throw Exception("Cannot convert parameter of aggregate function " + getName() + " (" + applyVisitor(FieldVisitorToString(), default_value) + ")"
" to type " + type->getName() + " to be used as default value in array", ErrorCodes::CANNOT_CONVERT_TYPE);
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert parameter of aggregate function {} ({}) "
"to type {} to be used as default value in array",
getName(), applyVisitor(FieldVisitorToString(), default_value), type->getName());
default_value = converted;
}
@ -113,9 +114,9 @@ public:
return;
if (position >= AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE)
throw Exception("Too large array size: position argument (" + toString(position) + ")"
" is greater or equals to limit (" + toString(AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE) + ")",
ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: "
"position argument ({}) is greater or equals to limit ({})",
position, AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE);
Array & arr = data(place).value;
@ -166,7 +167,7 @@ public:
readVarUInt(size, buf);
if (size > AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE)
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size");
Array & arr = data(place).value;

View File

@ -50,8 +50,8 @@ inline AggregateFunctionPtr createAggregateFunctionMovingImpl(const std::string
res.reset(createWithNumericType<Function, HasLimit>(*argument_type, argument_type, std::forward<TArgs>(args)...));
if (!res)
throw Exception("Illegal type " + argument_type->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_type->getName(), name);
return res;
}
@ -74,18 +74,18 @@ AggregateFunctionPtr createAggregateFunctionMoving(
{
auto type = parameters[0].getType();
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception("Parameter for aggregate function " + name + " should be positive integer", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
throw Exception("Parameter for aggregate function " + name + " should be positive integer", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name);
limit_size = true;
max_elems = parameters[0].get<UInt64>();
}
else
throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 0 or 1",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of parameters for aggregate function {}, should be 0 or 1", name);
const DataTypePtr & argument_type = argument_types[0];
if (!limit_size)

View File

@ -141,7 +141,7 @@ public:
readVarUInt(size, buf);
if (unlikely(size > AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE))
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size");
if (size > 0)
{

View File

@ -41,17 +41,16 @@ namespace
assertUnary(name, argument_types);
if (!argument_types[0]->canBeUsedInBitOperations())
throw Exception(
"The type " + argument_types[0]->getName() + " of argument for aggregate function " + name
+ " is illegal, because it cannot be used in Bitmap operations",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The type {} of argument for aggregate function {} "
"is illegal, because it cannot be used in Bitmap operations",
argument_types[0]->getName(), name);
AggregateFunctionPtr res(createWithIntegerType<AggregateFunctionBitmap, Data>(*argument_types[0], argument_types[0]));
if (!res)
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_types[0]->getName(), name);
return res;
}
@ -67,9 +66,8 @@ namespace
DataTypePtr argument_type_ptr = argument_types[0];
WhichDataType which(*argument_type_ptr);
if (which.idx != TypeIndex::AggregateFunction)
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_types[0]->getName(), name);
/// groupBitmap needs to know about the data type that was used to create bitmaps.
/// We need to look inside the type of its argument to obtain it.
@ -77,9 +75,8 @@ namespace
AggregateFunctionPtr aggfunc = datatype_aggfunc.getFunction();
if (aggfunc->getName() != AggregateFunctionGroupBitmapData<UInt8>::name())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_types[0]->getName(), name);
DataTypePtr nested_argument_type_ptr = aggfunc->getArgumentTypes()[0];
@ -87,9 +84,8 @@ namespace
*nested_argument_type_ptr, argument_type_ptr));
if (!res)
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_types[0]->getName(), name);
return res;
}

View File

@ -65,8 +65,8 @@ inline AggregateFunctionPtr createAggregateFunctionGroupUniqArrayImpl(const std:
res = AggregateFunctionPtr(createWithExtraTypes<HasLimit>(argument_type, std::forward<TArgs>(args)...));
if (!res)
throw Exception("Illegal type " + argument_type->getName() +
" of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_type->getName(), name);
return res;
@ -88,18 +88,18 @@ AggregateFunctionPtr createAggregateFunctionGroupUniqArray(
{
auto type = parameters[0].getType();
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
limit_size = true;
max_elems = parameters[0].get<UInt64>();
}
else
throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 0 or 1",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of parameters for aggregate function {}, should be 0 or 1", name);
if (!limit_size)
return createAggregateFunctionGroupUniqArrayImpl<std::false_type>(name, argument_types[0], parameters);

View File

@ -25,25 +25,26 @@ namespace
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
{
if (params.size() != 1)
throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires single parameter: bins count", name);
if (params[0].getType() != Field::Types::UInt64)
throw Exception("Invalid type for bins count", ErrorCodes::UNSUPPORTED_PARAMETER);
throw Exception(ErrorCodes::UNSUPPORTED_PARAMETER, "Invalid type for bins count");
UInt32 bins_count = applyVisitor(FieldVisitorConvertToNumber<UInt32>(), params[0]);
auto limit = AggregateFunctionHistogramData::bins_count_limit;
if (bins_count > limit)
throw Exception("Unsupported bins count. Should not be greater than " + std::to_string(limit), ErrorCodes::PARAMETER_OUT_OF_BOUND);
throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Unsupported bins count. Should not be greater than {}", limit);
if (bins_count == 0)
throw Exception("Bin count should be positive", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bin count should be positive");
assertUnary(name, arguments);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionHistogram>(*arguments[0], bins_count, arguments, params));
if (!res)
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument for aggregate function {}", arguments[0]->getName(), name);
return res;
}

View File

@ -256,7 +256,7 @@ public:
// nans break sort and compression
// infs don't fit in bins partition method
if (!isFinite(value))
throw Exception("Invalid value (inf or nan) for aggregation by 'histogram' function", ErrorCodes::INCORRECT_DATA);
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid value (inf or nan) for aggregation by 'histogram' function");
points[size] = {value, weight};
++size;
@ -291,7 +291,7 @@ public:
readVarUInt(size, buf);
if (size > max_bins * 2)
throw Exception("Too many bins", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too many bins");
buf.readStrict(reinterpret_cast<char *>(points), size * sizeof(WeightedValue));
}

View File

@ -20,12 +20,12 @@ public:
DataTypes transformArguments(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception("Incorrect number of arguments for aggregate function with " + getName() + " suffix",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of arguments for aggregate function with {} suffix", getName());
if (!isUInt8(arguments.back()) && !arguments.back()->onlyNull())
throw Exception("Illegal type " + arguments.back()->getName() + " of last argument for aggregate function with " + getName() + " suffix",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of last argument for "
"aggregate function with {} suffix", arguments.back()->getName(), getName());
return DataTypes(arguments.begin(), std::prev(arguments.end()));
}
@ -249,11 +249,11 @@ public:
: Base(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size())
{
if (number_of_arguments == 1)
throw Exception("Logical error: single argument is passed to AggregateFunctionIfNullVariadic", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionIfNullVariadic");
if (number_of_arguments > MAX_ARGS)
throw Exception("Maximum number of arguments for aggregate function with Nullable types is " + toString(MAX_ARGS),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Maximum number of arguments for aggregate function with Nullable types is {}", toString(MAX_ARGS));
for (size_t i = 0; i < number_of_arguments; ++i)
is_nullable[i] = arguments[i]->isNullable();

View File

@ -40,10 +40,10 @@ public:
, nested_func(nested), num_arguments(types.size())
{
if (num_arguments == 0)
throw Exception("Aggregate function " + getName() + " require at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require at least one argument", getName());
if (!isUInt8(types.back()) && !types.back()->onlyNull())
throw Exception("Last argument for aggregate function " + getName() + " must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Last argument for aggregate function {} must be UInt8", getName());
}
String getName() const override

View File

@ -25,24 +25,23 @@ namespace
createAggregateFunctionIntervalLengthSum(const std::string & name, const DataTypes & arguments, const Array &, const Settings *)
{
if (arguments.size() != 2)
throw Exception(
"Aggregate function " + name + " requires two timestamps argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Aggregate function {} requires two timestamps argument.", name);
auto args = {arguments[0].get(), arguments[1].get()};
if (WhichDataType{args.begin()[0]}.idx != WhichDataType{args.begin()[1]}.idx)
throw Exception(
"Illegal types " + args.begin()[0]->getName() + " and " + args.begin()[1]->getName() + " of arguments of aggregate function "
+ name + ", both arguments should have same data type",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal types {} and {} of arguments "
"of aggregate function {}, both arguments should have same data type",
args.begin()[0]->getName(), args.begin()[1]->getName(), name);
for (const auto & arg : args)
{
if (!isNativeNumber(arg) && !isDate(arg) && !isDateTime(arg))
throw Exception(
"Illegal type " + arg->getName() + " of argument of aggregate function " + name
+ ", must be native integral type, Date/DateTime or Float",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of aggregate function {}, must "
"be native integral type, Date/DateTime or Float", arg->getName(), name);
}
AggregateFunctionPtr res(createWithBasicNumberOrDateOrDateTime<AggregateFunctionIntervalLengthSum, Data>(*arguments[0], arguments));
@ -50,10 +49,9 @@ namespace
if (res)
return res;
throw Exception(
"Illegal type " + arguments.front().get()->getName() + " of argument of aggregate function " + name
+ ", must be native integral type, Date/DateTime or Float",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of aggregate function {}, must "
"be native integral type, Date/DateTime or Float", arguments.front().get()->getName(), name);
}
}

View File

@ -117,7 +117,7 @@ struct AggregateFunctionIntervalLengthSumData
readBinary(size, buf);
if (unlikely(size > MAX_ARRAY_SIZE))
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size");
segments.clear();
segments.reserve(size);

View File

@ -34,24 +34,20 @@ namespace
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
if (parameters.size() > 4)
throw Exception(
"Aggregate function " + name
+ " requires at most four parameters: learning_rate, l2_regularization_coef, mini-batch size and weights_updater "
"method",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Aggregate function {} requires at most four parameters: "
"learning_rate, l2_regularization_coef, mini-batch size and weights_updater method", name);
if (argument_types.size() < 2)
throw Exception(
"Aggregate function " + name + " requires at least two arguments: target and model's parameters",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Aggregate function {} requires at least two arguments: target and model's parameters", name);
for (size_t i = 0; i < argument_types.size(); ++i)
{
if (!isNativeNumber(argument_types[i]))
throw Exception(
"Argument " + std::to_string(i) + " of type " + argument_types[i]->getName()
+ " must be numeric for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Argument {} of type {} must be numeric for aggregate function {}",
i, argument_types[i]->getName(), name);
}
/// Such default parameters were picked because they did good on some tests,
@ -79,8 +75,8 @@ namespace
{
weights_updater_name = parameters[3].safeGet<String>();
if (weights_updater_name != "SGD" && weights_updater_name != "Momentum" && weights_updater_name != "Nesterov" && weights_updater_name != "Adam")
throw Exception("Invalid parameter for weights updater. The only supported are 'SGD', 'Momentum' and 'Nesterov'",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid parameter for weights updater. "
"The only supported are 'SGD', 'Momentum' and 'Nesterov'");
}
if constexpr (std::is_same_v<Method, FuncLinearRegression>)
@ -255,7 +251,7 @@ void Adam::merge(const IWeightsUpdater & rhs, Float64 frac, Float64 rhs_frac)
{
if (!average_squared_gradient.empty() ||
adam_rhs.average_gradient.size() != adam_rhs.average_squared_gradient.size())
throw Exception("Average_gradient and average_squared_gradient must have same size", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Average_gradient and average_squared_gradient must have same size");
average_gradient.resize(adam_rhs.average_gradient.size(), Float64{0.0});
average_squared_gradient.resize(adam_rhs.average_squared_gradient.size(), Float64{0.0});
@ -275,7 +271,7 @@ void Adam::update(UInt64 batch_size, std::vector<Float64> & weights, Float64 & b
if (average_gradient.empty())
{
if (!average_squared_gradient.empty())
throw Exception("Average_gradient and average_squared_gradient must have same size", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Average_gradient and average_squared_gradient must have same size");
average_gradient.resize(batch_gradient.size(), Float64{0.0});
average_squared_gradient.resize(batch_gradient.size(), Float64{0.0});
@ -460,9 +456,9 @@ void LogisticRegression::predict(
size_t rows_num = arguments.front().column->size();
if (offset > rows_num || offset + limit > rows_num)
throw Exception("Invalid offset and limit for LogisticRegression::predict. "
"Block has " + toString(rows_num) + " rows, but offset is " + toString(offset) +
" and limit is " + toString(limit), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid offset and limit for LogisticRegression::predict. "
"Block has {} rows, but offset is {} and limit is {}",
rows_num, offset, toString(limit));
std::vector<Float64> results(limit, bias);
@ -471,7 +467,7 @@ void LogisticRegression::predict(
const ColumnWithTypeAndName & cur_col = arguments[i];
if (!isNativeNumber(cur_col.type))
throw Exception("Prediction arguments must have numeric type", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Prediction arguments must have numeric type");
const auto & features_column = cur_col.column;
@ -527,15 +523,15 @@ void LinearRegression::predict(
{
if (weights.size() + 1 != arguments.size())
{
throw Exception("In predict function number of arguments differs from the size of weights vector", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "In predict function number of arguments differs from the size of weights vector");
}
size_t rows_num = arguments.front().column->size();
if (offset > rows_num || offset + limit > rows_num)
throw Exception("Invalid offset and limit for LogisticRegression::predict. "
"Block has " + toString(rows_num) + " rows, but offset is " + toString(offset) +
" and limit is " + toString(limit), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid offset and limit for LogisticRegression::predict. "
"Block has {} rows, but offset is {} and limit is {}",
rows_num, offset, toString(limit));
std::vector<Float64> results(limit, bias);
@ -544,12 +540,12 @@ void LinearRegression::predict(
const ColumnWithTypeAndName & cur_col = arguments[i];
if (!isNativeNumber(cur_col.type))
throw Exception("Prediction arguments must have numeric type", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Prediction arguments must have numeric type");
auto features_column = cur_col.column;
if (!features_column)
throw Exception("Unexpectedly cannot dynamically cast features column " + std::to_string(i), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpectedly cannot dynamically cast features column {}", i);
for (size_t row_num = 0; row_num < limit; ++row_num)
results[row_num] += weights[i - 1] * features_column->getFloat64(row_num + offset);

View File

@ -344,7 +344,7 @@ public:
else if (weights_updater_name == "Adam")
new_weights_updater = std::make_shared<Adam>();
else
throw Exception("Illegal name of weights updater (should have been checked earlier)", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal name of weights updater (should have been checked earlier)");
new (place) Data(learning_rate, l2_reg_coef, param_num, batch_size, gradient_computer, new_weights_updater);
}
@ -369,16 +369,15 @@ public:
ContextPtr context) const override
{
if (arguments.size() != param_num + 1)
throw Exception(
"Predict got incorrect number of arguments. Got: " + std::to_string(arguments.size())
+ ". Required: " + std::to_string(param_num + 1),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Predict got incorrect number of arguments. Got: {}. Required: {}",
arguments.size(), param_num + 1);
/// This cast might be correct because column type is based on getReturnTypeToPredict.
auto * column = typeid_cast<ColumnFloat64 *>(&to);
if (!column)
throw Exception("Cast of column of predictions is incorrect. getReturnTypeToPredict must return same value as it is casted to",
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cast of column of predictions is incorrect. "
"getReturnTypeToPredict must return same value as it is casted to");
this->data(place).predict(column->getData(), arguments, offset, limit, context);
}

View File

@ -22,7 +22,7 @@ AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(
assertBinary(name, argument_types);
if (!isNumber(argument_types[0]) || !isNumber(argument_types[1]))
throw Exception("Aggregate function " + name + " only supports numerical types", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Aggregate function {} only supports numerical types", name);
return std::make_shared<AggregateFunctionMannWhitney>(argument_types, parameters);
}

View File

@ -136,7 +136,7 @@ public:
: IAggregateFunctionDataHelper<MannWhitneyData, AggregateFunctionMannWhitney> ({arguments}, {}, createResultType())
{
if (params.size() > 2)
throw Exception("Aggregate function " + getName() + " require two parameter or less", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require two parameter or less", getName());
if (params.empty())
{
@ -145,7 +145,7 @@ public:
}
if (params[0].getType() != Field::Types::String)
throw Exception("Aggregate function " + getName() + " require first parameter to be a String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
if (param == "two-sided")
@ -155,14 +155,14 @@ public:
else if (param == "greater")
alternative = Alternative::Greater;
else
throw Exception("Unknown parameter in aggregate function " + getName() +
". It must be one of: 'two-sided', 'less', 'greater'", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown parameter in aggregate function {}. "
"It must be one of: 'two-sided', 'less', 'greater'", getName());
if (params.size() != 2)
return;
if (params[1].getType() != Field::Types::UInt64)
throw Exception("Aggregate function " + getName() + " require second parameter to be a UInt64", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a UInt64", getName());
continuity_correction = static_cast<bool>(params[1].get<UInt64>());
}

View File

@ -18,15 +18,14 @@ public:
DataTypes transformArguments(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of arguments for aggregate function with " + getName() + " suffix");
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of arguments for aggregate function with {} suffix", getName());
const auto * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].get());
if (map_type)
{
if (arguments.size() > 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, getName() + " combinator takes only one map argument");
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "{} combinator takes only one map argument", getName());
return DataTypes({map_type->getValueType()});
}
@ -55,7 +54,7 @@ public:
}
}
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function " + getName() + " requires map as argument");
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires map as argument", getName());
}
AggregateFunctionPtr transformAggregateFunction(
@ -121,8 +120,8 @@ public:
return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties);
}
else
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregation '" + nested_func_name + "Map' is not implemented for mapped arrays");
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregation '{}Map' is not implemented for mapped arrays",
nested_func_name);
}
}
};

View File

@ -24,8 +24,8 @@ namespace
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionIntersectionsMax>(*argument_types[0], kind, argument_types));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types {} and {} of argument for aggregate function {}",
argument_types[0]->getName(), argument_types[1]->getName(), name);
return res;
}

View File

@ -126,7 +126,7 @@ public:
readVarUInt(size, buf);
if (unlikely(size > AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE))
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size");
auto & value = this->data(place).value;

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