Merge pull request #64022 from ClickHouse/fix-analyzer

Fix a typo in Analyzer
This commit is contained in:
Alexey Milovidov 2024-05-17 20:02:13 +02:00 committed by GitHub
commit bd9709adee
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
29 changed files with 48 additions and 48 deletions

View File

@ -138,7 +138,7 @@ function filter_exists_and_template
# but it doesn't allow to use regex # but it doesn't allow to use regex
echo "$path" | sed 's/\.sql\.j2$/.gen.sql/' echo "$path" | sed 's/\.sql\.j2$/.gen.sql/'
else else
echo "'$path' does not exists" >&2 echo "'$path' does not exist" >&2
fi fi
done done
} }

View File

@ -93,7 +93,7 @@ sidebar_label: 2022
* `(U)Int128` and `(U)Int256` values are correctly checked in `PREWHERE`. [#42605](https://github.com/ClickHouse/ClickHouse/pull/42605) ([Antonio Andelic](https://github.com/antonio2368)). * `(U)Int128` and `(U)Int256` values are correctly checked in `PREWHERE`. [#42605](https://github.com/ClickHouse/ClickHouse/pull/42605) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix a bug in ParserFunction that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)). * Fix a bug in ParserFunction that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix truncate table does not hold lock correctly. [#42728](https://github.com/ClickHouse/ClickHouse/pull/42728) ([flynn](https://github.com/ucasfl)). * Fix truncate table does not hold lock correctly. [#42728](https://github.com/ClickHouse/ClickHouse/pull/42728) ([flynn](https://github.com/ucasfl)).
* Fix possible SIGSEGV for web disks when file does not exists (or `OPTIMIZE TABLE FINAL`, that also can got the same error eventually). [#42767](https://github.com/ClickHouse/ClickHouse/pull/42767) ([Azat Khuzhin](https://github.com/azat)). * Fix possible SIGSEGV for web disks when file does not exist (or `OPTIMIZE TABLE FINAL`, that also can got the same error eventually). [#42767](https://github.com/ClickHouse/ClickHouse/pull/42767) ([Azat Khuzhin](https://github.com/azat)).
* Fix `auth_type` mapping in `system.session_log`, by including `SSL_CERTIFICATE` for the enum values. [#42782](https://github.com/ClickHouse/ClickHouse/pull/42782) ([Miel Donkers](https://github.com/mdonkers)). * Fix `auth_type` mapping in `system.session_log`, by including `SSL_CERTIFICATE` for the enum values. [#42782](https://github.com/ClickHouse/ClickHouse/pull/42782) ([Miel Donkers](https://github.com/mdonkers)).
* Fix stack-use-after-return under ASAN build in ParserCreateUserQuery. [#42804](https://github.com/ClickHouse/ClickHouse/pull/42804) ([Nikolay Degterinsky](https://github.com/evillique)). * Fix stack-use-after-return under ASAN build in ParserCreateUserQuery. [#42804](https://github.com/ClickHouse/ClickHouse/pull/42804) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix lowerUTF8()/upperUTF8() in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)). * Fix lowerUTF8()/upperUTF8() in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)).

View File

@ -53,7 +53,7 @@ sidebar_label: 2022
* Store Keeper API version inside a predefined path. [#39096](https://github.com/ClickHouse/ClickHouse/pull/39096) ([Antonio Andelic](https://github.com/antonio2368)). * Store Keeper API version inside a predefined path. [#39096](https://github.com/ClickHouse/ClickHouse/pull/39096) ([Antonio Andelic](https://github.com/antonio2368)).
* Now entrypoint.sh in docker image creates and executes chown for all folders it found in config for multidisk setup [#17717](https://github.com/ClickHouse/ClickHouse/issues/17717). [#39121](https://github.com/ClickHouse/ClickHouse/pull/39121) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Now entrypoint.sh in docker image creates and executes chown for all folders it found in config for multidisk setup [#17717](https://github.com/ClickHouse/ClickHouse/issues/17717). [#39121](https://github.com/ClickHouse/ClickHouse/pull/39121) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Add profile events for fsync. [#39179](https://github.com/ClickHouse/ClickHouse/pull/39179) ([Azat Khuzhin](https://github.com/azat)). * Add profile events for fsync. [#39179](https://github.com/ClickHouse/ClickHouse/pull/39179) ([Azat Khuzhin](https://github.com/azat)).
* Add the second argument to the ordinary function `file(path[, default])`, which function returns in the case when a file does not exists. [#39218](https://github.com/ClickHouse/ClickHouse/pull/39218) ([Nikolay Degterinsky](https://github.com/evillique)). * Add the second argument to the ordinary function `file(path[, default])`, which function returns in the case when a file does not exist. [#39218](https://github.com/ClickHouse/ClickHouse/pull/39218) ([Nikolay Degterinsky](https://github.com/evillique)).
* Some small fixes for reading via http, allow to retry partial content in case if got 200OK. [#39244](https://github.com/ClickHouse/ClickHouse/pull/39244) ([Kseniia Sumarokova](https://github.com/kssenii)). * Some small fixes for reading via http, allow to retry partial content in case if got 200OK. [#39244](https://github.com/ClickHouse/ClickHouse/pull/39244) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Improved Base58 encoding/decoding. [#39292](https://github.com/ClickHouse/ClickHouse/pull/39292) ([Andrey Zvonov](https://github.com/zvonand)). * Improved Base58 encoding/decoding. [#39292](https://github.com/ClickHouse/ClickHouse/pull/39292) ([Andrey Zvonov](https://github.com/zvonand)).
* Normalize `AggregateFunction` types and state representations because optimizations like https://github.com/ClickHouse/ClickHouse/pull/35788 will treat `count(not null columns)` as `count()`, which might confuses distributed interpreters with the following error : `Conversion from AggregateFunction(count) to AggregateFunction(count, Int64) is not supported`. [#39420](https://github.com/ClickHouse/ClickHouse/pull/39420) ([Amos Bird](https://github.com/amosbird)). * Normalize `AggregateFunction` types and state representations because optimizations like https://github.com/ClickHouse/ClickHouse/pull/35788 will treat `count(not null columns)` as `count()`, which might confuses distributed interpreters with the following error : `Conversion from AggregateFunction(count) to AggregateFunction(count, Int64) is not supported`. [#39420](https://github.com/ClickHouse/ClickHouse/pull/39420) ([Amos Bird](https://github.com/amosbird)).

View File

@ -291,7 +291,7 @@ sidebar_label: 2023
* Fix replica groups for Replicated database engine [#55587](https://github.com/ClickHouse/ClickHouse/pull/55587) ([Azat Khuzhin](https://github.com/azat)). * Fix replica groups for Replicated database engine [#55587](https://github.com/ClickHouse/ClickHouse/pull/55587) ([Azat Khuzhin](https://github.com/azat)).
* Remove unused protobuf includes [#55590](https://github.com/ClickHouse/ClickHouse/pull/55590) ([Raúl Marín](https://github.com/Algunenano)). * Remove unused protobuf includes [#55590](https://github.com/ClickHouse/ClickHouse/pull/55590) ([Raúl Marín](https://github.com/Algunenano)).
* Apply Context changes to standalone Keeper [#55591](https://github.com/ClickHouse/ClickHouse/pull/55591) ([Antonio Andelic](https://github.com/antonio2368)). * Apply Context changes to standalone Keeper [#55591](https://github.com/ClickHouse/ClickHouse/pull/55591) ([Antonio Andelic](https://github.com/antonio2368)).
* Do not fail if label-to-remove does not exists in PR [#55592](https://github.com/ClickHouse/ClickHouse/pull/55592) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Do not fail if label-to-remove does not exist in PR [#55592](https://github.com/ClickHouse/ClickHouse/pull/55592) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* CI: cast extra column expression `pull_request_number` to Int32 [#55599](https://github.com/ClickHouse/ClickHouse/pull/55599) ([Han Fei](https://github.com/hanfei1991)). * CI: cast extra column expression `pull_request_number` to Int32 [#55599](https://github.com/ClickHouse/ClickHouse/pull/55599) ([Han Fei](https://github.com/hanfei1991)).
* Add back a test that was removed by mistake [#55605](https://github.com/ClickHouse/ClickHouse/pull/55605) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add back a test that was removed by mistake [#55605](https://github.com/ClickHouse/ClickHouse/pull/55605) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Bump croaring to v2.0.4 [#55606](https://github.com/ClickHouse/ClickHouse/pull/55606) ([Robert Schulze](https://github.com/rschu1ze)). * Bump croaring to v2.0.4 [#55606](https://github.com/ClickHouse/ClickHouse/pull/55606) ([Robert Schulze](https://github.com/rschu1ze)).

View File

@ -35,7 +35,7 @@ api_version
/keeper/api_version :) ls /keeper/api_version :) ls
/keeper/api_version :) cd xyz /keeper/api_version :) cd xyz
Path /keeper/api_version/xyz does not exists Path /keeper/api_version/xyz does not exist
/keeper/api_version :) cd ../../ /keeper/api_version :) cd ../../
/ :) ls / :) ls
keeper foo bar keeper foo bar

View File

@ -263,7 +263,7 @@ void Client::initialize(Poco::Util::Application & self)
config().add(loaded_config.configuration); config().add(loaded_config.configuration);
} }
else if (config().has("connection")) else if (config().has("connection"))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "--connection was specified, but config does not exists"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "--connection was specified, but config does not exist");
/** getenv is thread-safe in Linux glibc and in all sane libc implementations. /** getenv is thread-safe in Linux glibc and in all sane libc implementations.
* But the standard does not guarantee that subsequent calls will not rewrite the value by returned pointer. * But the standard does not guarantee that subsequent calls will not rewrite the value by returned pointer.

View File

@ -63,7 +63,7 @@ void CDCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con
auto new_path = client->getAbsolutePath(query->args[0].safeGet<String>()); auto new_path = client->getAbsolutePath(query->args[0].safeGet<String>());
if (!client->zookeeper->exists(new_path)) if (!client->zookeeper->exists(new_path))
std::cerr << "Path " << new_path << " does not exists\n"; std::cerr << "Path " << new_path << " does not exist\n";
else else
client->cwd = new_path; client->cwd = new_path;
} }

View File

@ -5090,14 +5090,14 @@ ProjectionName QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierR
auto * nearest_query_scope = scope.getNearestQueryScope(); auto * nearest_query_scope = scope.getNearestQueryScope();
if (!nearest_query_scope) if (!nearest_query_scope)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window '{}' does not exists.", parent_window_name); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window '{}' does not exist.", parent_window_name);
auto & scope_window_name_to_window_node = nearest_query_scope->window_name_to_window_node; auto & scope_window_name_to_window_node = nearest_query_scope->window_name_to_window_node;
auto window_node_it = scope_window_name_to_window_node.find(parent_window_name); auto window_node_it = scope_window_name_to_window_node.find(parent_window_name);
if (window_node_it == scope_window_name_to_window_node.end()) if (window_node_it == scope_window_name_to_window_node.end())
throw Exception(ErrorCodes::BAD_ARGUMENTS, throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Window '{}' does not exists. In scope {}", "Window '{}' does not exist. In scope {}",
parent_window_name, parent_window_name,
nearest_query_scope->scope_node->formatASTForErrorMessage()); nearest_query_scope->scope_node->formatASTForErrorMessage());
@ -5861,7 +5861,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
{ {
if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name)) if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name))
{ {
throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, "Aggregate function with name '{}' does not exists. In scope {}{}", throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, "Aggregate function with name '{}' does not exist. In scope {}{}",
function_name, scope.scope_node->formatASTForErrorMessage(), function_name, scope.scope_node->formatASTForErrorMessage(),
getHintsErrorMessageSuffix(AggregateFunctionFactory::instance().getHints(function_name))); getHintsErrorMessageSuffix(AggregateFunctionFactory::instance().getHints(function_name)));
} }
@ -5942,7 +5942,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
auto hints = NamePrompter<2>::getHints(function_name, possible_function_names); auto hints = NamePrompter<2>::getHints(function_name, possible_function_names);
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, throw Exception(ErrorCodes::UNKNOWN_FUNCTION,
"Function with name '{}' does not exists. In scope {}{}", "Function with name '{}' does not exist. In scope {}{}",
function_name, function_name,
scope.scope_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage(),
getHintsErrorMessageSuffix(hints)); getHintsErrorMessageSuffix(hints));
@ -8070,7 +8070,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
auto window_node_it = scope.window_name_to_window_node.find(parent_window_name); auto window_node_it = scope.window_name_to_window_node.find(parent_window_name);
if (window_node_it == scope.window_name_to_window_node.end()) if (window_node_it == scope.window_name_to_window_node.end())
throw Exception(ErrorCodes::BAD_ARGUMENTS, throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Window '{}' does not exists. In scope {}", "Window '{}' does not exist. In scope {}",
parent_window_name, parent_window_name,
scope.scope_node->formatASTForErrorMessage()); scope.scope_node->formatASTForErrorMessage());
@ -8268,7 +8268,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
if (!has_node_in_alias_table) if (!has_node_in_alias_table)
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Node {} with duplicate alias {} does not exists in alias table. In scope {}", "Node {} with duplicate alias {} does not exist in alias table. In scope {}",
node->formatASTForErrorMessage(), node->formatASTForErrorMessage(),
node_alias, node_alias,
scope.scope_node->formatASTForErrorMessage()); scope.scope_node->formatASTForErrorMessage());

View File

@ -26,12 +26,12 @@ using LoggerRawPtr = Poco::Logger *;
* Then it must be destroyed when underlying table is destroyed. * Then it must be destroyed when underlying table is destroyed.
*/ */
/** Get Logger with specified name. If the Logger does not exists, it is created. /** Get Logger with specified name. If the Logger does not exist, it is created.
* Logger is destroyed, when last shared ptr that refers to Logger with specified name is destroyed. * Logger is destroyed, when last shared ptr that refers to Logger with specified name is destroyed.
*/ */
LoggerPtr getLogger(const std::string & name); LoggerPtr getLogger(const std::string & name);
/** Get Logger with specified name. If the Logger does not exists, it is created. /** Get Logger with specified name. If the Logger does not exist, it is created.
* This overload was added for specific purpose, when logger is constructed from constexpr string. * This overload was added for specific purpose, when logger is constructed from constexpr string.
* Logger is destroyed only during program shutdown. * Logger is destroyed only during program shutdown.
*/ */

View File

@ -440,7 +440,7 @@ void DatabaseMySQL::detachTablePermanently(ContextPtr, const String & table_name
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name));
if (fs::exists(remove_flag)) if (fs::exists(remove_flag))
throw Exception(ErrorCodes::LOGICAL_ERROR, "The remove flag file already exists but the {}.{} does not exists remove tables, it is bug.", throw Exception(ErrorCodes::LOGICAL_ERROR, "The remove flag file already exists but the {}.{} does not exist remove tables, it is bug.",
backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); backQuoteIfNeed(database_name), backQuoteIfNeed(table_name));
auto table_iter = local_tables_cache.find(table_name); auto table_iter = local_tables_cache.find(table_name);

View File

@ -470,7 +470,7 @@ public:
auto path = std::filesystem::path{file_path}; auto path = std::filesystem::path{file_path};
auto parent_path_directory = path.parent_path(); auto parent_path_directory = path.parent_path();
/// If cache file is in directory that does not exists create it /// If cache file is in directory that does not exist create it
if (!std::filesystem::exists(parent_path_directory)) if (!std::filesystem::exists(parent_path_directory))
if (!std::filesystem::create_directories(parent_path_directory)) if (!std::filesystem::create_directories(parent_path_directory))
throw Exception(ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories."); throw Exception(ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories.");

View File

@ -1641,7 +1641,7 @@ void Context::updateExternalTable(const String & table_name, std::shared_ptr<Tem
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
auto it = external_tables_mapping.find(table_name); auto it = external_tables_mapping.find(table_name);
if (it == external_tables_mapping.end()) if (it == external_tables_mapping.end())
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary table {} does not exists", backQuoteIfNeed(table_name)); throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary table {} does not exist", backQuoteIfNeed(table_name));
it->second = std::move(temporary_table); it->second = std::move(temporary_table);
} }

