mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
less empty patterns
This commit is contained in:
parent
870cfcc36a
commit
522686f78b
@ -130,7 +130,7 @@ BackupEntries BackupEntriesCollector::run()
|
||||
|
||||
Strings BackupEntriesCollector::setStage(const String & new_stage, const String & message)
|
||||
{
|
||||
LOG_TRACE(log, "{}", toUpperFirst(new_stage));
|
||||
LOG_TRACE(log, fmt::runtime(toUpperFirst(new_stage)));
|
||||
current_stage = new_stage;
|
||||
|
||||
backup_coordination->setStage(backup_settings.host_id, new_stage, message);
|
||||
@ -215,7 +215,7 @@ void BackupEntriesCollector::gatherMetadataAndCheckConsistency()
|
||||
if (std::chrono::steady_clock::now() > consistent_metadata_snapshot_end_time)
|
||||
inconsistency_error->rethrow();
|
||||
else
|
||||
LOG_WARNING(log, "{}", inconsistency_error->displayText());
|
||||
LOG_WARNING(log, getExceptionMessageAndPattern(*inconsistency_error, /* with_stacktrace */ false));
|
||||
}
|
||||
|
||||
auto sleep_time = getSleepTimeAfterInconsistencyError(pass);
|
||||
|
@ -145,7 +145,7 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode)
|
||||
|
||||
void RestorerFromBackup::setStage(const String & new_stage, const String & message)
|
||||
{
|
||||
LOG_TRACE(log, "{}", toUpperFirst(new_stage));
|
||||
LOG_TRACE(log, fmt::runtime(toUpperFirst(new_stage)));
|
||||
current_stage = new_stage;
|
||||
|
||||
if (restore_coordination)
|
||||
|
@ -212,7 +212,7 @@ bool checkPermissionsImpl()
|
||||
{
|
||||
/// This error happens all the time when running inside Docker - consider it ok,
|
||||
/// don't create noise with this error.
|
||||
LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "{}", getCurrentExceptionMessage(false));
|
||||
LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false));
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -278,7 +278,7 @@ private:
|
||||
if (next_pos != std::string_view::npos)
|
||||
size = next_pos - pos;
|
||||
|
||||
LOG_FATAL(log, "{}", message.substr(pos, size));
|
||||
LOG_FATAL(log, fmt::runtime(message.substr(pos, size)));
|
||||
pos = next_pos;
|
||||
}
|
||||
}
|
||||
|
@ -100,7 +100,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co
|
||||
if (!load_result.config)
|
||||
{
|
||||
if (throw_on_error)
|
||||
throw Exception{"Dictionary " + backQuote(table_name) + " doesn't exist", ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY};
|
||||
throw Exception(ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY, "Dictionary {} doesn't exist", backQuote(table_name));
|
||||
return {};
|
||||
}
|
||||
|
||||
|
@ -102,7 +102,7 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name
|
||||
auto it = tables_cache.find(table_name);
|
||||
if (it != tables_cache.end())
|
||||
return it->second.metadata_modification_time;
|
||||
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.", backQuote(database_name), backQuote(table_name));
|
||||
}
|
||||
|
||||
void DatabaseLazy::alterTable(
|
||||
@ -185,7 +185,7 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables_cache.find(table_name);
|
||||
if (it == tables_cache.end())
|
||||
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.", backQuote(database_name), backQuote(table_name));
|
||||
res = it->second.table;
|
||||
if (it->second.expiration_iterator != cache_expiration_queue.end())
|
||||
cache_expiration_queue.erase(it->second.expiration_iterator);
|
||||
|
@ -463,8 +463,7 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, Contex
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (!has_table && e.code() == ErrorCodes::FILE_DOESNT_EXIST && throw_on_error)
|
||||
throw Exception{"Table " + backQuote(table_name) + " doesn't exist",
|
||||
ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY};
|
||||
throw Exception(ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY, "Table {} doesn't exist", backQuote(table_name));
|
||||
else if (!is_system_storage && throw_on_error)
|
||||
throw;
|
||||
}
|
||||
|
@ -136,8 +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("MySQL table " + database_name_in_mysql + "." + table_name + " doesn't exist..",
|
||||
ErrorCodes::UNKNOWN_TABLE);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
@ -181,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("MySQL table " + database_name_in_mysql + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
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);
|
||||
}
|
||||
|
@ -26,7 +26,7 @@ inline void throwIfDivisionLeadsToFPE(A a, B b)
|
||||
/// Is it better to use siglongjmp instead of checks?
|
||||
|
||||
if (unlikely(b == 0))
|
||||
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION);
|
||||
throw Exception(ErrorCodes::ILLEGAL_DIVISION, "Division by zero");
|
||||
|
||||
/// http://avva.livejournal.com/2548306.html
|
||||
if (unlikely(is_signed_v<A> && is_signed_v<B> && a == std::numeric_limits<A>::min() && b == -1))
|
||||
|
@ -207,7 +207,7 @@ public:
|
||||
|
||||
if (arguments.size() < 2)
|
||||
{
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "{}", arguments.size());
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect number of arguments: {}", arguments.size());
|
||||
}
|
||||
|
||||
const auto * first_array_type = checkAndGetDataType<typename Impl::data_type>(arguments[1].type.get());
|
||||
|
@ -22,7 +22,7 @@ struct DivideDecimalsImpl
|
||||
execute(FirstType a, SecondType b, UInt16 scale_a, UInt16 scale_b, UInt16 result_scale)
|
||||
{
|
||||
if (b.value == 0)
|
||||
throw DB::Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION);
|
||||
throw DB::Exception(ErrorCodes::ILLEGAL_DIVISION, "Division by zero");
|
||||
if (a.value == 0)
|
||||
return Decimal256(0);
|
||||
|
||||
|
@ -78,7 +78,7 @@ struct DivideIntegralByConstantImpl
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
if (unlikely(static_cast<A>(b) == 0))
|
||||
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION);
|
||||
throw Exception(ErrorCodes::ILLEGAL_DIVISION, "Division by zero");
|
||||
|
||||
divideImpl(a_pos, b, c_pos, size);
|
||||
}
|
||||
|
@ -78,7 +78,7 @@ struct ModuloByConstantImpl
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
if (unlikely(static_cast<A>(b) == 0))
|
||||
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION);
|
||||
throw Exception(ErrorCodes::ILLEGAL_DIVISION, "Division by zero");
|
||||
|
||||
/// Division by min negative value.
|
||||
if (std::is_signed_v<B> && b == std::numeric_limits<B>::lowest())
|
||||
|
@ -126,7 +126,7 @@ public:
|
||||
const auto & [level, prio] = convertLogLevel(log_level);
|
||||
if (tag_loggers.contains(tag))
|
||||
{
|
||||
LOG_IMPL(tag_loggers[tag], level, prio, "{}", message);
|
||||
LOG_IMPL(tag_loggers[tag], level, prio, fmt::runtime(message));
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -268,7 +268,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
{
|
||||
assert(!db_and_table.first && !db_and_table.second);
|
||||
if (exception)
|
||||
exception->emplace(fmt::format("Table {} doesn't exist", table_id.getNameForLogs()), ErrorCodes::UNKNOWN_TABLE);
|
||||
exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs()));
|
||||
return {};
|
||||
}
|
||||
|
||||
@ -317,7 +317,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
|
||||
auto table = database->tryGetTable(table_id.table_name, context_);
|
||||
if (!table && exception)
|
||||
exception->emplace(fmt::format("Table {} doesn't exist", table_id.getNameForLogs()), ErrorCodes::UNKNOWN_TABLE);
|
||||
exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs()));
|
||||
if (!table)
|
||||
database = nullptr;
|
||||
|
||||
|
@ -111,8 +111,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
|
||||
{
|
||||
if (query.if_exists)
|
||||
return {};
|
||||
throw Exception("Temporary table " + backQuoteIfNeed(table_id.table_name) + " doesn't exist",
|
||||
ErrorCodes::UNKNOWN_TABLE);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Temporary table {} doesn't exist", backQuoteIfNeed(table_id.table_name));
|
||||
}
|
||||
|
||||
auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr);
|
||||
|
@ -61,8 +61,7 @@ QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline()
|
||||
storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext());
|
||||
|
||||
if (!storage)
|
||||
throw Exception("Table " + table_id.getNameForLogs() + " doesn't exist.",
|
||||
ErrorCodes::UNKNOWN_TABLE);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist.", table_id.getNameForLogs());
|
||||
|
||||
auto storage_name = storage->getName();
|
||||
if (storage_name == "LiveView"
|
||||
|
@ -2629,15 +2629,13 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
}
|
||||
else if (command.type == AlterCommand::DROP_COLUMN)
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP version " + backQuoteIfNeed(command.column_name) + " column",
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
|
||||
"Trying to ALTER DROP version {} column", backQuoteIfNeed(command.column_name));
|
||||
}
|
||||
else if (command.type == AlterCommand::RENAME_COLUMN)
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER RENAME version " + backQuoteIfNeed(command.column_name) + " column",
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
|
||||
"Trying to ALTER RENAME version {} column", backQuoteIfNeed(command.column_name));
|
||||
}
|
||||
}
|
||||
|
||||
@ -2689,9 +2687,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
{
|
||||
if (columns_in_keys.contains(command.column_name))
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP key " + backQuoteIfNeed(command.column_name) + " column which is a part of key expression",
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
|
||||
"Trying to ALTER DROP key {} column which is a part of key expression", backQuoteIfNeed(command.column_name));
|
||||
}
|
||||
|
||||
if (!command.clear)
|
||||
@ -2699,10 +2696,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
|
||||
"Trying to ALTER DROP column {} which is referenced by materialized view {}",
|
||||
backQuoteIfNeed(command.column_name), toString(deps_mv));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -98,9 +98,9 @@ bool MutatePlainMergeTreeTask::executeStep()
|
||||
{
|
||||
if (merge_mutate_entry->txn)
|
||||
merge_mutate_entry->txn->onException();
|
||||
String exception_message = getCurrentExceptionMessage(false);
|
||||
LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message);
|
||||
storage.updateMutationEntriesErrors(future_part, false, exception_message);
|
||||
PreformattedMessage exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false);
|
||||
LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), exception_message);
|
||||
storage.updateMutationEntriesErrors(future_part, false, exception_message.message);
|
||||
write_part_log(ExecutionStatus::fromCurrentException());
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
return false;
|
||||
|
@ -1026,10 +1026,9 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
|
||||
"Trying to ALTER DROP column {} which is referenced by materialized view {}",
|
||||
backQuoteIfNeed(command.column_name), toString(deps_mv));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1065,10 +1065,9 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, Co
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
|
||||
"Trying to ALTER DROP column {} which is referenced by materialized view {}",
|
||||
backQuoteIfNeed(command.column_name), toString(deps_mv));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -838,10 +838,9 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, ContextP
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
|
||||
"Trying to ALTER DROP column {} which is referenced by materialized view {}",
|
||||
backQuoteIfNeed(command.column_name), toString(deps_mv));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -54,10 +54,10 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPt
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
|
||||
"Trying to ALTER DROP column {} which is referenced by materialized view {}",
|
||||
backQuoteIfNeed(command.column_name), toString(deps_mv)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -67,8 +67,8 @@ ColumnsDescription TableFunctionMySQL::getActualTableStructure(ContextPtr contex
|
||||
|
||||
const auto columns = tables_and_columns.find(configuration->table);
|
||||
if (columns == tables_and_columns.end())
|
||||
throw Exception("MySQL table " + (configuration->database.empty() ? "" : (backQuote(configuration->database) + "."))
|
||||
+ backQuote(configuration->table) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.",
|
||||
(configuration->database.empty() ? "" : (backQuote(configuration->database) + "." + backQuote(configuration->table))));
|
||||
|
||||
return columns->second;
|
||||
}
|
||||
|
@ -1851,7 +1851,8 @@ def reportLogStats(args):
|
||||
SELECT
|
||||
count() AS count,
|
||||
substr(replaceRegexpAll(message, '[^A-Za-z]+', ''), 1, 32) AS pattern,
|
||||
substr(any(message), 1, 256) as runtime_message
|
||||
substr(any(message), 1, 256) as runtime_message,
|
||||
any((extract(source_file, '\/[a-zA-Z0-9_]+\.[a-z]+'), source_line)) as line
|
||||
FROM system.text_log
|
||||
WHERE (now() - toIntervalMinute(mins)) < event_time AND message_format_string = ''
|
||||
GROUP BY pattern
|
||||
|
Loading…
Reference in New Issue
Block a user