diff --git a/CHANGELOG.md b/CHANGELOG.md index 0355b21c962..1b36142cc9f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,7 +22,7 @@ * The MergeTree setting `clean_deleted_rows` is deprecated, it has no effect anymore. The `CLEANUP` keyword for the `OPTIMIZE` is not allowed by default (it can be unlocked with the `allow_experimental_replacing_merge_with_cleanup` setting). [#58267](https://github.com/ClickHouse/ClickHouse/pull/58267) ([Alexander Tokmakov](https://github.com/tavplubix)). This fixes [#57930](https://github.com/ClickHouse/ClickHouse/issues/57930). This closes [#54988](https://github.com/ClickHouse/ClickHouse/issues/54988). This closes [#54570](https://github.com/ClickHouse/ClickHouse/issues/54570). This closes [#50346](https://github.com/ClickHouse/ClickHouse/issues/50346). This closes [#47579](https://github.com/ClickHouse/ClickHouse/issues/47579). The feature has to be removed because it is not good. We have to remove it as quickly as possible, because there is no other option. [#57932](https://github.com/ClickHouse/ClickHouse/pull/57932) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature -* Implement Refreshable Materialized Views, requested in [#33919](https://github.com/ClickHouse/ClickHouse/issues/57995). [#56946](https://github.com/ClickHouse/ClickHouse/pull/56946) ([Michael Kolupaev](https://github.com/al13n321), [Michael Guzov](https://github.com/koloshmet)). +* Implement Refreshable Materialized Views, requested in [#33919](https://github.com/ClickHouse/ClickHouse/issues/33919). [#56946](https://github.com/ClickHouse/ClickHouse/pull/56946) ([Michael Kolupaev](https://github.com/al13n321), [Michael Guzov](https://github.com/koloshmet)). * Introduce `PASTE JOIN`, which allows users to join tables without `ON` clause simply by row numbers. Example: `SELECT * FROM (SELECT number AS a FROM numbers(2)) AS t1 PASTE JOIN (SELECT number AS a FROM numbers(2) ORDER BY a DESC) AS t2`. [#57995](https://github.com/ClickHouse/ClickHouse/pull/57995) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * The `ORDER BY` clause now supports specifying `ALL`, meaning that ClickHouse sorts by all columns in the `SELECT` clause. Example: `SELECT col1, col2 FROM tab WHERE [...] ORDER BY ALL`. [#57875](https://github.com/ClickHouse/ClickHouse/pull/57875) ([zhongyuankai](https://github.com/zhongyuankai)). * Added a new mutation command `ALTER TABLE APPLY DELETED MASK`, which allows to enforce applying of mask written by lightweight delete and to remove rows marked as deleted from disk. [#57433](https://github.com/ClickHouse/ClickHouse/pull/57433) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/contrib/NuRaft b/contrib/NuRaft index b7ea89b817a..2f5f52c4d8c 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit b7ea89b817a18dc0eafc1f909d568869f02d2d04 +Subproject commit 2f5f52c4d8c87c2a3a3d101ca3a0194c9b77526f diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 836b1f2f637..ed67af48af7 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1262,6 +1262,7 @@ SELECT * FROM json_each_row_nested - [input_format_import_nested_json](/docs/en/operations/settings/settings-formats.md/#input_format_import_nested_json) - map nested JSON data to nested tables (it works for JSONEachRow format). Default value - `false`. - [input_format_json_read_bools_as_numbers](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`. +- [input_format_json_read_bools_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_bools_as_strings) - allow to parse bools as strings in JSON input formats. Default value - `true`. - [input_format_json_read_numbers_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `true`. - [input_format_json_read_arrays_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_arrays_as_strings) - allow to parse JSON arrays as strings in JSON input formats. Default value - `true`. - [input_format_json_read_objects_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `true`. diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index ef858796936..4db1d53987a 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -614,6 +614,26 @@ DESC format(JSONEachRow, $$ └───────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` +##### input_format_json_read_bools_as_strings + +Enabling this setting allows reading Bool values as strings. + +This setting is enabled by default. + +**Example:** + +```sql +SET input_format_json_read_bools_as_strings = 1; +DESC format(JSONEachRow, $$ + {"value" : true} + {"value" : "Hello, World"} + $$) +``` +```response +┌─name──┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ +│ value │ Nullable(String) │ │ │ │ │ │ +└───────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +``` ##### input_format_json_read_arrays_as_strings Enabling this setting allows reading JSON array values as strings. diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 3d76bd9df73..43a73844b79 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -377,6 +377,12 @@ Allow parsing bools as numbers in JSON input formats. Enabled by default. +## input_format_json_read_bools_as_strings {#input_format_json_read_bools_as_strings} + +Allow parsing bools as strings in JSON input formats. + +Enabled by default. + ## input_format_json_read_numbers_as_strings {#input_format_json_read_numbers_as_strings} Allow parsing numbers as strings in JSON input formats. diff --git a/docs/en/operations/system-tables/server_settings.md b/docs/en/operations/system-tables/server_settings.md index 7efe605ccef..417c3460a53 100644 --- a/docs/en/operations/system-tables/server_settings.md +++ b/docs/en/operations/system-tables/server_settings.md @@ -14,6 +14,11 @@ Columns: - `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting was specified in `config.xml` - `description` ([String](../../sql-reference/data-types/string.md)) — Short server setting description. - `type` ([String](../../sql-reference/data-types/string.md)) — Server setting value type. +- `changeable_without_restart` ([Enum8](../../sql-reference/data-types/enum.md)) — Whether the setting can be changed at server runtime. Values: + - `'No' ` + - `'IncreaseOnly'` + - `'DecreaseOnly'` + - `'Yes'` - `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether a setting is obsolete. **Example** @@ -27,22 +32,21 @@ WHERE name LIKE '%thread_pool%' ``` ``` text -┌─name────────────────────────────────────────_─value─_─default─_─changed─_─description────────────────────────────────────────────────────────────────────────────────────────────────────── -───────────────────────────────────_─type───_─is_obsolete─┐ -│ max_thread_pool_size │ 10000 │ 10000 │ 1 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ 0 │ -│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ 0 │ -│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ 0 │ -│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ 0 │ -│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ 0 │ -│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ 0 │ -│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ 0 │ -│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ 0 │ -│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ 0 │ -│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ 0 │ -│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ 0 │ -│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ 0 │ -└─────────────────────────────────────────────┴───────┴─────────┴─────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────── -───────────────────────────────────┴────────┴─────────────┘ +┌─name────────────────────────────────────────┬─value─┬─default─┬─changed─┬─description─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─type───┬─changeable_without_restart─┬─is_obsolete─┐ +│ max_thread_pool_size │ 10000 │ 10000 │ 0 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ No │ 0 │ +│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ No │ 0 │ +│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ No │ 0 │ +│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ No │ 0 │ +│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ No │ 0 │ +│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ No │ 0 │ +│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ No │ 0 │ +│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ No │ 0 │ +│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ No │ 0 │ +│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ No │ 0 │ +│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ No │ 0 │ +│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ No │ 0 │ +└─────────────────────────────────────────────┴───────┴─────────┴─────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┴────────────────────────────┴─────────────┘ + ``` Using of `WHERE changed` can be useful, for example, when you want to check diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 48d26233d94..109884ec899 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -335,7 +335,7 @@ try else if (std::filesystem::is_directory(std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination")) { throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "By default 'keeper.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper.storage_path' in the keeper configuration explicitly", + "By default 'keeper_server.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper_server.storage_path' in the keeper configuration explicitly", KEEPER_DEFAULT_PATH, String{std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination"}); } else diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 922eaabe75c..b07aa2d31b0 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -143,9 +143,17 @@ public: return alias; } + const String & getOriginalAlias() const + { + return original_alias.empty() ? alias : original_alias; + } + /// Set node alias void setAlias(String alias_value) { + if (original_alias.empty()) + original_alias = std::move(alias); + alias = std::move(alias_value); } @@ -276,6 +284,9 @@ protected: private: String alias; + /// An alias from query. Alias can be replaced by query passes, + /// but we need to keep the original one to support additional_table_filters. + String original_alias; ASTPtr original_ast; }; diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 3290d918a8b..4ad9581b5b6 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -52,6 +52,7 @@ #include +#include #include #include #include @@ -1198,7 +1199,7 @@ private: static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope); - static void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope); + void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope); static void convertLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); @@ -2168,7 +2169,12 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ scope.scope_node->formatASTForErrorMessage()); --positional_argument_number; - *node_to_replace = projection_nodes[positional_argument_number]; + *node_to_replace = projection_nodes[positional_argument_number]->clone(); + if (auto it = resolved_expressions.find(projection_nodes[positional_argument_number]); + it != resolved_expressions.end()) + { + resolved_expressions[*node_to_replace] = it->second; + } } } @@ -7366,6 +7372,7 @@ void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context { QueryAnalyzer analyzer; analyzer.resolve(query_tree_node, table_expression, context); + createUniqueTableAliases(query_tree_node, table_expression, context); } } diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index f75022220e7..53fcf534f64 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -326,7 +326,7 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q } } -QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node) +QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join) { QueryTreeNodes result; @@ -357,6 +357,8 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node) { auto & array_join_node = node_to_process->as(); nodes_to_process.push_front(array_join_node.getTableExpression()); + if (add_array_join) + result.push_back(std::move(node_to_process)); break; } case QueryTreeNodeType::JOIN: diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index e3316f5ad6b..d3eb6ba3cc2 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -51,7 +51,7 @@ std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options); /// Extract table, table function, query, union from join tree -QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node); +QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false); /// Extract left table expression from join tree QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node); diff --git a/src/Analyzer/createUniqueTableAliases.cpp b/src/Analyzer/createUniqueTableAliases.cpp new file mode 100644 index 00000000000..8f850fe8dec --- /dev/null +++ b/src/Analyzer/createUniqueTableAliases.cpp @@ -0,0 +1,141 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class CreateUniqueTableAliasesVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + + explicit CreateUniqueTableAliasesVisitor(const ContextPtr & context) + : Base(context) + { + // Insert a fake node on top of the stack. + scope_nodes_stack.push_back(std::make_shared(Names{}, nullptr)); + } + + void enterImpl(QueryTreeNodePtr & node) + { + auto node_type = node->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: + { + /// Queries like `(SELECT 1) as t` have invalid syntax. To avoid creating such queries (e.g. in StorageDistributed) + /// we need to remove aliases for top level queries. + /// N.B. Subquery depth starts count from 1, so the following condition checks if it's a top level. + if (getSubqueryDepth() == 1) + { + node->removeAlias(); + break; + } + [[fallthrough]]; + } + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + [[fallthrough]]; + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & alias = table_expression_to_alias[node]; + if (alias.empty()) + { + scope_to_nodes_with_aliases[scope_nodes_stack.back()].push_back(node); + alias = fmt::format("__table{}", ++next_id); + node->setAlias(alias); + } + break; + } + default: + break; + } + + switch (node_type) + { + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; + case QueryTreeNodeType::LAMBDA: + scope_nodes_stack.push_back(node); + break; + default: + break; + } + } + + void leaveImpl(QueryTreeNodePtr & node) + { + if (scope_nodes_stack.back() == node) + { + if (auto it = scope_to_nodes_with_aliases.find(scope_nodes_stack.back()); + it != scope_to_nodes_with_aliases.end()) + { + for (const auto & node_with_alias : it->second) + { + table_expression_to_alias.erase(node_with_alias); + } + scope_to_nodes_with_aliases.erase(it); + } + scope_nodes_stack.pop_back(); + } + + /// Here we revisit subquery for IN function. Reasons: + /// * For remote query execution, query tree may be traversed a few times. + /// In such a case, it is possible to get AST like + /// `IN ((SELECT ... FROM table AS __table4) AS __table1)` which result in + /// `Multiple expressions for the alias` exception + /// * Tables in subqueries could have different aliases => different three hashes, + /// which is important to be able to find a set in PreparedSets + /// See 01253_subquery_in_aggregate_function_JustStranger. + /// + /// So, we revisit this subquery to make aliases stable. + /// This should be safe cause columns from IN subquery can't be used in main query anyway. + if (node->getNodeType() == QueryTreeNodeType::FUNCTION) + { + auto * function_node = node->as(); + if (isNameOfInFunction(function_node->getFunctionName())) + { + auto arg = function_node->getArguments().getNodes().back(); + /// Avoid aliasing IN `table` + if (arg->getNodeType() != QueryTreeNodeType::TABLE) + CreateUniqueTableAliasesVisitor(getContext()).visit(function_node->getArguments().getNodes().back()); + } + } + } + +private: + size_t next_id = 0; + + // Stack of nodes which create scopes: QUERY, UNION and LAMBDA. + QueryTreeNodes scope_nodes_stack; + + std::unordered_map scope_to_nodes_with_aliases; + + // We need to use raw pointer as a key, not a QueryTreeNodePtrWithHash. + std::unordered_map table_expression_to_alias; +}; + +} + + +void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & /*table_expression*/, const ContextPtr & context) +{ + CreateUniqueTableAliasesVisitor(context).visit(node); +} + +} diff --git a/src/Analyzer/createUniqueTableAliases.h b/src/Analyzer/createUniqueTableAliases.h new file mode 100644 index 00000000000..d57a198498c --- /dev/null +++ b/src/Analyzer/createUniqueTableAliases.h @@ -0,0 +1,18 @@ +#pragma once + +#include +#include + +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + +namespace DB +{ + +/* + * For each table expression in the Query Tree generate and add a unique alias. + * If table expression had an alias in initial query tree, override it. + */ +void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression, const ContextPtr & context); + +} diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 836c03d81ff..eb98c3a5740 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -77,7 +77,6 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti }; add_column("name", "functions", false, {}); - add_column("name", "database_engines", false, {}); add_column("name", "table_engines", false, {}); add_column("name", "formats", false, {}); add_column("name", "table_functions", false, {}); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index fb56d58cb72..965e743da39 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -4,6 +4,7 @@ #include "config.h" #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -196,13 +198,9 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server nuraft::raft_server::commit_in_bg(); } - void commitLogs(uint64_t index_to_commit, bool initial_commit_exec) + std::unique_lock lockRaft() { - leader_commit_index_.store(index_to_commit); - quick_commit_index_ = index_to_commit; - lagging_sm_target_index_ = index_to_commit; - - commit_in_bg_exec(0, initial_commit_exec); + return std::unique_lock(lock_); } using nuraft::raft_server::raft_server; @@ -518,6 +516,7 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions) { std::vector> entries; + entries.reserve(requests_for_sessions.size()); for (const auto & request_for_session : requests_for_sessions) entries.push_back(getZooKeeperLogEntry(request_for_session)); @@ -630,32 +629,32 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ { const auto preprocess_logs = [&] { + auto lock = raft_instance->lockRaft(); keeper_context->local_logs_preprocessed = true; auto log_store = state_manager->load_log_store(); - if (last_log_idx_on_disk > 0 && last_log_idx_on_disk > state_machine->last_commit_index()) - { - auto log_entries = log_store->log_entries(state_machine->last_commit_index() + 1, last_log_idx_on_disk + 1); + auto log_entries = log_store->log_entries(state_machine->last_commit_index() + 1, log_store->next_slot()); - size_t preprocessed = 0; - LOG_INFO(log, "Preprocessing {} log entries", log_entries->size()); - auto idx = state_machine->last_commit_index() + 1; - for (const auto & entry : *log_entries) - { - if (entry && entry->get_val_type() == nuraft::log_val_type::app_log) - state_machine->pre_commit(idx, entry->get_buf()); - - ++idx; - ++preprocessed; - - if (preprocessed % 50000 == 0) - LOG_TRACE(log, "Preprocessed {}/{} entries", preprocessed, log_entries->size()); - } - LOG_INFO(log, "Preprocessing done"); - } - else + if (log_entries->empty()) { LOG_INFO(log, "All local log entries preprocessed"); + return; } + + size_t preprocessed = 0; + LOG_INFO(log, "Preprocessing {} log entries", log_entries->size()); + auto idx = state_machine->last_commit_index() + 1; + for (const auto & entry : *log_entries) + { + if (entry && entry->get_val_type() == nuraft::log_val_type::app_log) + state_machine->pre_commit(idx, entry->get_buf()); + + ++idx; + ++preprocessed; + + if (preprocessed % 50000 == 0) + LOG_TRACE(log, "Preprocessed {}/{} entries", preprocessed, log_entries->size()); + } + LOG_INFO(log, "Preprocessing done"); }; switch (type) @@ -666,43 +665,34 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ /// until we preprocess all stored logs return nuraft::cb_func::ReturnCode::ReturnNull; } - case nuraft::cb_func::InitialBatchCommited: - { - preprocess_logs(); - break; - } case nuraft::cb_func::GotAppendEntryReqFromLeader: { - auto & req = *static_cast(param->ctx); - - if (req.get_commit_idx() == 0 || req.log_entries().empty()) - break; - - auto last_committed_index = state_machine->last_commit_index(); - // Actual log number. - auto index_to_commit = std::min({last_log_idx_on_disk, req.get_last_log_idx(), req.get_commit_idx()}); - - if (index_to_commit > last_committed_index) - { - LOG_TRACE(log, "Trying to commit local log entries, committing upto {}", index_to_commit); - raft_instance->commitLogs(index_to_commit, true); - /// after we manually committed all the local logs we can, we assert that all of the local logs are either - /// committed or preprocessed - if (!keeper_context->local_logs_preprocessed) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Local logs are not preprocessed"); - } - else if (last_log_idx_on_disk <= last_committed_index) - { - keeper_context->local_logs_preprocessed = true; - } - else if - ( - index_to_commit == 0 || - (index_to_commit == last_committed_index && last_log_idx_on_disk > index_to_commit) /// we need to rollback all the logs so we preprocess all of them - ) + /// maybe we got snapshot installed + if (state_machine->last_commit_index() >= last_log_idx_on_disk) { preprocess_logs(); + break; } + + auto & req = *static_cast(param->ctx); + + if (req.log_entries().empty()) + break; + + if (req.get_last_log_idx() < last_log_idx_on_disk) + last_log_idx_on_disk = req.get_last_log_idx(); + /// we don't want to accept too many new logs before we preprocess all the local logs + /// because the next log index is decreased on each failure we need to also accept requests when it's near last_log_idx_on_disk + /// so the counter is reset on the leader side + else if (raft_instance->get_target_committed_log_idx() >= last_log_idx_on_disk && req.get_last_log_idx() > last_log_idx_on_disk) + return nuraft::cb_func::ReturnNull; + + break; + } + case nuraft::cb_func::StateMachineExecution: + { + if (state_machine->last_commit_index() >= last_log_idx_on_disk) + preprocess_logs(); break; } default: diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index ae3ff1553b0..d092a8d4440 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -13,6 +13,7 @@ private: static inline const std::unordered_map LEVELS = { + {LogsLevel::test, Poco::Message::Priority::PRIO_TEST}, {LogsLevel::trace, Poco::Message::Priority::PRIO_TRACE}, {LogsLevel::debug, Poco::Message::Priority::PRIO_DEBUG}, {LogsLevel::information, Poco::Message::PRIO_INFORMATION}, diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 310b3585eab..2a9fa8e744c 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -113,6 +113,8 @@ namespace DB M(Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0) \ M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \ + /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp + DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) struct ServerSettings : public BaseSettings diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b41e7869fae..58b7cbab4c9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -157,7 +157,7 @@ class IColumn; M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \ M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \ - M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ + M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ @@ -709,7 +709,6 @@ class IColumn; M(Bool, query_plan_execute_functions_after_sorting, true, "Allow to re-order functions after sorting", 0) \ M(Bool, query_plan_reuse_storage_ordering_for_window_functions, true, "Allow to use the storage sorting for window functions", 0) \ M(Bool, query_plan_lift_up_union, true, "Allow to move UNIONs up so that more parts of the query plan can be optimized", 0) \ - M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \ M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimization", 0) \ M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimization", 0) \ M(Bool, query_plan_remove_redundant_sorting, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries", 0) \ @@ -845,7 +844,7 @@ class IColumn; M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ - M(Bool, print_pretty_type_names, false, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ + M(Bool, print_pretty_type_names, true, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \ M(Bool, allow_named_collection_override_by_default, true, "Allow named collections' fields override by default.", 0)\ M(Bool, allow_experimental_shared_merge_tree, false, "Only available in ClickHouse Cloud", 0) \ @@ -918,6 +917,7 @@ class IColumn; MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \ MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_s3queue, true) \ + MAKE_OBSOLETE(M, Bool, query_plan_optimize_primary_key, true) \ /** The section above is for obsolete settings. Do not add anything there. */ @@ -983,6 +983,7 @@ class IColumn; M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \ M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ + M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \ M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \ M(Bool, input_format_json_read_numbers_as_strings, true, "Allow to parse numbers as strings in JSON input formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index aad57ffebb7..fdee1fd5b13 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -81,6 +81,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, + {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}}}, {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index acd26ca338b..1f246af74d3 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -85,10 +85,7 @@ std::string DataTypeMap::doGetName() const std::string DataTypeMap::doGetPrettyName(size_t indent) const { WriteBufferFromOwnString s; - s << "Map(\n" - << fourSpaceIndent(indent + 1) << key_type->getPrettyName(indent + 1) << ",\n" - << fourSpaceIndent(indent + 1) << value_type->getPrettyName(indent + 1) << '\n' - << fourSpaceIndent(indent) << ')'; + s << "Map(" << key_type->getPrettyName(indent) << ", " << value_type->getPrettyName(indent) << ')'; return s.str(); } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index fd2e5e6a784..db8a14c537a 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -98,21 +98,38 @@ std::string DataTypeTuple::doGetPrettyName(size_t indent) const { size_t size = elems.size(); WriteBufferFromOwnString s; - s << "Tuple(\n"; - for (size_t i = 0; i != size; ++i) + /// If the Tuple is named, we will output it in multiple lines with indentation. + if (have_explicit_names) { - if (i != 0) - s << ",\n"; + s << "Tuple(\n"; - s << fourSpaceIndent(indent + 1); - if (have_explicit_names) - s << backQuoteIfNeed(names[i]) << ' '; + for (size_t i = 0; i != size; ++i) + { + if (i != 0) + s << ",\n"; - s << elems[i]->getPrettyName(indent + 1); + s << fourSpaceIndent(indent + 1) + << backQuoteIfNeed(names[i]) << ' ' + << elems[i]->getPrettyName(indent + 1); + } + + s << ')'; + } + else + { + s << "Tuple("; + + for (size_t i = 0; i != size; ++i) + { + if (i != 0) + s << ", "; + s << elems[i]->getPrettyName(indent); + } + + s << ')'; } - s << '\n' << fourSpaceIndent(indent) << ')'; return s.str(); } diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 788ff429088..b2b083fd466 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -335,6 +335,22 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist { read(column, [&](ColumnString::Chars & data) { readJSONArrayInto(data, istr); }); } + else if (settings.json.read_bools_as_strings && !istr.eof() && (*istr.position() == 't' || *istr.position() == 'f')) + { + String str_value; + if (*istr.position() == 't') + { + assertString("true", istr); + str_value = "true"; + } + else if (*istr.position() == 'f') + { + assertString("false", istr); + str_value = "false"; + } + + read(column, [&](ColumnString::Chars & data) { data.insert(str_value.begin(), str_value.end()); }); + } else if (settings.json.read_numbers_as_strings && !istr.eof() && *istr.position() != '"') { String field; diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 9cc7cb3b89e..a7e9fb8e99f 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -450,10 +450,11 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo break; case FormatSettings::EscapingRule::JSON: result += fmt::format( - ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_objects_as_strings={}, read_numbers_as_strings={}, " + ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_bools_as_strings={}, read_objects_as_strings={}, read_numbers_as_strings={}, " "read_arrays_as_strings={}, try_infer_objects_as_tuples={}, infer_incomplete_types_as_strings={}, try_infer_objects={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers, + settings.json.read_bools_as_strings, settings.json.read_objects_as_strings, settings.json.read_numbers_as_strings, settings.json.read_arrays_as_strings, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 15743365d7d..0344ed54ae3 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -111,6 +111,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; format_settings.json.quote_decimals = settings.output_format_json_quote_decimals; format_settings.json.read_bools_as_numbers = settings.input_format_json_read_bools_as_numbers; + format_settings.json.read_bools_as_strings = settings.input_format_json_read_bools_as_strings; format_settings.json.read_numbers_as_strings = settings.input_format_json_read_numbers_as_strings; format_settings.json.read_objects_as_strings = settings.input_format_json_read_objects_as_strings; format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8d5c044a311..5982d30f6a7 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -204,6 +204,7 @@ struct FormatSettings bool ignore_unknown_keys_in_named_tuple = false; bool serialize_as_strings = false; bool read_bools_as_numbers = true; + bool read_bools_as_strings = true; bool read_numbers_as_strings = true; bool read_objects_as_strings = true; bool read_arrays_as_strings = true; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index e2ba188d015..f065d2f0f4d 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -377,6 +377,22 @@ namespace type_indexes.erase(TypeIndex::UInt8); } + /// If we have Bool and String types convert all numbers to String. + /// It's applied only when setting input_format_json_read_bools_as_strings is enabled. + void transformJSONBoolsAndStringsToString(DataTypes & data_types, TypeIndexesSet & type_indexes) + { + if (!type_indexes.contains(TypeIndex::String) || !type_indexes.contains(TypeIndex::UInt8)) + return; + + for (auto & type : data_types) + { + if (isBool(type)) + type = std::make_shared(); + } + + type_indexes.erase(TypeIndex::UInt8); + } + /// If we have type Nothing/Nullable(Nothing) and some other non Nothing types, /// convert all Nothing/Nullable(Nothing) types to the first non Nothing. /// For example, when we have [Nothing, Array(Int64)] it will convert it to [Array(Int64), Array(Int64)] @@ -628,6 +644,10 @@ namespace if (settings.json.read_bools_as_numbers) transformBoolsAndNumbersToNumbers(data_types, type_indexes); + /// Convert Bool to String if needed. + if (settings.json.read_bools_as_strings) + transformJSONBoolsAndStringsToString(data_types, type_indexes); + if (settings.json.try_infer_objects_as_tuples) mergeJSONPaths(data_types, type_indexes, settings, json_info); }; diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 256354b2833..05d35a57b12 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1382,8 +1382,12 @@ void skipJSONField(ReadBuffer & buf, StringRef name_of_field) } else { - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol '{}' for key '{}'", - std::string(*buf.position(), 1), name_of_field.toString()); + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Cannot read JSON field here: '{}'. Unexpected symbol '{}'{}", + String(buf.position(), std::min(buf.available(), size_t(10))), + std::string(1, *buf.position()), + name_of_field.empty() ? "" : " for key " + name_of_field.toString()); } } @@ -1753,7 +1757,7 @@ void readQuotedField(String & s, ReadBuffer & buf) void readJSONField(String & s, ReadBuffer & buf) { s.clear(); - auto parse_func = [](ReadBuffer & in) { skipJSONField(in, "json_field"); }; + auto parse_func = [](ReadBuffer & in) { skipJSONField(in, ""); }; readParsedValueInto(s, buf, parse_func); } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 827914eaefe..1789cc6c4b1 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1419,7 +1419,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return set; } - FutureSetPtr external_table_set; + FutureSetFromSubqueryPtr external_table_set; /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 07c52d50e18..cdc4292a79c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -664,26 +664,26 @@ void Aggregator::compileAggregateFunctionsIfNeeded() for (size_t i = 0; i < aggregate_functions.size(); ++i) { const auto * function = aggregate_functions[i]; - bool function_is_compilable = function->isCompilable(); - if (!function_is_compilable) - continue; - size_t offset_of_aggregate_function = offsets_of_aggregate_states[i]; - AggregateFunctionWithOffset function_to_compile + + if (function->isCompilable()) { - .function = function, - .aggregate_data_offset = offset_of_aggregate_function - }; + AggregateFunctionWithOffset function_to_compile + { + .function = function, + .aggregate_data_offset = offset_of_aggregate_function + }; - functions_to_compile.emplace_back(std::move(function_to_compile)); + functions_to_compile.emplace_back(std::move(function_to_compile)); - functions_description += function->getDescription(); - functions_description += ' '; + functions_description += function->getDescription(); + functions_description += ' '; - functions_description += std::to_string(offset_of_aggregate_function); - functions_description += ' '; + functions_description += std::to_string(offset_of_aggregate_function); + functions_description += ' '; + } - is_aggregate_function_compiled[i] = true; + is_aggregate_function_compiled[i] = function->isCompilable(); } if (functions_to_compile.empty()) @@ -1685,13 +1685,14 @@ bool Aggregator::executeOnBlock(Columns columns, /// For the case when there are no keys (all aggregate into one row). if (result.type == AggregatedDataVariants::Type::without_key) { -#if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_functions_instructions.data())) - { - executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); - } - else -#endif + /// TODO: Enable compilation after investigation +// #if USE_EMBEDDED_COMPILER +// if (compiled_aggregate_functions_holder) +// { +// executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); +// } +// else +// #endif { executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 6d8fd84557c..19882b0b828 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -82,8 +82,8 @@ private: using DDLGuardPtr = std::unique_ptr; -class FutureSet; -using FutureSetPtr = std::shared_ptr; +class FutureSetFromSubquery; +using FutureSetFromSubqueryPtr = std::shared_ptr; /// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID. /// Such table can be accessed from everywhere by its ID. @@ -116,7 +116,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext IDatabase * temporary_tables = nullptr; UUID id = UUIDHelpers::Nil; - FutureSetPtr future_set; + FutureSetFromSubqueryPtr future_set; }; ///TODO maybe remove shared_ptr from here? diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b6c9b8cdba3..8e8482ccbd7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2378,12 +2378,25 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle else { // It's possible to optimize count() given only partition predicates - SelectQueryInfo temp_query_info; - temp_query_info.query = query_ptr; - temp_query_info.syntax_analyzer_result = syntax_analyzer_result; - temp_query_info.prepared_sets = query_analyzer->getPreparedSets(); + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (analysis_result.hasPrewhere()) + { + auto & prewhere_info = analysis_result.prewhere_info; + filter_nodes.push_back(&prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name)); - return storage->totalRowsByPartitionPredicate(temp_query_info, context); + if (prewhere_info->row_level_filter) + filter_nodes.push_back(&prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name)); + } + if (analysis_result.hasWhere()) + { + filter_nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); + } + + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes, {}, context); + if (!filter_actions_dag) + return {}; + + return storage->totalRowsByPartitionPredicate(filter_actions_dag, context); } } diff --git a/src/Interpreters/JIT/compileFunction.cpp b/src/Interpreters/JIT/compileFunction.cpp index 1c6b324dad7..f50a122f9a2 100644 --- a/src/Interpreters/JIT/compileFunction.cpp +++ b/src/Interpreters/JIT/compileFunction.cpp @@ -67,8 +67,7 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio { const auto & function_argument_types = function.getArgumentTypes(); - auto & context = module.getContext(); - llvm::IRBuilder<> b(context); + llvm::IRBuilder<> b(module.getContext()); auto * size_type = b.getIntNTy(sizeof(size_t) * 8); auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy()); auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false); @@ -76,8 +75,6 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio /// Create function in module auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, function.getName(), module); - func->setAttributes(llvm::AttributeList::get(context, {{2, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}})); - auto * args = func->args().begin(); llvm::Value * rows_count_arg = args++; llvm::Value * columns_arg = args++; @@ -199,9 +196,6 @@ static void compileCreateAggregateStatesFunctions(llvm::Module & module, const s auto * create_aggregate_states_function_type = llvm::FunctionType::get(b.getVoidTy(), { aggregate_data_places_type }, false); auto * create_aggregate_states_function = llvm::Function::Create(create_aggregate_states_function_type, llvm::Function::ExternalLinkage, name, module); - create_aggregate_states_function->setAttributes( - llvm::AttributeList::get(context, {{1, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}})); - auto * arguments = create_aggregate_states_function->args().begin(); llvm::Value * aggregate_data_place_arg = arguments++; @@ -247,11 +241,6 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, auto * add_into_aggregate_states_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_type->getPointerTo(), places_type }, false); auto * add_into_aggregate_states_func = llvm::Function::Create(add_into_aggregate_states_func_declaration, llvm::Function::ExternalLinkage, name, module); - add_into_aggregate_states_func->setAttributes(llvm::AttributeList::get( - context, - {{3, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}, - {4, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}})); - auto * arguments = add_into_aggregate_states_func->args().begin(); llvm::Value * row_start_arg = arguments++; llvm::Value * row_end_arg = arguments++; @@ -307,7 +296,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, llvm::Value * aggregation_place = nullptr; if (places_argument_type == AddIntoAggregateStatesPlacesArgumentType::MultiplePlaces) - aggregation_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateInBoundsGEP(b.getInt8Ty()->getPointerTo(), places_arg, counter_phi)); + aggregation_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateGEP(b.getInt8Ty()->getPointerTo(), places_arg, counter_phi)); else aggregation_place = places_arg; @@ -324,7 +313,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, auto & column = columns[previous_columns_size + column_argument_index]; const auto & argument_type = arguments_types[column_argument_index]; - auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateInBoundsGEP(column.data_element_type, column.data_ptr, counter_phi)); + auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateGEP(column.data_element_type, column.data_ptr, counter_phi)); if (!argument_type->isNullable()) { @@ -332,7 +321,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, continue; } - auto * column_null_data_with_offset = b.CreateInBoundsGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi); + auto * column_null_data_with_offset = b.CreateGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi); auto * is_null = b.CreateICmpNE(b.CreateLoad(b.getInt8Ty(), column_null_data_with_offset), b.getInt8(0)); auto * nullable_unitialized = llvm::Constant::getNullValue(toNullableType(b, column.data_element_type)); auto * first_insert = b.CreateInsertValue(nullable_unitialized, column_data_element, {0}); @@ -365,8 +354,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, static void compileMergeAggregatesStates(llvm::Module & module, const std::vector & functions, const std::string & name) { - auto & context = module.getContext(); - llvm::IRBuilder<> b(context); + llvm::IRBuilder<> b(module.getContext()); auto * aggregate_data_place_type = b.getInt8Ty()->getPointerTo(); auto * aggregate_data_places_type = aggregate_data_place_type->getPointerTo(); @@ -377,11 +365,6 @@ static void compileMergeAggregatesStates(llvm::Module & module, const std::vecto auto * merge_aggregates_states_func = llvm::Function::Create(merge_aggregates_states_func_declaration, llvm::Function::ExternalLinkage, name, module); - merge_aggregates_states_func->setAttributes(llvm::AttributeList::get( - context, - {{1, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}, - {2, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}})); - auto * arguments = merge_aggregates_states_func->args().begin(); llvm::Value * aggregate_data_places_dst_arg = arguments++; llvm::Value * aggregate_data_places_src_arg = arguments++; @@ -443,11 +426,6 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons auto * insert_aggregates_into_result_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_type->getPointerTo(), aggregate_data_places_type }, false); auto * insert_aggregates_into_result_func = llvm::Function::Create(insert_aggregates_into_result_func_declaration, llvm::Function::ExternalLinkage, name, module); - insert_aggregates_into_result_func->setAttributes(llvm::AttributeList::get( - context, - {{3, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}, - {4, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}})); - auto * arguments = insert_aggregates_into_result_func->args().begin(); llvm::Value * row_start_arg = arguments++; llvm::Value * row_end_arg = arguments++; @@ -482,7 +460,7 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons auto * counter_phi = b.CreatePHI(row_start_arg->getType(), 2); counter_phi->addIncoming(row_start_arg, entry); - auto * aggregate_data_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateInBoundsGEP(b.getInt8Ty()->getPointerTo(), aggregate_data_places_arg, counter_phi)); + auto * aggregate_data_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateGEP(b.getInt8Ty()->getPointerTo(), aggregate_data_places_arg, counter_phi)); for (size_t i = 0; i < functions.size(); ++i) { @@ -492,11 +470,11 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons const auto * aggregate_function_ptr = functions[i].function; auto * final_value = aggregate_function_ptr->compileGetResult(b, aggregation_place_with_offset); - auto * result_column_data_element = b.CreateInBoundsGEP(columns[i].data_element_type, columns[i].data_ptr, counter_phi); + auto * result_column_data_element = b.CreateGEP(columns[i].data_element_type, columns[i].data_ptr, counter_phi); if (columns[i].null_data_ptr) { b.CreateStore(b.CreateExtractValue(final_value, {0}), result_column_data_element); - auto * result_column_is_null_element = b.CreateInBoundsGEP(b.getInt8Ty(), columns[i].null_data_ptr, counter_phi); + auto * result_column_is_null_element = b.CreateGEP(b.getInt8Ty(), columns[i].null_data_ptr, counter_phi); b.CreateStore(b.CreateSelect(b.CreateExtractValue(final_value, {1}), b.getInt8(1), b.getInt8(0)), result_column_is_null_element); } else diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 18a25482b7f..cc3db726f01 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -97,7 +97,7 @@ FutureSetFromSubquery::FutureSetFromSubquery( String key, std::unique_ptr source_, StoragePtr external_table_, - FutureSetPtr external_table_set_, + std::shared_ptr external_table_set_, const Settings & settings, bool in_subquery_) : external_table(std::move(external_table_)) @@ -168,6 +168,24 @@ std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & conte return plan; } +void FutureSetFromSubquery::buildSetInplace(const ContextPtr & context) +{ + if (external_table_set) + external_table_set->buildSetInplace(context); + + auto plan = build(context); + + if (!plan) + return; + + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); +} + SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) { if (!context->getSettingsRef().use_index_for_in_with_subqueries) @@ -233,7 +251,7 @@ String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & return buf.str(); } -FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) +FutureSetFromTuplePtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { auto from_tuple = std::make_shared(std::move(block), settings); const auto & set_types = from_tuple->getTypes(); @@ -247,7 +265,7 @@ FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Set return from_tuple; } -FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) +FutureSetFromStoragePtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) { auto from_storage = std::make_shared(std::move(set_)); auto [it, inserted] = sets_from_storage.emplace(key, from_storage); @@ -258,11 +276,11 @@ FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) return from_storage; } -FutureSetPtr PreparedSets::addFromSubquery( +FutureSetFromSubqueryPtr PreparedSets::addFromSubquery( const Hash & key, std::unique_ptr source, StoragePtr external_table, - FutureSetPtr external_table_set, + FutureSetFromSubqueryPtr external_table_set, const Settings & settings, bool in_subquery) { @@ -282,7 +300,7 @@ FutureSetPtr PreparedSets::addFromSubquery( return from_subquery; } -FutureSetPtr PreparedSets::addFromSubquery( +FutureSetFromSubqueryPtr PreparedSets::addFromSubquery( const Hash & key, QueryTreeNodePtr query_tree, const Settings & settings) @@ -300,7 +318,7 @@ FutureSetPtr PreparedSets::addFromSubquery( return from_subquery; } -FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const +FutureSetFromTuplePtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const { auto it = sets_from_tuple.find(key); if (it == sets_from_tuple.end()) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 9f8bac9f71c..7178cff73b9 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -69,6 +69,8 @@ private: SetPtr set; }; +using FutureSetFromStoragePtr = std::shared_ptr; + /// Set from tuple is filled as well as set from storage. /// Additionally, it can be converted to set useful for PK. class FutureSetFromTuple final : public FutureSet @@ -86,6 +88,8 @@ private: SetKeyColumns set_key_columns; }; +using FutureSetFromTuplePtr = std::shared_ptr; + /// Set from subquery can be built inplace for PK or in CreatingSet step. /// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created, /// but ordinary set would be created instead. @@ -96,7 +100,7 @@ public: String key, std::unique_ptr source_, StoragePtr external_table_, - FutureSetPtr external_table_set_, + std::shared_ptr external_table_set_, const Settings & settings, bool in_subquery_); @@ -110,6 +114,7 @@ public: SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context); + void buildSetInplace(const ContextPtr & context); QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); } void setQueryPlan(std::unique_ptr source_); @@ -119,7 +124,7 @@ public: private: SetAndKeyPtr set_and_key; StoragePtr external_table; - FutureSetPtr external_table_set; + std::shared_ptr external_table_set; std::unique_ptr source; QueryTreeNodePtr query_tree; @@ -130,6 +135,8 @@ private: // with new analyzer it's not a case }; +using FutureSetFromSubqueryPtr = std::shared_ptr; + /// Container for all the sets used in query. class PreparedSets { @@ -141,32 +148,32 @@ public: UInt64 operator()(const Hash & key) const { return key.low64 ^ key.high64; } }; - using SetsFromTuple = std::unordered_map>, Hashing>; - using SetsFromStorage = std::unordered_map, Hashing>; - using SetsFromSubqueries = std::unordered_map, Hashing>; + using SetsFromTuple = std::unordered_map, Hashing>; + using SetsFromStorage = std::unordered_map; + using SetsFromSubqueries = std::unordered_map; - FutureSetPtr addFromStorage(const Hash & key, SetPtr set_); - FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings); + FutureSetFromStoragePtr addFromStorage(const Hash & key, SetPtr set_); + FutureSetFromTuplePtr addFromTuple(const Hash & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery( + FutureSetFromSubqueryPtr addFromSubquery( const Hash & key, std::unique_ptr source, StoragePtr external_table, - FutureSetPtr external_table_set, + FutureSetFromSubqueryPtr external_table_set, const Settings & settings, bool in_subquery = false); - FutureSetPtr addFromSubquery( + FutureSetFromSubqueryPtr addFromSubquery( const Hash & key, QueryTreeNodePtr query_tree, const Settings & settings); - FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const; - std::shared_ptr findStorage(const Hash & key) const; - std::shared_ptr findSubquery(const Hash & key) const; + FutureSetFromTuplePtr findTuple(const Hash & key, const DataTypes & types) const; + FutureSetFromStoragePtr findStorage(const Hash & key) const; + FutureSetFromSubqueryPtr findSubquery(const Hash & key) const; void markAsINSubquery(const Hash & key); - using Subqueries = std::vector>; + using Subqueries = std::vector; Subqueries getSubqueries() const; bool hasSubqueries() const { return !sets_from_subqueries.empty(); } diff --git a/src/Interpreters/RequiredSourceColumnsData.h b/src/Interpreters/RequiredSourceColumnsData.h index dd4e2dc3d68..501f6961efa 100644 --- a/src/Interpreters/RequiredSourceColumnsData.h +++ b/src/Interpreters/RequiredSourceColumnsData.h @@ -36,7 +36,6 @@ struct RequiredSourceColumnsData bool has_table_join = false; bool has_array_join = false; - bool visit_index_hint = false; bool addColumnAliasIfAny(const IAST & ast); void addColumnIdentifier(const ASTIdentifier & node); diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index c07d783788a..3971c8b58f4 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -72,11 +72,6 @@ void RequiredSourceColumnsMatcher::visit(const ASTPtr & ast, Data & data) } if (auto * t = ast->as()) { - /// "indexHint" is a special function for index analysis. - /// Everything that is inside it is not calculated. See KeyCondition - if (!data.visit_index_hint && t->name == "indexHint") - return; - data.addColumnAliasIfAny(*ast); visit(*t, ast, data); return; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9cbf24091e3..6ed3ff2f1e6 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -995,13 +995,12 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) /// Calculate which columns are required to execute the expression. /// Then, delete all other columns from the list of available columns. /// After execution, columns will only contain the list of columns needed to read from the table. -bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw) +bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool no_throw) { /// We calculate required_source_columns with source_columns modifications and swap them on exit required_source_columns = source_columns; RequiredSourceColumnsVisitor::Data columns_context; - columns_context.visit_index_hint = visit_index_hint; RequiredSourceColumnsVisitor(columns_context).visit(query); NameSet source_column_names; @@ -1385,7 +1384,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.window_function_asts = getWindowFunctions(query, *select_query); result.expressions_with_window_function = getExpressionsWithWindowFunctions(query); - result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key); + result.collectUsedColumns(query, true); if (!result.missed_subcolumns.empty()) { @@ -1422,7 +1421,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.aggregates = getAggregates(query, *select_query); result.window_function_asts = getWindowFunctions(query, *select_query); result.expressions_with_window_function = getExpressionsWithWindowFunctions(query); - result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key); + result.collectUsedColumns(query, true); } } @@ -1499,7 +1498,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( else assertNoAggregates(query, "in wrong place"); - bool is_ok = result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key, no_throw); + bool is_ok = result.collectUsedColumns(query, false, no_throw); if (!is_ok) return {}; diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 1858488afa3..205b4760423 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -88,7 +88,7 @@ struct TreeRewriterResult bool add_special = true); void collectSourceColumns(bool add_special); - bool collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw = false); + bool collectUsedColumns(const ASTPtr & query, bool is_select, bool no_throw = false); Names requiredSourceColumns() const { return required_source_columns.getNames(); } const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; } NameSet getArrayJoinSourceNameSet() const; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d2ffd47c500..a0c0fce4934 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1057,7 +1057,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( Planner subquery_planner( query_tree, subquery_options, - planner_context->getGlobalPlannerContext()); + std::make_shared()); //planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); subquery->setQueryPlan(std::make_unique(std::move(subquery_planner).extractQueryPlan())); diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 0fde034b87a..422c8c1d01f 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -20,12 +20,15 @@ const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const Quer return createColumnIdentifier(column_node_typed.getColumn(), column_source_node); } -const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & /*column_source_node*/) +const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & column_source_node) { std::string column_identifier; - column_identifier += column.name; - column_identifier += '_' + std::to_string(column_identifiers.size()); + const auto & source_alias = column_source_node->getAlias(); + if (!source_alias.empty()) + column_identifier = source_alias + "." + column.name; + else + column_identifier = column.name; auto [it, inserted] = column_identifiers.emplace(column_identifier); assert(inserted); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4b2a799f146..f6569d998f1 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -817,7 +817,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } } - const auto & table_expression_alias = table_expression->getAlias(); + const auto & table_expression_alias = table_expression->getOriginalAlias(); auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context); add_filter(additional_filters_info, "additional filter"); @@ -1058,6 +1058,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto right_plan = std::move(right_join_tree_query_plan.query_plan); auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + // { + // WriteBufferFromOwnString buf; + // left_plan.explainPlan(buf, {.header = true, .actions = true}); + // std::cerr << "left plan \n "<< buf.str() << std::endl; + // } + + // { + // WriteBufferFromOwnString buf; + // right_plan.explainPlan(buf, {.header = true, .actions = true}); + // std::cerr << "right plan \n "<< buf.str() << std::endl; + // } + JoinClausesAndActions join_clauses_and_actions; JoinKind join_kind = join_node.getKind(); JoinStrictness join_strictness = join_node.getStrictness(); diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 5e9de4dedcf..9b249d21a24 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -20,6 +20,7 @@ #include #include +#include #include #include #include @@ -113,41 +114,96 @@ String JoinClause::dump() const namespace { -std::optional extractJoinTableSideFromExpression(const ActionsDAG::Node * expression_root_node, - const std::unordered_set & join_expression_dag_input_nodes, - const NameSet & left_table_expression_columns_names, - const NameSet & right_table_expression_columns_names, +using TableExpressionSet = std::unordered_set; + +TableExpressionSet extractTableExpressionsSet(const QueryTreeNodePtr & node) +{ + TableExpressionSet res; + for (const auto & expr : extractTableExpressions(node, true)) + res.insert(expr.get()); + + return res; +} + +std::optional extractJoinTableSideFromExpression(//const ActionsDAG::Node * expression_root_node, + const IQueryTreeNode * expression_root_node, + //const std::unordered_set & join_expression_dag_input_nodes, + const TableExpressionSet & left_table_expressions, + const TableExpressionSet & right_table_expressions, const JoinNode & join_node) { std::optional table_side; - std::vector nodes_to_process; + std::vector nodes_to_process; nodes_to_process.push_back(expression_root_node); + // std::cerr << "==== extractJoinTableSideFromExpression\n"; + // std::cerr << "inp nodes" << std::endl; + // for (const auto * node : join_expression_dag_input_nodes) + // std::cerr << reinterpret_cast(node) << ' ' << node->result_name << std::endl; + + + // std::cerr << "l names" << std::endl; + // for (const auto & l : left_table_expression_columns_names) + // std::cerr << l << std::endl; + + // std::cerr << "r names" << std::endl; + // for (const auto & r : right_table_expression_columns_names) + // std::cerr << r << std::endl; + + // const auto * left_table_expr = join_node.getLeftTableExpression().get(); + // const auto * right_table_expr = join_node.getRightTableExpression().get(); + while (!nodes_to_process.empty()) { const auto * node_to_process = nodes_to_process.back(); nodes_to_process.pop_back(); - for (const auto & child : node_to_process->children) - nodes_to_process.push_back(child); + //std::cerr << "... " << reinterpret_cast(node_to_process) << ' ' << node_to_process->result_name << std::endl; - if (!join_expression_dag_input_nodes.contains(node_to_process)) + if (const auto * function_node = node_to_process->as()) + { + for (const auto & child : function_node->getArguments()) + nodes_to_process.push_back(child.get()); + + continue; + } + + const auto * column_node = node_to_process->as(); + if (!column_node) continue; - const auto & input_name = node_to_process->result_name; + // if (!join_expression_dag_input_nodes.contains(node_to_process)) + // continue; - bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name); - bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name); + const auto & input_name = column_node->getColumnName(); - if (!left_table_expression_contains_input && !right_table_expression_contains_input) + // bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name); + // bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name); + + // if (!left_table_expression_contains_input && !right_table_expression_contains_input) + // throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + // "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns", + // join_node.formatASTForErrorMessage(), + // input_name, + // boost::join(left_table_expression_columns_names, ", "), + // boost::join(right_table_expression_columns_names, ", ")); + + const auto * column_source = column_node->getColumnSource().get(); + if (!column_source) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No source for column {} in JOIN {}", input_name, join_node.formatASTForErrorMessage()); + + bool is_column_from_left_expr = left_table_expressions.contains(column_source); + bool is_column_from_right_expr = right_table_expressions.contains(column_source); + + if (!is_column_from_left_expr && !is_column_from_right_expr) throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns", join_node.formatASTForErrorMessage(), - input_name, - boost::join(left_table_expression_columns_names, ", "), - boost::join(right_table_expression_columns_names, ", ")); + column_source->formatASTForErrorMessage(), + join_node.getLeftTableExpression()->formatASTForErrorMessage(), + join_node.getRightTableExpression()->formatASTForErrorMessage()); - auto input_table_side = left_table_expression_contains_input ? JoinTableSide::Left : JoinTableSide::Right; + auto input_table_side = is_column_from_left_expr ? JoinTableSide::Left : JoinTableSide::Right; if (table_side && (*table_side) != input_table_side) throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "JOIN {} join expression contains column from left and right table", @@ -159,29 +215,58 @@ std::optional extractJoinTableSideFromExpression(const ActionsDAG return table_side; } -void buildJoinClause(ActionsDAGPtr join_expression_dag, - const std::unordered_set & join_expression_dag_input_nodes, - const ActionsDAG::Node * join_expressions_actions_node, - const NameSet & left_table_expression_columns_names, - const NameSet & right_table_expression_columns_names, +const ActionsDAG::Node * appendExpression( + ActionsDAGPtr & dag, + const QueryTreeNodePtr & expression, + const PlannerContextPtr & planner_context, + const JoinNode & join_node) +{ + PlannerActionsVisitor join_expression_visitor(planner_context); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression); + if (join_expression_dag_node_raw_pointers.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} ON clause contains multiple expressions", + join_node.formatASTForErrorMessage()); + + return join_expression_dag_node_raw_pointers[0]; +} + +void buildJoinClause( + ActionsDAGPtr & left_dag, + ActionsDAGPtr & right_dag, + const PlannerContextPtr & planner_context, + //ActionsDAGPtr join_expression_dag, + //const std::unordered_set & join_expression_dag_input_nodes, + //const ActionsDAG::Node * join_expressions_actions_node, + const QueryTreeNodePtr & join_expression, + const TableExpressionSet & left_table_expressions, + const TableExpressionSet & right_table_expressions, const JoinNode & join_node, JoinClause & join_clause) { std::string function_name; - if (join_expressions_actions_node->function) - function_name = join_expressions_actions_node->function->getName(); + //std::cerr << join_expression_dag->dumpDAG() << std::endl; + auto * function_node = join_expression->as(); + if (function_node) + function_name = function_node->getFunction()->getName(); + + // if (join_expressions_actions_node->function) + // function_name = join_expressions_actions_node->function->getName(); /// For 'and' function go into children if (function_name == "and") { - for (const auto & child : join_expressions_actions_node->children) + for (const auto & child : function_node->getArguments()) { - buildJoinClause(join_expression_dag, - join_expression_dag_input_nodes, + buildJoinClause(//join_expression_dag, + //join_expression_dag_input_nodes, + left_dag, + right_dag, + planner_context, child, - left_table_expression_columns_names, - right_table_expression_columns_names, + left_table_expressions, + right_table_expressions, join_node, join_clause); } @@ -194,45 +279,49 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, if (function_name == "equals" || function_name == "isNotDistinctFrom" || is_asof_join_inequality) { - const auto * left_child = join_expressions_actions_node->children.at(0); - const auto * right_child = join_expressions_actions_node->children.at(1); + const auto left_child = function_node->getArguments().getNodes().at(0);//join_expressions_actions_node->children.at(0); + const auto right_child = function_node->getArguments().getNodes().at(1); //join_expressions_actions_node->children.at(1); - auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child, - join_expression_dag_input_nodes, - left_table_expression_columns_names, - right_table_expression_columns_names, + auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child.get(), + //join_expression_dag_input_nodes, + left_table_expressions, + right_table_expressions, join_node); - auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child, - join_expression_dag_input_nodes, - left_table_expression_columns_names, - right_table_expression_columns_names, + auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child.get(), + //join_expression_dag_input_nodes, + left_table_expressions, + right_table_expressions, join_node); if (!left_expression_side_optional && !right_expression_side_optional) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} ON expression {} with constants is not supported", - join_node.formatASTForErrorMessage(), - join_expressions_actions_node->result_name); + "JOIN {} ON expression with constants is not supported", + join_node.formatASTForErrorMessage()); } else if (left_expression_side_optional && !right_expression_side_optional) { - join_clause.addCondition(*left_expression_side_optional, join_expressions_actions_node); + auto & dag = *left_expression_side_optional == JoinTableSide::Left ? left_dag : right_dag; + const auto * node = appendExpression(dag, join_expression, planner_context, join_node); + join_clause.addCondition(*left_expression_side_optional, node); } else if (!left_expression_side_optional && right_expression_side_optional) { - join_clause.addCondition(*right_expression_side_optional, join_expressions_actions_node); + auto & dag = *right_expression_side_optional == JoinTableSide::Left ? left_dag : right_dag; + const auto * node = appendExpression(dag, join_expression, planner_context, join_node); + join_clause.addCondition(*right_expression_side_optional, node); } else { + // std::cerr << "===============\n"; auto left_expression_side = *left_expression_side_optional; auto right_expression_side = *right_expression_side_optional; if (left_expression_side != right_expression_side) { - const ActionsDAG::Node * left_key = left_child; - const ActionsDAG::Node * right_key = right_child; + auto left_key = left_child; + auto right_key = right_child; if (left_expression_side == JoinTableSide::Right) { @@ -241,6 +330,9 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, asof_inequality = reverseASOFJoinInequality(asof_inequality); } + const auto * left_node = appendExpression(left_dag, left_key, planner_context, join_node); + const auto * right_node = appendExpression(right_dag, right_key, planner_context, join_node); + if (is_asof_join_inequality) { if (join_clause.hasASOF()) @@ -250,55 +342,66 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, join_node.formatASTForErrorMessage()); } - join_clause.addASOFKey(left_key, right_key, asof_inequality); + join_clause.addASOFKey(left_node, right_node, asof_inequality); } else { bool null_safe_comparison = function_name == "isNotDistinctFrom"; - join_clause.addKey(left_key, right_key, null_safe_comparison); + join_clause.addKey(left_node, right_node, null_safe_comparison); } } else { - join_clause.addCondition(left_expression_side, join_expressions_actions_node); + auto & dag = left_expression_side == JoinTableSide::Left ? left_dag : right_dag; + const auto * node = appendExpression(dag, join_expression, planner_context, join_node); + join_clause.addCondition(left_expression_side, node); } } return; } - auto expression_side_optional = extractJoinTableSideFromExpression(join_expressions_actions_node, - join_expression_dag_input_nodes, - left_table_expression_columns_names, - right_table_expression_columns_names, + auto expression_side_optional = extractJoinTableSideFromExpression(//join_expressions_actions_node, + //join_expression_dag_input_nodes, + join_expression.get(), + left_table_expressions, + right_table_expressions, join_node); if (!expression_side_optional) expression_side_optional = JoinTableSide::Right; auto expression_side = *expression_side_optional; - join_clause.addCondition(expression_side, join_expressions_actions_node); + auto & dag = expression_side == JoinTableSide::Left ? left_dag : right_dag; + const auto * node = appendExpression(dag, join_expression, planner_context, join_node); + join_clause.addCondition(expression_side, node); } -JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & join_expression_input_columns, +JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName & join_expression_input_columns, const ColumnsWithTypeAndName & left_table_expression_columns, const ColumnsWithTypeAndName & right_table_expression_columns, const JoinNode & join_node, const PlannerContextPtr & planner_context) { - ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); + //ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); + + ActionsDAGPtr left_join_actions = std::make_shared(left_table_expression_columns); + ActionsDAGPtr right_join_actions = std::make_shared(right_table_expression_columns); + + // LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions cols {} ", left_join_actions->dumpDAG()); + // LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions cols {} ", right_join_actions->dumpDAG()); /** In ActionsDAG if input node has constant representation additional constant column is added. * That way we cannot simply check that node has INPUT type during resolution of expression join table side. * Put all nodes after actions dag initialization in set. * To check if actions dag node is input column, we check if set contains it. */ - const auto & join_expression_actions_nodes = join_expression_actions->getNodes(); + // const auto & join_expression_actions_nodes = join_expression_actions->getNodes(); - std::unordered_set join_expression_dag_input_nodes; - join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size()); - for (const auto & node : join_expression_actions_nodes) - join_expression_dag_input_nodes.insert(&node); + // std::unordered_set join_expression_dag_input_nodes; + // join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size()); + // for (const auto & node : join_expression_actions_nodes) + // join_expression_dag_input_nodes.insert(&node); /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. @@ -308,6 +411,9 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & * ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value); */ auto join_expression = join_node.getJoinExpression(); + // LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->formatConvertedASTForErrorMessage()); + // LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->dumpTree()); + auto * constant_join_expression = join_expression->as(); if (constant_join_expression && constant_join_expression->hasSourceExpression()) @@ -319,18 +425,18 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & "JOIN {} join expression expected function", join_node.formatASTForErrorMessage()); - PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression); - if (join_expression_dag_node_raw_pointers.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} ON clause contains multiple expressions", - join_node.formatASTForErrorMessage()); + // PlannerActionsVisitor join_expression_visitor(planner_context); + // auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression); + // if (join_expression_dag_node_raw_pointers.size() != 1) + // throw Exception(ErrorCodes::LOGICAL_ERROR, + // "JOIN {} ON clause contains multiple expressions", + // join_node.formatASTForErrorMessage()); - const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; - if (!join_expressions_actions_root_node->function) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} join expression expected function", - join_node.formatASTForErrorMessage()); + // const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; + // if (!join_expressions_actions_root_node->function) + // throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + // "JOIN {} join expression expected function", + // join_node.formatASTForErrorMessage()); size_t left_table_expression_columns_size = left_table_expression_columns.size(); @@ -360,21 +466,27 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & join_right_actions_names_set.insert(right_table_expression_column.name); } - JoinClausesAndActions result; - result.join_expression_actions = join_expression_actions; + auto join_left_table_expressions = extractTableExpressionsSet(join_node.getLeftTableExpression()); + auto join_right_table_expressions = extractTableExpressionsSet(join_node.getRightTableExpression()); - const auto & function_name = join_expressions_actions_root_node->function->getName(); + JoinClausesAndActions result; + //result.join_expression_actions = join_expression_actions; + + const auto & function_name = function_node->getFunction()->getName(); if (function_name == "or") { - for (const auto & child : join_expressions_actions_root_node->children) + for (const auto & child : function_node->getArguments()) { result.join_clauses.emplace_back(); - buildJoinClause(join_expression_actions, - join_expression_dag_input_nodes, + buildJoinClause(//join_expression_actions, + //join_expression_dag_input_nodes, + left_join_actions, + right_join_actions, + planner_context, child, - join_left_actions_names_set, - join_right_actions_names_set, + join_left_table_expressions, + join_right_table_expressions, join_node, result.join_clauses.back()); } @@ -383,11 +495,15 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & { result.join_clauses.emplace_back(); - buildJoinClause(join_expression_actions, - join_expression_dag_input_nodes, - join_expressions_actions_root_node, - join_left_actions_names_set, - join_right_actions_names_set, + buildJoinClause( + left_join_actions, + right_join_actions, + planner_context, + //join_expression_actions, + //join_expression_dag_input_nodes, + join_expression, //join_expressions_actions_root_node, + join_left_table_expressions, + join_right_table_expressions, join_node, result.join_clauses.back()); } @@ -412,12 +528,12 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & const ActionsDAG::Node * dag_filter_condition_node = nullptr; if (left_filter_condition_nodes.size() > 1) - dag_filter_condition_node = &join_expression_actions->addFunction(and_function, left_filter_condition_nodes, {}); + dag_filter_condition_node = &left_join_actions->addFunction(and_function, left_filter_condition_nodes, {}); else dag_filter_condition_node = left_filter_condition_nodes[0]; join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node}; - join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + left_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node); add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); } @@ -428,12 +544,12 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & const ActionsDAG::Node * dag_filter_condition_node = nullptr; if (right_filter_condition_nodes.size() > 1) - dag_filter_condition_node = &join_expression_actions->addFunction(and_function, right_filter_condition_nodes, {}); + dag_filter_condition_node = &right_join_actions->addFunction(and_function, right_filter_condition_nodes, {}); else dag_filter_condition_node = right_filter_condition_nodes[0]; join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node}; - join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + right_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node); add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); } @@ -470,10 +586,10 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & } if (!left_key_node->result_type->equals(*common_type)) - left_key_node = &join_expression_actions->addCast(*left_key_node, common_type, {}); + left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {}); if (!right_key_node->result_type->equals(*common_type)) - right_key_node = &join_expression_actions->addCast(*right_key_node, common_type, {}); + right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {}); } if (join_clause.isNullsafeCompareKey(i) && left_key_node->result_type->isNullable() && right_key_node->result_type->isNullable()) @@ -490,22 +606,29 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & * SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b) */ auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext()); - left_key_node = &join_expression_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {}); - right_key_node = &join_expression_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {}); + left_key_node = &left_join_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {}); + right_key_node = &right_join_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {}); } - join_expression_actions->addOrReplaceInOutputs(*left_key_node); - join_expression_actions->addOrReplaceInOutputs(*right_key_node); + left_join_actions->addOrReplaceInOutputs(*left_key_node); + right_join_actions->addOrReplaceInOutputs(*right_key_node); add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name); } } - result.left_join_expressions_actions = join_expression_actions->clone(); + result.left_join_expressions_actions = left_join_actions->clone(); + result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = join_expression_actions->clone(); + // for (const auto & name : join_right_actions_names) + // std::cerr << ".. " << name << std::endl; + + // std::cerr << right_join_actions->dumpDAG() << std::endl; + + result.right_join_expressions_actions = right_join_actions->clone(); + result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); return result; @@ -525,10 +648,10 @@ JoinClausesAndActions buildJoinClausesAndActions( "JOIN {} join does not have ON section", join_node_typed.formatASTForErrorMessage()); - auto join_expression_input_columns = left_table_expression_columns; - join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end()); + // auto join_expression_input_columns = left_table_expression_columns; + // join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end()); - return buildJoinClausesAndActions(join_expression_input_columns, left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context); + return buildJoinClausesAndActions(/*join_expression_input_columns,*/ left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context); } std::optional tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join_node) diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 94f32e7ad51..7bc65cfb544 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -165,7 +165,8 @@ struct JoinClausesAndActions /// Join clauses. Actions dag nodes point into join_expression_actions. JoinClauses join_clauses; /// Whole JOIN ON section expressions - ActionsDAGPtr join_expression_actions; + ActionsDAGPtr left_join_tmp_expression_actions; + ActionsDAGPtr right_join_tmp_expression_actions; /// Left join expressions actions ActionsDAGPtr left_join_expressions_actions; /// Right join expressions actions diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 9a6ef6f5d83..ba29cab5956 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -357,6 +357,7 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node, const ContextPtr & context, + //PlannerContext & planner_context, ResultReplacementMap * result_replacement_map) { auto & query_node_typed = query_node->as(); @@ -406,6 +407,13 @@ QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & if (result_replacement_map) result_replacement_map->emplace(table_expression, dummy_table_node); + dummy_table_node->setAlias(table_expression->getAlias()); + + // auto & src_table_expression_data = planner_context.getOrCreateTableExpressionData(table_expression); + // auto & dst_table_expression_data = planner_context.getOrCreateTableExpressionData(dummy_table_node); + + // dst_table_expression_data = src_table_expression_data; + replacement_map.emplace(table_expression.get(), std::move(dummy_table_node)); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index c5e42e76653..d1f0c1ebe5e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -436,7 +436,6 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( AggregateProjectionCandidates candidates; const auto & parts = reading.getParts(); - const auto & query_info = reading.getQueryInfo(); const auto metadata = reading.getStorageMetadata(); ContextPtr context = reading.getContext(); @@ -481,8 +480,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, candidate.dag->getRequiredColumnsNames(), - dag.filter_node != nullptr, - query_info, + (dag.filter_node ? dag.dag : nullptr), parts, max_added_blocks.get(), context); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6f0429459cd..68786bdec6c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -23,6 +23,8 @@ #include #include #include +#include +#include #include #include #include @@ -1337,26 +1339,12 @@ static void buildIndexes( const Names & primary_key_column_names = primary_key.column_names; const auto & settings = context->getSettingsRef(); - if (settings.query_plan_optimize_primary_key) - { - NameSet array_join_name_set; - if (query_info.syntax_analyzer_result) - array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - indexes.emplace(ReadFromMergeTree::Indexes{{ - filter_actions_dag, - context, - primary_key_column_names, - primary_key.expression}, {}, {}, {}, {}, false, {}}); - } - else - { - indexes.emplace(ReadFromMergeTree::Indexes{{ - query_info, - context, - primary_key_column_names, - primary_key.expression}, {}, {}, {}, {}, false, {}}); - } + indexes.emplace(ReadFromMergeTree::Indexes{{ + filter_actions_dag, + context, + primary_key_column_names, + primary_key.expression}, {}, {}, {}, {}, false, {}}); if (metadata_snapshot->hasPartitionKey()) { @@ -1369,11 +1357,7 @@ static void buildIndexes( } /// TODO Support row_policy_filter and additional_filters - if (settings.allow_experimental_analyzer) - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); - else - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); - + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(indexes->part_offset_condition, filter_actions_dag, context); indexes->use_skip_indexes = settings.use_skip_indexes; @@ -1385,14 +1369,18 @@ static void buildIndexes( if (!indexes->use_skip_indexes) return; - const SelectQueryInfo * info = &query_info; std::optional info_copy; - if (settings.allow_experimental_analyzer) + auto get_query_info = [&]() -> const SelectQueryInfo & { - info_copy.emplace(query_info); - info_copy->filter_actions_dag = filter_actions_dag; - info = &*info_copy; - } + if (settings.allow_experimental_analyzer) + { + info_copy.emplace(query_info); + info_copy->filter_actions_dag = filter_actions_dag; + return *info_copy; + } + + return query_info; + }; std::unordered_set ignored_index_names; @@ -1433,14 +1421,30 @@ static void buildIndexes( if (inserted) { skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(get_query_info(), metadata_snapshot); } skip_indexes.merged_indices[it->second].addIndex(index_helper); } else { - auto condition = index_helper->createIndexCondition(*info, context); + MergeTreeIndexConditionPtr condition; + if (index_helper->isVectorSearch()) + { +#ifdef ENABLE_ANNOY + if (const auto * annoy = typeid_cast(index_helper.get())) + condition = annoy->createIndexCondition(get_query_info(), context); +#endif +#ifdef ENABLE_USEARCH + if (const auto * usearch = typeid_cast(index_helper.get())) + condition = usearch->createIndexCondition(get_query_info(), context); +#endif + if (!condition) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); + } + else + condition = index_helper->createIndexCondition(filter_actions_dag, context); + if (!condition->alwaysUnknownOrTrue()) skip_indexes.useful_indices.emplace_back(index_helper, condition); } @@ -1473,34 +1477,15 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( Poco::Logger * log, std::optional & indexes) { - const auto & settings = context->getSettingsRef(); - if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key) - { - auto updated_query_info_with_filter_dag = query_info; - updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info); - - return selectRangesToReadImpl( - std::move(parts), - std::move(alter_conversions), - metadata_snapshot_base, - metadata_snapshot, - updated_query_info_with_filter_dag, - context, - num_streams, - max_block_numbers_to_read, - data, - real_column_names, - sample_factor_column_queried, - log, - indexes); - } + auto updated_query_info_with_filter_dag = query_info; + updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info); return selectRangesToReadImpl( std::move(parts), std::move(alter_conversions), metadata_snapshot_base, metadata_snapshot, - query_info, + updated_query_info_with_filter_dag, context, num_streams, max_block_numbers_to_read, diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index 798073f94d3..e7b170f0f91 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -30,19 +30,9 @@ void ReadFromStorageStep::applyFilters() if (!context) return; - std::shared_ptr key_condition; - if (!context->getSettingsRef().allow_experimental_analyzer) - { - for (const auto & processor : pipe.getProcessors()) - if (auto * source = dynamic_cast(processor.get())) - source->setKeyCondition(query_info, context); - } - else - { - for (const auto & processor : pipe.getProcessors()) - if (auto * source = dynamic_cast(processor.get())) - source->setKeyCondition(filter_nodes.nodes, context); - } + for (const auto & processor : pipe.getProcessors()) + if (auto * source = dynamic_cast(processor.get())) + source->setKeyCondition(filter_nodes.nodes, context); } } diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h index c9617d3e73e..82d46eb74a4 100644 --- a/src/Processors/SourceWithKeyCondition.h +++ b/src/Processors/SourceWithKeyCondition.h @@ -16,15 +16,6 @@ protected: /// Represents pushed down filters in source std::shared_ptr key_condition; - void setKeyConditionImpl(const SelectQueryInfo & query_info, ContextPtr context, const Block & keys) - { - key_condition = std::make_shared( - query_info, - context, - keys.getNames(), - std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName()))); - } - void setKeyConditionImpl(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context, const Block & keys) { std::unordered_map node_name_to_input_column; @@ -46,10 +37,7 @@ public: /// Set key_condition directly. It is used for filter push down in source. virtual void setKeyCondition(const std::shared_ptr & key_condition_) { key_condition = key_condition_; } - /// Set key_condition created by query_info and context. It is used for filter push down when allow_experimental_analyzer is false. - virtual void setKeyCondition(const SelectQueryInfo & /*query_info*/, ContextPtr /*context*/) { } - - /// Set key_condition created by nodes and context. It is used for filter push down when allow_experimental_analyzer is true. + /// Set key_condition created by nodes and context. virtual void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & /*nodes*/, ContextPtr /*context*/) { } }; } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index f03136e4edf..0c4e4f956a0 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -29,10 +29,14 @@ #include #include #include +#include #include #include #include #include +#include +#include +#include #include #include #include @@ -123,7 +127,6 @@ public: String compression_method_, Block sample_block_, ContextPtr context_, - const SelectQueryInfo & query_info_, UInt64 max_block_size_, const StorageHive & storage_, const Names & text_input_field_names_ = {}) @@ -140,7 +143,6 @@ public: , text_input_field_names(text_input_field_names_) , format_settings(getFormatSettings(getContext())) , read_settings(getContext()->getReadSettings()) - , query_info(query_info_) { to_read_block = sample_block; @@ -395,7 +397,6 @@ private: const Names & text_input_field_names; FormatSettings format_settings; ReadSettings read_settings; - SelectQueryInfo query_info; HiveFilePtr current_file; String current_path; @@ -574,7 +575,7 @@ static HiveFilePtr createHiveFile( HiveFiles StorageHive::collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -638,7 +639,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( for (size_t i = 0; i < partition_names.size(); ++i) ranges.emplace_back(fields[i]); - const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); + const KeyCondition partition_key_condition(filter_actions_dag, getContext(), partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) return {}; } @@ -648,7 +649,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( hive_files.reserve(file_infos.size()); for (const auto & file_info : file_infos) { - auto hive_file = getHiveFileIfNeeded(file_info, fields, query_info, hive_table_metadata, context_, prune_level); + auto hive_file = getHiveFileIfNeeded(file_info, fields, filter_actions_dag, hive_table_metadata, context_, prune_level); if (hive_file) { LOG_TRACE( @@ -672,7 +673,7 @@ StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hiv HiveFilePtr StorageHive::getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const ContextPtr & context_, PruneLevel prune_level) const @@ -706,7 +707,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( if (prune_level >= PruneLevel::File) { - const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + const KeyCondition hivefile_key_condition(filter_actions_dag, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { /// Load file level minmax index and apply @@ -758,10 +759,77 @@ bool StorageHive::supportsSubsetOfColumns() const return format_name == "Parquet" || format_name == "ORC"; } -Pipe StorageHive::read( +class ReadFromHive : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromHive"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters() override; + + ReadFromHive( + Block header, + std::shared_ptr storage_, + std::shared_ptr sources_info_, + HDFSBuilderWrapper builder_, + HDFSFSPtr fs_, + HiveMetastoreClient::HiveTableMetadataPtr hive_table_metadata_, + Block sample_block_, + Poco::Logger * log_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = std::move(header)}) + , storage(std::move(storage_)) + , sources_info(std::move(sources_info_)) + , builder(std::move(builder_)) + , fs(std::move(fs_)) + , hive_table_metadata(std::move(hive_table_metadata_)) + , sample_block(std::move(sample_block_)) + , log(log_) + , context(std::move(context_)) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + { + } + +private: + std::shared_ptr storage; + std::shared_ptr sources_info; + HDFSBuilderWrapper builder; + HDFSFSPtr fs; + HiveMetastoreClient::HiveTableMetadataPtr hive_table_metadata; + Block sample_block; + Poco::Logger * log; + + ContextPtr context; + size_t max_block_size; + size_t num_streams; + + std::optional hive_files; + + void createFiles(const ActionsDAGPtr & filter_actions_dag); +}; + +void ReadFromHive::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + createFiles(filter_actions_dag); +} + +void ReadFromHive::createFiles(const ActionsDAGPtr & filter_actions_dag) +{ + if (hive_files) + return; + + hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag, hive_table_metadata, fs, context); + LOG_INFO(log, "Collect {} hive files to read", hive_files->size()); +} + +void StorageHive::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, + SelectQueryInfo &, ContextPtr context_, QueryProcessingStage::Enum /* processed_stage */, size_t max_block_size, @@ -774,15 +842,7 @@ Pipe StorageHive::read( auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); - /// Collect Hive files to read - HiveFiles hive_files = collectHiveFiles(num_streams, query_info, hive_table_metadata, fs, context_); - LOG_INFO(log, "Collect {} hive files to read", hive_files.size()); - - if (hive_files.empty()) - return {}; - auto sources_info = std::make_shared(); - sources_info->hive_files = std::move(hive_files); sources_info->database_name = hive_database; sources_info->table_name = hive_table; sources_info->hive_metastore_client = hive_metastore_client; @@ -822,6 +882,36 @@ Pipe StorageHive::read( sources_info->need_file_column = true; } + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + StorageHiveSource::getHeader(sample_block, sources_info), + std::move(this_ptr), + std::move(sources_info), + std::move(builder), + std::move(fs), + std::move(hive_table_metadata), + std::move(sample_block), + log, + context_, + max_block_size, + num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromHive::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + createFiles(nullptr); + + if (hive_files->empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + sources_info->hive_files = std::move(*hive_files); + if (num_streams > sources_info->hive_files.size()) num_streams = sources_info->hive_files.size(); @@ -830,22 +920,29 @@ Pipe StorageHive::read( { pipes.emplace_back(std::make_shared( sources_info, - hdfs_namenode_url, - format_name, - compression_method, + storage->hdfs_namenode_url, + storage->format_name, + storage->compression_method, sample_block, - context_, - query_info, + context, max_block_size, - *this, - text_input_field_names)); + *storage, + storage->text_input_field_names)); } - return Pipe::unitePipes(std::move(pipes)); + + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(getOutputStream().header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); } HiveFiles StorageHive::collectHiveFiles( size_t max_threads, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -871,7 +968,7 @@ HiveFiles StorageHive::collectHiveFiles( [&]() { auto hive_files_in_partition - = collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_, prune_level); + = collectHiveFilesFromPartition(partition, filter_actions_dag, hive_table_metadata, fs, context_, prune_level); if (!hive_files_in_partition.empty()) { std::lock_guard lock(hive_files_mutex); @@ -897,7 +994,7 @@ HiveFiles StorageHive::collectHiveFiles( pool.scheduleOrThrowOnError( [&]() { - auto hive_file = getHiveFileIfNeeded(file_info, {}, query_info, hive_table_metadata, context_, prune_level); + auto hive_file = getHiveFileIfNeeded(file_info, {}, filter_actions_dag, hive_table_metadata, context_, prune_level); if (hive_file) { std::lock_guard lock(hive_files_mutex); @@ -925,13 +1022,12 @@ NamesAndTypesList StorageHive::getVirtuals() const std::optional StorageHive::totalRows(const Settings & settings) const { /// query_info is not used when prune_level == PruneLevel::None - SelectQueryInfo query_info; - return totalRowsImpl(settings, query_info, getContext(), PruneLevel::None); + return totalRowsImpl(settings, nullptr, getContext(), PruneLevel::None); } -std::optional StorageHive::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const +std::optional StorageHive::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const { - return totalRowsImpl(context_->getSettingsRef(), query_info, context_, PruneLevel::Partition); + return totalRowsImpl(context_->getSettingsRef(), filter_actions_dag, context_, PruneLevel::Partition); } void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const @@ -946,7 +1042,7 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt } std::optional -StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const +StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const { /// Row-based format like Text doesn't support totalRowsByPartitionPredicate if (!supportsSubsetOfColumns()) @@ -958,7 +1054,7 @@ StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & qu HDFSFSPtr fs = createHDFSFS(builder.get()); HiveFiles hive_files = collectHiveFiles( settings.max_threads, - query_info, + filter_actions_dag, hive_table_metadata, fs, context_, diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 8b378bf9e54..b0ec96604cc 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -42,10 +42,11 @@ public: bool supportsSubcolumns() const override { return true; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, + SelectQueryInfo &, ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, @@ -58,9 +59,12 @@ public: bool supportsSubsetOfColumns() const; std::optional totalRows(const Settings & settings) const override; - std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const override; void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; +protected: + friend class ReadFromHive; + private: using FileFormat = IHiveFile::FileFormat; using FileInfo = HiveMetastoreClient::FileInfo; @@ -88,7 +92,7 @@ private: HiveFiles collectHiveFiles( size_t max_threads, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -96,7 +100,7 @@ private: HiveFiles collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -105,7 +109,7 @@ private: HiveFilePtr getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; @@ -113,7 +117,7 @@ private: void lazyInitialize(); std::optional - totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const; + totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const; String hive_metastore_url; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 1102c77ca58..4fa6bfdd617 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -669,7 +669,7 @@ public: virtual std::optional totalRows(const Settings &) const { return {}; } /// Same as above but also take partition predicate into account. - virtual std::optional totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const { return {}; } + virtual std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr &, ContextPtr) const { return {}; } /// If it is possible to quickly determine exact number of bytes for the table on storage: /// - memory (approximated, resident) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 1cc672fb98f..d5922ae1bc2 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -762,92 +762,6 @@ void KeyCondition::getAllSpaceFillingCurves() } } -KeyCondition::KeyCondition( - const ASTPtr & query, - const ASTs & additional_filter_asts, - Block block_with_constants, - PreparedSetsPtr prepared_sets, - ContextPtr context, - const Names & key_column_names, - const ExpressionActionsPtr & key_expr_, - NameSet array_joined_column_names_, - bool single_point_, - bool strict_) - : key_expr(key_expr_) - , key_subexpr_names(getAllSubexpressionNames(*key_expr)) - , array_joined_column_names(std::move(array_joined_column_names_)) - , single_point(single_point_) - , strict(strict_) -{ - size_t key_index = 0; - for (const auto & name : key_column_names) - { - if (!key_columns.contains(name)) - { - key_columns[name] = key_columns.size(); - key_indices.push_back(key_index); - } - ++key_index; - } - - if (context->getSettingsRef().analyze_index_with_space_filling_curves) - getAllSpaceFillingCurves(); - - ASTPtr filter_node; - if (query) - filter_node = buildFilterNode(query, additional_filter_asts); - - if (!filter_node) - { - has_filter = false; - rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); - return; - } - - has_filter = true; - - /** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)), - * the use of NOT operator in predicate will result in the indexing algorithm leave out some data. - * This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict - * when parsing the AST into internal RPN representation. - * To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's - * are pushed down and applied (when possible) to leaf nodes. - */ - auto inverted_filter_node = DB::cloneASTWithInversionPushDown(filter_node); - - RPNBuilder builder( - inverted_filter_node, - std::move(context), - std::move(block_with_constants), - std::move(prepared_sets), - [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); - - rpn = std::move(builder).extractRPN(); - - findHyperrectanglesForArgumentsOfSpaceFillingCurves(); -} - -KeyCondition::KeyCondition( - const SelectQueryInfo & query_info, - ContextPtr context, - const Names & key_column_names, - const ExpressionActionsPtr & key_expr_, - bool single_point_, - bool strict_) - : KeyCondition( - query_info.query, - query_info.filter_asts, - KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context), - query_info.prepared_sets, - context, - key_column_names, - key_expr_, - query_info.syntax_analyzer_result ? query_info.syntax_analyzer_result->getArrayJoinSourceNameSet() : NameSet{}, - single_point_, - strict_) -{ -} - KeyCondition::KeyCondition( ActionsDAGPtr filter_dag, ContextPtr context, @@ -883,6 +797,13 @@ KeyCondition::KeyCondition( has_filter = true; + /** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)), + * the use of NOT operator in predicate will result in the indexing algorithm leave out some data. + * This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict + * when parsing the AST into internal RPN representation. + * To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's + * are pushed down and applied (when possible) to leaf nodes. + */ auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context); assert(inverted_dag->getOutputs().size() == 1); diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 980c248835d..6e248dd664a 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -39,30 +39,6 @@ struct ActionDAGNodes; class KeyCondition { public: - /// Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters - KeyCondition( - const ASTPtr & query, - const ASTs & additional_filter_asts, - Block block_with_constants, - PreparedSetsPtr prepared_sets_, - ContextPtr context, - const Names & key_column_names, - const ExpressionActionsPtr & key_expr, - NameSet array_joined_column_names, - bool single_point_ = false, - bool strict_ = false); - - /** Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters. - * Select query, additional filters, prepared sets are initialized using query info. - */ - KeyCondition( - const SelectQueryInfo & query_info, - ContextPtr context, - const Names & key_column_names, - const ExpressionActionsPtr & key_expr_, - bool single_point_ = false, - bool strict_ = false); - /// Construct key condition from ActionsDAG nodes KeyCondition( ActionsDAGPtr filter_dag, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a23d59055ca..4ad6c564a18 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1075,26 +1075,30 @@ Block MergeTreeData::getBlockWithVirtualPartColumns(const MergeTreeData::DataPar std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( - const SelectQueryInfo & query_info, ContextPtr local_context, const DataPartsVector & parts) const + const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const { if (parts.empty()) return 0u; auto metadata_snapshot = getInMemoryMetadataPtr(); - ASTPtr expression_ast; Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, true /* one_part */); - // Generate valid expressions for filtering - bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, local_context, virtual_columns_block, expression_ast); + auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr); - PartitionPruner partition_pruner(metadata_snapshot, query_info, local_context, true /* strict */); + // Generate valid expressions for filtering + bool valid = true; + for (const auto * input : filter_dag->getInputs()) + if (!virtual_columns_block.has(input->result_name)) + valid = false; + + PartitionPruner partition_pruner(metadata_snapshot, filter_dag, local_context, true /* strict */); if (partition_pruner.isUseless() && !valid) return {}; std::unordered_set part_values; - if (valid && expression_ast) + if (valid) { virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */); - VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, local_context, expression_ast); + VirtualColumnUtils::filterBlockWithDAG(filter_dag, virtual_columns_block, local_context); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return 0; @@ -4002,9 +4006,13 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW /// We don't need to commit it to zk, and don't even need to activate it. MergeTreePartInfo empty_info = drop_range; - empty_info.min_block = empty_info.level = empty_info.mutation = 0; + empty_info.level = empty_info.mutation = 0; + empty_info.min_block = MergeTreePartInfo::MAX_BLOCK_NUMBER; for (const auto & part : parts_to_remove) { + /// We still have to take min_block into account to avoid creating multiple covering ranges + /// that intersect each other + empty_info.min_block = std::min(empty_info.min_block, part->info.min_block); empty_info.level = std::max(empty_info.level, part->info.level); empty_info.mutation = std::max(empty_info.mutation, part->info.mutation); } @@ -6621,8 +6629,7 @@ using PartitionIdToMaxBlock = std::unordered_map; Block MergeTreeData::getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, - bool has_filter, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_dag, const DataPartsVector & parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context) const @@ -6672,7 +6679,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( Block virtual_columns_block; auto virtual_block = getSampleBlockWithVirtualColumns(); bool has_virtual_column = std::any_of(required_columns.begin(), required_columns.end(), [&](const auto & name) { return virtual_block.has(name); }); - if (has_virtual_column || has_filter) + if (has_virtual_column || filter_dag) { virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */, true /* ignore_empty */); if (virtual_columns_block.rows() == 0) @@ -6684,7 +6691,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( std::optional partition_pruner; std::optional minmax_idx_condition; DataTypes minmax_columns_types; - if (has_filter) + if (filter_dag) { if (metadata_snapshot->hasPartitionKey()) { @@ -6693,16 +6700,15 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( minmax_columns_types = getMinMaxColumnsTypes(partition_key); minmax_idx_condition.emplace( - query_info, query_context, minmax_columns_names, + filter_dag, query_context, minmax_columns_names, getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(query_context))); - partition_pruner.emplace(metadata_snapshot, query_info, query_context, false /* strict */); + partition_pruner.emplace(metadata_snapshot, filter_dag, query_context, false /* strict */); } + const auto * predicate = filter_dag->getOutputs().at(0); + // Generate valid expressions for filtering - ASTPtr expression_ast; - VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, query_context, virtual_columns_block, expression_ast); - if (expression_ast) - VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, query_context, expression_ast); + VirtualColumnUtils::filterBlockWithPredicate(predicate, virtual_columns_block, query_context); rows = virtual_columns_block.rows(); part_name_column = virtual_columns_block.getByName("_part").column; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index dfa13eca11d..f0dbaf0e307 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -404,8 +404,7 @@ public: Block getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, - bool has_filter, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_dag, const DataPartsVector & parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context) const; @@ -1222,7 +1221,7 @@ protected: boost::iterator_range range, const ColumnsDescription & storage_columns); std::optional totalRowsByPartitionPredicateImpl( - const SelectQueryInfo & query_info, ContextPtr context, const DataPartsVector & parts) const; + const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const; static decltype(auto) getStateModifier(DataPartState state) { diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 7b30622a4fc..d5b9b4423a9 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -784,7 +784,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( = {ColumnWithTypeAndName(part_offset_type->createColumn(), part_offset_type, "_part_offset"), ColumnWithTypeAndName(part_type->createColumn(), part_type, "_part")}; - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), sample); + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), &sample); if (!dag) return; @@ -810,7 +810,7 @@ std::optional> MergeTreeDataSelectExecutor::filterPar if (!filter_dag) return {}; auto sample = data.getSampleBlockWithVirtualColumns(); - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), sample); + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), &sample); if (!dag) return {}; @@ -819,34 +819,6 @@ std::optional> MergeTreeDataSelectExecutor::filterPar return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); } - -std::optional> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns( - const MergeTreeData & data, - const MergeTreeData::DataPartsVector & parts, - const ASTPtr & query, - ContextPtr context) -{ - std::unordered_set part_values; - ASTPtr expression_ast; - auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); - - if (virtual_columns_block.rows() == 0) - return {}; - - // Generate valid expressions for filtering - VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); - - // If there is still something left, fill the virtual block and do the filtering. - if (expression_ast) - { - virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); - VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast); - return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); - } - - return {}; -} - void MergeTreeDataSelectExecutor::filterPartsByPartition( const std::optional & partition_pruner, const std::optional & minmax_idx_condition, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 11c8e172a4f..4c6e1086cbc 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -169,12 +169,6 @@ public: /// If possible, filter using expression on virtual columns. /// Example: SELECT count() FROM table WHERE _part = 'part_name' /// If expression found, return a set with allowed part names (std::nullopt otherwise). - static std::optional> filterPartsByVirtualColumns( - const MergeTreeData & data, - const MergeTreeData::DataPartsVector & parts, - const ASTPtr & query, - ContextPtr context); - static std::optional> filterPartsByVirtualColumns( const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 4411d46e124..e36459b019f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } template @@ -331,6 +332,11 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec return std::make_shared(index, query, distance_function, context); }; +MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG"); +} + MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2; diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index dead12fe66f..d511ab84859 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -88,7 +88,7 @@ private: }; -class MergeTreeIndexAnnoy : public IMergeTreeIndex +class MergeTreeIndexAnnoy final : public IMergeTreeIndex { public: @@ -98,7 +98,9 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; - MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override; + bool isVectorSearch() const override { return true; } private: const UInt64 trees; diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index fa05f9e61e1..dbd33609a00 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -43,9 +43,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(con return std::make_shared(bits_per_row, hash_functions, index.column_names); } -MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const +MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const { - return std::make_shared(query_info, context, index.sample_block, hash_functions); + return std::make_shared(filter_actions_dag, context, index.sample_block, hash_functions); } static void assertIndexColumnsType(const Block & header) diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h index 4d688ae3cfc..d6f4d6f2cf5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h @@ -20,7 +20,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; - MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; private: size_t bits_per_row; diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 398a85e92ac..da49814b83a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -97,39 +97,18 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & } MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter( - const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_) - : WithContext(context_), header(header_), query_info(info_), hash_functions(hash_functions_) + const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) + : WithContext(context_), header(header_), hash_functions(hash_functions_) { - if (context_->getSettingsRef().allow_experimental_analyzer) - { - if (!query_info.filter_actions_dag) - { - rpn.push_back(RPNElement::FUNCTION_UNKNOWN); - return; - } - - RPNBuilder builder( - query_info.filter_actions_dag->getOutputs().at(0), - context_, - [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); - rpn = std::move(builder).extractRPN(); - return; - } - - ASTPtr filter_node = buildFilterNode(query_info.query); - - if (!filter_node) + if (!filter_actions_dag) { rpn.push_back(RPNElement::FUNCTION_UNKNOWN); return; } - auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_); RPNBuilder builder( - filter_node, + filter_actions_dag->getOutputs().at(0), context_, - std::move(block_with_constants), - query_info.prepared_sets, [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); rpn = std::move(builder).extractRPN(); } diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h index 952948fd582..db85c804d8d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h @@ -44,7 +44,7 @@ public: std::vector> predicate; }; - MergeTreeIndexConditionBloomFilter(const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_); + MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); bool alwaysUnknownOrTrue() const override; @@ -58,7 +58,6 @@ public: private: const Block & header; - const SelectQueryInfo & query_info; const size_t hash_functions; std::vector rpn; diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index b740f33593c..4cd616513ac 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -138,7 +138,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, } MergeTreeConditionFullText::MergeTreeConditionFullText( - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & index_sample_block, const BloomFilterParameters & params_, @@ -147,38 +147,16 @@ MergeTreeConditionFullText::MergeTreeConditionFullText( , index_data_types(index_sample_block.getNamesAndTypesList().getTypes()) , params(params_) , token_extractor(token_extactor_) - , prepared_sets(query_info.prepared_sets) { - if (context->getSettingsRef().allow_experimental_analyzer) - { - if (!query_info.filter_actions_dag) - { - rpn.push_back(RPNElement::FUNCTION_UNKNOWN); - return; - } - - RPNBuilder builder( - query_info.filter_actions_dag->getOutputs().at(0), - context, - [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); - rpn = std::move(builder).extractRPN(); - return; - } - - ASTPtr filter_node = buildFilterNode(query_info.query); - - if (!filter_node) + if (!filter_actions_dag) { rpn.push_back(RPNElement::FUNCTION_UNKNOWN); return; } - auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context); RPNBuilder builder( - filter_node, + filter_actions_dag->getOutputs().at(0), context, - std::move(block_with_constants), - query_info.prepared_sets, [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); rpn = std::move(builder).extractRPN(); } @@ -747,9 +725,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator(const } MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const + const ActionsDAGPtr & filter_dag, ContextPtr context) const { - return std::make_shared(query, context, index.sample_block, params, token_extractor.get()); + return std::make_shared(filter_dag, context, index.sample_block, params, token_extractor.get()); } MergeTreeIndexPtr bloomFilterIndexCreator( diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index 5b1cf8c1f1d..e66f498ce1d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -62,7 +62,7 @@ class MergeTreeConditionFullText final : public IMergeTreeIndexCondition { public: MergeTreeConditionFullText( - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & index_sample_block, const BloomFilterParameters & params_, @@ -144,9 +144,6 @@ private: BloomFilterParameters params; TokenExtractorPtr token_extractor; RPN rpn; - - /// Sets from syntax analyzer. - PreparedSetsPtr prepared_sets; }; class MergeTreeIndexFullText final : public IMergeTreeIndex @@ -166,7 +163,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const override; + const ActionsDAGPtr & filter_dag, ContextPtr context) const override; BloomFilterParameters params; /// Function for selecting next token. diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index 818bae40067..0995e2724ec 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -79,7 +79,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator(cons } MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( - const SelectQueryInfo &, ContextPtr) const + const ActionsDAGPtr &, ContextPtr) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h index 1cd0e3daf27..2296e1b717d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -70,7 +70,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const override; + const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; MergeTreeIndexMergedConditionPtr createIndexMergedCondition( const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 5e2a034cb97..4c28fe8f00b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -184,7 +184,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, } MergeTreeConditionInverted::MergeTreeConditionInverted( - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & index_sample_block, const GinFilterParameters & params_, @@ -192,41 +192,20 @@ MergeTreeConditionInverted::MergeTreeConditionInverted( : WithContext(context_), header(index_sample_block) , params(params_) , token_extractor(token_extactor_) - , prepared_sets(query_info.prepared_sets) { - if (context_->getSettingsRef().allow_experimental_analyzer) - { - if (!query_info.filter_actions_dag) - { - rpn.push_back(RPNElement::FUNCTION_UNKNOWN); - return; - } - - rpn = std::move( - RPNBuilder( - query_info.filter_actions_dag->getOutputs().at(0), context_, - [&](const RPNBuilderTreeNode & node, RPNElement & out) - { - return this->traverseAtomAST(node, out); - }).extractRPN()); - return; - } - - ASTPtr filter_node = buildFilterNode(query_info.query); - if (!filter_node) + if (!filter_actions_dag) { rpn.push_back(RPNElement::FUNCTION_UNKNOWN); return; } - auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_); - RPNBuilder builder( - filter_node, - context_, - std::move(block_with_constants), - query_info.prepared_sets, - [&](const RPNBuilderTreeNode & node, RPNElement & out) { return traverseAtomAST(node, out); }); - rpn = std::move(builder).extractRPN(); + rpn = std::move( + RPNBuilder( + filter_actions_dag->getOutputs().at(0), context_, + [&](const RPNBuilderTreeNode & node, RPNElement & out) + { + return this->traverseAtomAST(node, out); + }).extractRPN()); } /// Keep in-sync with MergeTreeConditionFullText::alwaysUnknownOrTrue @@ -721,9 +700,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregatorForPart } MergeTreeIndexConditionPtr MergeTreeIndexInverted::createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const + const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const { - return std::make_shared(query, context, index.sample_block, params, token_extractor.get()); + return std::make_shared(filter_actions_dag, context, index.sample_block, params, token_extractor.get()); }; MergeTreeIndexPtr invertedIndexCreator( diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.h b/src/Storages/MergeTree/MergeTreeIndexInverted.h index 413cf206f0e..807651d0c26 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.h +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.h @@ -64,7 +64,7 @@ class MergeTreeConditionInverted final : public IMergeTreeIndexCondition, WithCo { public: MergeTreeConditionInverted( - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & index_sample_block, const GinFilterParameters & params_, @@ -169,7 +169,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store, const MergeTreeWriterSettings & /*settings*/) const override; - MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; GinFilterParameters params; /// Function for selecting next token. diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index 535fef45872..b1f8e09be9f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -156,20 +156,17 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s namespace { -KeyCondition buildCondition(const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context) +KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context) { - if (context->getSettingsRef().allow_experimental_analyzer) - return KeyCondition{query_info.filter_actions_dag, context, index.column_names, index.expression}; - - return KeyCondition{query_info, context, index.column_names, index.expression}; + return KeyCondition{filter_actions_dag, context, index.column_names, index.expression}; } } MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax( - const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context) + const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context) : index_data_types(index.data_types) - , condition(buildCondition(index, query_info, context)) + , condition(buildCondition(index, filter_actions_dag, context)) { } @@ -200,9 +197,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator(const Me } MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const + const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const { - return std::make_shared(index, query, context); + return std::make_shared(index, filter_actions_dag, context); } MergeTreeIndexFormat MergeTreeIndexMinMax::getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & relative_path_prefix) const diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.h b/src/Storages/MergeTree/MergeTreeIndexMinMax.h index a1a216fdf72..1e2abe6983f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.h +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.h @@ -52,7 +52,7 @@ class MergeTreeIndexConditionMinMax final : public IMergeTreeIndexCondition public: MergeTreeIndexConditionMinMax( const IndexDescription & index, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_actions_dag, ContextPtr context); bool alwaysUnknownOrTrue() const override; @@ -79,7 +79,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const override; + const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; const char* getSerializedFileExtension() const override { return ".idx2"; } MergeTreeIndexFormat getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & path_prefix) const override; /// NOLINT diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 612c5d868cb..831856f8085 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -247,7 +247,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( const String & index_name_, const Block & index_sample_block, size_t max_rows_, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_dag, ContextPtr context) : index_name(index_name_) , max_rows(max_rows_) @@ -256,42 +256,20 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( if (!key_columns.contains(name)) key_columns.insert(name); - if (context->getSettingsRef().allow_experimental_analyzer) - { - if (!query_info.filter_actions_dag) - return; + if (!filter_dag) + return; - if (checkDAGUseless(*query_info.filter_actions_dag->getOutputs().at(0), context)) - return; + if (checkDAGUseless(*filter_dag->getOutputs().at(0), context)) + return; - const auto * filter_node = query_info.filter_actions_dag->getOutputs().at(0); - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG({filter_node}, {}, context); - const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0); + auto filter_actions_dag = filter_dag->clone(); + const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0); - std::unordered_map node_to_result_node; - filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); + std::unordered_map node_to_result_node; + filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); - filter_actions_dag->removeUnusedActions(); - actions = std::make_shared(filter_actions_dag); - } - else - { - ASTPtr ast_filter_node = buildFilterNode(query_info.query); - if (!ast_filter_node) - return; - - if (checkASTUseless(ast_filter_node)) - return; - - auto expression_ast = ast_filter_node->clone(); - - /// Replace logical functions with bit functions. - /// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). - traverseAST(expression_ast); - - auto syntax_analyzer_result = TreeRewriter(context).analyze(expression_ast, index_sample_block.getNamesAndTypesList()); - actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true); - } + filter_actions_dag->removeUnusedActions(); + actions = std::make_shared(filter_actions_dag); } bool MergeTreeIndexConditionSet::alwaysUnknownOrTrue() const @@ -704,9 +682,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const Merge } MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const + const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const { - return std::make_shared(index.name, index.sample_block, max_rows, query, context); + return std::make_shared(index.name, index.sample_block, max_rows, filter_actions_dag, context); } MergeTreeIndexPtr setIndexCreator(const IndexDescription & index) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index a53476ca751..ea9f7ddef3d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -87,7 +87,7 @@ public: const String & index_name_, const Block & index_sample_block, size_t max_rows_, - const SelectQueryInfo & query_info, + const ActionsDAGPtr & filter_dag, ContextPtr context); bool alwaysUnknownOrTrue() const override; @@ -149,7 +149,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const override; + const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; size_t max_rows = 0; }; diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index dc8ed368011..c9df7210569 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -36,6 +36,7 @@ namespace ErrorCodes extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } namespace @@ -366,6 +367,11 @@ MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const Sel return std::make_shared(index, query, distance_function, context); }; +MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG"); +} + MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) { static constexpr auto default_distance_function = DISTANCE_FUNCTION_L2; diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h index a7675620a2e..5107cfee371 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -100,7 +100,9 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; - MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override; + bool isVectorSearch() const override { return true; } private: const String distance_function; diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index da1e914b90e..4749470bedd 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -170,7 +170,9 @@ struct IMergeTreeIndex } virtual MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query_info, ContextPtr context) const = 0; + const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const = 0; + + virtual bool isVectorSearch() const { return false; } virtual MergeTreeIndexMergedConditionPtr createIndexMergedCondition( const SelectQueryInfo & /*query_info*/, StorageMetadataPtr /*storage_metadata*/) const diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index c559ba4371a..668576f9021 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -9,10 +9,7 @@ namespace KeyCondition buildKeyCondition(const KeyDescription & partition_key, const SelectQueryInfo & query_info, ContextPtr context, bool strict) { - if (context->getSettingsRef().allow_experimental_analyzer) - return {query_info.filter_actions_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict}; - - return {query_info, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict}; + return {query_info.filter_actions_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict}; } } diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index f14f241cac8..b0755ccd3ca 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -202,17 +202,6 @@ public: traverseTree(RPNBuilderTreeNode(filter_actions_dag_node, tree_context)); } - RPNBuilder(const ASTPtr & filter_node, - ContextPtr query_context_, - Block block_with_constants_, - PreparedSetsPtr prepared_sets_, - const ExtractAtomFromTreeFunction & extract_atom_from_tree_function_) - : tree_context(std::move(query_context_), std::move(block_with_constants_), std::move(prepared_sets_)) - , extract_atom_from_tree_function(extract_atom_from_tree_function_) - { - traverseTree(RPNBuilderTreeNode(filter_node.get(), tree_context)); - } - RPNElements && extractRPN() && { return std::move(rpn_elements); } private: diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f3917b878d6..3c1e13679b5 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -118,8 +118,12 @@ void listFilesWithRegexpMatchingImpl( { try { - fs::path path = fs::canonical(path_for_ls + for_match); - result.push_back(path.string()); + /// We use fs::canonical to resolve the canonical path and check if the file does exists + /// but the result path will be fs::absolute. + /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. + fs::canonical(path_for_ls + for_match); + fs::path absolute_path = fs::absolute(path_for_ls + for_match); + result.push_back(absolute_path.string()); } catch (const std::exception &) // NOLINT { @@ -1056,11 +1060,6 @@ StorageFileSource::~StorageFileSource() beforeDestroy(); } -void StorageFileSource::setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) -{ - setKeyConditionImpl(query_info_, context_, block_for_format); -} - void StorageFileSource::setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) { setKeyConditionImpl(nodes, context_, block_for_format); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 4f8cbfd4795..b74868597a6 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -256,8 +256,6 @@ private: return storage->getName(); } - void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override; - void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override; bool tryGetCountFromCache(const struct stat & file_stat); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e7ca50f4a5c..b8804ad3c6d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -262,10 +262,10 @@ std::optional StorageMergeTree::totalRows(const Settings &) const return getTotalActiveSizeInRows(); } -std::optional StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const +std::optional StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const { auto parts = getVisibleDataPartsVector(local_context); - return totalRowsByPartitionPredicateImpl(query_info, local_context, parts); + return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts); } std::optional StorageMergeTree::totalBytes(const Settings &) const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index b2829ecb17f..51bf6aa42e7 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -66,7 +66,7 @@ public: size_t num_streams) override; std::optional totalRows(const Settings &) const override; - std::optional totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr) const override; std::optional totalBytes(const Settings &) const override; std::optional totalBytesUncompressed(const Settings &) const override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 780698c31c2..a8404052c59 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5470,11 +5470,11 @@ std::optional StorageReplicatedMergeTree::totalRows(const Settings & set return res; } -std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const +std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const { DataPartsVector parts; foreachActiveParts([&](auto & part) { parts.push_back(part); }, local_context->getSettingsRef().select_sequential_consistency); - return totalRowsByPartitionPredicateImpl(query_info, local_context, parts); + return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts); } std::optional StorageReplicatedMergeTree::totalBytes(const Settings & settings) const diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 556d23d6903..2bd1fcbc693 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -163,7 +163,7 @@ public: size_t num_streams) override; std::optional totalRows(const Settings & settings) const override; - std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; std::optional totalBytes(const Settings & settings) const override; std::optional totalBytesUncompressed(const Settings & settings) const override; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f63bf3a8e90..b90a0d394cb 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -151,11 +151,6 @@ public: String getName() const override; - void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override - { - setKeyConditionImpl(query_info_, context_, sample_block); - } - void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override { setKeyConditionImpl(nodes, context_, sample_block); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index c9e6f6311bf..07d4d0cad38 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -172,11 +172,6 @@ public: String getName() const override { return name; } - void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override - { - setKeyConditionImpl(query_info_, context_, block_for_format); - } - void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override { setKeyConditionImpl(nodes, context_, block_for_format); diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index 439e3972bc7..7c42c610c22 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -1,11 +1,15 @@ -#include #include +#include +#include #include #include +#include +#include #include #include -#include +#include #include +#include namespace CurrentMetrics @@ -18,8 +22,26 @@ namespace CurrentMetrics namespace DB { + +enum class ChangeableWithoutRestart +{ + No, + IncreaseOnly, + DecreaseOnly, + Yes +}; + NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() { + auto changeable_without_restart_type = std::make_shared( + DataTypeEnum8::Values + { + {"No", static_cast(ChangeableWithoutRestart::No)}, + {"IncreaseOnly", static_cast(ChangeableWithoutRestart::IncreaseOnly)}, + {"DecreaseOnly", static_cast(ChangeableWithoutRestart::DecreaseOnly)}, + {"Yes", static_cast(ChangeableWithoutRestart::Yes)}, + }); + return { {"name", std::make_shared()}, {"value", std::make_shared()}, @@ -27,37 +49,45 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() {"changed", std::make_shared()}, {"description", std::make_shared()}, {"type", std::make_shared()}, - {"is_obsolete", std::make_shared()}, - {"is_hot_reloadable", std::make_shared()} + {"changeable_without_restart", std::move(changeable_without_restart_type)}, + {"is_obsolete", std::make_shared()} }; } void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - // Server settings that have been reloaded from the config file. - std::unordered_map updated = { - {"max_server_memory_usage", std::to_string(total_memory_tracker.getHardLimit())}, - {"allow_use_jemalloc_memory", std::to_string(total_memory_tracker.getAllowUseJemallocMmemory())}, + /// When the server configuration file is periodically re-loaded from disk, the server components (e.g. memory tracking) are updated + /// with new the setting values but the settings themselves are not stored between re-loads. As a result, if one wants to know the + /// current setting values, one needs to ask the components directly. + std::unordered_map> changeable_settings = { + {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}}, + {"allow_use_jemalloc_memory", {std::to_string(total_memory_tracker.getAllowUseJemallocMmemory()), ChangeableWithoutRestart::Yes}}, - {"max_table_size_to_drop", std::to_string(context->getMaxTableSizeToDrop())}, - {"max_partition_size_to_drop", std::to_string(context->getMaxPartitionSizeToDrop())}, + {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}}, + {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}}, - {"max_concurrent_queries", std::to_string(context->getProcessList().getMaxSize())}, - {"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())}, - {"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())}, + {"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), ChangeableWithoutRestart::Yes}}, + {"max_concurrent_insert_queries", {std::to_string(context->getProcessList().getMaxInsertQueriesAmount()), ChangeableWithoutRestart::Yes}}, + {"max_concurrent_select_queries", {std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), ChangeableWithoutRestart::Yes}}, - {"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))}, - {"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))}, - {"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))}, - {"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))} + {"background_buffer_flush_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_message_broker_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_distributed_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + + {"mark_cache_size", {std::to_string(context->getMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"uncompressed_cache_size", {std::to_string(context->getUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"index_mark_cache_size", {std::to_string(context->getIndexMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"index_uncompressed_cache_size", {std::to_string(context->getIndexUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}} }; if (context->areBackgroundExecutorsInitialized()) { - updated.insert({"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}); - updated.insert({"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}); - updated.insert({"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}); - updated.insert({"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}); + changeable_settings.insert({"background_pool_size", {std::to_string(context->getMergeMutateExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); + changeable_settings.insert({"background_move_pool_size", {std::to_string(context->getMovesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); + changeable_settings.insert({"background_fetches_pool_size", {std::to_string(context->getFetchesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); + changeable_settings.insert({"background_common_pool_size", {std::to_string(context->getCommonExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); } const auto & config = context->getConfigRef(); @@ -67,16 +97,18 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context for (const auto & setting : settings.all()) { const auto & setting_name = setting.getName(); - const auto & it = updated.find(setting_name); + + const auto & changeable_settings_it = changeable_settings.find(setting_name); + const bool is_changeable = (changeable_settings_it != changeable_settings.end()); res_columns[0]->insert(setting_name); - res_columns[1]->insert((it != updated.end()) ? it->second : setting.getValueString()); + res_columns[1]->insert(is_changeable ? changeable_settings_it->second.first : setting.getValueString()); res_columns[2]->insert(setting.getDefaultValueString()); res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); - res_columns[6]->insert(setting.isObsolete()); - res_columns[7]->insert((it != updated.end()) ? true : false); + res_columns[6]->insert(is_changeable ? changeable_settings_it->second.second : ChangeableWithoutRestart::No); + res_columns[7]->insert(setting.isObsolete()); } } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 53b28543bf1..d2c01ec3dea 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -104,7 +104,7 @@ ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr MutableColumnPtr database_column = ColumnString::create(); MutableColumnPtr engine_column; - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, sample); + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); if (dag) { bool filter_by_engine = false; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 76138bbea87..e845e03d122 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -253,19 +253,7 @@ static void makeSets(const ExpressionActionsPtr & actions, const ContextPtr & co if (!future_set->get()) { if (auto * set_from_subquery = typeid_cast(future_set.get())) - { - auto plan = set_from_subquery->build(context); - - if (!plan) - continue; - - auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); - - CompletedPipelineExecutor executor(pipeline); - executor.execute(); - } + set_from_subquery->buildSetInplace(context); } } } @@ -406,7 +394,7 @@ ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, con } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); - return splitFilterDagForAllowedInputs(predicate, block); + return splitFilterDagForAllowedInputs(predicate, &block); } ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context) @@ -480,7 +468,7 @@ static bool canEvaluateSubtree(const ActionsDAG::Node * node, const Block & allo static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( const ActionsDAG::Node * node, - const Block & allowed_inputs, + const Block * allowed_inputs, ActionsDAG::Nodes & additional_nodes) { if (node->type == ActionsDAG::ActionType::FUNCTION) @@ -555,13 +543,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( } } - if (!canEvaluateSubtree(node, allowed_inputs)) + if (allowed_inputs && !canEvaluateSubtree(node, *allowed_inputs)) return nullptr; return node; } -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block & allowed_inputs) +ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs) { if (!predicate) return nullptr; @@ -576,7 +564,7 @@ ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context) { - auto dag = splitFilterDagForAllowedInputs(predicate, block); + auto dag = splitFilterDagForAllowedInputs(predicate, &block); if (dag) filterBlockWithDAG(dag, block, context); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 3c07e33a177..7a9b2605339 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -42,7 +42,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context); /// Extract a part of predicate that can be evaluated using only columns from input_names. -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block & allowed_inputs); +ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); /// Extract from the input stream a set of `name` column values template diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 74f2709f458..00cc5e3ee58 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -1,6 +1,7 @@ #include +#include #include #include #include @@ -372,6 +373,10 @@ QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeN removeGroupingFunctionSpecializations(query_tree_to_modify); + // std::cerr << "====================== build 1 \n" << query_tree_to_modify->dumpTree() << std::endl; + createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext()); + // std::cerr << "====================== build 2 \n" << query_tree_to_modify->dumpTree() << std::endl; + return query_tree_to_modify; } diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 031ab0be8a0..895a12313da 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -72,10 +72,20 @@ class BuildConfig: include_paths=[ "./src", "./contrib/*-cmake", + "./contrib/consistent-hashing", + "./contrib/murmurhash", + "./contrib/libfarmhash", + "./contrib/pdqsort", + "./contrib/cityhash102", + "./contrib/sparse-checkout", + "./contrib/libmetrohash", + "./contrib/update-submodules.sh", + "./contrib/CMakeLists.txt", "./cmake", "./base", "./programs", "./packages", + "./docker/packager/packager", ], exclude_files=[".md"], docker=["clickhouse/binary-builder"], diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index d5f8d04e258..9a75dc50d61 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -463,7 +463,9 @@ def test_schema_inference(started_cluster, format_version): create_iceberg_table(instance, TABLE_NAME, format) - res = instance.query(f"DESC {TABLE_NAME} FORMAT TSVRaw") + res = instance.query( + f"DESC {TABLE_NAME} FORMAT TSVRaw", settings={"print_pretty_type_names": 0} + ) expected = TSV( [ ["intC", "Nullable(Int32)"], diff --git a/tests/queries/0_stateless/00547_named_tuples.reference b/tests/queries/0_stateless/00547_named_tuples.reference index 70cd0054bdd..041ead4ca79 100644 --- a/tests/queries/0_stateless/00547_named_tuples.reference +++ b/tests/queries/0_stateless/00547_named_tuples.reference @@ -1 +1 @@ -(1,'Hello') Tuple(x UInt64, s String) 1 Hello 1 Hello +(1,'Hello') Tuple(\n x UInt64,\n s String) 1 Hello 1 Hello diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.reference b/tests/queries/0_stateless/00621_regression_for_in_operator.reference index ab8bcf307eb..b68f550a742 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.reference +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.reference @@ -10,7 +10,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.regression_for_in_operator_view + TABLE id: 3, alias: __table1, table_name: default.regression_for_in_operator_view WHERE FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -27,7 +27,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.regression_for_in_operator_view + TABLE id: 3, alias: __table1, table_name: default.regression_for_in_operator_view WHERE FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.reference b/tests/queries/0_stateless/00736_disjunction_optimisation.reference index 84477a64057..f28dcacef0e 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.reference +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.reference @@ -34,7 +34,7 @@ QUERY id: 0 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -77,7 +77,7 @@ QUERY id: 0 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS k UInt64 s UInt64 @@ -86,7 +86,7 @@ QUERY id: 0 COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7 COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.bug + TABLE id: 7, alias: __table2, table_name: default.bug WHERE FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -151,7 +151,7 @@ QUERY id: 0 COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 21 1 22 1 @@ -184,7 +184,7 @@ QUERY id: 0 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 1 21 1 22 @@ -222,7 +222,7 @@ QUERY id: 0 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -265,7 +265,7 @@ QUERY id: 0 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS k UInt64 s UInt64 @@ -274,7 +274,7 @@ QUERY id: 0 COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7 COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.bug + TABLE id: 7, alias: __table2, table_name: default.bug WHERE FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -347,7 +347,7 @@ QUERY id: 0 COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 21 1 22 1 @@ -380,7 +380,7 @@ QUERY id: 0 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 21 1 22 1 @@ -413,5 +413,5 @@ QUERY id: 0 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index be8e603f8dc..5264d51fa73 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -69,8 +69,8 @@ hello 123456.1234 Decimal(20, 4) 123456.1234 Decimal(20, 4) 123456789012345.12 Decimal(30, 4) -(1234567890.1234567890123456789,'test') Tuple(a Decimal(35, 20), b LowCardinality(String)) -(1234567890.12345678901234567890123456789,'test') Tuple(a Decimal(45, 30), b LowCardinality(String)) +(1234567890.1234567890123456789,'test') Tuple(\n a Decimal(35, 20),\n b LowCardinality(String)) +(1234567890.12345678901234567890123456789,'test') Tuple(\n a Decimal(45, 30),\n b LowCardinality(String)) 123456789012345.1136 123456789012345.1136 1234567890.12345677879616925706 (1234567890.12345677879616925706,'test') 1234567890.123456695758468374595199311875 (1234567890.123456695758468374595199311875,'test') diff --git a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference index a9be79800c1..f861da3da2b 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference +++ b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference @@ -49,7 +49,7 @@ QUERY id: 0 LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -124,7 +124,7 @@ QUERY id: 0 LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -194,7 +194,7 @@ QUERY id: 0 COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7 CONSTANT id: 11, constant_value: UInt64_5, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 7, table_function_name: numbers + TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -276,7 +276,7 @@ QUERY id: 0 LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 diff --git a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference index bf184d142ec..d47f12ff4d1 100644 --- a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference @@ -49,14 +49,14 @@ QUERY id: 0 LIST id: 3, nodes: 1 COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5 JOIN TREE - QUERY id: 5, is_subquery: 1 + QUERY id: 5, alias: __table1, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 6, nodes: 1 COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 JOIN TREE - TABLE_FUNCTION id: 8, table_function_name: numbers + TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 @@ -83,14 +83,14 @@ QUERY id: 0 LIST id: 3, nodes: 1 COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5 JOIN TREE - QUERY id: 5, is_subquery: 1 + QUERY id: 5, alias: __table1, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 6, nodes: 1 COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 JOIN TREE - TABLE_FUNCTION id: 8, table_function_name: numbers + TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 @@ -119,14 +119,14 @@ QUERY id: 0 LIST id: 3, nodes: 1 COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5 JOIN TREE - QUERY id: 5, is_subquery: 1 + QUERY id: 5, alias: __table1, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 6, nodes: 1 COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 JOIN TREE - TABLE_FUNCTION id: 8, table_function_name: numbers + TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 @@ -171,7 +171,7 @@ QUERY id: 0 JOIN TREE JOIN id: 8, strictness: ALL, kind: FULL LEFT TABLE EXPRESSION - QUERY id: 3, alias: s, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS key UInt64 PROJECTION @@ -182,12 +182,12 @@ QUERY id: 0 COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13 CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 13, table_function_name: numbers + TABLE_FUNCTION id: 13, alias: __table2, table_function_name: numbers ARGUMENTS LIST id: 15, nodes: 1 CONSTANT id: 16, constant_value: UInt64_4, constant_value_type: UInt8 RIGHT TABLE EXPRESSION - TABLE id: 5, alias: t, table_name: default.test + TABLE id: 5, alias: __table3, table_name: default.test JOIN EXPRESSION LIST id: 17, nodes: 1 COLUMN id: 18, column_name: key, result_type: UInt64, source_id: 8 @@ -220,7 +220,7 @@ QUERY id: 0 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.test + TABLE id: 3, alias: __table1, table_name: default.test ORDER BY LIST id: 5, nodes: 2 SORT id: 6, sort_direction: ASCENDING, with_fill: 0 @@ -246,7 +246,7 @@ QUERY id: 0 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.test + TABLE id: 3, alias: __table1, table_name: default.test ORDER BY LIST id: 5, nodes: 2 SORT id: 6, sort_direction: ASCENDING, with_fill: 0 @@ -270,7 +270,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.test + TABLE id: 3, alias: __table1, table_name: default.test GROUP BY LIST id: 4, nodes: 1 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 @@ -297,9 +297,9 @@ QUERY id: 0 JOIN TREE JOIN id: 6, strictness: ALL, kind: INNER LEFT TABLE EXPRESSION - TABLE id: 3, table_name: default.t1 + TABLE id: 3, alias: __table1, table_name: default.t1 RIGHT TABLE EXPRESSION - TABLE id: 5, table_name: default.t2 + TABLE id: 5, alias: __table2, table_name: default.t2 JOIN EXPRESSION FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference index a6d43856aec..2920b387aa2 100644 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference @@ -1,9 +1,9 @@ ===http=== {"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"DESC TABLE system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} -{"query":"SELECT 1 AS `1` FROM `system`.`one`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} +{"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"DESC TABLE system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} -{"query":"SELECT 1 AS `1` FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} +{"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"} {"initial query spans with proper parent":"2"} diff --git a/tests/queries/0_stateless/01458_named_tuple_millin.reference b/tests/queries/0_stateless/01458_named_tuple_millin.reference index d6d6d7ae8d4..954dfe36563 100644 --- a/tests/queries/0_stateless/01458_named_tuple_millin.reference +++ b/tests/queries/0_stateless/01458_named_tuple_millin.reference @@ -3,10 +3,10 @@ CREATE TABLE default.tuple `j` Tuple(a Int8, b String) ) ENGINE = Memory -j Tuple(a Int8, b String) +j Tuple(\n a Int8,\n b String) CREATE TABLE default.tuple ( `j` Tuple(a Int8, b String) ) ENGINE = Memory -j Tuple(a Int8, b String) +j Tuple(\n a Int8,\n b String) diff --git a/tests/queries/0_stateless/01532_tuple_with_name_type.reference b/tests/queries/0_stateless/01532_tuple_with_name_type.reference index 8a3e57d9016..66b85f05fa6 100644 --- a/tests/queries/0_stateless/01532_tuple_with_name_type.reference +++ b/tests/queries/0_stateless/01532_tuple_with_name_type.reference @@ -1,4 +1,4 @@ -a Tuple(key String, value String) -a Tuple(Tuple(key String, value String)) -a Array(Tuple(key String, value String)) -a Tuple(UInt8, Tuple(key String, value String)) +a Tuple(\n key String,\n value String) +a Tuple(Tuple(\n key String,\n value String)) +a Array(Tuple(\n key String,\n value String)) +a Tuple(UInt8, Tuple(\n key String,\n value String)) diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference index 8a34751b071..2631199cbab 100644 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference @@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1 "foo" 1 execute: --allow_experimental_analyzer=1 --stage fetch_columns -"dummy_0" +"__table1.dummy" 0 execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 5d12a09a846..156f36f7dba 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -917,9 +917,9 @@ from ; Expression ((Project names + Projection)) Window (Window step for window \'\') - Window (Window step for window \'PARTITION BY p_0\') - Window (Window step for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') - Sorting (Sorting for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') + Window (Window step for window \'PARTITION BY __table1.p\') + Window (Window step for window \'PARTITION BY __table1.p ORDER BY __table1.o ASC\') + Sorting (Sorting for window \'PARTITION BY __table1.p ORDER BY __table1.o ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromSystemNumbers explain select @@ -930,11 +930,11 @@ from from numbers(16)) t ; Expression ((Project names + Projection)) - Window (Window step for window \'ORDER BY o_0 ASC, number_1 ASC\') - Sorting (Sorting for window \'ORDER BY o_0 ASC, number_1 ASC\') - Window (Window step for window \'ORDER BY number_1 ASC\') + Window (Window step for window \'ORDER BY __table1.o ASC, __table1.number ASC\') + Sorting (Sorting for window \'ORDER BY __table1.o ASC, __table1.number ASC\') + Window (Window step for window \'ORDER BY __table1.number ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part]) - Sorting (Sorting for window \'ORDER BY number_1 ASC\') + Sorting (Sorting for window \'ORDER BY __table1.number ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromSystemNumbers -- A test case for the sort comparator found by fuzzer. diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index ef6425b485b..d267df2237f 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -45,7 +45,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants + TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants WHERE FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -63,7 +63,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants + TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants WHERE FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -80,5 +80,5 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants + TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference index b5520d75b0e..3f6e8211f1a 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -8,7 +8,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 @@ -22,7 +22,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 @@ -36,7 +36,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 @@ -50,7 +50,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -68,7 +68,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where PREWHERE FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -85,5 +85,5 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 3639ad47228..555a4c93f70 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -20,7 +20,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -50,7 +50,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test PREWHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -80,7 +80,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -110,7 +110,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -140,7 +140,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -162,7 +162,7 @@ QUERY id: 0 COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -191,7 +191,7 @@ QUERY id: 0 CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 COLUMN id: 9, column_name: a, result_type: String, source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.column_swap_test_test + TABLE id: 7, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -223,7 +223,7 @@ QUERY id: 0 CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 COLUMN id: 9, column_name: a, result_type: String, source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.column_swap_test_test + TABLE id: 7, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -248,7 +248,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: String, source_id: 3 COLUMN id: 4, column_name: a, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.column_swap_test_test + TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -270,7 +270,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: String, source_id: 3 COLUMN id: 4, column_name: a, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.column_swap_test_test + TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -292,7 +292,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: String, source_id: 3 COLUMN id: 4, column_name: a, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.column_swap_test_test + TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -310,7 +310,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.column_swap_test_test + TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 4, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -327,5 +327,5 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: UInt32, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_bad_constraint + TABLE id: 3, alias: __table1, table_name: default.t_bad_constraint SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01646_rewrite_sum_if.reference b/tests/queries/0_stateless/01646_rewrite_sum_if.reference index 871c75737c6..af582908f03 100644 --- a/tests/queries/0_stateless/01646_rewrite_sum_if.reference +++ b/tests/queries/0_stateless/01646_rewrite_sum_if.reference @@ -56,7 +56,7 @@ QUERY id: 0 CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 12, table_function_name: numbers + TABLE_FUNCTION id: 12, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 15, nodes: 1 CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8 @@ -82,7 +82,7 @@ QUERY id: 0 CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 12, table_function_name: numbers + TABLE_FUNCTION id: 12, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 15, nodes: 1 CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8 @@ -111,7 +111,7 @@ QUERY id: 0 CONSTANT id: 15, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 16, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 14, table_function_name: numbers + TABLE_FUNCTION id: 14, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 17, nodes: 1 CONSTANT id: 18, constant_value: UInt64_100, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 54ca55d2068..436d06c5076 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -28,7 +28,7 @@ Aggregating Filter Filter > (analyzer) filter should be pushed down after aggregating, column after aggregation is const -COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8) +COLUMN Const(UInt8) -> notEquals(__table1.y, 0_UInt8) Aggregating Filter Filter @@ -49,9 +49,9 @@ Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased Filter column -ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8)) +ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8)) Aggregating -Filter column: notEquals(y_1, 0_UInt8) +Filter column: notEquals(__table1.y, 0_UInt8) 0 1 1 2 2 3 @@ -68,9 +68,9 @@ Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2 +FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2 Aggregating -Filter column: notEquals(y_1, 0_UInt8) +Filter column: notEquals(__table1.y, 0_UInt8) 0 1 1 2 2 3 @@ -87,9 +87,9 @@ Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8)) +FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8)) Aggregating -Filter column: notEquals(y_1, 0_UInt8) +Filter column: notEquals(__table1.y, 0_UInt8) 0 1 1 2 2 3 @@ -105,9 +105,9 @@ Aggregating Filter column: and(notEquals(y, 0), minus(y, 4)) > (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column -ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8)) +ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8)) Aggregating -Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8)) +Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8)) 0 1 1 2 2 3 @@ -121,9 +121,9 @@ Filter column: and(notEquals(y, 2), notEquals(x, 0)) ARRAY JOIN x Filter column: notEquals(y, 2) > (analyzer) filter is split, one part is filtered before ARRAY JOIN -Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8)) -ARRAY JOIN x_0 -Filter column: notEquals(y_1, 2_UInt8) +Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8)) +ARRAY JOIN __table1.x +Filter column: notEquals(__table2.y, 2_UInt8) 1 3 > filter is pushed down before Distinct Distinct @@ -132,7 +132,7 @@ Filter column: notEquals(y, 2) > (analyzer) filter is pushed down before Distinct Distinct Distinct -Filter column: notEquals(y_1, 2_UInt8) +Filter column: notEquals(__table1.y, 2_UInt8) 0 0 0 1 1 0 @@ -144,7 +144,7 @@ Filter column: and(notEquals(x, 0), notEquals(y, 0)) > (analyzer) filter is pushed down before sorting steps Sorting Sorting -Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8)) +Filter column: and(notEquals(__table1.x, 0_UInt8), notEquals(__table1.y, 0_UInt8)) 1 2 1 1 > filter is pushed down before TOTALS HAVING and aggregating @@ -154,7 +154,7 @@ Filter column: notEquals(y, 2) > (analyzer) filter is pushed down before TOTALS HAVING and aggregating TotalsHaving Aggregating -Filter column: notEquals(y_0, 2_UInt8) +Filter column: notEquals(__table1.y, 2_UInt8) 0 12 1 15 3 10 @@ -174,7 +174,7 @@ Join > (analyzer) one condition of filter is pushed down before LEFT JOIN Join Join -Filter column: notEquals(number_0, 1_UInt8) +Filter column: notEquals(__table1.number, 1_UInt8) 0 0 3 3 > one condition of filter is pushed down before INNER JOIN @@ -185,7 +185,7 @@ Join > (analyzer) one condition of filter is pushed down before INNER JOIN Join Join -Filter column: notEquals(number_0, 1_UInt8) +Filter column: notEquals(__table1.number, 1_UInt8) 3 3 > filter is pushed down before UNION Union diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index a765a6ea4fa..5a517264243 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -36,7 +36,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 - settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8)" + settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(__table1.y, 0_UInt8)" $CLICKHOUSE_CLIENT -q " select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y @@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -76,7 +76,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -96,7 +96,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -116,7 +116,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8))\|ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -134,7 +134,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select x, y from ( select range(number) as x, number + 1 as y from numbers(3) ) array join x where y != 2 and x != 0" | - grep -o "Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8))\|ARRAY JOIN x_0\|Filter column: notEquals(y_1, 2_UInt8)" + grep -o "Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8))\|ARRAY JOIN __table1.x\|Filter column: notEquals(__table2.y, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select x, y from ( select range(number) as x, number + 1 as y from numbers(3) @@ -166,7 +166,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) ) where y != 2 settings enable_optimize_predicate_expression=0" | - grep -o "Distinct\|Filter column: notEquals(y_1, 2_UInt8)" + grep -o "Distinct\|Filter column: notEquals(__table1.y, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select x, y from ( select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) @@ -186,7 +186,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 settings enable_optimize_predicate_expression = 0" | - grep -o "Sorting\|Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8))" + grep -o "Sorting\|Filter column: and(notEquals(__table1.x, 0_UInt8), notEquals(__table1.y, 0_UInt8))" $CLICKHOUSE_CLIENT -q " select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc @@ -206,7 +206,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals ) where y != 2 settings enable_optimize_predicate_expression=0" | - grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(y_0, 2_UInt8)" + grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(__table1.y, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select * from ( select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals @@ -236,7 +236,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | - grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)" + grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)" $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) @@ -255,7 +255,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | - grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)" + grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)" $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference index 8a33df9fad2..7c2753124b3 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference @@ -7,19 +7,19 @@ Partial sorting plan Prefix sort description: n ASC Result sort description: n ASC, x ASC optimize_read_in_window_order=1, allow_experimental_analyzer=1 - Prefix sort description: n_0 ASC - Result sort description: n_0 ASC, x_1 ASC + Prefix sort description: __table1.n ASC + Result sort description: __table1.n ASC, __table1.x ASC No sorting plan optimize_read_in_window_order=0 Sort description: n ASC, x ASC optimize_read_in_window_order=0, allow_experimental_analyzer=1 - Sort description: n_0 ASC, x_1 ASC + Sort description: __table1.n ASC, __table1.x ASC optimize_read_in_window_order=1 Prefix sort description: n ASC, x ASC Result sort description: n ASC, x ASC optimize_read_in_window_order=1, allow_experimental_analyzer=1 - Prefix sort description: n_0 ASC, x_1 ASC - Result sort description: n_0 ASC, x_1 ASC + Prefix sort description: __table1.n ASC, __table1.x ASC + Result sort description: __table1.n ASC, __table1.x ASC Complex ORDER BY optimize_read_in_window_order=0 3 3 1 diff --git a/tests/queries/0_stateless/01823_explain_json.reference b/tests/queries/0_stateless/01823_explain_json.reference index befbf82f4fb..23fb34c2192 100644 --- a/tests/queries/0_stateless/01823_explain_json.reference +++ b/tests/queries/0_stateless/01823_explain_json.reference @@ -37,59 +37,59 @@ "Node Type": "Aggregating", "Header": [ { - "Name": "number_0", + "Name": "__table1.number", "Type": "UInt64" }, { - "Name": "quantile(0.2_Float64)(number_0)", + "Name": "quantile(0.2_Float64)(__table1.number)", "Type": "Float64" }, { - "Name": "sumIf(number_0, greater(number_0, 0_UInt8))", + "Name": "sumIf(__table1.number, greater(__table1.number, 0_UInt8))", "Type": "UInt64" } ], - "Keys": ["number_0"], + "Keys": ["__table1.number"], "Aggregates": [ { - "Name": "quantile(0.2_Float64)(number_0)", + "Name": "quantile(0.2_Float64)(__table1.number)", "Function": { "Name": "quantile", "Parameters": ["0.2"], "Argument Types": ["UInt64"], "Result Type": "Float64" }, - "Arguments": ["number_0"] + "Arguments": ["__table1.number"] }, { - "Name": "sumIf(number_0, greater(number_0, 0_UInt8))", + "Name": "sumIf(__table1.number, greater(__table1.number, 0_UInt8))", "Function": { "Name": "sumIf", "Argument Types": ["UInt64", "UInt8"], "Result Type": "UInt64" }, - "Arguments": ["number_0", "greater(number_0, 0_UInt8)"] + "Arguments": ["__table1.number", "greater(__table1.number, 0_UInt8)"] } ], -------- "Node Type": "ArrayJoin", "Left": false, - "Columns": ["x_0", "y_1"], + "Columns": ["__table1.x", "__table1.y"], -------- "Node Type": "Distinct", - "Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"], + "Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"], -- "Node Type": "Distinct", - "Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"], + "Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"], -------- "Sort Description": [ { - "Column": "number_0", + "Column": "__table1.number", "Ascending": false, "With Fill": false }, { - "Column": "plus(number_0, 1_UInt8)", + "Column": "plus(__table1.number, 1_UInt8)", "Ascending": true, "With Fill": false } diff --git a/tests/queries/0_stateless/01825_type_json_10.reference b/tests/queries/0_stateless/01825_type_json_10.reference index 53fe604fa51..4161fb59c93 100644 --- a/tests/queries/0_stateless/01825_type_json_10.reference +++ b/tests/queries/0_stateless/01825_type_json_10.reference @@ -1,4 +1,4 @@ -Tuple(a Tuple(b Int8, c Nested(d Int8, e Array(Int16), f Int8))) +Tuple(\n a Tuple(\n b Int8,\n c Nested(d Int8, e Array(Int16), f Int8))) {"o":{"a":{"b":1,"c":[{"d":10,"e":[31],"f":0},{"d":20,"e":[63,127],"f":0}]}}} {"o":{"a":{"b":2,"c":[]}}} {"o":{"a":{"b":3,"c":[{"d":0,"e":[32],"f":20},{"d":0,"e":[64,128],"f":30}]}}} diff --git a/tests/queries/0_stateless/01825_type_json_11.reference b/tests/queries/0_stateless/01825_type_json_11.reference index 27569620cd7..0575743e019 100644 --- a/tests/queries/0_stateless/01825_type_json_11.reference +++ b/tests/queries/0_stateless/01825_type_json_11.reference @@ -1,4 +1,4 @@ -Tuple(id Int8, key_1 Nested(key_2 Int32, key_3 Nested(key_4 Nested(key_5 Int8), key_7 Int16))) +Tuple(\n id Int8,\n key_1 Nested(key_2 Int32, key_3 Nested(key_4 Nested(key_5 Int8), key_7 Int16))) {"obj":{"id":1,"key_1":[{"key_2":100,"key_3":[{"key_4":[{"key_5":-2}],"key_7":257}]},{"key_2":65536,"key_3":[]}]}} {"obj":{"id":2,"key_1":[{"key_2":101,"key_3":[{"key_4":[{"key_5":-2}],"key_7":0}]},{"key_2":102,"key_3":[{"key_4":[],"key_7":257}]},{"key_2":65536,"key_3":[]}]}} {"obj.key_1.key_3":[[{"key_4":[{"key_5":-2}],"key_7":257}],[]]} diff --git a/tests/queries/0_stateless/01825_type_json_12.reference b/tests/queries/0_stateless/01825_type_json_12.reference index 7f4f5bf190e..ff60ba33f94 100644 --- a/tests/queries/0_stateless/01825_type_json_12.reference +++ b/tests/queries/0_stateless/01825_type_json_12.reference @@ -1,3 +1,3 @@ -Tuple(id Int8, key_0 Nested(key_1 Nested(key_3 Nested(key_4 String, key_5 Float64, key_6 String, key_7 Float64)))) +Tuple(\n id Int8,\n key_0 Nested(key_1 Nested(key_3 Nested(key_4 String, key_5 Float64, key_6 String, key_7 Float64)))) {"obj":{"id":1,"key_0":[{"key_1":[{"key_3":[{"key_4":"1048576","key_5":0.0001048576,"key_6":"25.5","key_7":1025},{"key_4":"","key_5":0,"key_6":"","key_7":2}]}]},{"key_1":[]},{"key_1":[{"key_3":[{"key_4":"","key_5":-1,"key_6":"aqbjfiruu","key_7":-922337203685477600},{"key_4":"","key_5":0,"key_6":"","key_7":65537}]},{"key_3":[{"key_4":"ghdqyeiom","key_5":1048575,"key_6":"","key_7":21474836.48}]}]}]}} [[['1048576','']],[],[['',''],['ghdqyeiom']]] [[[0.0001048576,0]],[],[[-1,0],[1048575]]] [[['25.5','']],[],[['aqbjfiruu',''],['']]] [[[1025,2]],[],[[-922337203685477600,65537],[21474836.48]]] diff --git a/tests/queries/0_stateless/01825_type_json_13.reference b/tests/queries/0_stateless/01825_type_json_13.reference index e420021f406..fa105f1a4c6 100644 --- a/tests/queries/0_stateless/01825_type_json_13.reference +++ b/tests/queries/0_stateless/01825_type_json_13.reference @@ -1,3 +1,3 @@ -Tuple(id Int8, key_1 Nested(key_2 Nested(key_3 Nested(key_4 Nested(key_5 Float64, key_6 Int64, key_7 Int32), key_8 Int32)))) +Tuple(\n id Int8,\n key_1 Nested(key_2 Nested(key_3 Nested(key_4 Nested(key_5 Float64, key_6 Int64, key_7 Int32), key_8 Int32)))) {"obj":{"id":1,"key_1":[{"key_2":[{"key_3":[{"key_4":[],"key_8":65537},{"key_4":[{"key_5":-0.02,"key_6":"0","key_7":0},{"key_5":0,"key_6":"0","key_7":1023},{"key_5":0,"key_6":"9223372036854775807","key_7":1}],"key_8":0},{"key_4":[{"key_5":0,"key_6":"0","key_7":65537}],"key_8":0}]}]}]}} [[[65537,0,0]]] [[[[],[-0.02,0,0],[0]]]] [[[[],[0,0,9223372036854775807],[0]]]] [[[[],[0,1023,1],[65537]]]] diff --git a/tests/queries/0_stateless/01825_type_json_15.reference b/tests/queries/0_stateless/01825_type_json_15.reference index ab4b1b82877..4f13731d35a 100644 --- a/tests/queries/0_stateless/01825_type_json_15.reference +++ b/tests/queries/0_stateless/01825_type_json_15.reference @@ -1,3 +1,3 @@ -Tuple(id Int8, key_0 Nested(key_0 Float64, key_1 Tuple(key_2 Array(Int8), key_8 String), key_10 Float64)) +Tuple(\n id Int8,\n key_0 Nested(key_0 Float64, key_1 Tuple(key_2 Array(Int8), key_8 String), key_10 Float64)) {"obj":{"id":1,"key_0":[{"key_0":-1,"key_1":{"key_2":[1,2,3],"key_8":"sffjx"},"key_10":65535},{"key_0":922337203.685,"key_1":{"key_2":[],"key_8":""},"key_10":10.23}]}} [[1,2,3],[]] ['sffjx',''] [65535,10.23] [-1,922337203.685] diff --git a/tests/queries/0_stateless/01825_type_json_16.reference b/tests/queries/0_stateless/01825_type_json_16.reference index f40f0d747d5..a8cc682f8e1 100644 --- a/tests/queries/0_stateless/01825_type_json_16.reference +++ b/tests/queries/0_stateless/01825_type_json_16.reference @@ -1,3 +1,3 @@ -Tuple(id Int8, key_0 Nested(key_1 Nested(key_2 Tuple(key_3 Nested(key_4 Int32, key_6 Int8, key_7 Int16), key_5 Nested(key_6 Int8, key_7 String))))) +Tuple(\n id Int8,\n key_0 Nested(key_1 Nested(key_2 Tuple(key_3 Nested(key_4 Int32, key_6 Int8, key_7 Int16), key_5 Nested(key_6 Int8, key_7 String))))) {"obj":{"id":1,"key_0":[{"key_1":[{"key_2":{"key_3":[{"key_4":255,"key_6":0,"key_7":0},{"key_4":65535,"key_6":0,"key_7":0},{"key_4":0,"key_6":3,"key_7":255}],"key_5":[{"key_6":1,"key_7":"nnpqx"},{"key_6":3,"key_7":"255"}]}}]}]}} [[[255,65535,0]]] [[[0,0,3]]] [[[0,0,255]]] [[[1,3]]] [[['nnpqx','255']]] diff --git a/tests/queries/0_stateless/01825_type_json_17.reference b/tests/queries/0_stateless/01825_type_json_17.reference index 0f97bfed5bc..c830cf41cf1 100644 --- a/tests/queries/0_stateless/01825_type_json_17.reference +++ b/tests/queries/0_stateless/01825_type_json_17.reference @@ -1,4 +1,4 @@ -Tuple(arr Nested(k1 Nested(k2 String, k3 String, k4 Int8), k5 Tuple(k6 String)), id Int8) +Tuple(\n arr Nested(k1 Nested(k2 String, k3 String, k4 Int8), k5 Tuple(k6 String)),\n id Int8) {"obj":{"arr":[{"k1":[{"k2":"aaa","k3":"bbb","k4":0},{"k2":"ccc","k3":"","k4":0}],"k5":{"k6":""}}],"id":1}} {"obj":{"arr":[{"k1":[{"k2":"","k3":"ddd","k4":10},{"k2":"","k3":"","k4":20}],"k5":{"k6":"foo"}}],"id":2}} [['bbb','']] [['aaa','ccc']] @@ -6,7 +6,7 @@ Tuple(arr Nested(k1 Nested(k2 String, k3 String, k4 Int8), k5 Tuple(k6 String)), 1 [[0,0]] [[10,20]] -Tuple(arr Nested(k1 Nested(k2 String, k3 Nested(k4 Int8))), id Int8) +Tuple(\n arr Nested(k1 Nested(k2 String, k3 Nested(k4 Int8))),\n id Int8) {"obj":{"arr":[{"k1":[{"k2":"aaa","k3":[]}]}],"id":1}} {"obj":{"arr":[{"k1":[{"k2":"bbb","k3":[{"k4":10}]},{"k2":"ccc","k3":[{"k4":20}]}]}],"id":2}} [['aaa']] [[[]]] @@ -14,7 +14,7 @@ Tuple(arr Nested(k1 Nested(k2 String, k3 Nested(k4 Int8))), id Int8) 1 [[[]]] [[[10],[20]]] -Tuple(arr Nested(k1 Nested(k2 String, k4 Nested(k5 Int8)), k3 String), id Int8) +Tuple(\n arr Nested(k1 Nested(k2 String, k4 Nested(k5 Int8)), k3 String),\n id Int8) {"obj":{"arr":[{"k1":[],"k3":"qqq"},{"k1":[],"k3":"www"}],"id":1}} {"obj":{"arr":[{"k1":[{"k2":"aaa","k4":[]}],"k3":"eee"}],"id":2}} {"obj":{"arr":[{"k1":[{"k2":"bbb","k4":[{"k5":10}]},{"k2":"ccc","k4":[{"k5":20}]}],"k3":"rrr"}],"id":3}} diff --git a/tests/queries/0_stateless/01825_type_json_18.reference b/tests/queries/0_stateless/01825_type_json_18.reference index d93f9bda63c..d61baf5eb6f 100644 --- a/tests/queries/0_stateless/01825_type_json_18.reference +++ b/tests/queries/0_stateless/01825_type_json_18.reference @@ -1,2 +1,2 @@ -1 (1) Tuple(k1 Int8) -1 ([1,2]) Tuple(k1 Array(Int8)) +1 (1) Tuple(\n k1 Int8) +1 ([1,2]) Tuple(\n k1 Array(Int8)) diff --git a/tests/queries/0_stateless/01825_type_json_2.reference b/tests/queries/0_stateless/01825_type_json_2.reference index 8524035a3a4..790d825a894 100644 --- a/tests/queries/0_stateless/01825_type_json_2.reference +++ b/tests/queries/0_stateless/01825_type_json_2.reference @@ -1,24 +1,24 @@ -1 (1,2,0) Tuple(k1 Int8, k2 Int8, k3 Int8) -2 (0,3,4) Tuple(k1 Int8, k2 Int8, k3 Int8) +1 (1,2,0) Tuple(\n k1 Int8,\n k2 Int8,\n k3 Int8) +2 (0,3,4) Tuple(\n k1 Int8,\n k2 Int8,\n k3 Int8) 1 1 2 0 2 0 3 4 -1 (1,2,'0') Tuple(k1 Int8, k2 Int8, k3 String) -2 (0,3,'4') Tuple(k1 Int8, k2 Int8, k3 String) -3 (0,0,'10') Tuple(k1 Int8, k2 Int8, k3 String) -4 (0,5,'str') Tuple(k1 Int8, k2 Int8, k3 String) +1 (1,2,'0') Tuple(\n k1 Int8,\n k2 Int8,\n k3 String) +2 (0,3,'4') Tuple(\n k1 Int8,\n k2 Int8,\n k3 String) +3 (0,0,'10') Tuple(\n k1 Int8,\n k2 Int8,\n k3 String) +4 (0,5,'str') Tuple(\n k1 Int8,\n k2 Int8,\n k3 String) 1 1 2 0 2 0 3 4 3 0 0 10 4 0 5 str ============ -1 ([1,2,3.3]) Tuple(k1 Array(Float64)) +1 ([1,2,3.3]) Tuple(\n k1 Array(Float64)) 1 [1,2,3.3] -1 (['1','2','3.3']) Tuple(k1 Array(String)) -2 (['a','4','b']) Tuple(k1 Array(String)) +1 (['1','2','3.3']) Tuple(\n k1 Array(String)) +2 (['a','4','b']) Tuple(\n k1 Array(String)) 1 ['1','2','3.3'] 2 ['a','4','b'] ============ -1 ([(11,0,0),(0,22,0)]) Tuple(k1 Nested(k2 Int8, k3 Int8, k4 Int8)) -2 ([(0,33,0),(0,0,44),(0,55,66)]) Tuple(k1 Nested(k2 Int8, k3 Int8, k4 Int8)) +1 ([(11,0,0),(0,22,0)]) Tuple(\n k1 Nested(k2 Int8, k3 Int8, k4 Int8)) +2 ([(0,33,0),(0,0,44),(0,55,66)]) Tuple(\n k1 Nested(k2 Int8, k3 Int8, k4 Int8)) 1 [11,0] [0,22] [0,0] 2 [0,0,0] [33,0,55] [0,44,66] diff --git a/tests/queries/0_stateless/01825_type_json_3.reference.j2 b/tests/queries/0_stateless/01825_type_json_3.reference.j2 index 23f38b74fd1..8646cf48872 100644 --- a/tests/queries/0_stateless/01825_type_json_3.reference.j2 +++ b/tests/queries/0_stateless/01825_type_json_3.reference.j2 @@ -1,17 +1,17 @@ {% for engine in ["ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_3/t_json_3', 'r1') ORDER BY tuple()", "Memory"] -%} -1 ('',0) Tuple(k1 String, k2 Int8) -2 ('v1',2) Tuple(k1 String, k2 Int8) +1 ('',0) Tuple(\n k1 String,\n k2 Int8) +2 ('v1',2) Tuple(\n k1 String,\n k2 Int8) 1 0 2 v1 2 ======== -1 ([]) Tuple(k1 Nested(k2 String, k3 String)) -2 ([('v1','v3'),('v4','')]) Tuple(k1 Nested(k2 String, k3 String)) +1 ([]) Tuple(\n k1 Nested(k2 String, k3 String)) +2 ([('v1','v3'),('v4','')]) Tuple(\n k1 Nested(k2 String, k3 String)) 1 [] [] 2 ['v1','v4'] ['v3',''] -1 ([]) Tuple(k1 Nested(k2 String, k3 String)) -2 ([('v1','v3'),('v4','')]) Tuple(k1 Nested(k2 String, k3 String)) -3 ([]) Tuple(k1 Nested(k2 String, k3 String)) -4 ([]) Tuple(k1 Nested(k2 String, k3 String)) +1 ([]) Tuple(\n k1 Nested(k2 String, k3 String)) +2 ([('v1','v3'),('v4','')]) Tuple(\n k1 Nested(k2 String, k3 String)) +3 ([]) Tuple(\n k1 Nested(k2 String, k3 String)) +4 ([]) Tuple(\n k1 Nested(k2 String, k3 String)) 1 [] [] 2 ['v1','v4'] ['v3',''] 3 [] [] @@ -26,9 +26,9 @@ data Tuple(k1 Nested(k2 String, k3 String)) 3 [] [] 4 [] [] ======== -1 ((1,'foo'),[]) Tuple(k1 Tuple(k2 Int8, k3 String), k4 Array(Int8)) -2 ((0,''),[1,2,3]) Tuple(k1 Tuple(k2 Int8, k3 String), k4 Array(Int8)) -3 ((10,''),[]) Tuple(k1 Tuple(k2 Int8, k3 String), k4 Array(Int8)) +1 ((1,'foo'),[]) Tuple(\n k1 Tuple(\n k2 Int8,\n k3 String),\n k4 Array(Int8)) +2 ((0,''),[1,2,3]) Tuple(\n k1 Tuple(\n k2 Int8,\n k3 String),\n k4 Array(Int8)) +3 ((10,''),[]) Tuple(\n k1 Tuple(\n k2 Int8,\n k3 String),\n k4 Array(Int8)) 1 1 foo [] 2 0 [1,2,3] 3 10 [] diff --git a/tests/queries/0_stateless/01825_type_json_4.reference b/tests/queries/0_stateless/01825_type_json_4.reference index 1b23bf2213e..58b1d067a2b 100644 --- a/tests/queries/0_stateless/01825_type_json_4.reference +++ b/tests/queries/0_stateless/01825_type_json_4.reference @@ -1,5 +1,5 @@ Code: 645 Code: 15 Code: 53 -1 ('v1') Tuple(k1 String) +1 ('v1') Tuple(\n k1 String) 1 v1 diff --git a/tests/queries/0_stateless/01825_type_json_5.reference b/tests/queries/0_stateless/01825_type_json_5.reference index 4ac0aa26ffd..3c21f2840a2 100644 --- a/tests/queries/0_stateless/01825_type_json_5.reference +++ b/tests/queries/0_stateless/01825_type_json_5.reference @@ -2,4 +2,4 @@ {"s":{"a.b":1,"a.c":2}} 1 [22,33] 2 qqq [44] -Tuple(k1 Int8, k2 Tuple(k3 String, k4 Array(Int8))) +Tuple(\n k1 Int8,\n k2 Tuple(\n k3 String,\n k4 Array(Int8))) diff --git a/tests/queries/0_stateless/01825_type_json_6.reference b/tests/queries/0_stateless/01825_type_json_6.reference index 7fcd2a40826..15e1ab3ac80 100644 --- a/tests/queries/0_stateless/01825_type_json_6.reference +++ b/tests/queries/0_stateless/01825_type_json_6.reference @@ -1,3 +1,3 @@ -Tuple(key String, out Nested(outputs Nested(index Int32, n Int8), type Int8, value Int8)) +Tuple(\n key String,\n out Nested(outputs Nested(index Int32, n Int8), type Int8, value Int8)) v1 [0,0] [1,2] [[],[1960131]] [[],[0]] v2 [1,1] [4,3] [[1881212],[]] [[1],[]] diff --git a/tests/queries/0_stateless/01825_type_json_7.reference b/tests/queries/0_stateless/01825_type_json_7.reference index 263f1688a91..cf6b32d73e8 100644 --- a/tests/queries/0_stateless/01825_type_json_7.reference +++ b/tests/queries/0_stateless/01825_type_json_7.reference @@ -1,4 +1,4 @@ -Tuple(categories Array(String), key String) +Tuple(\n categories Array(String),\n key String) v1 [] v2 ['foo','bar'] v3 [] diff --git a/tests/queries/0_stateless/01825_type_json_8.reference b/tests/queries/0_stateless/01825_type_json_8.reference index b64e6d0c9b9..27770317862 100644 --- a/tests/queries/0_stateless/01825_type_json_8.reference +++ b/tests/queries/0_stateless/01825_type_json_8.reference @@ -1,2 +1,2 @@ -([[(1,2),(3,4)],[(5,6)]]) Tuple(k1 Array(Nested(k2 Int8, k3 Int8))) -([([1,3,4,5],[6,7]),([8],[9,10,11])]) Tuple(k1 Nested(k2 Array(Int8), k3 Array(Int8))) +([[(1,2),(3,4)],[(5,6)]]) Tuple(\n k1 Array(Nested(k2 Int8, k3 Int8))) +([([1,3,4,5],[6,7]),([8],[9,10,11])]) Tuple(\n k1 Nested(k2 Array(Int8), k3 Array(Int8))) diff --git a/tests/queries/0_stateless/01825_type_json_9.reference b/tests/queries/0_stateless/01825_type_json_9.reference index a426b09a100..f58a64eda5a 100644 --- a/tests/queries/0_stateless/01825_type_json_9.reference +++ b/tests/queries/0_stateless/01825_type_json_9.reference @@ -1 +1 @@ -Tuple(foo Int8, k1 Int8, k2 Int8) +Tuple(\n foo Int8,\n k1 Int8,\n k2 Int8) diff --git a/tests/queries/0_stateless/01825_type_json_bools.reference b/tests/queries/0_stateless/01825_type_json_bools.reference index bed8c2ad2c3..6b4d2382dc2 100644 --- a/tests/queries/0_stateless/01825_type_json_bools.reference +++ b/tests/queries/0_stateless/01825_type_json_bools.reference @@ -1 +1 @@ -(1,0) Tuple(k1 UInt8, k2 UInt8) +(1,0) Tuple(\n k1 UInt8,\n k2 UInt8) diff --git a/tests/queries/0_stateless/01825_type_json_btc.reference b/tests/queries/0_stateless/01825_type_json_btc.reference index cee3b31a798..e85c0ef45bd 100644 --- a/tests/queries/0_stateless/01825_type_json_btc.reference +++ b/tests/queries/0_stateless/01825_type_json_btc.reference @@ -1,5 +1,5 @@ 100 -data Tuple(double_spend UInt8, fee Int32, hash String, inputs Nested(index Int8, prev_out Tuple(addr String, n Int16, script String, spending_outpoints Nested(n Int8, tx_index Int64), spent UInt8, tx_index Int64, type Int8, value Int64), script String, sequence Int64, witness String), lock_time Int32, out Nested(addr String, n Int8, script String, spending_outpoints Nested(n Int8, tx_index Int64), spent UInt8, tx_index Int64, type Int8, value Int64), rbf UInt8, relayed_by String, size Int16, time Int32, tx_index Int64, ver Int8, vin_sz Int8, vout_sz Int8, weight Int16) +data Tuple(\n double_spend UInt8,\n fee Int32,\n hash String,\n inputs Nested(index Int8, prev_out Tuple(addr String, n Int16, script String, spending_outpoints Nested(n Int8, tx_index Int64), spent UInt8, tx_index Int64, type Int8, value Int64), script String, sequence Int64, witness String),\n lock_time Int32,\n out Nested(addr String, n Int8, script String, spending_outpoints Nested(n Int8, tx_index Int64), spent UInt8, tx_index Int64, type Int8, value Int64),\n rbf UInt8,\n relayed_by String,\n size Int16,\n time Int32,\n tx_index Int64,\n ver Int8,\n vin_sz Int8,\n vout_sz Int8,\n weight Int16) 8174.56 2680 2.32 1 [[],[(0,359661801933760)]] diff --git a/tests/queries/0_stateless/01825_type_json_describe.reference b/tests/queries/0_stateless/01825_type_json_describe.reference index 629b60cb629..98b2bf8be83 100644 --- a/tests/queries/0_stateless/01825_type_json_describe.reference +++ b/tests/queries/0_stateless/01825_type_json_describe.reference @@ -1,3 +1,3 @@ data Object(\'json\') -data Tuple(k1 Int8) -data Tuple(k1 String, k2 Array(Int8)) +data Tuple(\n k1 Int8) +data Tuple(\n k1 String,\n k2 Array(Int8)) diff --git a/tests/queries/0_stateless/01825_type_json_distributed.reference b/tests/queries/0_stateless/01825_type_json_distributed.reference index 9ae85ac888c..9735fec2fe5 100644 --- a/tests/queries/0_stateless/01825_type_json_distributed.reference +++ b/tests/queries/0_stateless/01825_type_json_distributed.reference @@ -1,4 +1,4 @@ -(2,('qqq',[44,55])) Tuple(k1 Int8, k2 Tuple(k3 String, k4 Array(Int8))) -(2,('qqq',[44,55])) Tuple(k1 Int8, k2 Tuple(k3 String, k4 Array(Int8))) +(2,('qqq',[44,55])) Tuple(\n k1 Int8,\n k2 Tuple(\n k3 String,\n k4 Array(Int8))) +(2,('qqq',[44,55])) Tuple(\n k1 Int8,\n k2 Tuple(\n k3 String,\n k4 Array(Int8))) 2 qqq [44,55] 2 qqq [44,55] diff --git a/tests/queries/0_stateless/01825_type_json_field.reference b/tests/queries/0_stateless/01825_type_json_field.reference index b5637b1fbb7..8afd0110b63 100644 --- a/tests/queries/0_stateless/01825_type_json_field.reference +++ b/tests/queries/0_stateless/01825_type_json_field.reference @@ -1,12 +1,12 @@ 1 10 a -Tuple(a UInt8, s String) +Tuple(\n a UInt8,\n s String) 1 10 a 0 2 sss b 300 3 20 c 0 -Tuple(a String, b UInt16, s String) +Tuple(\n a String,\n b UInt16,\n s String) 1 10 a 0 2 sss b 300 3 20 c 0 4 30 400 5 0 qqq 0 foo -Tuple(a String, b UInt16, s String, t String) +Tuple(\n a String,\n b UInt16,\n s String,\n t String) diff --git a/tests/queries/0_stateless/01825_type_json_from_map.reference b/tests/queries/0_stateless/01825_type_json_from_map.reference index dbcf67faef3..90680ee383b 100644 --- a/tests/queries/0_stateless/01825_type_json_from_map.reference +++ b/tests/queries/0_stateless/01825_type_json_from_map.reference @@ -1,4 +1,4 @@ 800000 2000000 1400000 900000 800000 2000000 1400000 900000 -Tuple(col0 UInt64, col1 UInt64, col2 UInt64, col3 UInt64, col4 UInt64, col5 UInt64, col6 UInt64, col7 UInt64, col8 UInt64) +Tuple(\n col0 UInt64,\n col1 UInt64,\n col2 UInt64,\n col3 UInt64,\n col4 UInt64,\n col5 UInt64,\n col6 UInt64,\n col7 UInt64,\n col8 UInt64) 1600000 4000000 2800000 1800000 diff --git a/tests/queries/0_stateless/01825_type_json_in_array.reference b/tests/queries/0_stateless/01825_type_json_in_array.reference index c36a22e6951..82207f53a21 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.reference +++ b/tests/queries/0_stateless/01825_type_json_in_array.reference @@ -5,7 +5,7 @@ {"arr":{"k1":1,"k2":{"k3":2,"k4":3,"k5":""}}} {"arr":{"k1":2,"k2":{"k3":0,"k4":0,"k5":"foo"}}} {"arr":{"k1":3,"k2":{"k3":4,"k4":5,"k5":""}}} -Array(Tuple(k1 Int8, k2 Tuple(k3 Int8, k4 Int8, k5 String))) +Array(Tuple(\n k1 Int8,\n k2 Tuple(\n k3 Int8,\n k4 Int8,\n k5 String))) {"id":1,"arr":[{"k1":[{"k2":"aaa","k3":"bbb","k4":0},{"k2":"ccc","k3":"","k4":0}],"k5":{"k6":""}}]} {"id":2,"arr":[{"k1":[{"k2":"","k3":"ddd","k4":10},{"k2":"","k3":"","k4":20}],"k5":{"k6":"foo"}}]} 1 [['aaa','ccc']] [['bbb','']] [[0,0]] [''] @@ -14,7 +14,7 @@ Array(Tuple(k1 Int8, k2 Tuple(k3 Int8, k4 Int8, k5 String))) {"k1":{"k2":"","k3":"ddd","k4":10}} {"k1":{"k2":"aaa","k3":"bbb","k4":0}} {"k1":{"k2":"ccc","k3":"","k4":0}} -Tuple(k2 String, k3 String, k4 Int8) +Tuple(\n k2 String,\n k3 String,\n k4 Int8) {"arr":[{"x":1}]} {"arr":{"x":{"y":1},"t":{"y":2}}} {"arr":[1,{"y":1}]} diff --git a/tests/queries/0_stateless/01825_type_json_in_other_types.reference b/tests/queries/0_stateless/01825_type_json_in_other_types.reference index b94885a65ab..fa8af729cc7 100644 --- a/tests/queries/0_stateless/01825_type_json_in_other_types.reference +++ b/tests/queries/0_stateless/01825_type_json_in_other_types.reference @@ -1,4 +1,4 @@ -Tuple(String, Map(String, Array(Tuple(k1 Nested(k2 Int8, k3 Int8, k5 String), k4 String))), Tuple(k1 String, k2 Tuple(k3 String, k4 String))) +Tuple(String, Map(String, Array(Tuple(\n k1 Nested(k2 Int8, k3 Int8, k5 String),\n k4 String))), Tuple(\n k1 String,\n k2 Tuple(\n k3 String,\n k4 String))) ============= {"id":1,"data":["foo",{"aa":[{"k1":[{"k2":1,"k3":2,"k5":""},{"k2":0,"k3":3,"k5":""}],"k4":""},{"k1":[{"k2":4,"k3":0,"k5":""},{"k2":0,"k3":5,"k5":""},{"k2":6,"k3":0,"k5":""}],"k4":"qqq"}],"bb":[{"k1":[],"k4":"www"},{"k1":[{"k2":7,"k3":8,"k5":""},{"k2":9,"k3":10,"k5":""},{"k2":11,"k3":12,"k5":""}],"k4":""}]},{"k1":"aa","k2":{"k3":"bb","k4":"c"}}]} {"id":2,"data":["bar",{"aa":[{"k1":[{"k2":13,"k3":14,"k5":""},{"k2":15,"k3":16,"k5":""}],"k4":"www"}]},{"k1":"","k2":{"k3":"","k4":""}}]} diff --git a/tests/queries/0_stateless/01825_type_json_insert_select.reference b/tests/queries/0_stateless/01825_type_json_insert_select.reference index 6778da508f2..cb46a9c607e 100644 --- a/tests/queries/0_stateless/01825_type_json_insert_select.reference +++ b/tests/queries/0_stateless/01825_type_json_insert_select.reference @@ -1,10 +1,10 @@ -Tuple(k1 Int8, k2 String) +Tuple(\n k1 Int8,\n k2 String) 1 (1,'foo') -Tuple(k1 Int8, k2 String, k3 String) +Tuple(\n k1 Int8,\n k2 String,\n k3 String) 1 (1,'foo','') 2 (2,'bar','') 3 (3,'','aaa') -Tuple(arr Nested(k11 Int8, k22 String, k33 Int8), k1 Int8, k2 String, k3 String) +Tuple(\n arr Nested(k11 Int8, k22 String, k33 Int8),\n k1 Int8,\n k2 String,\n k3 String) 1 ([],1,'foo','') 2 ([],2,'bar','') 3 ([],3,'','aaa') @@ -12,7 +12,7 @@ Tuple(arr Nested(k11 Int8, k22 String, k33 Int8), k1 Int8, k2 String, k3 String) 5 ([(0,'str1',0)],0,'','') {"data":{"k1":1,"k10":[{"a":"1","b":"2","c":{"k11":""}},{"a":"2","b":"3","c":{"k11":""}}]}} {"data":{"k1":2,"k10":[{"a":"1","b":"2","c":{"k11":"haha"}}]}} -Tuple(k1 Int8, k10 Nested(a String, b String, c Tuple(k11 String))) +Tuple(\n k1 Int8,\n k10 Nested(a String, b String, c Tuple(k11 String))) {"data":{"k1":1,"k10":[{"a":"1","b":"2","c":{"k11":""}},{"a":"2","b":"3","c":{"k11":""}}]}} {"data":{"k1":2,"k10":[{"a":"1","b":"2","c":{"k11":"haha"}}]}} -Tuple(k1 Int8, k10 Nested(a String, b String, c Tuple(k11 String))) +Tuple(\n k1 Int8,\n k10 Nested(a String, b String, c Tuple(k11 String))) diff --git a/tests/queries/0_stateless/01825_type_json_missed_values.reference b/tests/queries/0_stateless/01825_type_json_missed_values.reference index b480493995b..2a4b3a6f671 100644 --- a/tests/queries/0_stateless/01825_type_json_missed_values.reference +++ b/tests/queries/0_stateless/01825_type_json_missed_values.reference @@ -1,2 +1,2 @@ -Tuple(foo Int8, k1 Int8, k2 Int8) +Tuple(\n foo Int8,\n k1 Int8,\n k2 Int8) 1 diff --git a/tests/queries/0_stateless/01825_type_json_multiple_files.reference b/tests/queries/0_stateless/01825_type_json_multiple_files.reference index b887abc8590..6dcdb00e139 100644 --- a/tests/queries/0_stateless/01825_type_json_multiple_files.reference +++ b/tests/queries/0_stateless/01825_type_json_multiple_files.reference @@ -4,11 +4,11 @@ {"data":{"k0":0,"k1":0,"k2":0,"k3":100,"k4":0,"k5":0}} {"data":{"k0":0,"k1":0,"k2":0,"k3":0,"k4":100,"k5":0}} {"data":{"k0":0,"k1":0,"k2":0,"k3":0,"k4":0,"k5":100}} -Tuple(k0 Int8, k1 Int8, k2 Int8, k3 Int8, k4 Int8, k5 Int8) +Tuple(\n k0 Int8,\n k1 Int8,\n k2 Int8,\n k3 Int8,\n k4 Int8,\n k5 Int8) {"data":{"k0":100,"k1":0,"k2":0}} {"data":{"k0":0,"k1":100,"k2":0}} {"data":{"k0":0,"k1":0,"k2":100}} -Tuple(k0 Int8, k1 Int8, k2 Int8) +Tuple(\n k0 Int8,\n k1 Int8,\n k2 Int8) {"data":{"k1":100,"k3":0}} {"data":{"k1":0,"k3":100}} -Tuple(k1 Int8, k3 Int8) +Tuple(\n k1 Int8,\n k3 Int8) diff --git a/tests/queries/0_stateless/01825_type_json_nbagames.reference b/tests/queries/0_stateless/01825_type_json_nbagames.reference index 5aa63dceb86..70df8f967f3 100644 --- a/tests/queries/0_stateless/01825_type_json_nbagames.reference +++ b/tests/queries/0_stateless/01825_type_json_nbagames.reference @@ -1,5 +1,5 @@ 1000 -data Tuple(_id Tuple(`$oid` String), date Tuple(`$date` String), teams Nested(abbreviation String, city String, home UInt8, name String, players Nested(ast Int8, blk Int8, drb Int8, fg Int8, fg3 Int8, fg3_pct String, fg3a Int8, fg_pct String, fga Int8, ft Int8, ft_pct String, fta Int8, mp String, orb Int8, pf Int8, player String, pts Int8, stl Int8, tov Int8, trb Int8), results Tuple(ast Int8, blk Int8, drb Int8, fg Int8, fg3 Int8, fg3_pct String, fg3a Int8, fg_pct String, fga Int8, ft Int8, ft_pct String, fta Int8, mp Int16, orb Int8, pf Int8, pts Int16, stl Int8, tov Int8, trb Int8), score Int16, won Int8)) +data Tuple(\n _id Tuple(\n `$oid` String),\n date Tuple(\n `$date` String),\n teams Nested(abbreviation String, city String, home UInt8, name String, players Nested(ast Int8, blk Int8, drb Int8, fg Int8, fg3 Int8, fg3_pct String, fg3a Int8, fg_pct String, fga Int8, ft Int8, ft_pct String, fta Int8, mp String, orb Int8, pf Int8, player String, pts Int8, stl Int8, tov Int8, trb Int8), results Tuple(ast Int8, blk Int8, drb Int8, fg Int8, fg3 Int8, fg3_pct String, fg3a Int8, fg_pct String, fga Int8, ft Int8, ft_pct String, fta Int8, mp Int16, orb Int8, pf Int8, pts Int16, stl Int8, tov Int8, trb Int8), score Int16, won Int8)) Boston Celtics 70 Los Angeles Lakers 64 Milwaukee Bucks 61 diff --git a/tests/queries/0_stateless/01825_type_json_nullable.reference b/tests/queries/0_stateless/01825_type_json_nullable.reference index 587fb1b1bc9..597ede47615 100644 --- a/tests/queries/0_stateless/01825_type_json_nullable.reference +++ b/tests/queries/0_stateless/01825_type_json_nullable.reference @@ -1,17 +1,17 @@ -1 (1,2,NULL) Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(Int8)) -2 (NULL,3,4) Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(Int8)) +1 (1,2,NULL) Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(Int8)) +2 (NULL,3,4) Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(Int8)) 1 1 2 \N 2 \N 3 4 -1 (1,2,NULL) Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(String)) -2 (NULL,3,'4') Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(String)) -3 (NULL,NULL,'10') Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(String)) -4 (NULL,5,'str') Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(String)) +1 (1,2,NULL) Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(String)) +2 (NULL,3,'4') Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(String)) +3 (NULL,NULL,'10') Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(String)) +4 (NULL,5,'str') Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(String)) 1 1 2 \N 2 \N 3 4 3 \N \N 10 4 \N 5 str ============ -1 ([(11,NULL,NULL),(NULL,22,NULL)]) Tuple(k1 Nested(k2 Nullable(Int8), k3 Nullable(Int8), k4 Nullable(Int8))) -2 ([(NULL,33,NULL),(NULL,NULL,44),(NULL,55,66)]) Tuple(k1 Nested(k2 Nullable(Int8), k3 Nullable(Int8), k4 Nullable(Int8))) +1 ([(11,NULL,NULL),(NULL,22,NULL)]) Tuple(\n k1 Nested(k2 Nullable(Int8), k3 Nullable(Int8), k4 Nullable(Int8))) +2 ([(NULL,33,NULL),(NULL,NULL,44),(NULL,55,66)]) Tuple(\n k1 Nested(k2 Nullable(Int8), k3 Nullable(Int8), k4 Nullable(Int8))) 1 [11,NULL] [NULL,22] [NULL,NULL] 2 [NULL,NULL,NULL] [33,NULL,55] [NULL,44,66] diff --git a/tests/queries/0_stateless/01825_type_json_parallel_insert.reference b/tests/queries/0_stateless/01825_type_json_parallel_insert.reference index 158d61d46f7..e93e0aeb956 100644 --- a/tests/queries/0_stateless/01825_type_json_parallel_insert.reference +++ b/tests/queries/0_stateless/01825_type_json_parallel_insert.reference @@ -1 +1 @@ -Tuple(k1 Int8, k2 String) 500000 +Tuple(\n k1 Int8,\n k2 String) 500000 diff --git a/tests/queries/0_stateless/01825_type_json_schema_inference.reference b/tests/queries/0_stateless/01825_type_json_schema_inference.reference index a1dd269f9b4..72e3b58b8a8 100644 --- a/tests/queries/0_stateless/01825_type_json_schema_inference.reference +++ b/tests/queries/0_stateless/01825_type_json_schema_inference.reference @@ -1,5 +1,5 @@ {"id":"1","obj":{"k1":1,"k2":{"k3":"2","k4":[{"k5":3,"k6":null},{"k5":4,"k6":null}]},"some":null},"s":"foo"} {"id":"2","obj":{"k1":null,"k2":{"k3":"str","k4":[{"k5":null,"k6":55}]},"some":42},"s":"bar"} -Tuple(k1 Nullable(Int8), k2 Tuple(k3 Nullable(String), k4 Nested(k5 Nullable(Int8), k6 Nullable(Int8))), some Nullable(Int8)) +Tuple(\n k1 Nullable(Int8),\n k2 Tuple(\n k3 Nullable(String),\n k4 Nested(k5 Nullable(Int8), k6 Nullable(Int8))),\n some Nullable(Int8)) {"id":"1","obj":"aaa","s":"foo"} {"id":"2","obj":"bbb","s":"bar"} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index fafefd72cb8..bd20d34b684 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -30,7 +30,7 @@ SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == SELECT '--'; SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; -SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_FOUND_COLUMN_IN_BLOCK } SELECT '--'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2; diff --git a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference index ba792ea9f74..ac114a03837 100644 --- a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference +++ b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference @@ -1,23 +1,33 @@ -┌─name─┬─type────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┐ -│ d │ Date │ │ │ │ │ │ -│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ -│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ -│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ -│ t │ Tuple(s String, a Array(Tuple(a UInt32, b UInt32))) │ │ │ │ ZSTD(1) │ │ -└──────┴─────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┘ -┌─name───────┬─type────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┬─is_subcolumn─┐ -│ d │ Date │ │ │ │ │ │ 0 │ -│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ 0 │ -│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 0 │ -│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ 0 │ -│ t │ Tuple(s String, a Array(Tuple(a UInt32, b UInt32))) │ │ │ │ ZSTD(1) │ │ 0 │ -│ n.null │ UInt8 │ │ │ It is a nullable column │ │ │ 1 │ -│ arr1.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ arr2.size0 │ UInt64 │ │ │ │ │ d + toIntervalDay(1) │ 1 │ -│ arr2.size1 │ Array(UInt64) │ │ │ │ │ d + toIntervalDay(1) │ 1 │ -│ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.a │ Array(Tuple(a UInt32, b UInt32)) │ │ │ │ │ │ 1 │ -│ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ -└────────────┴─────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┴──────────────┘ +┌─name─┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┐ +│ d │ Date │ │ │ │ │ │ +│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ +│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ +│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ +│ t │ Tuple( + s String, + a Array(Tuple( + a UInt32, + b UInt32))) │ │ │ │ ZSTD(1) │ │ +└──────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┘ +┌─name───────┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┬─is_subcolumn─┐ +│ d │ Date │ │ │ │ │ │ 0 │ +│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ 0 │ +│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 0 │ +│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ 0 │ +│ t │ Tuple( + s String, + a Array(Tuple( + a UInt32, + b UInt32))) │ │ │ │ ZSTD(1) │ │ 0 │ +│ n.null │ UInt8 │ │ │ It is a nullable column │ │ │ 1 │ +│ arr1.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ arr2.size0 │ UInt64 │ │ │ │ │ d + toIntervalDay(1) │ 1 │ +│ arr2.size1 │ Array(UInt64) │ │ │ │ │ d + toIntervalDay(1) │ 1 │ +│ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.a │ Array(Tuple( + a UInt32, + b UInt32)) │ │ │ │ │ │ 1 │ +│ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ +└────────────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference index 8a34751b071..2631199cbab 100644 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1 "foo" 1 execute: --allow_experimental_analyzer=1 --stage fetch_columns -"dummy_0" +"__table1.dummy" 0 execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.reference b/tests/queries/0_stateless/02051_symlinks_to_user_files.reference index d86bac9de59..2c94e483710 100644 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.reference +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.reference @@ -1 +1,2 @@ OK +OK diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index 0af71e4deee..eab44e74d88 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -8,13 +8,14 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # See 01658_read_file_to_string_column.sh user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -FILE_PATH="${user_files_path}/file/" +FILE_PATH="${user_files_path}/file" mkdir -p ${FILE_PATH} chmod 777 ${FILE_PATH} FILE="test_symlink_${CLICKHOUSE_DATABASE}" symlink_path=${FILE_PATH}/${FILE} +symlink_path_with_regex="${FILE_PATH}*/${FILE}" file_path=$CUR_DIR/${FILE} touch ${file_path} @@ -29,3 +30,4 @@ trap cleanup EXIT ${CLICKHOUSE_CLIENT} --query="insert into table function file('${symlink_path}', 'Values', 'a String') select 'OK'"; ${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String') order by a"; +${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path_with_regex}', 'Values', 'a String') order by a"; diff --git a/tests/queries/0_stateless/02149_external_schema_inference.reference b/tests/queries/0_stateless/02149_external_schema_inference.reference index ebc30e874da..194c8ca62cb 100644 --- a/tests/queries/0_stateless/02149_external_schema_inference.reference +++ b/tests/queries/0_stateless/02149_external_schema_inference.reference @@ -31,8 +31,8 @@ lotteryWin Float64 someRatio Float32 temperature Float32 randomBigNumber Int64 -measureUnits Array(Tuple(unit String, coef Float32)) -nestiness_a_b_c Tuple(d UInt32, e Array(UInt32)) +measureUnits Array(Tuple(\n unit String,\n coef Float32)) +nestiness_a_b_c Tuple(\n d UInt32,\n e Array(UInt32)) location Array(Int32) pi Float32 @@ -78,8 +78,8 @@ lotteryWin String someRatio String temperature String randomBigNumber String -measureUnits Tuple(unit Array(String), coef Array(String)) -nestiness_a_b_c Tuple(d String, e Array(String)) +measureUnits Tuple(\n unit Array(String),\n coef Array(String)) +nestiness_a_b_c Tuple(\n d String,\n e Array(String)) uuid String name String @@ -101,14 +101,14 @@ lotteryWin Float64 someRatio Float32 temperature Float32 randomBigNumber Int64 -measureunits Tuple(coef Array(Float32), unit Array(String)) -nestiness_a_b_c Tuple(d UInt32, e Array(UInt32)) +measureunits Tuple(\n coef Array(Float32),\n unit Array(String)) +nestiness_a_b_c Tuple(\n d UInt32,\n e Array(UInt32)) newFieldStr String newFieldInt Int32 newBool UInt8 identifier String -modules Array(Tuple(module_id UInt32, supply UInt32, temp UInt32, nodes Array(Tuple(node_id UInt32, opening_time UInt32, closing_time UInt32, current UInt32, coords_y Float32)))) +modules Array(Tuple(\n module_id UInt32,\n supply UInt32,\n temp UInt32,\n nodes Array(Tuple(\n node_id UInt32,\n opening_time UInt32,\n closing_time UInt32,\n current UInt32,\n coords_y Float32)))) Capnproto @@ -123,15 +123,15 @@ lc2 Nullable(String) lc3 Array(Nullable(String)) value UInt64 -nested Tuple(a Tuple(b UInt64, c Array(Array(UInt64))), d Array(Tuple(e Array(Array(Tuple(f UInt64, g UInt64))), h Array(Tuple(k Array(UInt64)))))) +nested Tuple(\n a Tuple(\n b UInt64,\n c Array(Array(UInt64))),\n d Array(Tuple(\n e Array(Array(Tuple(\n f UInt64,\n g UInt64))),\n h Array(Tuple(\n k Array(UInt64)))))) -nested Tuple(value Array(UInt64), array Array(Array(UInt64)), tuple Array(Tuple(one UInt64, two UInt64))) +nested Tuple(\n value Array(UInt64),\n array Array(Array(UInt64)),\n tuple Array(Tuple(\n one UInt64,\n two UInt64))) -a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(f UInt64))) +a Tuple(\n b UInt64,\n c Tuple(\n d UInt64,\n e Tuple(\n f UInt64))) nullable Nullable(UInt64) array Array(Nullable(UInt64)) -tuple Tuple(nullable Nullable(UInt64)) +tuple Tuple(\n nullable Nullable(UInt64)) int8 Int8 uint8 UInt8 @@ -151,8 +151,8 @@ datetime UInt32 datetime64 Int64 value UInt64 -tuple1 Tuple(one UInt64, two Tuple(three UInt64, four UInt64)) -tuple2 Tuple(nested1 Tuple(nested2 Tuple(x UInt64))) +tuple1 Tuple(\n one UInt64,\n two Tuple(\n three UInt64,\n four UInt64)) +tuple2 Tuple(\n nested1 Tuple(\n nested2 Tuple(\n x UInt64))) RawBLOB diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index 6d70c4682f5..ca634ac1701 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -37,30 +37,30 @@ d Array(Nullable(Int64)) JSONCompactEachRow c1 Nullable(Float64) c2 Array(Tuple(Nullable(Int64), Nullable(String))) -c3 Tuple(key Nullable(Int64), key2 Nullable(Int64)) +c3 Tuple(\n key Nullable(Int64),\n key2 Nullable(Int64)) c4 Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] (42,24) true c1 Nullable(Int64) c2 Array(Tuple(Nullable(Int64), Nullable(String))) -c3 Tuple(key1 Nullable(Int64), key2 Nullable(Int64)) +c3 Tuple(\n key1 Nullable(Int64),\n key2 Nullable(Int64)) c4 Nullable(Bool) \N [(1,'String'),(2,NULL)] (NULL,24) \N 32 [(2,'String 2'),(3,'hello')] (4242,2424) true JSONCompactEachRowWithNames a Nullable(Float64) b Array(Tuple(Nullable(Int64), Nullable(String))) -c Tuple(key Nullable(Int64), key2 Nullable(Int64)) +c Tuple(\n key Nullable(Int64),\n key2 Nullable(Int64)) d Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] (42,24) true JSONEachRow a Nullable(Float64) b Array(Tuple(Nullable(Int64), Nullable(String))) -c Tuple(key Nullable(Int64), key2 Nullable(Int64)) +c Tuple(\n key Nullable(Int64),\n key2 Nullable(Int64)) d Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] (42,24) true a Nullable(Int64) b Array(Tuple(Nullable(Int64), Nullable(String))) -c Tuple(key1 Nullable(Int64), key2 Nullable(Int64)) +c Tuple(\n key1 Nullable(Int64),\n key2 Nullable(Int64)) d Nullable(Bool) \N [(1,'String'),(2,NULL)] (NULL,24) \N 32 [(2,'String 2'),(3,'hello')] (4242,2424) true diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference index 4e020427ad0..ee83ed63dc1 100644 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference @@ -24,12 +24,12 @@ fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +tuple Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String)) map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +nested1 Array(Tuple(\n `1` Array(Nullable(UInt64)),\n `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(\n `1` Tuple(\n `1` Array(Array(Nullable(UInt64))),\n `2` Map(UInt64, Array(Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String))))),\n `2` Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) ArrowStream @@ -58,12 +58,12 @@ fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +tuple Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String)) map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +nested1 Array(Tuple(\n `1` Array(Nullable(UInt64)),\n `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(\n `1` Tuple(\n `1` Array(Array(Nullable(UInt64))),\n `2` Map(UInt64, Array(Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String))))),\n `2` Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) Parquet @@ -92,12 +92,12 @@ fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +tuple Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String)) map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +nested1 Array(Tuple(\n `1` Array(Nullable(UInt64)),\n `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(\n `1` Tuple(\n `1` Array(Array(Nullable(UInt64))),\n `2` Map(UInt64, Array(Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String))))),\n `2` Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) ORC @@ -126,12 +126,12 @@ fixed_string Nullable(String) Str: 0 100 Str: 1 200 array Array(Nullable(Int64)) -tuple Tuple(`1` Nullable(Int64), `2` Nullable(String)) +tuple Tuple(\n `1` Nullable(Int64),\n `2` Nullable(String)) map Map(String, Nullable(Int64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(Int64)), `2` Map(String, Nullable(Int64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(Int64))), `2` Map(Int64, Array(Tuple(`1` Nullable(Int64), `2` Nullable(String))))), `2` Nullable(Int8)) +nested1 Array(Tuple(\n `1` Array(Nullable(Int64)),\n `2` Map(String, Nullable(Int64)))) +nested2 Tuple(\n `1` Tuple(\n `1` Array(Array(Nullable(Int64))),\n `2` Map(Int64, Array(Tuple(\n `1` Nullable(Int64),\n `2` Nullable(String))))),\n `2` Nullable(Int8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) Native diff --git a/tests/queries/0_stateless/02179_map_cast_to_array.reference b/tests/queries/0_stateless/02179_map_cast_to_array.reference index 81bb9fba537..e87d1c69c1b 100644 --- a/tests/queries/0_stateless/02179_map_cast_to_array.reference +++ b/tests/queries/0_stateless/02179_map_cast_to_array.reference @@ -6,4 +6,4 @@ {1:{1:'1234'}} [(1,{1:1234})] [(1,{1:1234})] {1:{1:'1234'}} [(1,[(1,'1234')])] [(1,[(1,'1234')])] {1:{1:'1234'}} [(1,[(1,1234)])] [(1,[(1,1234)])] -[(1,'val1'),(2,'val2')] Array(Tuple(k UInt32, v String)) +[(1,'val1'),(2,'val2')] Array(Tuple(\n k UInt32,\n v String)) diff --git a/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference b/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference index d741391067c..0ff24b39709 100644 --- a/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference +++ b/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference @@ -11,7 +11,7 @@ QUERY id: 0 LIST id: 3, nodes: 1 CONSTANT id: 4, constant_value: \'Привет, World\', constant_value_type: String JOIN TREE - TABLE id: 5, table_name: system.one + TABLE id: 5, alias: __table1, table_name: system.one WHERE FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -54,7 +54,7 @@ QUERY id: 0 LIST id: 3, nodes: 1 CONSTANT id: 4, constant_value: \'Привет, World\', constant_value_type: String JOIN TREE - TABLE id: 5, table_name: system.one + TABLE id: 5, alias: __table1, table_name: system.one WHERE FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02227_union_match_by_name.reference b/tests/queries/0_stateless/02227_union_match_by_name.reference index 42b9b01a529..c28035fab49 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.reference +++ b/tests/queries/0_stateless/02227_union_match_by_name.reference @@ -4,15 +4,15 @@ EXPLAIN header = 1, optimize = 0 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 Expression (Project names) Header: avgWeighted(x, y) Nullable(Float64) Expression (Projection) - Header: avgWeighted(x_0, y_1) Nullable(Float64) + Header: avgWeighted(__table1.x, __table1.y) Nullable(Float64) Aggregating - Header: avgWeighted(x_0, y_1) Nullable(Float64) + Header: avgWeighted(__table1.x, __table1.y) Nullable(Float64) Expression (Before GROUP BY) - Header: x_0 Nullable(UInt8) - y_1 UInt8 + Header: __table1.x Nullable(UInt8) + __table1.y UInt8 Expression (Change column names to column identifiers) - Header: x_0 Nullable(UInt8) - y_1 UInt8 + Header: __table1.x Nullable(UInt8) + __table1.y UInt8 Union Header: x Nullable(UInt8) y UInt8 @@ -26,7 +26,7 @@ Header: avgWeighted(x, y) Nullable(Float64) Header: 255_UInt8 UInt8 1_UInt8 UInt8 Expression (Change column names to column identifiers) - Header: dummy_0 UInt8 + Header: __table3.dummy UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 Expression (Conversion before UNION) @@ -39,7 +39,7 @@ Header: avgWeighted(x, y) Nullable(Float64) Header: NULL_Nullable(Nothing) Nullable(Nothing) 1_UInt8 UInt8 Expression (Change column names to column identifiers) - Header: dummy_0 UInt8 + Header: __table5.dummy UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); diff --git a/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference index 2ecce985eb4..cd39bf8879b 100644 --- a/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference +++ b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference @@ -2,7 +2,7 @@ Arrow x Nullable(UInt64) arr1 Array(Nullable(UInt64)) arr2 Array(Array(Nullable(String))) -arr3 Array(Tuple(`1` Nullable(String), `2` Nullable(UInt64))) +arr3 Array(Tuple(\n `1` Nullable(String),\n `2` Nullable(UInt64))) 0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] \N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] 2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] @@ -12,7 +12,7 @@ ArrowStream x Nullable(UInt64) arr1 Array(Nullable(UInt64)) arr2 Array(Array(Nullable(String))) -arr3 Array(Tuple(`1` Nullable(String), `2` Nullable(UInt64))) +arr3 Array(Tuple(\n `1` Nullable(String),\n `2` Nullable(UInt64))) 0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] \N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] 2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] @@ -22,7 +22,7 @@ Parquet x Nullable(UInt64) arr1 Array(Nullable(UInt64)) arr2 Array(Array(Nullable(String))) -arr3 Array(Tuple(`1` Nullable(String), `2` Nullable(UInt64))) +arr3 Array(Tuple(\n `1` Nullable(String),\n `2` Nullable(UInt64))) 0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] \N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] 2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] @@ -32,7 +32,7 @@ ORC x Nullable(Int64) arr1 Array(Nullable(Int64)) arr2 Array(Array(Nullable(String))) -arr3 Array(Tuple(`1` Nullable(String), `2` Nullable(Int64))) +arr3 Array(Tuple(\n `1` Nullable(String),\n `2` Nullable(Int64))) 0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] \N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] 2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] diff --git a/tests/queries/0_stateless/02246_flatten_tuple.reference b/tests/queries/0_stateless/02246_flatten_tuple.reference index 0320150025d..ad0ca1fa03a 100644 --- a/tests/queries/0_stateless/02246_flatten_tuple.reference +++ b/tests/queries/0_stateless/02246_flatten_tuple.reference @@ -1,4 +1,4 @@ -([1,2],['a','b'],3,'c',4) Tuple(`t1.a` Array(UInt32), `t1.s` Array(String), b UInt32, `t2.k` String, `t2.v` UInt32) -Tuple(id Int8, obj Tuple(k1 Int8, k2 Tuple(k3 String, k4 Nested(k5 Int8, k6 Int8)), some Int8), s String) Tuple(id Int8, `obj.k1` Int8, `obj.k2.k3` String, `obj.k2.k4.k5` Array(Int8), `obj.k2.k4.k6` Array(Int8), `obj.some` Int8, s String) +([1,2],['a','b'],3,'c',4) Tuple(\n `t1.a` Array(UInt32),\n `t1.s` Array(String),\n b UInt32,\n `t2.k` String,\n `t2.v` UInt32) +Tuple(\n id Int8,\n obj Tuple(\n k1 Int8,\n k2 Tuple(\n k3 String,\n k4 Nested(k5 Int8, k6 Int8)),\n some Int8),\n s String) Tuple(\n id Int8,\n `obj.k1` Int8,\n `obj.k2.k3` String,\n `obj.k2.k4.k5` Array(Int8),\n `obj.k2.k4.k6` Array(Int8),\n `obj.some` Int8,\n s String) 1 1 2 [3,4] [0,0] 0 foo 2 0 str [0] [55] 42 bar diff --git a/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference b/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference index 5f330409b2a..21348493d1d 100644 --- a/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference +++ b/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference @@ -4,7 +4,7 @@ CREATE TABLE default.t_tuple_numeric\n(\n `t` Tuple(`1` Tuple(`2` Int32, `3` 2 3 4 2 3 4 2 3 4 -Tuple(`1` Tuple(`2` Int8, `3` Int8), `4` Int8) +Tuple(\n `1` Tuple(\n `2` Int8,\n `3` Int8),\n `4` Int8) {"t":{"1":{"2":2,"3":3},"4":4}} 2 3 4 (('value')) diff --git a/tests/queries/0_stateless/02287_type_object_convert.reference b/tests/queries/0_stateless/02287_type_object_convert.reference index 2df54dcbcbc..501536f1f3e 100644 --- a/tests/queries/0_stateless/02287_type_object_convert.reference +++ b/tests/queries/0_stateless/02287_type_object_convert.reference @@ -1,15 +1,15 @@ -1 (1) Tuple(x Nullable(Int8)) -1 (1,NULL) Tuple(x Nullable(Int8), y Nullable(Int8)) -2 (NULL,2) Tuple(x Nullable(Int8), y Nullable(Int8)) -1 (1,NULL) Tuple(x Nullable(Int8), y Nullable(Int8)) -2 (NULL,2) Tuple(x Nullable(Int8), y Nullable(Int8)) -3 (1,2) Tuple(x Nullable(Int8), y Nullable(Int8)) +1 (1) Tuple(\n x Nullable(Int8)) +1 (1,NULL) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) +2 (NULL,2) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) +1 (1,NULL) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) +2 (NULL,2) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) +3 (1,2) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) 1 1 \N 2 \N 2 3 1 2 -1 (1) Tuple(x Int8) -1 (1,0) Tuple(x Int8, y Int8) -2 (0,2) Tuple(x Int8, y Int8) +1 (1) Tuple(\n x Int8) +1 (1,0) Tuple(\n x Int8,\n y Int8) +2 (0,2) Tuple(\n x Int8,\n y Int8) {"x":1} {"x":1} {"x":[[1],[1,2]]} diff --git a/tests/queries/0_stateless/02303_query_kind.reference b/tests/queries/0_stateless/02303_query_kind.reference index 8d119fb22b2..53a0df682b2 100644 --- a/tests/queries/0_stateless/02303_query_kind.reference +++ b/tests/queries/0_stateless/02303_query_kind.reference @@ -2,35 +2,35 @@ clickhouse-client --allow_experimental_analyzer=1 --query_kind secondary_query - Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-client --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02313_avro_records_and_maps.reference b/tests/queries/0_stateless/02313_avro_records_and_maps.reference index 24fc635cdce..329462a4dda 100644 --- a/tests/queries/0_stateless/02313_avro_records_and_maps.reference +++ b/tests/queries/0_stateless/02313_avro_records_and_maps.reference @@ -1,8 +1,8 @@ -t Tuple(a Int32, b String) +t Tuple(\n a Int32,\n b String) (0,'String') (1,'String') (2,'String') -t Tuple(a Int32, b Tuple(c Int32, d Int32), e Array(Int32)) +t Tuple(\n a Int32,\n b Tuple(\n c Int32,\n d Int32),\n e Array(Int32)) (0,(1,2),[]) (1,(2,3),[0]) (2,(3,4),[0,1]) @@ -11,7 +11,7 @@ a.c Array(Int32) [0,1] [2,3] [1,2] [3,4] [2,3] [4,5] -a.b Array(Array(Tuple(c Int32, d Int32))) +a.b Array(Array(Tuple(\n c Int32,\n d Int32))) [[(0,1),(2,3)]] [[(1,2),(3,4)]] [[(2,3),(4,5)]] @@ -19,7 +19,7 @@ m Map(String, Int64) {'key_0':0} {'key_1':1} {'key_2':2} -m Map(String, Tuple(`1` Int64, `2` Array(Int64))) +m Map(String, Tuple(\n `1` Int64,\n `2` Array(Int64))) {'key_0':(0,[])} {'key_1':(1,[0])} {'key_2':(2,[0,1])} diff --git a/tests/queries/0_stateless/02314_avro_null_as_default.reference b/tests/queries/0_stateless/02314_avro_null_as_default.reference index ba38a15f924..e5d1b1c3752 100644 --- a/tests/queries/0_stateless/02314_avro_null_as_default.reference +++ b/tests/queries/0_stateless/02314_avro_null_as_default.reference @@ -1,5 +1,5 @@ a Nullable(Int64) -b Array(Tuple(c Nullable(Int64), d Nullable(String))) +b Array(Tuple(\n c Nullable(Int64),\n d Nullable(String))) 1 [(100,'Q'),(200,'W')] 0 0 diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index da07e94cead..69571551c2b 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -83,36 +83,36 @@ Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC === enable new analyzer === -- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, b ASC -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns -Sorting (Stream): a_1 ASC -Sorting (Stream): a_1 ASC -Sorting (Stream): a_1 ASC +Sorting (Stream): __table1.a ASC +Sorting (Stream): __table1.a ASC +Sorting (Stream): __table1.a ASC Sorting (Stream): a ASC -- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): a ASC, b ASC -- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause -Sorting (Stream): a_1 DESC, b_0 DESC -Sorting (Stream): a_1 DESC, b_0 DESC -Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC Sorting (Stream): a DESC, b DESC -- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause -Sorting (Stream): a_0 ASC, b_1 ASC -Sorting (Stream): a_0 ASC, b_1 ASC -Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): a ASC, b ASC -- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause -Sorting (Stream): a_1 DESC, b_0 DESC -Sorting (Stream): a_1 DESC, b_0 DESC -Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC Sorting (Stream): a DESC, b DESC -- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization -Sorting (Stream): a_0 ASC, b_1 ASC -Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): a ASC, b ASC diff --git a/tests/queries/0_stateless/02325_dates_schema_inference.reference b/tests/queries/0_stateless/02325_dates_schema_inference.reference index a37360dae62..c8eebd3262e 100644 --- a/tests/queries/0_stateless/02325_dates_schema_inference.reference +++ b/tests/queries/0_stateless/02325_dates_schema_inference.reference @@ -5,14 +5,14 @@ x Nullable(DateTime64(9)) x Array(Nullable(Date)) x Array(Nullable(DateTime64(9))) x Array(Nullable(DateTime64(9))) -x Tuple(date1 Nullable(DateTime64(9)), date2 Nullable(Date)) +x Tuple(\n date1 Nullable(DateTime64(9)),\n date2 Nullable(Date)) x Array(Nullable(DateTime64(9))) x Array(Nullable(DateTime64(9))) x Nullable(DateTime64(9)) x Array(Nullable(String)) x Nullable(String) x Array(Nullable(String)) -x Tuple(key1 Array(Array(Nullable(DateTime64(9)))), key2 Array(Array(Nullable(String)))) +x Tuple(\n key1 Array(Array(Nullable(DateTime64(9)))),\n key2 Array(Array(Nullable(String)))) CSV c1 Nullable(Date) c1 Nullable(DateTime64(9)) diff --git a/tests/queries/0_stateless/02326_settings_changes_system_table.reference b/tests/queries/0_stateless/02326_settings_changes_system_table.reference index c4a3c71edfd..1c8c4fa1880 100644 --- a/tests/queries/0_stateless/02326_settings_changes_system_table.reference +++ b/tests/queries/0_stateless/02326_settings_changes_system_table.reference @@ -1,3 +1,3 @@ version String -changes Array(Tuple(name String, previous_value String, new_value String, reason String)) +changes Array(Tuple(\n name String,\n previous_value String,\n new_value String,\n reason String)) 22.5 [('memory_overcommit_ratio_denominator','0','1073741824','Enable memory overcommit feature by default'),('memory_overcommit_ratio_denominator_for_user','0','1073741824','Enable memory overcommit feature by default')] diff --git a/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference index a0e0f8f6b5e..d190476a7da 100644 --- a/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference +++ b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference @@ -1,12 +1,12 @@ JSONEachRow x Nullable(Int64) x Array(Nullable(Int64)) -x Tuple(a Array(Nullable(Int64))) -x Tuple(a Array(Nullable(Int64)), b Array(Nullable(Int64))) +x Tuple(\n a Array(Nullable(Int64))) +x Tuple(\n a Array(Nullable(Int64)),\n b Array(Nullable(Int64))) x Nullable(Float64) x Nullable(Float64) x Array(Nullable(Float64)) -x Tuple(a Array(Nullable(Int64)), b Array(Nullable(Float64))) +x Tuple(\n a Array(Nullable(Int64)),\n b Array(Nullable(Float64))) CSV c1 Nullable(Int64) c1 Array(Nullable(Int64)) diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.reference b/tests/queries/0_stateless/02342_analyzer_compound_types.reference index 51e0bbe6e92..c384b548473 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.reference +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.reference @@ -8,33 +8,33 @@ Constant tuple Tuple -- id UInt64 -value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) 0 (('value_0_level_1','value_1_level_1'),'value_1_level_0') -- id UInt64 -value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) 0 (('value_0_level_1','value_1_level_1'),'value_1_level_0') -- -value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +value.value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String) value.value_1_level_0 String ('value_0_level_1','value_1_level_1') value_1_level_0 -- -alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -alias_value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) +alias_value.value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String) alias_value.value_1_level_0 String (('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 -- -alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -alias_value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) +alias_value.value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String) alias_value.value_1_level_0 String (('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 -- -alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +alias_value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) toString(alias_value.value_0_level_0) String toString(alias_value.value_1_level_0) String (('value_0_level_1','value_1_level_1'),'value_1_level_0') (\'value_0_level_1\',\'value_1_level_1\') value_1_level_0 -- -value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +value.value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String) value.value_1_level_0 String ('value_0_level_1','value_1_level_1') value_1_level_0 -- @@ -46,17 +46,17 @@ value.value_0_level_0.value_0_level_1 String value.value_0_level_0.value_1_level_1 String value_0_level_1 value_1_level_1 -- -alias_value Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_1 String,\n value_1_level_1 String) alias_value.value_0_level_1 String alias_value.value_1_level_1 String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- -alias_value Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_1 String,\n value_1_level_1 String) alias_value.value_0_level_1 String alias_value.value_1_level_1 String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- -alias_value Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_1 String,\n value_1_level_1 String) toString(alias_value.value_0_level_1) String toString(alias_value.value_1_level_1) String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 diff --git a/tests/queries/0_stateless/02366_explain_query_tree.reference b/tests/queries/0_stateless/02366_explain_query_tree.reference index 769d7661e68..acbedbd0622 100644 --- a/tests/queries/0_stateless/02366_explain_query_tree.reference +++ b/tests/queries/0_stateless/02366_explain_query_tree.reference @@ -22,7 +22,7 @@ QUERY id: 0 COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: value, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.test_table + TABLE id: 3, alias: __table1, table_name: default.test_table -- QUERY id: 0 PROJECTION @@ -64,7 +64,7 @@ QUERY id: 0 CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 CONSTANT id: 10, constant_value: Array_[UInt64_1, UInt64_2, UInt64_3], constant_value_type: Array(UInt8) JOIN TREE - TABLE id: 11, table_name: default.test_table + TABLE id: 11, alias: __table1, table_name: default.test_table -- QUERY id: 0 WITH @@ -99,4 +99,4 @@ QUERY id: 0 COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5 CONSTANT id: 6, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.test_table + TABLE id: 5, alias: __table1, table_name: default.test_table diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index 5c9e39805b7..2c50d1028fe 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -8,7 +8,7 @@ Sorting (None) -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): a_0 ASC +Sorting (Global): __table1.a ASC Sorting (None) Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort @@ -36,8 +36,8 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): a_0 ASC -Sorting (Stream): a_0 ASC +Sorting (Global): __table1.a ASC +Sorting (Stream): __table1.a ASC Sorting (Stream): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) @@ -48,8 +48,8 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): plus(a_0, 1_UInt8) ASC -Sorting (Chunk): a_0 ASC +Sorting (Global): plus(__table1.a, 1_UInt8) ASC +Sorting (Chunk): __table1.a ASC Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 @@ -61,7 +61,7 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 Sorting (Global): plus(a, 1) ASC Sorting (Sorting for ORDER BY) -Sorting (Global): plus(a_0, 1_UInt8) ASC +Sorting (Global): plus(__table1.a, 1_UInt8) ASC Sorting (None) Sorting (Chunk): a ASC -- FilterStep preserves sort mode @@ -71,7 +71,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 Sorting (Chunk): a ASC -Sorting (Chunk): a_0 ASC +Sorting (Chunk): __table1.a ASC Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC @@ -79,7 +79,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC -Sorting (Chunk): a_0 ASC +Sorting (Chunk): __table1.a ASC Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC @@ -87,7 +87,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC -Sorting (Chunk): a_0 ASC +Sorting (Chunk): __table1.a ASC Sorting (Chunk): a ASC -- FilterStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 @@ -119,11 +119,11 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): a_0 ASC +Sorting (Global): __table1.a ASC Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): a_2 ASC -Sorting (Stream): a_2 ASC +Sorting (Global): __table3.a ASC +Sorting (Stream): __table3.a ASC Sorting (Stream): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) @@ -135,8 +135,8 @@ Sorting (Stream): a ASC, b ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) Sorting (Global): a ASC, b ASC Sorting (Sorting for ORDER BY) -Sorting (Global): x_2 ASC, y_3 ASC -Sorting (Stream): x_2 ASC, y_3 ASC +Sorting (Global): __table2.x ASC, __table2.y ASC +Sorting (Stream): __table2.x ASC, __table2.y ASC Sorting (Stream): a ASC, b ASC -- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1) -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 @@ -151,11 +151,11 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): plus(a_0, 1_UInt8) ASC -Sorting (Global): plus(a_3, 1_UInt8) ASC +Sorting (Global): plus(__table1.a, 1_UInt8) ASC +Sorting (Global): plus(__table3.a, 1_UInt8) ASC Sorting (Sorting for ORDER BY) -Sorting (Global): plus(a_3, 1_UInt8) ASC -Sorting (Chunk): a_3 ASC +Sorting (Global): plus(__table3.a, 1_UInt8) ASC +Sorting (Chunk): __table3.a ASC Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a @@ -167,6 +167,6 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): a_0 ASC -Sorting (Stream): a_0 ASC +Sorting (Global): __table1.a ASC +Sorting (Stream): __table1.a ASC Sorting (Stream): a ASC diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference index f8b18e6df15..fd5bc7d4ae8 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.reference +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -13,7 +13,7 @@ concat(\'Value_1\', \'Value_2\') String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)')); -CAST((1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(id UInt64, value String) +CAST((1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(\n id UInt64,\n value String) SELECT 'Columns'; Columns DESCRIBE (SELECT test_table.id, test_table.id, id FROM test_table); @@ -77,45 +77,45 @@ e String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.id, a.value); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) a.id UInt64 a.value String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.*); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) a.id UInt64 a.value String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT id); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) a.value String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) a.id UInt64 SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value APPLY toString); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) toString(a.id) String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value APPLY x -> toString(x)); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) toString(a.id) String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, untuple(a)); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) tupleElement(a, \'id\') UInt64 tupleElement(a, \'value\') String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, untuple(a) AS b); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) b.id UInt64 b.value String SELECT 'Columns with aliases'; @@ -199,63 +199,63 @@ arrayMap(lambda(tuple(x), toString(id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.*, [1,2,3])); -compound_value Tuple(id UInt64) +compound_value Tuple(\n id UInt64) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY x -> x, [1,2,3])); -compound_value Tuple(id UInt64) +compound_value Tuple(\n id UInt64) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY toString, [1,2,3])); -compound_value Tuple(id UInt64) +compound_value Tuple(\n id UInt64) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY x -> toString(x), [1,2,3])); -compound_value Tuple(id UInt64) +compound_value Tuple(\n id UInt64) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value, [1,2,3])); -compound_value Tuple(id UInt64, value String) +compound_value Tuple(\n id UInt64,\n value String) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY x -> x, [1,2,3])); -compound_value Tuple(id UInt64, value String) +compound_value Tuple(\n id UInt64,\n value String) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY toString, [1,2,3])); -compound_value Tuple(id UInt64, value String) +compound_value Tuple(\n id UInt64,\n value String) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY x -> toString(x), [1,2,3])); -compound_value Tuple(id UInt64, value String) +compound_value Tuple(\n id UInt64,\n value String) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a), [1,2,3]) FROM test_table); -a Tuple(id UInt64) +a Tuple(\n id UInt64) arrayMap(lambda(tuple(x), tupleElement(a, \'id\')), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a) AS untupled_value, [1,2,3]) FROM test_table); -a Tuple(id UInt64) +a Tuple(\n id UInt64) arrayMap(untupled_value, [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value, [1,2,3]) FROM test_table); -a Tuple(id UInt64) +a Tuple(\n id UInt64) untupled_value.id UInt64 arrayMap(lambda(tuple(x), untupled_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value AS untupled_value_in_lambda, [1,2,3]) FROM test_table); -a Tuple(id UInt64) +a Tuple(\n id UInt64) untupled_value.id UInt64 arrayMap(untupled_value_in_lambda, [1, 2, 3]) Array(UInt64) SELECT 'Standalone lambda'; @@ -285,13 +285,13 @@ arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) SELECT '--'; -- DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b); -c Tuple(a UInt8, b UInt8) +c Tuple(\n a UInt8,\n b UInt8) c.a UInt8 c.b UInt8 SELECT '--'; -- DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.*); -c Tuple(a UInt8, b UInt8) +c Tuple(\n a UInt8,\n b UInt8) c.a UInt8 c.b UInt8 SELECT '--'; @@ -311,13 +311,13 @@ arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) SELECT '--'; -- DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b UNION DISTINCT SELECT 1, 2) AS c, c.a, c.b); -c Tuple(a UInt8, b UInt8) +c Tuple(\n a UInt8,\n b UInt8) c.a UInt8 c.b UInt8 SELECT '--'; -- DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b UNION DISTINCT SELECT 1, 2) AS c, c.*); -c Tuple(a UInt8, b UInt8) +c Tuple(\n a UInt8,\n b UInt8) c.a UInt8 c.b UInt8 SELECT '--'; diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index dd5c9d4616e..5dd39c39852 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -2,51 +2,51 @@ Expression Header: key String value String Join - Header: key_0 String - value_1 String + Header: __table1.key String + __table3.value String Expression - Header: key_0 String + Header: __table1.key String ReadFromStorage Header: dummy UInt8 Union - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String Expression - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String ReadFromStorage Header: dummy UInt8 Expression - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String ReadFromStorage Header: dummy UInt8 Expression Header: key String value String Join - Header: key_0 String - key_2 String - value_1 String + Header: __table1.key String + __table3.key String + __table3.value String Sorting - Header: key_0 String + Header: __table1.key String Expression - Header: key_0 String + Header: __table1.key String ReadFromStorage Header: dummy UInt8 Sorting - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String Union - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String Expression - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String ReadFromStorage Header: dummy UInt8 Expression - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String ReadFromStorage Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference b/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference index 57cafb6c8e0..3f4eeac37b3 100644 --- a/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference +++ b/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference @@ -1,3 +1,3 @@ x Array(Array(Nullable(Int64))) x Tuple(Array(Array(Nullable(Int64))), Nullable(Int64)) -x Tuple(key Array(Nullable(Int64))) +x Tuple(\n key Array(Nullable(Int64))) diff --git a/tests/queries/0_stateless/02421_explain_subquery.sql b/tests/queries/0_stateless/02421_explain_subquery.sql index 4b970f81219..2970003cb1c 100644 --- a/tests/queries/0_stateless/02421_explain_subquery.sql +++ b/tests/queries/0_stateless/02421_explain_subquery.sql @@ -34,7 +34,7 @@ DROP TABLE t1; SET allow_experimental_analyzer = 1; -SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: number__ UInt64%'; +SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: \_\_table1.number UInt64%'; SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; SELECT count() > 0 FROM (EXPLAIN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; SELECT count() > 0 FROM (EXPLAIN CURRENT TRANSACTION); diff --git a/tests/queries/0_stateless/02421_type_json_empty_parts.reference b/tests/queries/0_stateless/02421_type_json_empty_parts.reference index f360b4b92cd..3c1d2aafec1 100644 --- a/tests/queries/0_stateless/02421_type_json_empty_parts.reference +++ b/tests/queries/0_stateless/02421_type_json_empty_parts.reference @@ -3,24 +3,24 @@ Collapsing 0 id UInt64 s Int8 -data Tuple(_dummy UInt8) +data Tuple(\n _dummy UInt8) DELETE all 2 1 id UInt64 -data Tuple(k1 String, k2 String) +data Tuple(\n k1 String,\n k2 String) 0 0 id UInt64 -data Tuple(_dummy UInt8) +data Tuple(\n _dummy UInt8) TTL 1 1 id UInt64 d Date -data Tuple(k1 String, k2 String) +data Tuple(\n k1 String,\n k2 String) 0 0 id UInt64 d Date -data Tuple(_dummy UInt8) +data Tuple(\n _dummy UInt8) diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.reference b/tests/queries/0_stateless/02451_order_by_monotonic.reference index 05f20a9bad8..4b2f9f7e227 100644 --- a/tests/queries/0_stateless/02451_order_by_monotonic.reference +++ b/tests/queries/0_stateless/02451_order_by_monotonic.reference @@ -4,19 +4,19 @@ 2022-09-09 12:00:00 0x 2022-09-09 12:00:00 1 2022-09-09 12:00:00 1x - Prefix sort description: toStartOfMinute(t_0) ASC - Result sort description: toStartOfMinute(t_0) ASC, c1_1 ASC - Prefix sort description: toStartOfMinute(t_0) ASC - Result sort description: toStartOfMinute(t_0) ASC - Prefix sort description: negate(a_0) ASC - Result sort description: negate(a_0) ASC - Prefix sort description: negate(a_0) ASC, negate(b_1) ASC - Result sort description: negate(a_0) ASC, negate(b_1) ASC - Prefix sort description: a_0 DESC, negate(b_1) ASC - Result sort description: a_0 DESC, negate(b_1) ASC - Prefix sort description: negate(a_0) ASC, b_1 DESC - Result sort description: negate(a_0) ASC, b_1 DESC - Prefix sort description: negate(a_0) ASC - Result sort description: negate(a_0) ASC, b_1 ASC - Prefix sort description: a_0 ASC - Result sort description: a_0 ASC, negate(b_1) ASC + Prefix sort description: toStartOfMinute(__table1.t) ASC + Result sort description: toStartOfMinute(__table1.t) ASC, __table1.c1 ASC + Prefix sort description: toStartOfMinute(__table1.t) ASC + Result sort description: toStartOfMinute(__table1.t) ASC + Prefix sort description: negate(__table1.a) ASC + Result sort description: negate(__table1.a) ASC + Prefix sort description: negate(__table1.a) ASC, negate(__table1.b) ASC + Result sort description: negate(__table1.a) ASC, negate(__table1.b) ASC + Prefix sort description: __table1.a DESC, negate(__table1.b) ASC + Result sort description: __table1.a DESC, negate(__table1.b) ASC + Prefix sort description: negate(__table1.a) ASC, __table1.b DESC + Result sort description: negate(__table1.a) ASC, __table1.b DESC + Prefix sort description: negate(__table1.a) ASC + Result sort description: negate(__table1.a) ASC, __table1.b ASC + Prefix sort description: __table1.a ASC + Result sort description: __table1.a ASC, negate(__table1.b) ASC diff --git a/tests/queries/0_stateless/02475_bson_each_row_format.reference b/tests/queries/0_stateless/02475_bson_each_row_format.reference index f90867d92b1..5659e5201b1 100644 --- a/tests/queries/0_stateless/02475_bson_each_row_format.reference +++ b/tests/queries/0_stateless/02475_bson_each_row_format.reference @@ -166,7 +166,7 @@ Tuple ('Hello',4) OK OK -tuple Tuple(x Nullable(Int64), s Nullable(String)) +tuple Tuple(\n x Nullable(Int64),\n s Nullable(String)) (0,'Hello') (1,'Hello') (2,'Hello') @@ -214,7 +214,7 @@ Nested types [[0,1,2],[0,1,2,3]] ((3,'Hello'),'Hello') {'a':{'a.a':3,'a.b':4},'b':{'b.a':3,'b.b':4}} [[0,1,2,3],[0,1,2,3,4]] ((4,'Hello'),'Hello') {'a':{'a.a':4,'a.b':5},'b':{'b.a':4,'b.b':5}} nested1 Array(Array(Nullable(Int64))) -nested2 Tuple(Tuple(x Nullable(Int64), s Nullable(String)), Nullable(String)) +nested2 Tuple(Tuple(\n x Nullable(Int64),\n s Nullable(String)), Nullable(String)) nested3 Map(String, Map(String, Nullable(Int64))) [[],[0]] ((0,'Hello'),'Hello') {'a':{'a.a':0,'a.b':1},'b':{'b.a':0,'b.b':1}} [[0],[0,1]] ((1,'Hello'),'Hello') {'a':{'a.a':1,'a.b':2},'b':{'b.a':1,'b.b':2}} diff --git a/tests/queries/0_stateless/02476_fuse_sum_count.reference b/tests/queries/0_stateless/02476_fuse_sum_count.reference index 43a39e8b7e5..1eb156743b0 100644 --- a/tests/queries/0_stateless/02476_fuse_sum_count.reference +++ b/tests/queries/0_stateless/02476_fuse_sum_count.reference @@ -21,7 +21,7 @@ QUERY id: 0 LIST id: 7, nodes: 1 COLUMN id: 4, column_name: a, result_type: Nullable(Int8), source_id: 5 JOIN TREE - TABLE id: 5, table_name: default.fuse_tbl + TABLE id: 5, alias: __table1, table_name: default.fuse_tbl QUERY id: 0 PROJECTION COLUMNS sum(b) Int64 @@ -59,7 +59,7 @@ QUERY id: 0 COLUMN id: 6, column_name: b, result_type: Int8, source_id: 7 CONSTANT id: 18, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE id: 7, table_name: default.fuse_tbl + TABLE id: 7, alias: __table1, table_name: default.fuse_tbl QUERY id: 0 PROJECTION COLUMNS sum(plus(a, 1)) Nullable(Int64) @@ -138,7 +138,7 @@ QUERY id: 0 LIST id: 39, nodes: 1 COLUMN id: 6, column_name: a, result_type: Nullable(Int8), source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.fuse_tbl + TABLE id: 7, alias: __table1, table_name: default.fuse_tbl QUERY id: 0 PROJECTION COLUMNS multiply(avg(b), 3) Float64 @@ -215,14 +215,14 @@ QUERY id: 0 COLUMN id: 10, column_name: b, result_type: Int8, source_id: 11 CONSTANT id: 37, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - QUERY id: 11, is_subquery: 1 + QUERY id: 11, alias: __table1, is_subquery: 1 PROJECTION COLUMNS b Int8 PROJECTION LIST id: 38, nodes: 1 COLUMN id: 39, column_name: b, result_type: Int8, source_id: 40 JOIN TREE - TABLE id: 40, table_name: default.fuse_tbl + TABLE id: 40, alias: __table2, table_name: default.fuse_tbl QUERY id: 0 PROJECTION COLUMNS sum(b) Int64 @@ -246,14 +246,14 @@ QUERY id: 0 COLUMN id: 6, column_name: b, result_type: Int64, source_id: 7 CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - QUERY id: 7, is_subquery: 1 + QUERY id: 7, alias: __table1, is_subquery: 1 PROJECTION COLUMNS b Int64 PROJECTION LIST id: 12, nodes: 1 COLUMN id: 13, column_name: x, result_type: Int64, source_id: 14 JOIN TREE - QUERY id: 14, is_subquery: 1 + QUERY id: 14, alias: __table2, is_subquery: 1 PROJECTION COLUMNS x Int64 count(b) UInt64 @@ -276,7 +276,7 @@ QUERY id: 0 COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21 CONSTANT id: 25, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE id: 21, table_name: default.fuse_tbl + TABLE id: 21, alias: __table3, table_name: default.fuse_tbl 0 0 nan 0 0 nan 45 10 4.5 Decimal(38, 0) UInt64 Float64 diff --git a/tests/queries/0_stateless/02477_fuse_quantiles.reference b/tests/queries/0_stateless/02477_fuse_quantiles.reference index 7c7d581f7fb..7603381416c 100644 --- a/tests/queries/0_stateless/02477_fuse_quantiles.reference +++ b/tests/queries/0_stateless/02477_fuse_quantiles.reference @@ -34,7 +34,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: Float64, source_id: 10 CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - QUERY id: 10, is_subquery: 1 + QUERY id: 10, alias: __table1, is_subquery: 1 PROJECTION COLUMNS b Float64 PROJECTION @@ -45,7 +45,7 @@ QUERY id: 0 COLUMN id: 18, column_name: x, result_type: Float64, source_id: 19 CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE - QUERY id: 19, is_subquery: 1 + QUERY id: 19, alias: __table2, is_subquery: 1 PROJECTION COLUMNS x Float64 quantile(0.9)(b) Float64 @@ -76,7 +76,7 @@ QUERY id: 0 COLUMN id: 29, column_name: b, result_type: Int32, source_id: 30 CONSTANT id: 34, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE id: 30, table_name: default.fuse_tbl + TABLE id: 30, alias: __table3, table_name: default.fuse_tbl GROUP BY LIST id: 35, nodes: 1 COLUMN id: 18, column_name: x, result_type: Float64, source_id: 19 diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference index ff5f7e5a687..649b037fafa 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference @@ -8,7 +8,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -26,7 +26,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -44,7 +44,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: notIn, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -62,7 +62,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: notIn, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -80,7 +80,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -106,7 +106,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql index 99f20290ff0..50248665bc9 100644 --- a/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql +++ b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql @@ -24,4 +24,4 @@ SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, SELECT '--'; -SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt); -- { serverError 403 } +SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt); -- { serverError 403, NOT_FOUND_COLUMN_IN_BLOCK } diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference index f4dd01bc184..6838dacc3b3 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference @@ -50,7 +50,7 @@ QUERY id: 0 COLUMN id: 5, column_name: b, result_type: String, source_id: 3 COLUMN id: 6, column_name: c, result_type: String, source_id: 3 JOIN TREE - TABLE_FUNCTION id: 3, table_function_name: mysql + TABLE_FUNCTION id: 3, alias: __table1, table_function_name: mysql ARGUMENTS LIST id: 7, nodes: 5 CONSTANT id: 8, constant_value: \'127.0.0.1:9004\', constant_value_type: String @@ -63,10 +63,10 @@ QUERY id: 0 SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3 SELECT - key AS key, - a AS a, - b AS b, - c AS c -FROM mysql(\'127.0.0.1:9004\', \'default\', foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) + __table1.key AS key, + __table1.a AS a, + __table1.b AS b, + __table1.c AS c +FROM mysql(\'127.0.0.1:9004\', \'default\', foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) AS __table1 --- 5 diff --git a/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference b/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference index b11f3e3a1d3..969ec320790 100644 --- a/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference +++ b/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference @@ -6,5 +6,5 @@ Order: a ASC, c ASC ReadFromMergeTree (default.tab) Aggregating - Order: a_0 ASC, c_2 ASC + Order: __table1.a ASC, __table1.c ASC ReadFromMergeTree (default.tab) diff --git a/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.reference b/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.reference index 22dda253066..a26773baae2 100644 --- a/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.reference +++ b/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.reference @@ -20,7 +20,7 @@ QUERY id: 0 LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 @@ -44,7 +44,7 @@ QUERY id: 0 LIST id: 10, nodes: 1 CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 7, table_function_name: numbers + TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.reference b/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.reference index 03722034708..9f9c1da5e88 100644 --- a/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.reference +++ b/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.reference @@ -17,7 +17,7 @@ QUERY id: 0, group_by_type: grouping_sets LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -103,7 +103,7 @@ QUERY id: 0, group_by_type: grouping_sets LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -180,7 +180,7 @@ QUERY id: 0, group_by_type: grouping_sets LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -253,7 +253,7 @@ QUERY id: 0, group_by_type: grouping_sets LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE_FUNCTION id: 3, table_function_name: numbers + TABLE_FUNCTION id: 3, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 4, nodes: 1 CONSTANT id: 5, constant_value: UInt64_1000, constant_value_type: UInt16 diff --git a/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.reference b/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.reference index eccf51501ed..23e91dc2703 100644 --- a/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.reference +++ b/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.reference @@ -16,7 +16,7 @@ QUERY id: 0 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 @@ -41,7 +41,7 @@ QUERY id: 0 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 @@ -69,7 +69,7 @@ QUERY id: 0 CONSTANT id: 12, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 13, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 14, nodes: 1 CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference b/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference index 5b808310f0e..01d7fa2a2cb 100644 --- a/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference +++ b/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference @@ -13,7 +13,7 @@ QUERY id: 0 LIST id: 6, nodes: 1 CONSTANT id: 7, constant_value: \'\', constant_value_type: String JOIN TREE - TABLE_FUNCTION id: 8, table_function_name: numbers + TABLE_FUNCTION id: 8, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference index 88f23334d31..d77fd1028f2 100644 --- a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference +++ b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference @@ -35,7 +35,7 @@ QUERY id: 0 CONSTANT id: 15, constant_value: \'other\', constant_value_type: String CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String JOIN TREE - TABLE id: 7, table_name: system.numbers + TABLE id: 7, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64 google @@ -78,7 +78,7 @@ QUERY id: 0 CONSTANT id: 17, constant_value: \'google\', constant_value_type: String CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String JOIN TREE - TABLE id: 9, table_name: system.numbers + TABLE id: 9, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64 other1 @@ -122,7 +122,7 @@ QUERY id: 0 CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String CONSTANT id: 19, constant_value: \'1\', constant_value_type: String JOIN TREE - TABLE id: 9, table_name: system.numbers + TABLE id: 9, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64 google1 @@ -169,7 +169,7 @@ QUERY id: 0 CONSTANT id: 20, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String CONSTANT id: 21, constant_value: \'1\', constant_value_type: String JOIN TREE - TABLE id: 11, table_name: system.numbers + TABLE id: 11, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64 google @@ -196,7 +196,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: String, source_id: 3 JOIN TREE - QUERY id: 3, alias: t1, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value String PROJECTION @@ -223,7 +223,7 @@ QUERY id: 0 CONSTANT id: 20, constant_value: \'google\', constant_value_type: String CONSTANT id: 21, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String JOIN TREE - TABLE id: 12, table_name: system.numbers + TABLE id: 12, alias: __table2, table_name: system.numbers LIMIT CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64 other @@ -250,7 +250,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: String, source_id: 3 JOIN TREE - QUERY id: 3, alias: t1, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value String PROJECTION @@ -274,7 +274,7 @@ QUERY id: 0 CONSTANT id: 18, constant_value: \'other\', constant_value_type: String CONSTANT id: 19, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String JOIN TREE - TABLE id: 10, table_name: system.numbers + TABLE id: 10, alias: __table2, table_name: system.numbers LIMIT CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64 google google @@ -341,7 +341,7 @@ QUERY id: 0 CONSTANT id: 17, constant_value: \'google\', constant_value_type: String CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String JOIN TREE - TABLE id: 9, table_name: system.numbers + TABLE id: 9, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64 other other @@ -402,7 +402,7 @@ QUERY id: 0 CONSTANT id: 15, constant_value: \'other\', constant_value_type: String CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String JOIN TREE - TABLE id: 7, table_name: system.numbers + TABLE id: 7, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64 other @@ -446,14 +446,14 @@ QUERY id: 0 CONSTANT id: 15, constant_value: \'other\', constant_value_type: String CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String JOIN TREE - QUERY id: 7, is_subquery: 1 + QUERY id: 7, alias: __table1, is_subquery: 1 PROJECTION COLUMNS number Nullable(Nothing) PROJECTION LIST id: 17, nodes: 1 CONSTANT id: 18, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE id: 19, table_name: system.numbers + TABLE id: 19, alias: __table2, table_name: system.numbers LIMIT CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64 other @@ -482,7 +482,7 @@ QUERY id: 0 CONSTANT id: 7, constant_value: Array_[\'google\', \'censor.net\', \'yahoo\'], constant_value_type: Array(String) CONSTANT id: 8, constant_value: \'other\', constant_value_type: String JOIN TREE - TABLE id: 5, table_name: system.numbers + TABLE id: 5, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt64 google @@ -514,6 +514,6 @@ QUERY id: 0 CONSTANT id: 9, constant_value: \'censor.net\', constant_value_type: String CONSTANT id: 10, constant_value: \'google\', constant_value_type: String JOIN TREE - TABLE id: 7, table_name: system.numbers + TABLE id: 7, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 11, constant_value: UInt64_10, constant_value_type: UInt64 diff --git a/tests/queries/0_stateless/02498_analyzer_settings_push_down.reference b/tests/queries/0_stateless/02498_analyzer_settings_push_down.reference index 583da07380e..f24edd96996 100644 --- a/tests/queries/0_stateless/02498_analyzer_settings_push_down.reference +++ b/tests/queries/0_stateless/02498_analyzer_settings_push_down.reference @@ -12,7 +12,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION @@ -23,7 +23,7 @@ QUERY id: 0 COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8 CONSTANT id: 9, constant_value: \'a\', constant_value_type: String JOIN TREE - TABLE id: 8, table_name: default.test_table + TABLE id: 8, alias: __table2, table_name: default.test_table SELECT '--'; -- EXPLAIN QUERY TREE SELECT value FROM ( @@ -36,14 +36,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: value.a, result_type: UInt64, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table2, table_name: default.test_table SETTINGS optimize_functions_to_subcolumns=1 SELECT '--'; -- @@ -57,7 +57,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION @@ -68,7 +68,7 @@ QUERY id: 0 COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8 CONSTANT id: 9, constant_value: \'a\', constant_value_type: String JOIN TREE - TABLE id: 8, table_name: default.test_table + TABLE id: 8, alias: __table2, table_name: default.test_table SETTINGS optimize_functions_to_subcolumns=0 SETTINGS optimize_functions_to_subcolumns=1 SELECT '--'; @@ -83,7 +83,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION @@ -94,7 +94,7 @@ QUERY id: 0 COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8 CONSTANT id: 9, constant_value: \'a\', constant_value_type: String JOIN TREE - TABLE id: 8, table_name: default.test_table + TABLE id: 8, alias: __table2, table_name: default.test_table SETTINGS optimize_functions_to_subcolumns=0 SELECT '--'; -- @@ -108,13 +108,13 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: value.a, result_type: UInt64, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table2, table_name: default.test_table SETTINGS optimize_functions_to_subcolumns=1 SETTINGS optimize_functions_to_subcolumns=0 diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 7e94fdf1a42..a5a71560d00 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -6,43 +6,43 @@ SELECT count() FROM a JOIN b ON b.b1 = a.a1 JOIN c ON c.c1 = b.b1 JOIN d ON d.d1 Expression ((Project names + Projection)) Header: count() UInt64 Aggregating - Header: a2_4 String + Header: __table1.a2 String count() UInt64 Expression ((Before GROUP BY + DROP unused columns after JOIN)) - Header: a2_4 String + Header: __table1.a2 String Join (JOIN FillRightFirst) - Header: a2_4 String - c1_2 UInt64 + Header: __table1.a2 String + __table3.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: a2_4 String - c1_2 UInt64 + Header: __table1.a2 String + __table3.c1 UInt64 Join (JOIN FillRightFirst) - Header: a2_4 String - b1_0 UInt64 - c1_2 UInt64 + Header: __table1.a2 String + __table2.b1 UInt64 + __table3.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: a2_4 String - b1_0 UInt64 + Header: __table1.a2 String + __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: a1_1 UInt64 - a2_4 String - b1_0 UInt64 + Header: __table1.a1 UInt64 + __table1.a2 String + __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: a1_1 UInt64 - a2_4 String + Header: __table1.a1 UInt64 + __table1.a2 String ReadFromMemoryStorage Header: a1 UInt64 a2 String Expression ((JOIN actions + Change column names to column identifiers)) - Header: b1_0 UInt64 + Header: __table2.b1 UInt64 ReadFromMemoryStorage Header: b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: c1_2 UInt64 + Header: __table3.c1 UInt64 ReadFromMemoryStorage Header: c1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: d1_3 UInt64 + Header: __table4.d1 UInt64 ReadFromMemoryStorage Header: d1 UInt64 EXPLAIN PLAN header = 1 @@ -52,38 +52,38 @@ Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: a2 String d2 String Join (JOIN FillRightFirst) - Header: a2_0 String - k_2 UInt64 - d2_1 String + Header: __table1.a2 String + __table1.k UInt64 + __table4.d2 String Expression ((Actions for left table alias column keys + DROP unused columns after JOIN)) - Header: a2_0 String - k_2 UInt64 + Header: __table1.a2 String + __table1.k UInt64 Join (JOIN FillRightFirst) - Header: a2_0 String - k_2 UInt64 + Header: __table1.a2 String + __table1.k UInt64 Expression ((Actions for left table alias column keys + DROP unused columns after JOIN)) - Header: a2_0 String - k_2 UInt64 + Header: __table1.a2 String + __table1.k UInt64 Join (JOIN FillRightFirst) - Header: a2_0 String - k_2 UInt64 + Header: __table1.a2 String + __table1.k UInt64 Expression ((Actions for left table alias column keys + Change column names to column identifiers)) - Header: a2_0 String - k_2 UInt64 + Header: __table1.a2 String + __table1.k UInt64 ReadFromMemoryStorage Header: a2 String k UInt64 Expression ((Actions for right table alias column keys + Change column names to column identifiers)) - Header: k_5 UInt64 + Header: __table2.k UInt64 ReadFromMemoryStorage Header: k UInt64 Expression ((Actions for right table alias column keys + Change column names to column identifiers)) - Header: k_4 UInt64 + Header: __table3.k UInt64 ReadFromMemoryStorage Header: k UInt64 Expression ((Actions for right table alias column keys + Change column names to column identifiers)) - Header: d2_1 String - k_3 UInt64 + Header: __table4.d2 String + __table4.k UInt64 ReadFromMemoryStorage Header: d2 String k UInt64 @@ -97,55 +97,55 @@ WHERE c.c2 != '' ORDER BY a.a2 Expression (Project names) Header: bx String Sorting (Sorting for ORDER BY) - Header: a2_6 String - bx_0 String + Header: __table1.a2 String + __table2.bx String Expression ((Before ORDER BY + (Projection + ))) - Header: a2_6 String - bx_0 String + Header: __table1.a2 String + __table2.bx String Join (JOIN FillRightFirst) - Header: a2_6 String - bx_0 String - c2_5 String - c1_3 UInt64 + Header: __table1.a2 String + __table2.bx String + __table4.c2 String + __table4.c1 UInt64 Expression - Header: a2_6 String - bx_0 String - c2_5 String - c1_3 UInt64 + Header: __table1.a2 String + __table2.bx String + __table4.c2 String + __table4.c1 UInt64 Join (JOIN FillRightFirst) - Header: a2_6 String - bx_0 String - b1_1 UInt64 - c2_5 String - c1_3 UInt64 + Header: __table1.a2 String + __table2.bx String + __table2.b1 UInt64 + __table4.c2 String + __table4.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: a2_6 String - bx_0 String - b1_1 UInt64 + Header: __table1.a2 String + __table2.bx String + __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: a1_2 UInt64 - a2_6 String - bx_0 String - b1_1 UInt64 + Header: __table1.a1 UInt64 + __table1.a2 String + __table2.bx String + __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: a1_2 UInt64 - a2_6 String + Header: __table1.a1 UInt64 + __table1.a2 String ReadFromMemoryStorage Header: a1 UInt64 a2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) - Header: b1_1 UInt64 - bx_0 String + Header: __table2.b1 UInt64 + __table2.bx String ReadFromMemoryStorage Header: b1 UInt64 b2 String Filter (( + (JOIN actions + Change column names to column identifiers))) - Header: c1_3 UInt64 - c2_5 String + Header: __table4.c1 UInt64 + __table4.c2 String ReadFromMemoryStorage Header: c1 UInt64 c2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) - Header: d1_4 UInt64 + Header: __table5.d1 UInt64 ReadFromSystemNumbers Header: number UInt64 diff --git a/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.reference b/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.reference index 37680adf8e0..15543789c1d 100644 --- a/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.reference +++ b/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.reference @@ -17,7 +17,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -40,7 +40,7 @@ QUERY id: 0 CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -63,7 +63,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -86,7 +86,7 @@ QUERY id: 0 CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing) COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -109,7 +109,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -132,7 +132,7 @@ QUERY id: 0 CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing) COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -160,7 +160,7 @@ QUERY id: 0 COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13 CONSTANT id: 15, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE_FUNCTION id: 13, table_function_name: numbers + TABLE_FUNCTION id: 13, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 16, nodes: 1 CONSTANT id: 17, constant_value: UInt64_100, constant_value_type: UInt8 @@ -188,7 +188,7 @@ QUERY id: 0 CONSTANT id: 15, constant_value: NULL, constant_value_type: Nullable(Nothing) COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13 JOIN TREE - TABLE_FUNCTION id: 13, table_function_name: numbers + TABLE_FUNCTION id: 13, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 16, nodes: 1 CONSTANT id: 17, constant_value: UInt64_100, constant_value_type: UInt8 @@ -207,7 +207,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8 @@ -229,7 +229,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 11, nodes: 1 CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8 @@ -248,7 +248,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8 @@ -270,7 +270,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 11, nodes: 1 CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8 @@ -289,7 +289,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8 @@ -311,7 +311,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 11, nodes: 1 CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8 @@ -335,7 +335,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 12, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 13, nodes: 1 CONSTANT id: 14, constant_value: UInt64_100, constant_value_type: UInt8 @@ -362,7 +362,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 15, nodes: 1 CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02521_avro_union_null_nested.reference b/tests/queries/0_stateless/02521_avro_union_null_nested.reference index e4818b4bcac..a3cb5ba4858 100644 --- a/tests/queries/0_stateless/02521_avro_union_null_nested.reference +++ b/tests/queries/0_stateless/02521_avro_union_null_nested.reference @@ -5,7 +5,7 @@ added_snapshot_id Nullable(Int64) added_data_files_count Nullable(Int32) existing_data_files_count Nullable(Int32) deleted_data_files_count Nullable(Int32) -partitions Array(Tuple(contains_null Bool, contains_nan Nullable(Bool), lower_bound Nullable(String), upper_bound Nullable(String))) +partitions Array(Tuple(\n contains_null Bool,\n contains_nan Nullable(Bool),\n lower_bound Nullable(String),\n upper_bound Nullable(String))) added_rows_count Nullable(Int64) existing_rows_count Nullable(Int64) deleted_rows_count Nullable(Int64) diff --git a/tests/queries/0_stateless/02522_avro_complicate_schema.reference b/tests/queries/0_stateless/02522_avro_complicate_schema.reference index 55c0369020f..a885163d609 100644 --- a/tests/queries/0_stateless/02522_avro_complicate_schema.reference +++ b/tests/queries/0_stateless/02522_avro_complicate_schema.reference @@ -1,5 +1,5 @@ status Int32 snapshot_id Nullable(Int64) -data_file Tuple(file_path String, file_format String, partition Tuple(vendor_id Nullable(Int64)), record_count Int64, file_size_in_bytes Int64, block_size_in_bytes Int64, column_sizes Array(Tuple(key Int32, value Int64)), value_counts Array(Tuple(key Int32, value Int64)), null_value_counts Array(Tuple(key Int32, value Int64)), nan_value_counts Array(Tuple(key Int32, value Int64)), lower_bounds Array(Tuple(key Int32, value String)), upper_bounds Array(Tuple(key Int32, value String)), key_metadata Nullable(String), split_offsets Array(Int64), sort_order_id Nullable(Int32)) +data_file Tuple(\n file_path String,\n file_format String,\n partition Tuple(\n vendor_id Nullable(Int64)),\n record_count Int64,\n file_size_in_bytes Int64,\n block_size_in_bytes Int64,\n column_sizes Array(Tuple(\n key Int32,\n value Int64)),\n value_counts Array(Tuple(\n key Int32,\n value Int64)),\n null_value_counts Array(Tuple(\n key Int32,\n value Int64)),\n nan_value_counts Array(Tuple(\n key Int32,\n value Int64)),\n lower_bounds Array(Tuple(\n key Int32,\n value String)),\n upper_bounds Array(Tuple(\n key Int32,\n value String)),\n key_metadata Nullable(String),\n split_offsets Array(Int64),\n sort_order_id Nullable(Int32)) 1 6850377589038341628 ('file:/warehouse/nyc.db/taxis/data/vendor_id=1/00000-0-c070e655-dc44-43d2-a01a-484f107210cb-00001.parquet','PARQUET',(1),2,1565,67108864,[(1,87),(2,51),(3,51),(4,57),(5,51)],[(1,2),(2,2),(3,2),(4,2),(5,2)],[(1,0),(2,0),(3,0),(4,0),(5,0)],[(3,0),(4,0)],[(1,'\0\0\0\0\0\0\0'),(2,'C\0\0\0\0\0'),(3,'ff?'),(4,'p=\nף.@'),(5,'N')],[(1,'\0\0\0\0\0\0\0'),(2,'C\0\0\0\0\0'),(3,'ffA'),(4,'q=\nףE@'),(5,'Y')],NULL,[4],0) 1 6850377589038341628 ('file:/warehouse/nyc.db/taxis/data/vendor_id=2/00000-0-c070e655-dc44-43d2-a01a-484f107210cb-00002.parquet','PARQUET',(2),2,1620,67108864,[(1,87),(2,51),(3,51),(4,57),(5,89)],[(1,2),(2,2),(3,2),(4,2),(5,2)],[(1,0),(2,0),(3,0),(4,0),(5,0)],[(3,0),(4,0)],[(1,'\0\0\0\0\0\0\0'),(2,'C\0\0\0\0\0'),(3,'fff?'),(4,'Q"@'),(5,'N')],[(1,'\0\0\0\0\0\0\0'),(2,'C\0\0\0\0\0'),(3,'\0\0 @'),(4,'fffff&6@'),(5,'N')],NULL,[4],0) diff --git a/tests/queries/0_stateless/02534_analyzer_grouping_function.reference b/tests/queries/0_stateless/02534_analyzer_grouping_function.reference index fcbf625ef22..1b496644547 100644 --- a/tests/queries/0_stateless/02534_analyzer_grouping_function.reference +++ b/tests/queries/0_stateless/02534_analyzer_grouping_function.reference @@ -16,7 +16,7 @@ QUERY id: 0 LIST id: 7, nodes: 1 COLUMN id: 8, column_name: value, result_type: String, source_id: 5 JOIN TREE - TABLE id: 5, table_name: default.test_table + TABLE id: 5, alias: __table1, table_name: default.test_table GROUP BY LIST id: 9, nodes: 2 COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5 @@ -42,7 +42,7 @@ QUERY id: 0, group_by_type: rollup COLUMN id: 9, column_name: __grouping_set, result_type: UInt64 COLUMN id: 10, column_name: value, result_type: String, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table1, table_name: default.test_table GROUP BY LIST id: 11, nodes: 2 COLUMN id: 5, column_name: id, result_type: UInt64, source_id: 6 @@ -70,7 +70,7 @@ QUERY id: 0, group_by_type: cube COLUMN id: 9, column_name: __grouping_set, result_type: UInt64 COLUMN id: 10, column_name: value, result_type: String, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table1, table_name: default.test_table GROUP BY LIST id: 11, nodes: 2 COLUMN id: 5, column_name: id, result_type: UInt64, source_id: 6 @@ -99,7 +99,7 @@ QUERY id: 0, group_by_type: grouping_sets COLUMN id: 9, column_name: __grouping_set, result_type: UInt64 COLUMN id: 10, column_name: value, result_type: String, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table1, table_name: default.test_table GROUP BY LIST id: 11, nodes: 2 LIST id: 12, nodes: 1 @@ -128,7 +128,7 @@ QUERY id: 0, group_by_type: grouping_sets COLUMN id: 9, column_name: __grouping_set, result_type: UInt64 COLUMN id: 10, column_name: value, result_type: String, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table1, table_name: default.test_table GROUP BY LIST id: 11, nodes: 2 LIST id: 12, nodes: 1 diff --git a/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference index e4d7ff55b86..5b9bc206695 100644 --- a/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference +++ b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference @@ -29,9 +29,9 @@ QUERY id: 0 LEFT TABLE EXPRESSION JOIN id: 11, strictness: ALL, kind: INNER LEFT TABLE EXPRESSION - TABLE id: 3, table_name: default.t1 + TABLE id: 3, alias: __table1, table_name: default.t1 RIGHT TABLE EXPRESSION - TABLE id: 6, table_name: default.t2 + TABLE id: 6, alias: __table2, table_name: default.t2 JOIN EXPRESSION FUNCTION id: 12, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -48,14 +48,14 @@ QUERY id: 0 COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6 CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8 RIGHT TABLE EXPRESSION - QUERY id: 9, alias: t3, is_subquery: 1 + QUERY id: 9, alias: __table3, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 23, nodes: 1 COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25 JOIN TREE - TABLE id: 25, table_name: default.t3 + TABLE id: 25, alias: __table4, table_name: default.t3 WHERE FUNCTION id: 26, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -97,18 +97,18 @@ QUERY id: 0 LEFT TABLE EXPRESSION JOIN id: 11, kind: COMMA LEFT TABLE EXPRESSION - TABLE id: 3, table_name: default.t1 + TABLE id: 3, alias: __table1, table_name: default.t1 RIGHT TABLE EXPRESSION - TABLE id: 6, table_name: default.t2 + TABLE id: 6, alias: __table2, table_name: default.t2 RIGHT TABLE EXPRESSION - QUERY id: 9, alias: t3, is_subquery: 1 + QUERY id: 9, alias: __table3, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 12, nodes: 1 COLUMN id: 13, column_name: a, result_type: UInt64, source_id: 14 JOIN TREE - TABLE id: 14, table_name: default.t3 + TABLE id: 14, alias: __table4, table_name: default.t3 WHERE FUNCTION id: 15, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -166,9 +166,9 @@ QUERY id: 0 LEFT TABLE EXPRESSION JOIN id: 11, strictness: ALL, kind: INNER LEFT TABLE EXPRESSION - TABLE id: 3, table_name: default.t1 + TABLE id: 3, alias: __table1, table_name: default.t1 RIGHT TABLE EXPRESSION - TABLE id: 6, table_name: default.t2 + TABLE id: 6, alias: __table2, table_name: default.t2 JOIN EXPRESSION FUNCTION id: 12, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -185,14 +185,14 @@ QUERY id: 0 COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6 CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8 RIGHT TABLE EXPRESSION - QUERY id: 9, alias: t3, is_subquery: 1 + QUERY id: 9, alias: __table3, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 23, nodes: 1 COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25 JOIN TREE - TABLE id: 25, table_name: default.t3 + TABLE id: 25, alias: __table4, table_name: default.t3 WHERE FUNCTION id: 26, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference b/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference index b8c68f90135..dd107065380 100644 --- a/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference +++ b/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference @@ -1,21 +1,21 @@ Expression ((Project names + (Projection + ))) Header: number UInt64 -Actions: INPUT : 0 -> number_1 UInt64 : 0 - ALIAS number_1 :: 0 -> number UInt64 : 1 - ALIAS number :: 1 -> number_0 UInt64 : 0 - ALIAS number_0 :: 0 -> number UInt64 : 1 +Actions: INPUT : 0 -> __table2.number UInt64 : 0 + ALIAS __table2.number :: 0 -> number UInt64 : 1 + ALIAS number :: 1 -> __table1.number UInt64 : 0 + ALIAS __table1.number :: 0 -> number UInt64 : 1 Positions: 1 Sorting (Sorting for ORDER BY) Header: ignore(2_UInt8) UInt8 - number_1 UInt64 + __table2.number UInt64 Sort description: ignore(2_UInt8) ASC Filter (( + (Before ORDER BY + (Projection + Change column names to column identifiers)))) Header: ignore(2_UInt8) UInt8 - number_1 UInt64 + __table2.number UInt64 Filter column: ignore(2_UInt8) Actions: INPUT : 0 -> number UInt64 : 0 COLUMN Const(UInt8) -> 2_UInt8 UInt8 : 1 - ALIAS number :: 0 -> number_1 UInt64 : 2 + ALIAS number :: 0 -> __table2.number UInt64 : 2 FUNCTION ignore(2_UInt8 :: 1) -> ignore(2_UInt8) UInt8 : 0 Positions: 0 2 ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.reference b/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.reference index b6964976c20..f4e09c4b4de 100644 --- a/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.reference +++ b/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.reference @@ -26,7 +26,7 @@ QUERY id: 0 LIST id: 14, nodes: 1 CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 16, table_function_name: numbers + TABLE_FUNCTION id: 16, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 17, nodes: 1 CONSTANT id: 18, constant_value: UInt64_10, constant_value_type: UInt8 @@ -58,7 +58,7 @@ QUERY id: 0 LIST id: 14, nodes: 1 CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 16, table_function_name: numbers + TABLE_FUNCTION id: 16, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 17, nodes: 1 CONSTANT id: 18, constant_value: UInt64_10, constant_value_type: UInt8 @@ -81,7 +81,7 @@ QUERY id: 0 CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8 CONSTANT id: 10, constant_value: UInt64_5, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 @@ -104,7 +104,7 @@ QUERY id: 0 CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8 CONSTANT id: 10, constant_value: UInt64_5, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference index 089d1849eb4..cf60d63b1cf 100644 --- a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference +++ b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference @@ -9,7 +9,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -26,7 +26,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -42,7 +42,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE CONSTANT id: 5, constant_value: UInt64_0, constant_value_type: UInt8 3 another @@ -55,7 +55,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -80,7 +80,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -97,7 +97,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -115,7 +115,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: notEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference index 2630c5b95b6..e6c4d5768af 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference @@ -2,31 +2,31 @@ Expression ((Project names + (Projection + ))) Header: t1.id UInt64 t1.value String t2.value String -Actions: INPUT : 0 -> id_0 UInt64 : 0 - INPUT : 1 -> value_1 String : 1 - INPUT : 2 -> value_2 String : 2 - ALIAS id_0 :: 0 -> t1.id UInt64 : 3 - ALIAS value_1 :: 1 -> t1.value String : 0 - ALIAS value_2 :: 2 -> t2.value String : 1 +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + ALIAS __table1.id :: 0 -> t1.id UInt64 : 3 + ALIAS __table1.value :: 1 -> t1.value String : 0 + ALIAS __table2.value :: 2 -> t2.value String : 1 Positions: 3 0 1 FilledJoin (Filled JOIN) - Header: id_0 UInt64 - value_1 String - value_2 String + Header: __table1.id UInt64 + __table1.value String + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin - Clauses: [(id_0) = (id)] + Clauses: [(__table1.id) = (id)] Filter (( + (JOIN actions + Change column names to column identifiers))) - Header: id_0 UInt64 - value_1 String - Filter column: equals(id_0, 0_UInt8) (removed) + Header: __table1.id UInt64 + __table1.value String + Filter column: equals(__table1.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2 - ALIAS id :: 0 -> id_0 UInt64 : 3 - ALIAS value :: 1 -> value_1 String : 0 - FUNCTION equals(id_0 : 3, 0_UInt8 :: 2) -> equals(id_0, 0_UInt8) UInt8 : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 0 + FUNCTION equals(__table1.id : 3, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1 Positions: 1 3 0 ReadFromMergeTree (default.test_table) Header: id UInt64 diff --git a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference index cc16a1fce02..4a4e338438b 100644 --- a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference +++ b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference @@ -29,10 +29,10 @@ Header: id UInt64 value String Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 - ALIAS id :: 0 -> id_0 UInt64 : 2 - ALIAS value :: 1 -> value_1 String : 0 - ALIAS id_0 :: 2 -> id UInt64 : 1 - ALIAS value_1 :: 0 -> value String : 2 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value :: 1 -> __table1.value String : 0 + ALIAS __table1.id :: 2 -> id UInt64 : 1 + ALIAS __table1.value :: 0 -> value String : 2 Positions: 1 2 ReadFromMergeTree (default.test_table) Header: id UInt64 diff --git a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference index e7f46a974e6..c25b446dcdc 100644 --- a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference +++ b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference @@ -9,7 +9,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02702_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer WHERE FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: Nullable(UInt8) ARGUMENTS @@ -41,7 +41,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02702_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer WHERE FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: Nullable(UInt8) ARGUMENTS @@ -68,7 +68,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Nullable(Int32), source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02702_logical_optimizer_with_null_column + TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer_with_null_column WHERE FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: Nullable(UInt8) ARGUMENTS diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference index 35573110550..3b8a394a522 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference @@ -9,4 +9,4 @@ 7885388429666205427 8124171311239967992 1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; -0 2 SELECT `join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1 +0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1 diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference index 0fd2f694aeb..63658890119 100644 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference @@ -8,7 +8,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -50,7 +50,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -92,7 +92,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -126,7 +126,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -160,7 +160,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -194,7 +194,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -228,7 +228,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -270,7 +270,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -335,7 +335,7 @@ QUERY id: 0 LIST id: 5, nodes: 1 COLUMN id: 6, column_name: date1, result_type: Date, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 7, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -377,7 +377,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -412,7 +412,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t PREWHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -452,7 +452,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -492,7 +492,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -529,7 +529,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -566,7 +566,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -608,7 +608,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -650,7 +650,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -692,7 +692,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -726,7 +726,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -760,7 +760,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -794,7 +794,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -828,7 +828,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -878,7 +878,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.datetime_t + TABLE id: 3, alias: __table1, table_name: default.datetime_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -920,7 +920,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.datetime_t + TABLE id: 3, alias: __table1, table_name: default.datetime_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -962,7 +962,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date32_t + TABLE id: 3, alias: __table1, table_name: default.date32_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -1004,7 +1004,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date32_t + TABLE id: 3, alias: __table1, table_name: default.date32_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -1046,7 +1046,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.datetime64_t + TABLE id: 3, alias: __table1, table_name: default.datetime64_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -1088,7 +1088,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.datetime64_t + TABLE id: 3, alias: __table1, table_name: default.datetime64_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index 0cc2e802682..06f4a9cfc99 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -3,31 +3,31 @@ Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String -Actions: INPUT : 0 -> id_0 UInt64 : 0 - INPUT : 1 -> value_1_1 String : 1 - INPUT : 2 -> value_1_3 String : 2 - INPUT : 3 -> id_2 UInt64 : 3 - ALIAS id_0 :: 0 -> id UInt64 : 4 - ALIAS value_1_1 :: 1 -> value_1 String : 0 - ALIAS value_1_3 :: 2 -> rhs.value_1 String : 1 - ALIAS id_2 :: 3 -> rhs.id UInt64 : 2 +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value_1 String : 1 + INPUT : 2 -> __table2.value_1 String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value_1 :: 1 -> value_1 String : 0 + ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 Positions: 4 0 2 1 Join (JOIN FillRightFirst) - Header: id_0 UInt64 - value_1_1 String - value_1_3 String - id_2 UInt64 + Header: __table1.id UInt64 + __table1.value_1 String + __table2.value_1 String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin - Clauses: [(id_0) = (id_2)] + Clauses: [(__table1.id) = (__table2.id)] Expression ((JOIN actions + Change column names to column identifiers)) - Header: id_0 UInt64 - value_1_1 String + Header: __table1.id UInt64 + __table1.value_1 String Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value_1 String : 1 - ALIAS id :: 0 -> id_0 UInt64 : 2 - ALIAS value_1 :: 1 -> value_1_1 String : 0 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value_1 :: 1 -> __table1.value_1 String : 0 Positions: 2 0 ReadFromMergeTree (default.test_table_1) Header: id UInt64 @@ -36,12 +36,12 @@ Positions: 4 0 2 1 Parts: 1 Granules: 1 Expression ((JOIN actions + Change column names to column identifiers)) - Header: id_2 UInt64 - value_1_3 String + Header: __table2.id UInt64 + __table2.value_1 String Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value_1 String : 1 - ALIAS id :: 0 -> id_2 UInt64 : 2 - ALIAS value_1 :: 1 -> value_1_3 String : 0 + ALIAS id :: 0 -> __table2.id UInt64 : 2 + ALIAS value_1 :: 1 -> __table2.value_1 String : 0 Positions: 2 0 ReadFromMergeTree (default.test_table_2) Header: id UInt64 @@ -55,39 +55,39 @@ Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String -Actions: INPUT : 0 -> id_0 UInt64 : 0 - INPUT : 1 -> value_1_1 String : 1 - INPUT :: 2 -> value_2_4 UInt64 : 2 - INPUT : 3 -> value_1_3 String : 3 - INPUT :: 4 -> value_2_5 UInt64 : 4 - INPUT : 5 -> id_2 UInt64 : 5 - ALIAS id_0 :: 0 -> id UInt64 : 6 - ALIAS value_1_1 :: 1 -> value_1 String : 0 - ALIAS value_1_3 :: 3 -> rhs.value_1 String : 1 - ALIAS id_2 :: 5 -> rhs.id UInt64 : 3 +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value_1 String : 1 + INPUT :: 2 -> __table1.value_2 UInt64 : 2 + INPUT : 3 -> __table2.value_1 String : 3 + INPUT :: 4 -> __table2.value_2 UInt64 : 4 + INPUT : 5 -> __table2.id UInt64 : 5 + ALIAS __table1.id :: 0 -> id UInt64 : 6 + ALIAS __table1.value_1 :: 1 -> value_1 String : 0 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 Positions: 6 0 3 1 Join (JOIN FillRightFirst) - Header: id_0 UInt64 - value_1_1 String - value_2_4 UInt64 - value_1_3 String - value_2_5 UInt64 - id_2 UInt64 + Header: __table1.id UInt64 + __table1.value_1 String + __table1.value_2 UInt64 + __table2.value_1 String + __table2.value_2 UInt64 + __table2.id UInt64 Type: INNER Strictness: ASOF Algorithm: HashJoin ASOF inequality: LESS - Clauses: [(id_0, value_2_4) = (id_2, value_2_5)] + Clauses: [(__table1.id, __table1.value_2) = (__table2.id, __table2.value_2)] Expression ((JOIN actions + Change column names to column identifiers)) - Header: id_0 UInt64 - value_1_1 String - value_2_4 UInt64 + Header: __table1.id UInt64 + __table1.value_1 String + __table1.value_2 UInt64 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value_1 String : 1 INPUT : 2 -> value_2 UInt64 : 2 - ALIAS id :: 0 -> id_0 UInt64 : 3 - ALIAS value_1 :: 1 -> value_1_1 String : 0 - ALIAS value_2 :: 2 -> value_2_4 UInt64 : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 3 + ALIAS value_1 :: 1 -> __table1.value_1 String : 0 + ALIAS value_2 :: 2 -> __table1.value_2 UInt64 : 1 Positions: 3 0 1 ReadFromMergeTree (default.test_table_1) Header: id UInt64 @@ -97,15 +97,15 @@ Positions: 6 0 3 1 Parts: 1 Granules: 1 Expression ((JOIN actions + Change column names to column identifiers)) - Header: id_2 UInt64 - value_1_3 String - value_2_5 UInt64 + Header: __table2.id UInt64 + __table2.value_1 String + __table2.value_2 UInt64 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value_1 String : 1 INPUT : 2 -> value_2 UInt64 : 2 - ALIAS id :: 0 -> id_2 UInt64 : 3 - ALIAS value_1 :: 1 -> value_1_3 String : 0 - ALIAS value_2 :: 2 -> value_2_5 UInt64 : 1 + ALIAS id :: 0 -> __table2.id UInt64 : 3 + ALIAS value_1 :: 1 -> __table2.value_1 String : 0 + ALIAS value_2 :: 2 -> __table2.value_2 UInt64 : 1 Positions: 3 0 1 ReadFromMergeTree (default.test_table_2) Header: id UInt64 diff --git a/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference b/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference index a2c441fa460..c2075f72f33 100644 --- a/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference +++ b/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference @@ -15,14 +15,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, is_subquery: 1, is_distinct: 1 + QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count SETTINGS allow_experimental_analyzer=1 2. test distinct with subquery alias 3 @@ -41,14 +41,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1 + QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count SETTINGS allow_experimental_analyzer=1 3. test distinct with compound column name 3 @@ -67,14 +67,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1 + QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count SETTINGS allow_experimental_analyzer=1 4. test distinct with select expression alias 3 @@ -93,14 +93,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1 + QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1 PROJECTION COLUMNS alias_of_a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count SETTINGS allow_experimental_analyzer=1 5. test simple group by 3 @@ -122,14 +122,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 @@ -154,14 +154,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 @@ -186,14 +186,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS alias_of_a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 @@ -218,14 +218,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS alias_of_a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 diff --git a/tests/queries/0_stateless/02874_infer_objects_as_named_tuples.reference b/tests/queries/0_stateless/02874_infer_objects_as_named_tuples.reference index 01ef288d81a..06c152a0a3c 100644 --- a/tests/queries/0_stateless/02874_infer_objects_as_named_tuples.reference +++ b/tests/queries/0_stateless/02874_infer_objects_as_named_tuples.reference @@ -1,34 +1,34 @@ -obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64))) +obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64))) (42,'Hello',[1,2,3]) -obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(Date)) +obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(Date)) (42,'Hello',[1,2,3],NULL) (43,'World',[],'2020-01-01') -obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(Date)) +obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(Date)) (42,'Hello',[1,2,3],NULL) (43,'World',[],'2020-01-01') (NULL,NULL,[],NULL) -obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(String)) +obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(String)) (42,'Hello',[1,2,3],NULL) (43,'World',[],'2020-01-01') (NULL,NULL,[],NULL) (NULL,'2020-01-01',[],'Hello') -obj Array(Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(Date))) +obj Array(Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(Date))) [(42,'Hello',[1,2,3],NULL),(43,'World',[],'2020-01-01')] [(NULL,NULL,[],NULL)] -obj Tuple(nested_obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(Date))) +obj Tuple(\n nested_obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(Date))) ((42,'Hello',[1,2,3],NULL)) ((43,'World',[],'2020-01-01')) ((NULL,NULL,[],NULL)) -obj Tuple(a Tuple(b Nullable(Int64)), `a.b` Nullable(Int64), `a.b.c` Nullable(String)) +obj Tuple(\n a Tuple(\n b Nullable(Int64)),\n `a.b` Nullable(Int64),\n `a.b.c` Nullable(String)) ((1),NULL,NULL) ((NULL),2,'Hello') -obj Tuple(a Tuple(b Tuple(c Nullable(Int64)))) +obj Tuple(\n a Tuple(\n b Tuple(\n c Nullable(Int64)))) (((NULL))) (((10))) -obj Tuple(a Nullable(String)) +obj Tuple(\n a Nullable(String)) ('{}') obj Nullable(String) {} -obj Tuple(a Array(Tuple(b Array(Nullable(Int64)), c Tuple(d Nullable(Int64)), e Nullable(String)))) +obj Tuple(\n a Array(Tuple(\n b Array(Nullable(Int64)),\n c Tuple(\n d Nullable(Int64)),\n e Nullable(String)))) ([([],(NULL),NULL),([],(NULL),NULL),([],(10),NULL)]) ([([1,2,3],(NULL),'Hello')]) diff --git a/tests/queries/0_stateless/02876_json_incomplete_types_as_strings_inference.reference b/tests/queries/0_stateless/02876_json_incomplete_types_as_strings_inference.reference index db94ffc9466..b904568391b 100644 --- a/tests/queries/0_stateless/02876_json_incomplete_types_as_strings_inference.reference +++ b/tests/queries/0_stateless/02876_json_incomplete_types_as_strings_inference.reference @@ -2,6 +2,6 @@ a Nullable(String) b Nullable(String) c Array(Nullable(String)) \N {} [] -a Tuple(b Nullable(String), c Array(Array(Nullable(String)))) -d Tuple(e Array(Nullable(String)), f Nullable(String)) +a Tuple(\n b Nullable(String),\n c Array(Array(Nullable(String)))) +d Tuple(\n e Array(Nullable(String)),\n f Nullable(String)) (NULL,[[],[]]) (['{}','{}'],NULL) diff --git a/tests/queries/0_stateless/02889_print_pretty_type_names.reference b/tests/queries/0_stateless/02889_print_pretty_type_names.reference index ea25df165bb..9af8e0142f8 100644 --- a/tests/queries/0_stateless/02889_print_pretty_type_names.reference +++ b/tests/queries/0_stateless/02889_print_pretty_type_names.reference @@ -5,18 +5,11 @@ a Tuple( e Array(UInt32), f Array(Tuple( g String, - h Map( - String, - Array(Tuple( - i String, - j UInt64 - )) - ) - )), - k Date - ), - l Nullable(String) -) + h Map(String, Array(Tuple( + i String, + j UInt64))))), + k Date), + l Nullable(String)) Tuple( b String, c Tuple( @@ -24,15 +17,8 @@ Tuple( e Array(UInt32), f Array(Tuple( g String, - h Map( - String, - Array(Tuple( - i String, - j UInt64 - )) - ) - )), - k Date - ), - l Nullable(String) -) + h Map(String, Array(Tuple( + i String, + j UInt64))))), + k Date), + l Nullable(String)) diff --git a/tests/queries/0_stateless/02890_describe_table_options.reference b/tests/queries/0_stateless/02890_describe_table_options.reference index 2974fd92f3c..5d99df36bb4 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.reference +++ b/tests/queries/0_stateless/02890_describe_table_options.reference @@ -2,205 +2,237 @@ SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ -│ id │ UInt64 │ │ │ index column │ │ │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ -└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ +┌─name─┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ +│ id │ UInt64 │ │ │ index column │ │ │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ +└──────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ -│ id │ UInt64 │ │ │ index column │ │ │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ -└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ +┌─name─┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ +│ id │ UInt64 │ │ │ index column │ │ │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ +└──────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ -└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ +┌─name──────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ +└───────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ -└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ +┌─name──────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ +└───────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _partition_value │ UInt8 │ │ │ │ │ │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ -└──────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ +┌─name─────────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ +│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _partition_value │ UInt8 │ │ │ │ │ │ 1 │ +│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ +│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ +│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ +│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ +└──────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _table │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ -│ _shard_num │ UInt32 │ │ │ │ │ │ 1 │ -└────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ +┌─name───────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ _table │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ +│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ +│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ +│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ +│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ +│ _shard_num │ UInt32 │ │ │ │ │ │ 1 │ +└────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _partition_value │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -└──────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ +┌─name─────────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ +│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _partition_value │ UInt8 │ │ │ │ │ │ 0 │ 1 │ +│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ +│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ +│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +└──────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _table │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _shard_num │ UInt32 │ │ │ │ │ │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -└────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ +┌─name───────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ +│ _table │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ +│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ +│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _shard_num │ UInt32 │ │ │ │ │ │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +└────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┐ -│ id │ UInt64 │ -│ arr │ Array(UInt64) │ -│ t │ Tuple(a String, b UInt64) │ -└──────┴───────────────────────────┘ +┌─name─┬─type─────────────────────────────┐ +│ id │ UInt64 │ +│ arr │ Array(UInt64) │ +│ t │ Tuple( + a String, + b UInt64) │ +└──────┴──────────────────────────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┐ -│ id │ UInt64 │ -│ arr │ Array(UInt64) │ -│ t │ Tuple(a String, b UInt64) │ -└──────┴───────────────────────────┘ +┌─name─┬─type─────────────────────────────┐ +│ id │ UInt64 │ +│ arr │ Array(UInt64) │ +│ t │ Tuple( + a String, + b UInt64) │ +└──────┴──────────────────────────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─is_subcolumn─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ arr.size0 │ UInt64 │ 1 │ -│ t.a │ String │ 1 │ -│ t.b │ UInt64 │ 1 │ -└───────────┴───────────────────────────┴──────────────┘ +┌─name──────┬─type─────────────────────────────┬─is_subcolumn─┐ +│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ +│ arr.size0 │ UInt64 │ 1 │ +│ t.a │ String │ 1 │ +│ t.b │ UInt64 │ 1 │ +└───────────┴──────────────────────────────────┴──────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─is_subcolumn─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ arr.size0 │ UInt64 │ 1 │ -│ t.a │ String │ 1 │ -│ t.b │ UInt64 │ 1 │ -└───────────┴───────────────────────────┴──────────────┘ +┌─name──────┬─type─────────────────────────────┬─is_subcolumn─┐ +│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ +│ arr.size0 │ UInt64 │ 1 │ +│ t.a │ String │ 1 │ +│ t.b │ UInt64 │ 1 │ +└───────────┴──────────────────────────────────┴──────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type──────────────────────┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ _part │ LowCardinality(String) │ 1 │ -│ _part_index │ UInt64 │ 1 │ -│ _part_uuid │ UUID │ 1 │ -│ _partition_id │ LowCardinality(String) │ 1 │ -│ _partition_value │ UInt8 │ 1 │ -│ _sample_factor │ Float64 │ 1 │ -│ _part_offset │ UInt64 │ 1 │ -│ _row_exists │ UInt8 │ 1 │ -│ _block_number │ UInt64 │ 1 │ -└──────────────────┴───────────────────────────┴────────────┘ +┌─name─────────────┬─type─────────────────────────────┬─is_virtual─┐ +│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ +│ _part │ LowCardinality(String) │ 1 │ +│ _part_index │ UInt64 │ 1 │ +│ _part_uuid │ UUID │ 1 │ +│ _partition_id │ LowCardinality(String) │ 1 │ +│ _partition_value │ UInt8 │ 1 │ +│ _sample_factor │ Float64 │ 1 │ +│ _part_offset │ UInt64 │ 1 │ +│ _row_exists │ UInt8 │ 1 │ +│ _block_number │ UInt64 │ 1 │ +└──────────────────┴──────────────────────────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ _table │ LowCardinality(String) │ 1 │ -│ _part │ LowCardinality(String) │ 1 │ -│ _part_index │ UInt64 │ 1 │ -│ _part_uuid │ UUID │ 1 │ -│ _partition_id │ LowCardinality(String) │ 1 │ -│ _sample_factor │ Float64 │ 1 │ -│ _part_offset │ UInt64 │ 1 │ -│ _row_exists │ UInt8 │ 1 │ -│ _block_number │ UInt64 │ 1 │ -│ _shard_num │ UInt32 │ 1 │ -└────────────────┴───────────────────────────┴────────────┘ +┌─name───────────┬─type─────────────────────────────┬─is_virtual─┐ +│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ +│ _table │ LowCardinality(String) │ 1 │ +│ _part │ LowCardinality(String) │ 1 │ +│ _part_index │ UInt64 │ 1 │ +│ _part_uuid │ UUID │ 1 │ +│ _partition_id │ LowCardinality(String) │ 1 │ +│ _sample_factor │ Float64 │ 1 │ +│ _part_offset │ UInt64 │ 1 │ +│ _row_exists │ UInt8 │ 1 │ +│ _block_number │ UInt64 │ 1 │ +│ _shard_num │ UInt32 │ 1 │ +└────────────────┴──────────────────────────────────┴────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ 0 │ 1 │ -│ _part_index │ UInt64 │ 0 │ 1 │ -│ _part_uuid │ UUID │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ -│ _partition_value │ UInt8 │ 0 │ 1 │ -│ _sample_factor │ Float64 │ 0 │ 1 │ -│ _part_offset │ UInt64 │ 0 │ 1 │ -│ _row_exists │ UInt8 │ 0 │ 1 │ -│ _block_number │ UInt64 │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ 1 │ 0 │ -│ t.a │ String │ 1 │ 0 │ -│ t.b │ UInt64 │ 1 │ 0 │ -└──────────────────┴───────────────────────────┴──────────────┴────────────┘ +┌─name─────────────┬─type─────────────────────────────┬─is_subcolumn─┬─is_virtual─┐ +│ id │ UInt64 │ 0 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ 0 │ +│ _part │ LowCardinality(String) │ 0 │ 1 │ +│ _part_index │ UInt64 │ 0 │ 1 │ +│ _part_uuid │ UUID │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ +│ _partition_value │ UInt8 │ 0 │ 1 │ +│ _sample_factor │ Float64 │ 0 │ 1 │ +│ _part_offset │ UInt64 │ 0 │ 1 │ +│ _row_exists │ UInt8 │ 0 │ 1 │ +│ _block_number │ UInt64 │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ 1 │ 0 │ +│ t.a │ String │ 1 │ 0 │ +│ t.b │ UInt64 │ 1 │ 0 │ +└──────────────────┴──────────────────────────────────┴──────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ -│ _table │ LowCardinality(String) │ 0 │ 1 │ -│ _part │ LowCardinality(String) │ 0 │ 1 │ -│ _part_index │ UInt64 │ 0 │ 1 │ -│ _part_uuid │ UUID │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ -│ _sample_factor │ Float64 │ 0 │ 1 │ -│ _part_offset │ UInt64 │ 0 │ 1 │ -│ _row_exists │ UInt8 │ 0 │ 1 │ -│ _block_number │ UInt64 │ 0 │ 1 │ -│ _shard_num │ UInt32 │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ 1 │ 0 │ -│ t.a │ String │ 1 │ 0 │ -│ t.b │ UInt64 │ 1 │ 0 │ -└────────────────┴───────────────────────────┴──────────────┴────────────┘ +┌─name───────────┬─type─────────────────────────────┬─is_subcolumn─┬─is_virtual─┐ +│ id │ UInt64 │ 0 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ 0 │ +│ _table │ LowCardinality(String) │ 0 │ 1 │ +│ _part │ LowCardinality(String) │ 0 │ 1 │ +│ _part_index │ UInt64 │ 0 │ 1 │ +│ _part_uuid │ UUID │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ +│ _sample_factor │ Float64 │ 0 │ 1 │ +│ _part_offset │ UInt64 │ 0 │ 1 │ +│ _row_exists │ UInt8 │ 0 │ 1 │ +│ _block_number │ UInt64 │ 0 │ 1 │ +│ _shard_num │ UInt32 │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ 1 │ 0 │ +│ t.a │ String │ 1 │ 0 │ +│ t.b │ UInt64 │ 1 │ 0 │ +└────────────────┴──────────────────────────────────┴──────────────┴────────────┘ diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.reference b/tests/queries/0_stateless/02900_union_schema_inference_mode.reference index 864cd780ddb..31172c41262 100644 --- a/tests/queries/0_stateless/02900_union_schema_inference_mode.reference +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.reference @@ -1,5 +1,5 @@ a Nullable(Int64) -obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +obj Tuple(\n f1 Nullable(Int64),\n f2 Nullable(String),\n f3 Nullable(Int64)) b Nullable(Int64) c Nullable(String) {"a":"1","obj":{"f1":"1","f2":"2020-01-01","f3":null},"b":null,"c":null} @@ -10,11 +10,11 @@ UNION data2.jsonl b Nullable(Int64), obj Tuple(f2 Nullable(String), f3 Nullable( UNION data3.jsonl c Nullable(String) c Nullable(String) a Nullable(Int64) -obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +obj Tuple(\n f1 Nullable(Int64),\n f2 Nullable(String),\n f3 Nullable(Int64)) b Nullable(Int64) c Nullable(String) a Nullable(Int64) -obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +obj Tuple(\n f1 Nullable(Int64),\n f2 Nullable(String),\n f3 Nullable(Int64)) b Nullable(Int64) c Nullable(String) {"a":"1","obj":{"f1":"1","f2":"2020-01-01","f3":null},"b":null,"c":null} @@ -25,7 +25,7 @@ UNION archive.tar::data2.jsonl b Nullable(Int64), obj Tuple(f2 Nullable(String), UNION archive.tar::data3.jsonl c Nullable(String) c Nullable(String) a Nullable(Int64) -obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +obj Tuple(\n f1 Nullable(Int64),\n f2 Nullable(String),\n f3 Nullable(Int64)) b Nullable(Int64) c Nullable(String) 1 diff --git a/tests/queries/0_stateless/02906_flatten_only_true_nested.reference b/tests/queries/0_stateless/02906_flatten_only_true_nested.reference index e7a96da8db9..b259b1e4563 100644 --- a/tests/queries/0_stateless/02906_flatten_only_true_nested.reference +++ b/tests/queries/0_stateless/02906_flatten_only_true_nested.reference @@ -1,3 +1,3 @@ data.x Array(UInt32) data.y Array(UInt32) -data Array(Tuple(x UInt64, y UInt64)) +data Array(Tuple(\n x UInt64,\n y UInt64)) diff --git a/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference b/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference index dfdd38f5e8e..46738c95cd5 100644 --- a/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference +++ b/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference @@ -1,9 +1,9 @@ int64_column Nullable(Int64) string_column Nullable(String) float64_column Nullable(Float64) -tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)) -array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))) -map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))) +tuple_column Tuple(\n a Nullable(String),\n b Nullable(Float64),\n c Nullable(Int64)) +array_tuple_column Array(Tuple(\n a Nullable(String),\n b Nullable(Float64),\n c Nullable(Int64))) +map_tuple_column Map(String, Tuple(\n a Nullable(String),\n b Nullable(Float64),\n c Nullable(Int64))) -- { echoOn } -- Test primitive types select int64_column, string_column, float64_column from file('02906.orc') where int64_column % 15 = 0; diff --git a/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.reference b/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.reference index 5dd0d0d1820..d8f2decba37 100644 --- a/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.reference +++ b/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.reference @@ -13,8 +13,8 @@ select * from tab order by (a + b) * c; 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by (a + b) * c) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC select * from tab order by (a + b) * c desc; 4 4 4 4 4 4 4 4 @@ -27,8 +27,8 @@ select * from tab order by (a + b) * c desc; 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC -- Exact match, full key select * from tab order by (a + b) * c, sin(a / b); 0 0 0 0 @@ -42,8 +42,8 @@ select * from tab order by (a + b) * c, sin(a / b); 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC select * from tab order by (a + b) * c desc, sin(a / b) desc; 4 4 4 4 4 4 4 4 @@ -56,8 +56,8 @@ select * from tab order by (a + b) * c desc, sin(a / b) desc; 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b) desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) DESC -- Exact match, mixed direction select * from tab order by (a + b) * c desc, sin(a / b); 4 4 4 4 @@ -71,8 +71,8 @@ select * from tab order by (a + b) * c desc, sin(a / b); 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) ASC select * from tab order by (a + b) * c, sin(a / b) desc; 0 0 0 0 0 0 0 0 @@ -85,8 +85,8 @@ select * from tab order by (a + b) * c, sin(a / b) desc; 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b) desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) DESC -- Wrong order, full sort select * from tab order by sin(a / b), (a + b) * c; 1 1 1 1 @@ -100,32 +100,32 @@ select * from tab order by sin(a / b), (a + b) * c; 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from tab order by sin(a / b), (a + b) * c) where explain ilike '%sort description%'; - Sort description: sin(divide(a_0, b_1)) ASC, multiply(plus(a_0, b_1), c_2) ASC + Sort description: sin(divide(__table1.a, __table1.b)) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC -- Fixed point select * from tab where (a + b) * c = 8 order by sin(a / b); 2 2 2 2 2 2 2 2 select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(a / b)) where explain ilike '%sort description%'; - Prefix sort description: sin(divide(a_0, b_1)) ASC - Result sort description: sin(divide(a_0, b_1)) ASC + Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC + Result sort description: sin(divide(__table1.a, __table1.b)) ASC select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c; 1 1 1 1 1 1 1 1 select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c) where explain ilike '%sort description%'; - Prefix sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, multiply(plus(a_0, b_1), c_2) ASC + Prefix sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b); 2 2 2 2 2 2 2 2 select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b)) where explain ilike '%sort description%'; - Prefix sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC -- Wrong order with fixed point select * from tab where (a + b) * c = 8 order by sin(b / a); 2 2 2 2 2 2 2 2 select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(b / a)) where explain ilike '%sort description%'; - Sort description: sin(divide(b_1, a_0)) ASC + Sort description: sin(divide(__table1.b, __table1.a)) ASC -- Monotonicity select * from tab order by intDiv((a + b) * c, 2); 0 0 0 0 @@ -139,8 +139,8 @@ select * from tab order by intDiv((a + b) * c, 2); 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2)) where explain like '%sort description%'; - Prefix sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC - Result sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC + Prefix sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC + Result sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC select * from tab order by intDiv((a + b) * c, 2), sin(a / b); 0 0 0 0 0 0 0 0 @@ -153,36 +153,36 @@ select * from tab order by intDiv((a + b) * c, 2), sin(a / b); 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2), sin(a / b)) where explain like '%sort description%'; - Prefix sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC - Result sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC + Result sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC -- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2); select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC -- select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2); select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC -- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc; select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC -- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc; select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC -- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2); select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), -2_Int8) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), -2_Int8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8) ASC -- select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3); select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC -- select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3); select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC -- Aliases select * from (select *, a + b as x from tab) order by x * c; 0 0 0 0 0 @@ -196,8 +196,8 @@ select * from (select *, a + b as x from tab) order by x * c; 4 4 4 4 8 4 4 4 4 8 select * from (explain plan actions = 1 select * from (select *, a + b as x from tab) order by x * c) where explain like '%sort description%'; - Prefix sort description: multiply(x_4, c_2) ASC - Result sort description: multiply(x_4, c_2) ASC + Prefix sort description: multiply(__table1.x, __table1.c) ASC + Result sort description: multiply(__table1.x, __table1.c) ASC select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y); 0 0 0 0 0 nan 0 0 0 0 0 nan @@ -210,8 +210,8 @@ select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y) 4 4 4 4 8 1 4 4 4 4 8 1 select * from (explain plan actions = 1 select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y)) where explain like '%sort description%'; - Prefix sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC - Result sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC + Prefix sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC + Result sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y); 0 0 0 0 0 nan 0 0 0 0 0 nan @@ -224,8 +224,8 @@ select * from (select *, a / b as y from (select *, a + b as x from tab)) order 4 4 4 4 8 1 4 4 4 4 8 1 select * from (explain plan actions = 1 select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y)) where explain like '%sort description%'; - Prefix sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC - Result sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC + Prefix sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC + Result sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC -- { echoOn } select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3); @@ -238,8 +238,8 @@ select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intD 2020-02-05 00:00:00 3 3 2020-02-05 00:00:00 3 3 select * from (explain plan actions = 1 select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; - Prefix sort description: toTimezone(toTimezone(x_0, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC - Result sort description: toTimezone(toTimezone(x_0, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC + Prefix sort description: toTimezone(toTimezone(__table1.x, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC + Result sort description: toTimezone(toTimezone(__table1.x, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); 2020-02-02 00:00:00 0 0 2020-02-02 00:00:00 0 0 @@ -250,12 +250,12 @@ select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); 2020-02-05 00:00:00 3 3 2020-02-05 00:00:00 3 3 select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; - Prefix sort description: toStartOfDay(x_0) ASC - Result sort description: toStartOfDay(x_0) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC + Prefix sort description: toStartOfDay(__table1.x) ASC + Result sort description: toStartOfDay(__table1.x) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC -- select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; - Prefix sort description: intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC - Result sort description: intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC + Prefix sort description: intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC + Result sort description: intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC -- { echoOn } -- Union (not fully supported) @@ -281,8 +281,8 @@ select * from (select * from tab union all select * from tab3) order by (a + b) 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab3) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder ReadType: InOrder select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b); @@ -291,8 +291,8 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t 3 3 3 3 3 3 3 3 select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: sin(divide(a_0, b_1)) ASC - Result sort description: sin(divide(a_0, b_1)) ASC + Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC + Result sort description: sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder ReadType: InOrder select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b); @@ -309,8 +309,8 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: sin(divide(a_0, b_1)) ASC - Result sort description: sin(divide(a_0, b_1)) ASC + Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC + Result sort description: sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder ReadType: InOrder select * from (select * from tab union all select * from tab5) order by (a + b) * c; @@ -335,8 +335,8 @@ select * from (select * from tab union all select * from tab5) order by (a + b) 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC ReadType: InOrder ReadType: InOrder select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b); @@ -361,11 +361,11 @@ select * from (select * from tab union all select * from tab5) order by (a + b) 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder -- Union with limit select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3; @@ -375,12 +375,12 @@ select * from (select * from tab union all select * from tab5) order by (a + b) select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%'; Limit (preliminary LIMIT (without OFFSET)) Limit 3 - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC Limit 3 ReadType: InOrder - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder -- In this example, we read-in-order from tab up to ((a + b) * c, sin(a / b)) and from tab5 up to ((a + b) * c). -- In case of tab5, there would be two finish sorting transforms: ((a + b) * c) -> ((a + b) * c, sin(a / b)) -> ((a + b) * c, sin(a / b), d). @@ -393,14 +393,14 @@ select * from (select * from tab union all select * from tab5 union all select * select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%'; Limit (preliminary LIMIT (without OFFSET)) Limit 3 - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC, d_3 ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC, __table1.d ASC Limit 3 ReadType: InOrder - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder - Sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC, d_3 ASC + Sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC, __table1.d ASC Limit 3 ReadType: Default drop table if exists tab; diff --git a/tests/queries/0_stateless/02932_kill_query_sleep.sh b/tests/queries/0_stateless/02932_kill_query_sleep.sh index 08c375b875d..84e84204aa1 100755 --- a/tests/queries/0_stateless/02932_kill_query_sleep.sh +++ b/tests/queries/0_stateless/02932_kill_query_sleep.sh @@ -8,18 +8,31 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function wait_query_started() { local query_id="$1" - $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.query_log WHERE query_id='$query_id' AND current_database = currentDatabase()") == 0 ]]; do - sleep 0.1; - $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS;" + timeout=60 + start=$EPOCHSECONDS + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") == 0 ]]; do + if ((EPOCHSECONDS-start > timeout )); then + echo "Timeout while waiting for query $query_id to start" + exit 1 + fi + sleep 0.1 done } + function kill_query() { local query_id="$1" $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id'" >/dev/null - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") != 0 ]]; do sleep 0.1; done + timeout=60 + start=$EPOCHSECONDS + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") != 0 ]]; do + if ((EPOCHSECONDS-start > timeout )); then + echo "Timeout while waiting for query $query_id to cancel" + exit 1 + fi + sleep 0.1 + done } diff --git a/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference b/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference index aac3e471264..aac8c4f777e 100644 --- a/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference +++ b/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference @@ -1 +1 @@ -data Array(Tuple(Nullable(Int64), Tuple(a Nullable(Int64), b Nullable(Int64)), Nullable(Int64), Nullable(String))) +data Array(Tuple(Nullable(Int64), Tuple(\n a Nullable(Int64),\n b Nullable(Int64)), Nullable(Int64), Nullable(String))) diff --git a/tests/queries/0_stateless/02952_conjunction_optimization.reference b/tests/queries/0_stateless/02952_conjunction_optimization.reference index 64663cea662..eeadfaae21d 100644 --- a/tests/queries/0_stateless/02952_conjunction_optimization.reference +++ b/tests/queries/0_stateless/02952_conjunction_optimization.reference @@ -9,7 +9,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02952_disjunction_optimization + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization WHERE FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -27,7 +27,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02952_disjunction_optimization + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: Bool ARGUMENTS @@ -48,7 +48,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02952_disjunction_optimization + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -73,7 +73,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02952_disjunction_optimization + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -100,7 +100,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02952_disjunction_optimization + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization WHERE FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02961_read_bool_as_string_json.reference b/tests/queries/0_stateless/02961_read_bool_as_string_json.reference new file mode 100644 index 00000000000..56f15989a45 --- /dev/null +++ b/tests/queries/0_stateless/02961_read_bool_as_string_json.reference @@ -0,0 +1,12 @@ +true +false +str +true +false +str +['true','false'] +['false','true'] +['str1','str2'] +['true','false'] +['false','true'] +['str1','str2'] diff --git a/tests/queries/0_stateless/02961_read_bool_as_string_json.sql b/tests/queries/0_stateless/02961_read_bool_as_string_json.sql new file mode 100644 index 00000000000..b9f4a7926f9 --- /dev/null +++ b/tests/queries/0_stateless/02961_read_bool_as_string_json.sql @@ -0,0 +1,9 @@ +set input_format_json_read_bools_as_strings=1; +select * from format(JSONEachRow, 'x String', '{"x" : true}, {"x" : false}, {"x" : "str"}'); +select * from format(JSONEachRow, '{"x" : true}, {"x" : false}, {"x" : "str"}'); +select * from format(JSONEachRow, 'x String', '{"x" : tru}'); -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} +select * from format(JSONEachRow, 'x String', '{"x" : fals}'); -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} +select * from format(JSONEachRow, 'x String', '{"x" : atru}'); -- {serverError INCORRECT_DATA} +select * from format(JSONEachRow, 'x Array(String)', '{"x" : [true, false]}, {"x" : [false, true]}, {"x" : ["str1", "str2"]}'); +select * from format(JSONEachRow, '{"x" : [true, false]}, {"x" : [false, true]}, {"x" : ["str1", "str2"]}'); +