View File

@ -137,7 +137,7 @@ public:
if (it == column_name_to_column.end()) if (it == column_name_to_column.end())
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Column for column name {} does not exists. There are only column names: {}", "Column for column name {} does not exist. There are only column names: {}",
column_name, column_name,
fmt::join(column_names.begin(), column_names.end(), ", ")); fmt::join(column_names.begin(), column_names.end(), ", "));
} }
@ -154,7 +154,7 @@ public:
if (it == column_name_to_column_identifier.end()) if (it == column_name_to_column_identifier.end())
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Column identifier for column name {} does not exists. There are only column names: {}", "Column identifier for column name {} does not exist. There are only column names: {}",
column_name, column_name,
fmt::join(column_names.begin(), column_names.end(), ", ")); fmt::join(column_names.begin(), column_names.end(), ", "));
} }

View File

@ -1107,7 +1107,7 @@ void TCPHandler::processTablesStatusRequest()
ContextPtr context_to_resolve_table_names; ContextPtr context_to_resolve_table_names;
if (is_interserver_mode) if (is_interserver_mode)
{ {
/// In interserver mode session context does not exists, because authentication is done for each query. /// In the interserver mode session context does not exist, because authentication is done for each query.
/// We also cannot create query context earlier, because it cannot be created before authentication, /// We also cannot create query context earlier, because it cannot be created before authentication,
/// but query is not received yet. So we have to do this trick. /// but query is not received yet. So we have to do this trick.
ContextMutablePtr fake_interserver_context = Context::createCopy(server.context()); ContextMutablePtr fake_interserver_context = Context::createCopy(server.context());

View File

@ -173,7 +173,7 @@ bool DistributedAsyncInsertBatch::valid()
{ {
if (!fs::exists(file)) if (!fs::exists(file))
{ {
LOG_WARNING(parent.log, "File {} does not exists, likely due abnormal shutdown", file); LOG_WARNING(parent.log, "File {} does not exist, likely due abnormal shutdown", file);
res = false; res = false;
} }
} }

View File

@ -554,7 +554,7 @@ void DistributedAsyncInsertDirectoryQueue::processFilesWithBatching(const Settin
{ {
if (!fs::exists(file_path)) if (!fs::exists(file_path))
{ {
LOG_WARNING(log, "File {} does not exists, likely due to current_batch.txt processing", file_path); LOG_WARNING(log, "File {} does not exist, likely due to current_batch.txt processing", file_path);
continue; continue;
} }

View File

@ -580,7 +580,7 @@ void DataPartStorageOnDiskBase::rename(
disk.setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); disk.setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr)));
disk.moveDirectory(from, to); disk.moveDirectory(from, to);
/// Only after moveDirectory() since before the directory does not exists. /// Only after moveDirectory() since before the directory does not exist.
SyncGuardPtr to_sync_guard; SyncGuardPtr to_sync_guard;
if (fsync_part_dir) if (fsync_part_dir)
to_sync_guard = volume->getDisk()->getDirectorySyncGuard(to); to_sync_guard = volume->getDisk()->getDirectorySyncGuard(to);

View File

@ -100,7 +100,7 @@ protected:
/// Position and level (of nesting). /// Position and level (of nesting).
using ColumnNameLevel = std::optional<std::pair<String, size_t>>; using ColumnNameLevel = std::optional<std::pair<String, size_t>>;
/// In case of part of the nested column does not exists, offsets should be /// In case of part of the nested column does not exist, offsets should be
/// read, but only the offsets for the current column, that is why it /// read, but only the offsets for the current column, that is why it
/// returns pair of size_t, not just one. /// returns pair of size_t, not just one.
ColumnNameLevel findColumnForOffsets(const NameAndTypePair & column) const; ColumnNameLevel findColumnForOffsets(const NameAndTypePair & column) const;

View File

@ -5126,7 +5126,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String
auto volume = getStoragePolicy()->getVolumeByName(name); auto volume = getStoragePolicy()->getVolumeByName(name);
if (!volume) if (!volume)
throw Exception(ErrorCodes::UNKNOWN_DISK, "Volume {} does not exists on policy {}", name, getStoragePolicy()->getName()); throw Exception(ErrorCodes::UNKNOWN_DISK, "Volume {} does not exist on policy {}", name, getStoragePolicy()->getName());
if (parts.empty()) if (parts.empty())
throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Nothing to move (check that the partition exists)."); throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Nothing to move (check that the partition exists).");

View File

@ -490,11 +490,11 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn(
if (key_node_function_name == "arrayElement") if (key_node_function_name == "arrayElement")
{ {
/** Try to parse arrayElement for mapKeys index. /** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map * It is important to ignore keys like column_map['Key'] IN ('') because if the key does not exist in the map
* we return default value for arrayElement. * we return the default value for arrayElement.
* *
* We cannot skip keys that does not exist in map if comparison is with default type value because * We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists. * that way we skip necessary granules where the map key does not exist.
*/ */
if (!prepared_set) if (!prepared_set)
return false; return false;
@ -781,11 +781,11 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals(
if (key_node_function_name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) if (key_node_function_name == "arrayElement" && (function_name == "equals" || function_name == "notEquals"))
{ {
/** Try to parse arrayElement for mapKeys index. /** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map * It is important to ignore keys like column_map['Key'] = '' because if key does not exist in the map
* we return default value for arrayElement. * we return default the value for arrayElement.
* *
* We cannot skip keys that does not exist in map if comparison is with default type value because * We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists. * that way we skip necessary granules where map key does not exist.
*/ */
if (value_field == value_type->getDefault()) if (value_field == value_type->getDefault())
return false; return false;

View File

@ -444,11 +444,11 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals(
if (key_function_node_function_name == "arrayElement") if (key_function_node_function_name == "arrayElement")
{ {
/** Try to parse arrayElement for mapKeys index. /** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map * It is important to ignore keys like column_map['Key'] = '' because if key does not exist in the map
* we return default value for arrayElement. * we return default the value for arrayElement.
* *
* We cannot skip keys that does not exist in map if comparison is with default type value because * We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists. * that way we skip necessary granules where map key does not exist.
*/ */
if (value_field == value_type->getDefault()) if (value_field == value_type->getDefault())
return false; return false;

View File

@ -477,11 +477,11 @@ bool MergeTreeConditionFullText::traverseASTEquals(
if (function.getFunctionName() == "arrayElement") if (function.getFunctionName() == "arrayElement")
{ {
/** Try to parse arrayElement for mapKeys index. /** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map * It is important to ignore keys like column_map['Key'] = '' because if key does not exist in the map
* we return default value for arrayElement. * we return default the value for arrayElement.
* *
* We cannot skip keys that does not exist in map if comparison is with default type value because * We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists. * that way we skip necessary granules where map key does not exist.
*/ */
if (value_field == value_type->getDefault()) if (value_field == value_type->getDefault())
return false; return false;

View File

@ -232,7 +232,7 @@ bool MergeTreeReaderCompact::needSkipStream(size_t column_pos, const ISerializat
/// ///
/// Consider the following columns in nested "root": /// Consider the following columns in nested "root":
/// - root.array Array(UInt8) - exists /// - root.array Array(UInt8) - exists
/// - root.nested_array Array(Array(UInt8)) - does not exists (only_offsets_level=1) /// - root.nested_array Array(Array(UInt8)) - does not exist (only_offsets_level=1)
/// ///
/// For root.nested_array it will try to read multiple streams: /// For root.nested_array it will try to read multiple streams:
/// - offsets (substream_path = {ArraySizes}) /// - offsets (substream_path = {ArraySizes})

View File

@ -58,7 +58,7 @@ static inline String generateInnerTableName(const StorageID & view_id)
return ".inner." + view_id.getTableName(); return ".inner." + view_id.getTableName();
} }
/// Remove columns from target_header that does not exists in src_header /// Remove columns from target_header that does not exist in src_header
static void removeNonCommonColumns(const Block & src_header, Block & target_header) static void removeNonCommonColumns(const Block & src_header, Block & target_header)
{ {
std::set<size_t> target_only_positions; std::set<size_t> target_only_positions;
@ -233,10 +233,10 @@ void StorageMaterializedView::read(
auto mv_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, context, processed_stage); auto mv_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, context, processed_stage);
auto target_header = query_plan.getCurrentDataStream().header; auto target_header = query_plan.getCurrentDataStream().header;
/// No need to convert columns that does not exists in MV /// No need to convert columns that does not exist in MV
removeNonCommonColumns(mv_header, target_header); removeNonCommonColumns(mv_header, target_header);
/// No need to convert columns that does not exists in the result header. /// No need to convert columns that does not exist in the result header.
/// ///
/// Distributed storage may process query up to the specific stage, and /// Distributed storage may process query up to the specific stage, and
/// so the result header may not include all the columns from the /// so the result header may not include all the columns from the

View File

@ -42,7 +42,7 @@ def test_access_rights_for_function():
function_resolution_error = instance.query_and_get_error("SELECT MySum(1, 2)") function_resolution_error = instance.query_and_get_error("SELECT MySum(1, 2)")
assert ( assert (
"Unknown function MySum" in function_resolution_error "Unknown function MySum" in function_resolution_error
or "Function with name 'MySum' does not exists." in function_resolution_error or "Function with name 'MySum' does not exist." in function_resolution_error
) )
instance.query("REVOKE CREATE FUNCTION ON *.* FROM A") instance.query("REVOKE CREATE FUNCTION ON *.* FROM A")

View File

@ -28,7 +28,7 @@ def test_sql_user_defined_functions_on_cluster():
error_message = node.query_and_get_error("SELECT test_function(1);") error_message = node.query_and_get_error("SELECT test_function(1);")
assert ( assert (
"Unknown function test_function" in error_message "Unknown function test_function" in error_message
or "Function with name 'test_function' does not exists. In scope SELECT test_function(1)" or "Function with name 'test_function' does not exist. In scope SELECT test_function(1)"
in error_message in error_message
) )

View File

@ -38,13 +38,13 @@ def test_persistence():
error_message = instance.query_and_get_error("SELECT MySum1(1, 2)") error_message = instance.query_and_get_error("SELECT MySum1(1, 2)")
assert ( assert (
"Unknown function MySum1" in error_message "Unknown function MySum1" in error_message
or "Function with name 'MySum1' does not exists. In scope SELECT MySum1(1, 2)" or "Function with name 'MySum1' does not exist. In scope SELECT MySum1(1, 2)"
in error_message in error_message
) )
error_message = instance.query_and_get_error("SELECT MySum2(1, 2)") error_message = instance.query_and_get_error("SELECT MySum2(1, 2)")
assert ( assert (
"Unknown function MySum2" in error_message "Unknown function MySum2" in error_message
or "Function with name 'MySum2' does not exists. In scope SELECT MySum2(1, 2)" or "Function with name 'MySum2' does not exist. In scope SELECT MySum2(1, 2)"
in error_message in error_message
) )

View File

@ -72,7 +72,7 @@ DROP DATABASE IF EXISTS test2;
CREATE DATABASE test2 ENGINE = Filesystem('/tmp'); CREATE DATABASE test2 ENGINE = Filesystem('/tmp');
""" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||:
# BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exists # BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exist
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
DROP DATABASE IF EXISTS test2; DROP DATABASE IF EXISTS test2;
CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir');

View File

@ -10,4 +10,4 @@ select count() > 0 from system.text_log where message_format_string = 'Peak memo
select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier \'{}\' in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count'; select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier \'{}\' in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count';
select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name \'{}\' does not exists. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%'; select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name \'{}\' does not exist. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%';