Merge pull request #58226 from Algunenano/cleanup_known_short

Cleanup some known short messages
This commit is contained in:
Alexey Milovidov 2023-12-26 14:40:58 +01:00 committed by GitHub
commit 31a081bd83
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 132 additions and 53 deletions

View File

@ -610,7 +610,6 @@ void BackupsWorker::doBackup(
void BackupsWorker::buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, const ReadSettings & read_settings, std::shared_ptr<IBackupCoordination> backup_coordination)
{
LOG_TRACE(log, "{}", Stage::BUILDING_FILE_INFOS);
backup_coordination->setStage(Stage::BUILDING_FILE_INFOS, "");
backup_coordination->waitForStage(Stage::BUILDING_FILE_INFOS);
backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), read_settings, getThreadPool(ThreadPoolId::BACKUP_MAKE_FILES_LIST)));

View File

@ -101,19 +101,35 @@ public:
const Float64 lon = data_col_lon[row];
const Float64 lat = data_col_lat[row];
if (isNaN(lon) || isNaN(lat))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments must not be NaN");
if (isNaN(lon))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for longitude in function {}. It must not be NaN", getName());
if (!isFinite(lon))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal argument for longitude in function {}. It must not be infinite",
getName());
if (!(isFinite(lon) && isFinite(lat)))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments must not be infinite");
if (isNaN(lat))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for latitude in function {}. It must not be NaN", getName());
if (!isFinite(lat))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal argument for latitude in function {}. It must not be infinite",
getName());
/// S2 acceptes point as (latitude, longitude)
/// S2 accepts point as (latitude, longitude)
S2LatLng lat_lng = S2LatLng::FromDegrees(lat, lon);
if (!lat_lng.is_valid())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive"
"and the longitude is between -180 and 180 degrees inclusive.");
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Point ({}, {}) is invalid in function {}. For valid point the latitude is between -90 and 90 degrees inclusive"
"and the longitude is between -180 and 180 degrees inclusive.",
lon,
lat,
getName());
S2CellId id(lat_lng);

View File

@ -84,7 +84,7 @@ public:
const UInt64 hindex = data[row];
if (!isValidCell(hindex))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {}", hindex);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {} in function {}", hindex, getName());
h3ToString(hindex, pos, H3_INDEX_STRING_LENGTH);

View File

@ -131,16 +131,16 @@ public:
const auto point = S2CellId(data_point[row]);
if (isNaN(degrees))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Radius of the cap must not be nan");
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Radius of the cap must not be nan in function {}", getName());
if (std::isinf(degrees))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Radius of the cap must not be infinite");
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Radius of the cap must not be infinite in function {}", getName());
if (!center.is_valid())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Center is not valid");
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Center (id {}) is not valid in function {}", data_center[row], getName());
if (!point.is_valid())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid");
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point (id {}) is not valid in function {}", data_point[row], getName());
S1Angle angle = S1Angle::Degrees(degrees);
S2Cap cap(center.ToPoint(), angle);

View File

@ -100,10 +100,12 @@ public:
const UInt64 id_second = data_id_second[row];
auto first_cell = S2CellId(id_first);
auto second_cell = S2CellId(id_second);
if (!first_cell.is_valid())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First cell (id {}) is not valid in function {}", id_first, getName());
if (!first_cell.is_valid() || !second_cell.is_valid())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cell is not valid");
auto second_cell = S2CellId(id_second);
if (!second_cell.is_valid())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second cell (id {}) is not valid in function {}", id_second, getName());
dst_data.emplace_back(S2CellId(id_first).intersects(S2CellId(id_second)));
}

View File

@ -94,7 +94,7 @@ public:
S2CellId cell_id(id);
if (!cell_id.is_valid())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cell is not valid");
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cell (id {}) is not valid in function {}", id, getName());
S2CellId neighbors[4];
cell_id.GetEdgeNeighbors(neighbors);

View File

@ -88,7 +88,7 @@ private:
if (res_data[row_num] == 0)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {}", h3index_str);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {} in function {}", h3index_str, name);
}
h3index_source.next();

View File

@ -61,7 +61,7 @@ QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline()
storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext());
if (!storage)
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist.", table_id.getNameForLogs());
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist.", table_id.getNameForLogs());
auto storage_name = storage->getName();
if (storage_name == "LiveView"

View File

@ -2803,8 +2803,6 @@ void MergeTreeData::dropAllData()
void MergeTreeData::dropIfEmpty()
{
LOG_TRACE(log, "dropIfEmpty");
auto lock = lockParts();
if (!data_parts_by_info.empty())

View File

@ -1300,8 +1300,6 @@ void StorageDistributed::drop()
disk->removeRecursive(relative_data_path);
}
LOG_DEBUG(log, "Removed");
}
Strings StorageDistributed::getDataPaths() const
@ -1328,8 +1326,6 @@ void StorageDistributed::truncate(const ASTPtr &, const StorageMetadataPtr &, Co
it->second.directory_queue->shutdownAndDropAllData();
it = cluster_nodes_data.erase(it);
}
LOG_DEBUG(log, "Removed");
}
StoragePolicyPtr StorageDistributed::getStoragePolicy() const

View File

