mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
check number of parameters in format string
This commit is contained in:
parent
3f6594f4c6
commit
bb4c8e169f
@ -1943,7 +1943,7 @@ void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & ta
|
||||
|
||||
if (!table_node && !table_function_node && !query_node && !union_node)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Unexpected table expression. Expected table, table function, query or union node. Actual {}",
|
||||
"Unexpected table expression. Expected table, table function, query or union node. Table node: {}, scope node: {}",
|
||||
table_expression_node->formatASTForErrorMessage(),
|
||||
scope.scope_node->formatASTForErrorMessage());
|
||||
|
||||
@ -5726,7 +5726,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node,
|
||||
case QueryTreeNodeType::IDENTIFIER:
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Identifiers in FROM section must be already resolved. In scope {}",
|
||||
"Identifiers in FROM section must be already resolved. Node {}, scope {}",
|
||||
join_tree_node->formatASTForErrorMessage(),
|
||||
scope.scope_node->formatASTForErrorMessage());
|
||||
}
|
||||
|
@ -23,7 +23,10 @@ struct FormatStringHelperImpl
|
||||
std::string_view message_format_string;
|
||||
fmt::format_string<Args...> fmt_str;
|
||||
template<typename T>
|
||||
consteval FormatStringHelperImpl(T && str) : message_format_string(tryGetStaticFormatString(str)), fmt_str(std::forward<T>(str)) {}
|
||||
consteval FormatStringHelperImpl(T && str) : message_format_string(tryGetStaticFormatString(str)), fmt_str(std::forward<T>(str))
|
||||
{
|
||||
formatStringCheckArgsNumImpl(message_format_string, sizeof...(Args));
|
||||
}
|
||||
template<typename T>
|
||||
FormatStringHelperImpl(fmt::basic_runtime<T> && str) : message_format_string(), fmt_str(std::forward<fmt::basic_runtime<T>>(str)) {}
|
||||
|
||||
|
@ -53,3 +53,64 @@ template <typename T, typename... Ts> constexpr auto firstArg(T && x, Ts &&...)
|
||||
/// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor
|
||||
template <typename T, typename... Ts> constexpr auto firstArg(fmt::basic_runtime<T> && data, Ts &&...) { return data.str.data(); }
|
||||
|
||||
consteval ssize_t formatStringCountArgsNum(const char * const str, size_t len)
|
||||
{
|
||||
/// It does not count named args, but we don't use them
|
||||
size_t cnt = 0;
|
||||
size_t i = 0;
|
||||
while (i + 1 < len)
|
||||
{
|
||||
if (str[i] == '{' && str[i + 1] == '}')
|
||||
{
|
||||
i += 2;
|
||||
cnt += 1;
|
||||
}
|
||||
else if (str[i] == '{')
|
||||
{
|
||||
/// Ignore checks for complex formatting like "{:.3f}"
|
||||
return -1;
|
||||
}
|
||||
else
|
||||
{
|
||||
i += 1;
|
||||
}
|
||||
}
|
||||
return cnt;
|
||||
}
|
||||
|
||||
[[noreturn]] void functionThatFailsCompilationOfConstevalFunctions(const char * error)
|
||||
{
|
||||
throw std::runtime_error(error);
|
||||
}
|
||||
|
||||
/// fmt::format checks that there are enough arguments, but ignores extra arguments (e.g. fmt::format("{}", 1, 2) compiles)
|
||||
/// This function will fail to compile if the number of "{}" substitutions does not exactly match
|
||||
consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs)
|
||||
{
|
||||
if (str.empty())
|
||||
return;
|
||||
ssize_t cnt = formatStringCountArgsNum(str.data(), str.size());
|
||||
if (0 <= cnt && cnt != nargs)
|
||||
functionThatFailsCompilationOfConstevalFunctions("unexpected number of arguments in a format string");
|
||||
}
|
||||
|
||||
template <typename... Args>
|
||||
struct CheckArgsNumHelperImpl
|
||||
{
|
||||
//std::enable_if_t<std::is_same_v<std::decay_t<T>, PreformattedMessage>>
|
||||
template<typename T>
|
||||
consteval CheckArgsNumHelperImpl(T && str)
|
||||
{
|
||||
formatStringCheckArgsNumImpl(tryGetStaticFormatString(str), sizeof...(Args));
|
||||
}
|
||||
|
||||
/// No checks for fmt::runtime and PreformattedMessage
|
||||
template<typename T> CheckArgsNumHelperImpl(fmt::basic_runtime<T> &&) {}
|
||||
template<> CheckArgsNumHelperImpl(PreformattedMessage &) {}
|
||||
template<> CheckArgsNumHelperImpl(const PreformattedMessage &) {}
|
||||
template<> CheckArgsNumHelperImpl(PreformattedMessage &&) {}
|
||||
|
||||
};
|
||||
|
||||
template <typename... Args> using CheckArgsNumHelper = CheckArgsNumHelperImpl<std::type_identity_t<Args>...>;
|
||||
template <typename... Args> void formatStringCheckArgsNum(CheckArgsNumHelper<Args...>, Args &&...) {}
|
||||
|
@ -187,7 +187,7 @@ public:
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Value with key `{}` is used twice in the SET query",
|
||||
"Value with key `{}` is used twice in the SET query (collection name: {})",
|
||||
name, query.collection_name);
|
||||
}
|
||||
}
|
||||
|
@ -100,7 +100,7 @@ size_t TLDListsHolder::parseAndAddTldList(const std::string & name, const std::s
|
||||
tld_list_tmp.emplace(line, TLDType::TLD_REGULAR);
|
||||
}
|
||||
if (!in.eof())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not all list had been read", name);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not all list had been read: {}", name);
|
||||
|
||||
TLDList tld_list(tld_list_tmp.size());
|
||||
for (const auto & [host, type] : tld_list_tmp)
|
||||
|
@ -57,6 +57,7 @@ namespace
|
||||
if (_is_clients_log || _logger->is((PRIORITY))) \
|
||||
{ \
|
||||
std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \
|
||||
formatStringCheckArgsNum(__VA_ARGS__); \
|
||||
if (auto _channel = _logger->getChannel()) \
|
||||
{ \
|
||||
std::string file_function; \
|
||||
|
@ -57,7 +57,7 @@ static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (func->name != "Nullable" || func->arguments->children.size() != 1)
|
||||
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
|
||||
"Expected 'Nullable(<schema_name>)' as parameter for type Object", func->name);
|
||||
"Expected 'Nullable(<schema_name>)' as parameter for type Object (function: {})", func->name);
|
||||
|
||||
schema_argument = func->arguments->children[0];
|
||||
is_nullable = true;
|
||||
|
@ -136,7 +136,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context
|
||||
if (local_tables_cache.find(table_name) == local_tables_cache.end())
|
||||
{
|
||||
if (throw_on_error)
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {}.{} doesn't exist.", database_name_in_mysql, table_name);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
@ -180,7 +180,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam
|
||||
fetchTablesIntoLocalCache(getContext());
|
||||
|
||||
if (local_tables_cache.find(table_name) == local_tables_cache.end())
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {}.{} doesn't exist.", database_name_in_mysql, table_name);
|
||||
|
||||
return time_t(local_tables_cache[table_name].first);
|
||||
}
|
||||
|
@ -214,12 +214,12 @@ void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & t
|
||||
|
||||
if (!checkPostgresTable(table_name))
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE,
|
||||
"Cannot attach PostgreSQL table {} because it does not exist in PostgreSQL",
|
||||
"Cannot attach PostgreSQL table {} because it does not exist in PostgreSQL (database: {})",
|
||||
getTableNameForLogs(table_name), database_name);
|
||||
|
||||
if (!detached_or_dropped.contains(table_name))
|
||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS,
|
||||
"Cannot attach PostgreSQL table {} because it already exists",
|
||||
"Cannot attach PostgreSQL table {} because it already exists (database: {})",
|
||||
getTableNameForLogs(table_name), database_name);
|
||||
|
||||
if (cache_tables)
|
||||
|
@ -597,7 +597,7 @@ private:
|
||||
if (input_value.size < tag_size)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted data is too short: only {} bytes, "
|
||||
"should contain at least {} bytes of a tag.",
|
||||
input_value.size, block_size, tag_size);
|
||||
input_value.size, tag_size);
|
||||
|
||||
input_value.size -= tag_size;
|
||||
}
|
||||
|
@ -182,7 +182,7 @@ public:
|
||||
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of first argument of function, expected a string",
|
||||
"Illegal type {} of first argument of function {}, expected a string",
|
||||
arguments[0]->getName(),
|
||||
getName());
|
||||
|
||||
@ -416,7 +416,7 @@ public:
|
||||
|
||||
if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64)))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of fourth argument of function must be convertible to Int64.",
|
||||
"Illegal type {} of fourth argument of function {} must be convertible to Int64.",
|
||||
range_col_type->getName(),
|
||||
getName());
|
||||
|
||||
|
@ -37,8 +37,7 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer(
|
||||
throw Exception(
|
||||
ErrorCodes::LZ4_ENCODER_FAILED,
|
||||
"creation of LZ4 compression context failed. LZ4F version: {}",
|
||||
LZ4F_VERSION,
|
||||
ErrorCodes::LZ4_ENCODER_FAILED);
|
||||
LZ4F_VERSION);
|
||||
}
|
||||
|
||||
Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer()
|
||||
|
@ -131,7 +131,7 @@ namespace
|
||||
LOG_TRACE(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", dest_bucket, dest_key, multipart_upload_id, part_tags.size());
|
||||
|
||||
if (part_tags.empty())
|
||||
throw Exception("Failed to complete multipart upload. No parts have uploaded", ErrorCodes::S3_ERROR);
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Failed to complete multipart upload. No parts have uploaded");
|
||||
|
||||
Aws::S3::Model::CompleteMultipartUploadRequest request;
|
||||
request.SetBucket(dest_bucket);
|
||||
@ -680,7 +680,7 @@ namespace
|
||||
if (!outcome.IsSuccess())
|
||||
{
|
||||
abortMultipartUpload();
|
||||
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
||||
throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
||||
}
|
||||
|
||||
return outcome.GetResult().GetCopyPartResult().GetETag();
|
||||
|
@ -2046,7 +2046,7 @@ void Context::setQueryResultCache(size_t max_size_in_bytes, size_t max_entries,
|
||||
auto lock = getLock();
|
||||
|
||||
if (shared->query_result_cache)
|
||||
throw Exception("Query result cache has been already created.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Query result cache has been already created.");
|
||||
|
||||
shared->query_result_cache = std::make_shared<QueryResultCache>(max_size_in_bytes, max_entries, max_entry_size_in_bytes, max_entry_size_in_records);
|
||||
}
|
||||
|
@ -288,7 +288,7 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
copyData(res->out, out);
|
||||
copyData(res->err, out);
|
||||
if (!out.str().empty())
|
||||
LOG_DEBUG(log, "The command returned output: {}", command, out.str());
|
||||
LOG_DEBUG(log, "The command {} returned output: {}", command, out.str());
|
||||
res->wait();
|
||||
break;
|
||||
}
|
||||
|
@ -637,7 +637,7 @@ void ProcessList::decreaseQueryKindAmount(const IAST::QueryKind & query_kind)
|
||||
if (found == query_kind_amounts.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease before increase on '{}'", query_kind);
|
||||
else if (found->second == 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease to negative on '{}'", query_kind, found->second);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease to negative on '{}', {}", query_kind, found->second);
|
||||
else
|
||||
found->second -= 1;
|
||||
}
|
||||
|
@ -458,7 +458,7 @@ bool BSONEachRowRowInputFormat::readField(IColumn & column, const DataTypePtr &
|
||||
if (!format_settings.null_as_default)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Cannot insert BSON Null value into non-nullable column with type {}",
|
||||
getBSONTypeName(bson_type), data_type->getName());
|
||||
data_type->getName());
|
||||
|
||||
column.insertDefault();
|
||||
return false;
|
||||
|
@ -1776,7 +1776,7 @@ catch (...)
|
||||
{
|
||||
LOG_ERROR(log, "Loading of outdated parts failed. "
|
||||
"Will terminate to avoid undefined behaviour due to inconsistent set of parts. "
|
||||
"Exception: ", getCurrentExceptionMessage(true));
|
||||
"Exception: {}", getCurrentExceptionMessage(true));
|
||||
std::terminate();
|
||||
}
|
||||
|
||||
|
@ -207,7 +207,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const
|
||||
if (!checksums.files.contains(bin_file_name))
|
||||
throw Exception(
|
||||
ErrorCodes::NO_FILE_IN_DATA_PART,
|
||||
"No {} file checksum for column {} in part ",
|
||||
"No {} file checksum for column {} in part {}",
|
||||
bin_file_name, name_type.name, getDataPartStorage().getFullPath());
|
||||
});
|
||||
}
|
||||
|
@ -177,7 +177,7 @@ void PartMetadataManagerWithCache::assertAllDeleted(bool include_projection) con
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Data part {} with type {} with meta file {} with projection name still in cache",
|
||||
"Data part {} with type {} with meta file {} with projection name {} still in cache",
|
||||
part->name,
|
||||
part->getType().toString(),
|
||||
file_path,
|
||||
|
@ -619,7 +619,8 @@ void PostgreSQLReplicationHandler::removeTableFromPublication(pqxx::nontransacti
|
||||
catch (const pqxx::undefined_table &)
|
||||
{
|
||||
/// Removing table from replication must succeed even if table does not exist in PostgreSQL.
|
||||
LOG_WARNING(log, "Did not remove table {} from publication, because table does not exist in PostgreSQL", doubleQuoteWithSchema(table_name), publication_name);
|
||||
LOG_WARNING(log, "Did not remove table {} from publication, because table does not exist in PostgreSQL (publication: {})",
|
||||
doubleQuoteWithSchema(table_name), publication_name);
|
||||
}
|
||||
}
|
||||
|
||||
@ -756,7 +757,7 @@ std::set<String> PostgreSQLReplicationHandler::fetchRequiredTables()
|
||||
}
|
||||
|
||||
LOG_ERROR(log,
|
||||
"Publication {} already exists, but specified tables list differs from publication tables list in tables: {}. ",
|
||||
"Publication {} already exists, but specified tables list differs from publication tables list in tables: {}. "
|
||||
"Will use tables list from setting. "
|
||||
"To avoid redundant work, you can try ALTER PUBLICATION query to remove redundant tables. "
|
||||
"Or you can you ALTER SETTING. "
|
||||
|
@ -153,7 +153,7 @@ void StorageDictionary::checkTableCanBeDropped() const
|
||||
dictionary_name);
|
||||
if (location == Location::DictionaryDatabase)
|
||||
throw Exception(ErrorCodes::CANNOT_DETACH_DICTIONARY_AS_TABLE,
|
||||
"Cannot drop/detach table from a database with DICTIONARY engine, use DROP DICTIONARY or DETACH DICTIONARY query instead",
|
||||
"Cannot drop/detach table '{}' from a database with DICTIONARY engine, use DROP DICTIONARY or DETACH DICTIONARY query instead",
|
||||
dictionary_name);
|
||||
}
|
||||
|
||||
|
@ -116,20 +116,17 @@ void S3Settings::RequestSettings::PartUploadSettings::validate()
|
||||
if (!upload_part_size_multiply_factor)
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_SETTING_VALUE,
|
||||
"Setting upload_part_size_multiply_factor cannot be zero",
|
||||
upload_part_size_multiply_factor);
|
||||
"Setting upload_part_size_multiply_factor cannot be zero");
|
||||
|
||||
if (!upload_part_size_multiply_parts_count_threshold)
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_SETTING_VALUE,
|
||||
"Setting upload_part_size_multiply_parts_count_threshold cannot be zero",
|
||||
upload_part_size_multiply_parts_count_threshold);
|
||||
"Setting upload_part_size_multiply_parts_count_threshold cannot be zero");
|
||||
|
||||
if (!max_part_number)
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_SETTING_VALUE,
|
||||
"Setting max_part_number cannot be zero",
|
||||
max_part_number);
|
||||
"Setting max_part_number cannot be zero");
|
||||
|
||||
static constexpr size_t max_part_number_limit = 10000;
|
||||
if (max_part_number > max_part_number_limit)
|
||||
|
Loading…
Reference in New Issue
Block a user