mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #58226 from Algunenano/cleanup_known_short
Cleanup some known short messages
This commit is contained in:
commit
31a081bd83
@ -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)));
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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)));
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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"
|
||||
|
@ -2803,8 +2803,6 @@ void MergeTreeData::dropAllData()
|
||||
|
||||
void MergeTreeData::dropIfEmpty()
|
||||
{
|
||||
LOG_TRACE(log, "dropIfEmpty");
|
||||
|
||||
auto lock = lockParts();
|
||||
|
||||
if (!data_parts_by_info.empty())
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user