@ -66,33 +66,101 @@ WHERE
-- FIXME some of the following messages are not informative and it has to be fixed
create temporary table known_short_messages (s String) as select * from (select
['', '{} ({})', '({}) Keys: {}', '({}) {}', 'Aggregating', 'Became leader', 'Cleaning queue',
'Creating set.', 'Cyclic aliases', 'Detaching {}', 'Executing {}', 'Fire events: {}',
'Found part {}', 'Loaded queue', 'No sharding key', 'No tables', 'Query: {}', 'Removed',
'Removed part {}', 'Removing parts.', 'Request URI: {}', 'Sending part {}',
'Sent handshake', 'Starting {}', 'Will mimic {}', 'Writing to {}', 'dropIfEmpty',
'loadAll {}', '{} ({}:{})', '{} -> {}', '{} {}', '{}: {}', '{}%', 'Read object: {}',
'New segment: {}', 'Convert overflow', 'Division by zero', 'Files set to {}',
'Bytes set to {}', 'Numeric overflow', 'Invalid mode: {}',
'Write file: {}', 'Unable to parse JSONPath', 'Host is empty in S3 URI.', 'Expected end of line',
'inflate failed: {}{}', 'Center is not valid', 'Column ''{}'' is ambiguous', 'Cannot parse object', 'Invalid date: {}',
'There is no cache by name: {}', 'No part {} in table', '`{}` should be a String', 'There are duplicate id {}',
'Invalid replica name: {}', 'Unexpected value {} in enum', 'Unknown BSON type: {}', 'Point is not valid',
'Invalid qualified name: {}', 'INTO OUTFILE is not allowed', 'Arguments must not be NaN', 'Cell is not valid',
'brotli decode error{}', 'Invalid H3 index: {}', 'Too large node state size', 'No additional keys found.',
'Attempt to read after EOF.', 'Replication was stopped', '{} building file infos', 'Cannot parse uuid {}',
'Query was cancelled', 'Cancelled merging parts', 'Cancelled mutating parts', 'Log pulling is cancelled',
'Transaction was cancelled', 'Could not find table: {}', 'Table {} does not exist',
'Database {} does not exist', 'Dictionary ({}) not found', 'Unknown table function {}',
'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}',
'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}',
'Attempt to read after eof', 'String size is too big ({}), maximum: {}',
'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}',
'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64',
'Unknown statistic column: {}',
'Bad SSH public key provided', 'Database {} does not exist', 'Substitution {} is not set', 'Invalid cache key hex: {}'
] as arr) array join arr;
create temporary table known_short_messages (s String) as select * from (select [
'',
'({}) Keys: {}',
'({}) {}',
'Aggregating',
'Attempt to read after EOF.',
'Attempt to read after eof',
'Bad SSH public key provided',
'Became leader',
'Bytes set to {}',
'Cancelled merging parts',
'Cancelled mutating parts',
'Cannot parse date here: {}',
'Cannot parse object',
'Cannot parse uuid {}',
'Cleaning queue',
'Column \'{}\' is ambiguous',
'Convert overflow',
'Could not find table: {}',
'Creating {}: {}',
'Cyclic aliases',
'Database {} does not exist',
'Detaching {}',
'Dictionary ({}) not found',
'Division by zero',
'Executing {}',
'Expected end of line',
'Expected function, got: {}',
'Files set to {}',
'Fire events: {}',
'Found part {}',
'Host is empty in S3 URI.',
'INTO OUTFILE is not allowed',
'Illegal type {} of argument of function {}. Should be DateTime or DateTime64',
'Illegal UTF-8 sequence, while processing \'{}\'',
'Invalid cache key hex: {}',
'Invalid date: {}',
'Invalid mode: {}',
'Invalid qualified name: {}',
'Invalid replica name: {}',
'Loaded queue',
'Log pulling is cancelled',
'New segment: {}',
'No additional keys found.',
'No part {} in table',
'No sharding key',
'No tables',
'Numeric overflow',
'Path to archive is empty',
'Processed: {}%',
'Query was cancelled',
'Query: {}',
'Read object: {}',
'Removed part {}',
'Removing parts.',
'Replication was stopped',
'Request URI: {}',
'Sending part {}',
'Sent handshake',
'Starting {}',
'String size is too big ({}), maximum: {}',
'Substitution {} is not set',
'Table {} does not exist',
'Table {}.{} doesn\'t exist',
'There are duplicate id {}',
'There is no cache by name: {}',
'Too large node state size',
'Transaction was cancelled',
'Unable to parse JSONPath',
'Unexpected value {} in enum',
'Unknown BSON type: {}',
'Unknown explain kind \'{}\'',
'Unknown format {}',
'Unknown identifier: \'{}\'',
'Unknown input format {}',
'Unknown setting {}',
'Unknown statistic column: {}',
'Unknown table function {}',
'User has been dropped',
'User name is empty',
'Will mimic {}',
'Write file: {}',
'Writing to {}',
'`{}` should be a String',
'brotli decode error{}',
'dropIfEmpty',
'inflate failed: {}{}',
'loadAll {}',
'{} ({})',
'{} ({}:{})',
'{} -> {}',
'{} {}',
'{}%',
'{}: {}'
] as arr) array join arr;
-- Check that we don't have too many short meaningless message patterns.
WITH 1 AS max_messages