diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 64372a90613..9a719a205d4 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -23,10 +23,10 @@ jobs: clear-repository: true # to ensure correct digests fetch-depth: 0 # to get version filter: tree:0 - - name: Check sync PR + - name: Merge sync PR run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 sync_pr.py || : + python3 sync_pr.py --merge || : - name: Python unit tests run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 74ce8452de8..0eac9a9a722 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -157,16 +157,25 @@ jobs: ################################# Stage Final ################################# # FinishCheck: - if: ${{ !failure() && !cancelled() && github.event_name != 'merge_group' }} - needs: [Tests_1, Tests_2] + if: ${{ !failure() && !cancelled() }} + needs: [Tests_1, Tests_2, Builds_1_Report, Builds_2_Report] runs-on: [self-hosted, style-checker] steps: - name: Check out repository code uses: ClickHouse/checkout@v1 + - name: Check sync status + if: ${{ github.event_name == 'merge_group' }} + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 sync_pr.py --status - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 finish_check.py + - name: Auto merge if approved + if: ${{ github.event_name != 'merge_group' }} + run: | + cd "$GITHUB_WORKSPACE/tests/ci" python3 merge_pr.py --check-approved diff --git a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml index 023f257253a..c31d2fd7f39 100644 --- a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml +++ b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml @@ -26,6 +26,11 @@ 200 + + + + + diff --git a/docs/en/operations/settings/composable-protocols.md b/docs/en/operations/settings/composable-protocols.md new file mode 100644 index 00000000000..b68a5906abf --- /dev/null +++ b/docs/en/operations/settings/composable-protocols.md @@ -0,0 +1,155 @@ +--- +slug: /en/operations/settings/composable-protocols +sidebar_position: 64 +sidebar_label: Composable Protocols +--- + +# Composable Protocols + +Composable protocols allows more flexible configuration of TCP access to the ClickHouse server. This configuration can co-exist with or replace conventional configuration. + +## Composable protocols section is denoted as `protocols` in configuration xml +**Example:** +``` xml + + + +``` + +## Basic modules define protocol layers +**Example:** +``` xml + + + + + http + + + +``` +where: +- `plain_http` - name which can be referred by another layer +- `type` - denotes protocol handler which will be instantiated to process data, set of protocol handlers is predefined: + * `tcp` - native clickhouse protocol handler + * `http` - http clickhouse protocol handler + * `tls` - TLS encryption layer + * `proxy1` - PROXYv1 layer + * `mysql` - MySQL compatibility protocol handler + * `postgres` - PostgreSQL compatibility protocol handler + * `prometheus` - Prometheus protocol handler + * `interserver` - clickhouse interserver handler + +:::note +`gRPC` protocol handler is not implemented for `Composable protocols` +::: + +## Endpoint (i.e. listening port) is denoted by `` and (optional) `` tags +**Example:** +``` xml + + + + + http + + 127.0.0.1 + 8123 + + + + +``` +If `` is omitted, then `` from root config is used. + +## Layers sequence is defined by `` tag, referencing another module +**Example:** definition for HTTPS protocol +``` xml + + + + + http + + + + + tls + plain_http + 127.0.0.1 + 8443 + + + +``` + +## Endpoint can be attached to any layer +**Example:** definition for HTTP (port 8123) and HTTPS (port 8443) endpoints +``` xml + + + + http + 127.0.0.1 + 8123 + + + + tls + plain_http + 127.0.0.1 + 8443 + + + +``` + +## Additional endpoints can be defined by referencing any module and omitting `` tag +**Example:** `another_http` endpoint is defined for `plain_http` module +``` xml + + + + http + 127.0.0.1 + 8123 + + + + tls + plain_http + 127.0.0.1 + 8443 + + + + plain_http + 127.0.0.1 + 8223 + + + +``` + +## Some modules can contain specific for its layer parameters +**Example:** for TLS layer private key (`privateKeyFile`) and certificate files (`certificateFile`) can be specified +``` xml + + + + http + 127.0.0.1 + 8123 + + + + tls + plain_http + 127.0.0.1 + 8443 + another_server.key + another_server.crt + + + +``` diff --git a/docs/en/operations/system-tables/backup_log.md b/docs/en/operations/system-tables/backup_log.md index c73fd26683e..d9c2a61cb81 100644 --- a/docs/en/operations/system-tables/backup_log.md +++ b/docs/en/operations/system-tables/backup_log.md @@ -9,6 +9,7 @@ Columns: - `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Date of the entry. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time of the entry. - `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Time of the entry with microseconds precision. - `id` ([String](../../sql-reference/data-types/string.md)) — Identifier of the backup or restore operation. - `name` ([String](../../sql-reference/data-types/string.md)) — Name of the backup storage (the contents of the `FROM` or `TO` clause). @@ -67,6 +68,7 @@ Row 2: ────── hostname: clickhouse.eu-central1.internal event_date: 2023-08-19 +event_time: 2023-08-19 11:08:56 event_time_microseconds: 2023-08-19 11:08:56.916192 id: e5b74ecb-f6f1-426a-80be-872f90043885 name: Disk('backups_disk', '1.zip') diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index f7e56e73520..df266b224fb 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -970,7 +970,7 @@ If the haystack or the LIKE expression are not valid UTF-8, the behavior is unde No automatic Unicode normalization is performed, you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that. -To match against literal `%`, `_` and `/` (which are LIKE metacharacters), prepend them with a backslash: `\%`, `\_` and `\\`. +To match against literal `%`, `_` and `\` (which are LIKE metacharacters), prepend them with a backslash: `\%`, `\_` and `\\`. The backslash loses its special meaning (i.e. is interpreted literally) if it prepends a character different than `%`, `_` or `\`. Note that ClickHouse requires backslashes in strings [to be quoted as well](../syntax.md#string), so you would actually need to write `\\%`, `\\_` and `\\\\`. @@ -1768,4 +1768,4 @@ SELECT hasTokenCaseInsensitiveOrNull('Hello World','hello,world'); ```response null -``` \ No newline at end of file +``` diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 941dc000a02..ce5cecf6fd6 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -133,8 +133,6 @@ For the query to run successfully, the following conditions must be met: - Both tables must have the same indices and projections. - Both tables must have the same storage policy. -If both tables have the same storage policy, use hardlink to attach partition. Otherwise, use copying the data to attach partition. - ## REPLACE PARTITION ``` sql diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 090dda383b3..d757a9192ce 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -482,6 +482,7 @@ void Client::connect() server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch); load_suggestions = is_interactive && (server_revision >= Suggest::MIN_SERVER_REVISION) && !config().getBool("disable_suggestion", false); + wait_for_suggestions_to_load = config().getBool("wait_for_suggestions_to_load", false); if (server_display_name = connection->getServerDisplayName(connection_parameters.timeouts); server_display_name.empty()) server_display_name = config().getString("host", "localhost"); diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 20c1a0ad4a8..f2ef3857d63 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -46,12 +46,12 @@ INCBIN(resource_users_xml, SOURCE_DIR "/programs/server/users.xml"); * * The following steps are performed: * - * - copying the binary to binary directory (/usr/bin). + * - copying the binary to binary directory (/usr/bin/) * - creation of symlinks for tools. * - creation of clickhouse user and group. - * - creation of config directory (/etc/clickhouse-server). + * - creation of config directory (/etc/clickhouse-server/). * - creation of default configuration files. - * - creation of a directory for logs (/var/log/clickhouse-server). + * - creation of a directory for logs (/var/log/clickhouse-server/). * - creation of a data directory if not exists. * - setting a password for default user. * - choose an option to listen connections. @@ -226,7 +226,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv) desc.add_options() ("help,h", "produce help message") ("prefix", po::value()->default_value("/"), "prefix for all paths") +#if defined (OS_DARWIN) + /// https://stackoverflow.com/a/36734569/22422288 + ("binary-path", po::value()->default_value("usr/local/bin"), "where to install binaries") +#else ("binary-path", po::value()->default_value("usr/bin"), "where to install binaries") +#endif ("config-path", po::value()->default_value("etc/clickhouse-server"), "where to install configs") ("log-path", po::value()->default_value("var/log/clickhouse-server"), "where to create log directory") ("data-path", po::value()->default_value("var/lib/clickhouse"), "directory for data") @@ -1216,7 +1221,12 @@ int mainEntryClickHouseStart(int argc, char ** argv) desc.add_options() ("help,h", "produce help message") ("prefix", po::value()->default_value("/"), "prefix for all paths") +#if defined (OS_DARWIN) + /// https://stackoverflow.com/a/36734569/22422288 + ("binary-path", po::value()->default_value("usr/local/bin"), "directory with binary") +#else ("binary-path", po::value()->default_value("usr/bin"), "directory with binary") +#endif ("config-path", po::value()->default_value("etc/clickhouse-server"), "directory with configs") ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user") @@ -1332,7 +1342,12 @@ int mainEntryClickHouseRestart(int argc, char ** argv) desc.add_options() ("help,h", "produce help message") ("prefix", po::value()->default_value("/"), "prefix for all paths") +#if defined (OS_DARWIN) + /// https://stackoverflow.com/a/36734569/22422288 + ("binary-path", po::value()->default_value("usr/local/bin"), "directory with binary") +#else ("binary-path", po::value()->default_value("usr/bin"), "directory with binary") +#endif ("config-path", po::value()->default_value("etc/clickhouse-server"), "directory with configs") ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user") diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index a0aa6d085d8..690e20b25be 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -572,6 +572,7 @@ void LocalServer::processConfig() const std::string clickhouse_dialect{"clickhouse"}; load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false) && config().getString("dialect", clickhouse_dialect) == clickhouse_dialect; + wait_for_suggestions_to_load = config().getBool("wait_for_suggestions_to_load", false); auto logging = (config().has("logger.console") || config().has("logger.level") @@ -847,6 +848,8 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp config().setString("logger.level", options["logger.level"].as()); if (options.count("send_logs_level")) config().setString("send_logs_level", options["send_logs_level"].as()); + if (options.count("wait_for_suggestions_to_load")) + config().setBool("wait_for_suggestions_to_load", true); } void LocalServer::readArguments(int argc, char ** argv, Arguments & common_arguments, std::vector &, std::vector &) diff --git a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp index d0a5656d334..3e2a2055fdb 100644 --- a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp +++ b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp @@ -15,6 +15,7 @@ #include #include +#include namespace DB @@ -61,47 +62,7 @@ const QueryTreeNodePtr & getEquiArgument(const QueryTreeNodePtr & cond, size_t i return func->getArguments().getNodes()[index]; } - -/// Check that node has only one source and return it. -/// {_, false} - multiple sources -/// {nullptr, true} - no sources -/// {source, true} - single source -std::pair getExpressionSource(const QueryTreeNodePtr & node) -{ - if (const auto * column = node->as()) - { - auto source = column->getColumnSourceOrNull(); - if (!source) - return {nullptr, false}; - return {source.get(), true}; - } - - if (const auto * func = node->as()) - { - const IQueryTreeNode * source = nullptr; - const auto & args = func->getArguments().getNodes(); - for (const auto & arg : args) - { - auto [arg_source, is_ok] = getExpressionSource(arg); - if (!is_ok) - return {nullptr, false}; - - if (!source) - source = arg_source; - else if (arg_source && !source->isEqual(*arg_source)) - return {nullptr, false}; - } - return {source, true}; - - } - - if (node->as()) - return {nullptr, true}; - - return {nullptr, false}; -} - -bool findInTableExpression(const IQueryTreeNode * source, const QueryTreeNodePtr & table_expression) +bool findInTableExpression(const QueryTreeNodePtr & source, const QueryTreeNodePtr & table_expression) { if (!source) return true; @@ -115,7 +76,6 @@ bool findInTableExpression(const IQueryTreeNode * source, const QueryTreeNodePtr || findInTableExpression(source, join_node->getRightTableExpression()); } - return false; } @@ -169,10 +129,10 @@ public: auto left_src = getExpressionSource(lhs_equi_argument); auto right_src = getExpressionSource(rhs_equi_argument); - if (left_src.second && right_src.second && left_src.first && right_src.first) + if (left_src && right_src) { - if ((findInTableExpression(left_src.first, left_table) && findInTableExpression(right_src.first, right_table)) || - (findInTableExpression(left_src.first, right_table) && findInTableExpression(right_src.first, left_table))) + if ((findInTableExpression(left_src, left_table) && findInTableExpression(right_src, right_table)) || + (findInTableExpression(left_src, right_table) && findInTableExpression(right_src, left_table))) { can_convert_cross_to_inner = true; continue; diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 5f08bb9035e..b798a256ebd 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -25,8 +25,9 @@ class JoinOnLogicalExpressionOptimizerVisitor : public InDepthQueryTreeVisitorWi public: using Base = InDepthQueryTreeVisitorWithContext; - explicit JoinOnLogicalExpressionOptimizerVisitor(ContextPtr context) + explicit JoinOnLogicalExpressionOptimizerVisitor(const JoinNode * join_node_, ContextPtr context) : Base(std::move(context)) + , join_node(join_node_) {} void enterImpl(QueryTreeNodePtr & node) @@ -55,10 +56,11 @@ public: } private: + const JoinNode * join_node; bool need_rerun_resolve = false; /// Returns true if type of some operand is changed and parent function needs to be re-resolved - static bool tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node, const ContextPtr & context) + bool tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node, const ContextPtr & context) { auto & function_node = node->as(); chassert(function_node.getFunctionName() == "or"); @@ -93,6 +95,21 @@ private: const auto & func_name = argument_function->getFunctionName(); if (func_name == "equals" || func_name == "isNotDistinctFrom") { + const auto & argument_nodes = argument_function->getArguments().getNodes(); + if (argument_nodes.size() != 2) + continue; + /// We can rewrite to a <=> b only if we are joining on a and b, + /// because the function is not yet implemented for other cases. + auto first_src = getExpressionSource(argument_nodes[0]); + auto second_src = getExpressionSource(argument_nodes[1]); + if (!first_src || !second_src) + continue; + const auto & lhs_join = *join_node->getLeftTableExpression(); + const auto & rhs_join = *join_node->getRightTableExpression(); + bool arguments_from_both_sides = (first_src->isEqual(lhs_join) && second_src->isEqual(rhs_join)) || + (first_src->isEqual(rhs_join) && second_src->isEqual(lhs_join)); + if (!arguments_from_both_sides) + continue; equals_functions_indices.push_back(or_operands.size() - 1); } else if (func_name == "and") @@ -231,7 +248,7 @@ public: /// Operator <=> is not supported outside of JOIN ON section if (join_node->hasJoinExpression()) { - JoinOnLogicalExpressionOptimizerVisitor join_on_visitor(getContext()); + JoinOnLogicalExpressionOptimizerVisitor join_on_visitor(join_node, getContext()); join_on_visitor.visit(join_node->getJoinExpression()); } return; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 2882c4e0c02..4299050a094 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -760,6 +760,54 @@ QueryTreeNodePtr createCastFunction(QueryTreeNodePtr node, DataTypePtr result_ty return function_node; } +/** Returns: + * {_, false} - multiple sources + * {nullptr, true} - no sources (for constants) + * {source, true} - single source + */ +std::pair getExpressionSourceImpl(const QueryTreeNodePtr & node) +{ + if (const auto * column = node->as()) + { + auto source = column->getColumnSourceOrNull(); + if (!source) + return {nullptr, false}; + return {source, true}; + } + + if (const auto * func = node->as()) + { + QueryTreeNodePtr source = nullptr; + const auto & args = func->getArguments().getNodes(); + for (const auto & arg : args) + { + auto [arg_source, is_ok] = getExpressionSourceImpl(arg); + if (!is_ok) + return {nullptr, false}; + + if (!source) + source = arg_source; + else if (arg_source && !source->isEqual(*arg_source)) + return {nullptr, false}; + } + return {source, true}; + + } + + if (node->as()) + return {nullptr, true}; + + return {nullptr, false}; +} + +QueryTreeNodePtr getExpressionSource(const QueryTreeNodePtr & node) +{ + auto [source, is_ok] = getExpressionSourceImpl(node); + if (!is_ok) + return nullptr; + return source; +} + QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(QueryTreeNodePtr table_node, const ContextPtr & context) { const auto & storage_snapshot = table_node->as()->getStorageSnapshot(); diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 1b4a7d5ef3c..93d439453c6 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -105,6 +105,9 @@ NameSet collectIdentifiersFullNames(const QueryTreeNodePtr & node); /// Wrap node into `_CAST` function QueryTreeNodePtr createCastFunction(QueryTreeNodePtr node, DataTypePtr result_type, ContextPtr context); +/// Checks that node has only one source and returns it +QueryTreeNodePtr getExpressionSource(const QueryTreeNodePtr & node); + /// Build subquery which we execute for `IN table` function. QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(QueryTreeNodePtr table_node, const ContextPtr & context); diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 2efb6bf84ad..58dac0c0843 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -125,7 +125,7 @@ BackupEntries BackupEntriesCollector::run() = BackupSettings::Util::filterHostIDs(backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); /// Do renaming in the create queries according to the renaming config. - renaming_map = makeRenamingMapFromBackupQuery(backup_query_elements); + renaming_map = BackupUtils::makeRenamingMap(backup_query_elements); /// Calculate the root path for collecting backup entries, it's either empty or has the format "shards//replicas//". calculateRootPathInBackup(); @@ -570,17 +570,16 @@ std::vector> BackupEntriesCollector::findTablesInD checkIsQueryCancelled(); - auto filter_by_table_name = [my_database_info = &database_info](const String & table_name) + auto filter_by_table_name = [&](const String & table_name) { - /// We skip inner tables of materialized views. - if (table_name.starts_with(".inner_id.")) + if (BackupUtils::isInnerTable(database_name, table_name)) return false; - if (my_database_info->tables.contains(table_name)) + if (database_info.tables.contains(table_name)) return true; - if (my_database_info->all_tables) - return !my_database_info->except_table_names.contains(table_name); + if (database_info.all_tables) + return !database_info.except_table_names.contains(table_name); return false; }; diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 6efca053f05..fb448fb64ad 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -8,10 +8,10 @@ #include -namespace DB +namespace DB::BackupUtils { -DDLRenamingMap makeRenamingMapFromBackupQuery(const ASTBackupQuery::Elements & elements) +DDLRenamingMap makeRenamingMap(const ASTBackupQuery::Elements & elements) { DDLRenamingMap map; @@ -120,4 +120,15 @@ bool compareRestoredDatabaseDef(const IAST & restored_database_create_query, con return compareRestoredTableDef(restored_database_create_query, create_query_from_backup, global_context); } +bool isInnerTable(const QualifiedTableName & table_name) +{ + return isInnerTable(table_name.database, table_name.table); +} + +bool isInnerTable(const String & /* database_name */, const String & table_name) +{ + /// We skip inner tables of materialized views. + return table_name.starts_with(".inner.") || table_name.starts_with(".inner_id."); +} + } diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index 7976de818e2..ba889eccc48 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -9,9 +9,13 @@ namespace DB class IBackup; class AccessRightsElements; class DDLRenamingMap; +struct QualifiedTableName; + +namespace BackupUtils +{ /// Initializes a DDLRenamingMap from a BACKUP or RESTORE query. -DDLRenamingMap makeRenamingMapFromBackupQuery(const ASTBackupQuery::Elements & elements); +DDLRenamingMap makeRenamingMap(const ASTBackupQuery::Elements & elements); /// Returns access required to execute BACKUP query. AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements & elements); @@ -20,4 +24,10 @@ AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements & bool compareRestoredTableDef(const IAST & restored_table_create_query, const IAST & create_query_from_backup, const ContextPtr & global_context); bool compareRestoredDatabaseDef(const IAST & restored_database_create_query, const IAST & create_query_from_backup, const ContextPtr & global_context); +/// Returns true if this table should be skipped while making a backup because it's an inner table. +bool isInnerTable(const QualifiedTableName & table_name); +bool isInnerTable(const String & database_name, const String & table_name); + +} + } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 9a3e1052e0b..bb04c72dd3c 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -564,7 +564,7 @@ void BackupsWorker::doBackup( /// Checks access rights if this is not ON CLUSTER query. /// (If this is ON CLUSTER query executeDDLQueryOnCluster() will check access rights later.) - auto required_access = getRequiredAccessToBackup(backup_query->elements); + auto required_access = BackupUtils::getRequiredAccessToBackup(backup_query->elements); if (!on_cluster) context->checkAccess(required_access); diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index d8383fdbb1d..ec0a717cfcd 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -124,7 +124,7 @@ void RestorerFromBackup::run(Mode mode) restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); /// Do renaming in the create queries according to the renaming config. - renaming_map = makeRenamingMapFromBackupQuery(restore_query_elements); + renaming_map = BackupUtils::makeRenamingMap(restore_query_elements); /// Calculate the root path in the backup for restoring, it's either empty or has the format "shards//replicas//". findRootPathsInBackup(); @@ -346,12 +346,12 @@ void RestorerFromBackup::findDatabasesAndTablesInBackup() { case ASTBackupQuery::ElementType::TABLE: { - findTableInBackup({element.database_name, element.table_name}, element.partitions); + findTableInBackup({element.database_name, element.table_name}, /* skip_if_inner_table= */ false, element.partitions); break; } case ASTBackupQuery::ElementType::TEMPORARY_TABLE: { - findTableInBackup({DatabaseCatalog::TEMPORARY_DATABASE, element.table_name}, element.partitions); + findTableInBackup({DatabaseCatalog::TEMPORARY_DATABASE, element.table_name}, /* skip_if_inner_table= */ false, element.partitions); break; } case ASTBackupQuery::ElementType::DATABASE: @@ -370,14 +370,14 @@ void RestorerFromBackup::findDatabasesAndTablesInBackup() LOG_INFO(log, "Will restore {} databases and {} tables", getNumDatabases(), getNumTables()); } -void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional & partitions) +void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional & partitions) { schedule( - [this, table_name_in_backup, partitions]() { findTableInBackupImpl(table_name_in_backup, partitions); }, + [this, table_name_in_backup, skip_if_inner_table, partitions]() { findTableInBackupImpl(table_name_in_backup, skip_if_inner_table, partitions); }, "Restore_FindTbl"); } -void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, const std::optional & partitions) +void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional & partitions) { bool is_temporary_table = (table_name_in_backup.database == DatabaseCatalog::TEMPORARY_DATABASE); @@ -422,6 +422,10 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_ = *root_path_in_use / "data" / escapeForFileName(table_name_in_backup.database) / escapeForFileName(table_name_in_backup.table); } + QualifiedTableName table_name = renaming_map.getNewTableName(table_name_in_backup); + if (skip_if_inner_table && BackupUtils::isInnerTable(table_name)) + return; + auto read_buffer = backup->readFile(*metadata_path); String create_query_str; readStringUntilEOF(create_query_str, *read_buffer); @@ -432,8 +436,6 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_ renameDatabaseAndTableNameInCreateQuery(create_table_query, renaming_map, context->getGlobalContext()); String create_table_query_str = serializeAST(*create_table_query); - QualifiedTableName table_name = renaming_map.getNewTableName(table_name_in_backup); - bool is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table}); auto table_dependencies = getDependenciesFromCreateQuery(context, table_name, create_table_query); bool table_has_data = backup->hasFiles(data_path_in_backup); @@ -568,7 +570,7 @@ void RestorerFromBackup::findDatabaseInBackupImpl(const String & database_name_i if (except_table_names.contains({database_name_in_backup, table_name_in_backup})) continue; - findTableInBackup({database_name_in_backup, table_name_in_backup}, /* partitions= */ {}); + findTableInBackup({database_name_in_backup, table_name_in_backup}, /* skip_if_inner_table= */ true, /* partitions= */ {}); } } @@ -767,7 +769,7 @@ void RestorerFromBackup::checkDatabase(const String & database_name) ASTPtr existing_database_def = database->getCreateDatabaseQuery(); ASTPtr database_def_from_backup = database_info.create_database_query; - if (!compareRestoredDatabaseDef(*existing_database_def, *database_def_from_backup, context->getGlobalContext())) + if (!BackupUtils::compareRestoredDatabaseDef(*existing_database_def, *database_def_from_backup, context->getGlobalContext())) { throw Exception( ErrorCodes::CANNOT_RESTORE_DATABASE, @@ -938,7 +940,7 @@ void RestorerFromBackup::checkTable(const QualifiedTableName & table_name) { ASTPtr existing_table_def = database->getCreateTableQuery(resolved_id.table_name, context); ASTPtr table_def_from_backup = table_info.create_table_query; - if (!compareRestoredTableDef(*existing_table_def, *table_def_from_backup, context->getGlobalContext())) + if (!BackupUtils::compareRestoredTableDef(*existing_table_def, *table_def_from_backup, context->getGlobalContext())) { throw Exception( ErrorCodes::CANNOT_RESTORE_TABLE, diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index 238569ac755..7b36eea0ba0 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -92,8 +92,8 @@ private: void findRootPathsInBackup(); void findDatabasesAndTablesInBackup(); - void findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional & partitions); - void findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, const std::optional & partitions); + void findTableInBackup(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional & partitions); + void findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional & partitions); void findDatabaseInBackup(const String & database_name_in_backup, const std::set & except_table_names); void findDatabaseInBackupImpl(const String & database_name_in_backup, const std::set & except_table_names); void findEverythingInBackup(const std::set & except_database_names, const std::set & except_table_names); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index d243a1d74b9..19b7c059f18 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2477,9 +2477,9 @@ void ClientBase::runInteractive() { /// Load suggestion data from the server. if (global_context->getApplicationType() == Context::ApplicationType::CLIENT) - suggest->load(global_context, connection_parameters, config().getInt("suggestion_limit")); + suggest->load(global_context, connection_parameters, config().getInt("suggestion_limit"), wait_for_suggestions_to_load); else if (global_context->getApplicationType() == Context::ApplicationType::LOCAL) - suggest->load(global_context, connection_parameters, config().getInt("suggestion_limit")); + suggest->load(global_context, connection_parameters, config().getInt("suggestion_limit"), wait_for_suggestions_to_load); } if (home_path.empty()) @@ -2975,6 +2975,7 @@ void ClientBase::init(int argc, char ** argv) ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off") ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") + ("wait_for_suggestions_to_load", "Load suggestion data synchonously.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") ("echo", "in batch mode, print query before execution") @@ -3104,6 +3105,8 @@ void ClientBase::init(int argc, char ** argv) config().setBool("echo", true); if (options.count("disable_suggestion")) config().setBool("disable_suggestion", true); + if (options.count("wait_for_suggestions_to_load")) + config().setBool("wait_for_suggestions_to_load", true); if (options.count("suggestion_limit")) config().setInt("suggestion_limit", options["suggestion_limit"].as()); if (options.count("highlight")) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 710a72a3238..9ee77122e2e 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -209,6 +209,7 @@ protected: std::optional suggest; bool load_suggestions = false; + bool wait_for_suggestions_to_load = false; std::vector queries; /// Queries passed via '--query' std::vector queries_files; /// If not empty, queries will be read from these files diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 03df582de10..f63dbc64d27 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -110,7 +110,7 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti } template -void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit) +void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit, bool wait_for_load) { loading_thread = std::thread([my_context = Context::createCopy(context), connection_parameters, suggestion_limit, this] { @@ -152,6 +152,9 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p /// Note that keyword suggestions are available even if we cannot load data from server. }); + + if (wait_for_load) + loading_thread.join(); } void Suggest::load(IServerConnection & connection, @@ -228,8 +231,8 @@ void Suggest::fillWordsFromBlock(const Block & block) } template -void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit); +void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit, bool wait_for_load); template -void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit); +void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit, bool wait_for_load); } diff --git a/src/Client/Suggest.h b/src/Client/Suggest.h index 5cecdc4501b..aac8a73f702 100644 --- a/src/Client/Suggest.h +++ b/src/Client/Suggest.h @@ -27,7 +27,7 @@ public: /// Load suggestions for clickhouse-client. template - void load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit); + void load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit, bool wait_for_load); void load(IServerConnection & connection, const ConnectionTimeouts & timeouts, diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 4634eae7759..48bb510e589 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -403,7 +403,7 @@ void ZooKeeperSetACLRequest::readImpl(ReadBuffer & in) std::string ZooKeeperSetACLRequest::toStringImpl() const { - return fmt::format("path = {}\n", "version = {}", path, version); + return fmt::format("path = {}\nversion = {}", path, version); } void ZooKeeperSetACLResponse::writeImpl(WriteBuffer & out) const @@ -457,7 +457,7 @@ void ZooKeeperCheckRequest::readImpl(ReadBuffer & in) std::string ZooKeeperCheckRequest::toStringImpl() const { - return fmt::format("path = {}\n", "version = {}", path, version); + return fmt::format("path = {}\nversion = {}", path, version); } void ZooKeeperErrorResponse::readImpl(ReadBuffer & in) diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index a5c1d21eda6..1d9830505f8 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -2,6 +2,7 @@ #include #include +#include namespace Coordination @@ -64,3 +65,12 @@ static constexpr int32_t DEFAULT_OPERATION_TIMEOUT_MS = 10000; static constexpr int32_t DEFAULT_CONNECTION_TIMEOUT_MS = 1000; } + +/// This is used by fmt::format to print OpNum as strings. +/// All OpNum values should be in range [min, max] to be printed. +template <> +struct magic_enum::customize::enum_range +{ + static constexpr int min = -100; + static constexpr int max = 1000; +}; diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 38167e4eb21..0969d188ffc 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -90,7 +90,8 @@ static std::map sett {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fileds"}, - }}, + {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, + }}, {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, @@ -130,7 +131,6 @@ static std::map sett {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index f2f1d985cc9..744cea9f4dc 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3721,8 +3721,23 @@ namespace return std::make_shared>(std::move(values)); } - std::optional getNameAndDataTypeFromField(const google::protobuf::FieldDescriptor * field_descriptor, bool skip_unsupported_fields, bool allow_repeat = true) + std::optional getNameAndDataTypeFromField( + const google::protobuf::FieldDescriptor * field_descriptor, bool skip_unsupported_fields, bool allow_repeat); + + std::optional getNameAndDataTypeFromFieldRecursive( + const google::protobuf::FieldDescriptor * field_descriptor, + bool skip_unsupported_fields, + bool allow_repeat, + std::unordered_set & pending_resolution) { + if (pending_resolution.contains(field_descriptor)) + { + if (skip_unsupported_fields) + return std::nullopt; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouse doesn't support type recursion ({})", field_descriptor->full_name()); + } + pending_resolution.emplace(field_descriptor); + if (allow_repeat && field_descriptor->is_map()) { auto name_and_type = getNameAndDataTypeFromField(field_descriptor, skip_unsupported_fields, false); @@ -3804,7 +3819,8 @@ namespace else if (message_descriptor->field_count() == 1) { const auto * nested_field_descriptor = message_descriptor->field(0); - auto nested_name_and_type = getNameAndDataTypeFromField(nested_field_descriptor, skip_unsupported_fields); + auto nested_name_and_type + = getNameAndDataTypeFromFieldRecursive(nested_field_descriptor, skip_unsupported_fields, true, pending_resolution); if (!nested_name_and_type) return std::nullopt; return NameAndTypePair{field_descriptor->name() + "_" + nested_name_and_type->name, nested_name_and_type->type}; @@ -3815,7 +3831,8 @@ namespace Strings nested_names; for (int i = 0; i != message_descriptor->field_count(); ++i) { - auto nested_name_and_type = getNameAndDataTypeFromField(message_descriptor->field(i), skip_unsupported_fields); + auto nested_name_and_type = getNameAndDataTypeFromFieldRecursive( + message_descriptor->field(i), skip_unsupported_fields, true, pending_resolution); if (!nested_name_and_type) continue; nested_types.push_back(nested_name_and_type->type); @@ -3831,6 +3848,14 @@ namespace UNREACHABLE(); } + + std::optional getNameAndDataTypeFromField( + const google::protobuf::FieldDescriptor * field_descriptor, bool skip_unsupported_fields, bool allow_repeat = true) + { + /// Keep track of the fields that are pending resolution to avoid recursive types, which are unsupported + std::unordered_set pending_resolution{}; + return getNameAndDataTypeFromFieldRecursive(field_descriptor, skip_unsupported_fields, allow_repeat, pending_resolution); + } } std::unique_ptr ProtobufSerializer::create( diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 09e9364a3f1..06e6e1f8fc8 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2135,13 +2135,6 @@ ConjunctionNodes getConjunctionNodes(ActionsDAG::Node * predicate, std::unordere } } - // std::cerr << "Allowed " << conjunction.allowed.size() << std::endl; - // for (const auto & node : conjunction.allowed) - // std::cerr << node->result_name << std::endl; - // std::cerr << "Rejected " << conjunction.rejected.size() << std::endl; - // for (const auto & node : conjunction.rejected) - // std::cerr << node->result_name << std::endl; - return conjunction; } @@ -2170,7 +2163,7 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt /// /// Result actions add single column with conjunction result (it is always first in outputs). /// No other columns are added or removed. -ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs) +ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs) { if (conjunction.empty()) return nullptr; @@ -2265,9 +2258,9 @@ ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunctio return actions; } -ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( +ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( const std::string & filter_name, - bool can_remove_filter, + bool removes_filter, const Names & available_inputs, const ColumnsWithTypeAndName & all_inputs) { @@ -2321,16 +2314,232 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( } } - auto actions = cloneActionsForConjunction(conjunction.allowed, all_inputs); + auto actions = createActionsForConjunction(conjunction.allowed, all_inputs); if (!actions) return nullptr; /// Now, when actions are created, update the current DAG. + removeUnusedConjunctions(std::move(conjunction.rejected), predicate, removes_filter); - if (conjunction.rejected.empty()) + return actions; +} + +ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPushDown( + const std::string & filter_name, + bool removes_filter, + const Names & left_stream_available_columns_to_push_down, + const Block & left_stream_header, + const Names & right_stream_available_columns_to_push_down, + const Block & right_stream_header, + const Names & equivalent_columns_to_push_down, + const std::unordered_map & equivalent_left_stream_column_to_right_stream_column, + const std::unordered_map & equivalent_right_stream_column_to_left_stream_column) +{ + Node * predicate = const_cast(tryFindInOutputs(filter_name)); + if (!predicate) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Output nodes for ActionsDAG do not contain filter column name {}. DAG:\n{}", + filter_name, + dumpDAG()); + + /// If condition is constant let's do nothing. + /// It means there is nothing to push down or optimization was already applied. + if (predicate->type == ActionType::COLUMN) + return {}; + + auto get_input_nodes = [this](const Names & inputs_names) + { + std::unordered_set allowed_nodes; + + std::unordered_map> inputs_map; + for (const auto & input_node : inputs) + inputs_map[input_node->result_name].emplace_back(input_node); + + for (const auto & name : inputs_names) + { + auto & inputs_list = inputs_map[name]; + if (inputs_list.empty()) + continue; + + allowed_nodes.emplace(inputs_list.front()); + inputs_list.pop_front(); + } + + return allowed_nodes; + }; + + auto left_stream_allowed_nodes = get_input_nodes(left_stream_available_columns_to_push_down); + auto right_stream_allowed_nodes = get_input_nodes(right_stream_available_columns_to_push_down); + auto both_streams_allowed_nodes = get_input_nodes(equivalent_columns_to_push_down); + + auto left_stream_push_down_conjunctions = getConjunctionNodes(predicate, left_stream_allowed_nodes); + auto right_stream_push_down_conjunctions = getConjunctionNodes(predicate, right_stream_allowed_nodes); + auto both_streams_push_down_conjunctions = getConjunctionNodes(predicate, both_streams_allowed_nodes); + + NodeRawConstPtrs left_stream_allowed_conjunctions = std::move(left_stream_push_down_conjunctions.allowed); + NodeRawConstPtrs right_stream_allowed_conjunctions = std::move(right_stream_push_down_conjunctions.allowed); + + std::unordered_set left_stream_allowed_conjunctions_set(left_stream_allowed_conjunctions.begin(), left_stream_allowed_conjunctions.end()); + std::unordered_set right_stream_allowed_conjunctions_set(right_stream_allowed_conjunctions.begin(), right_stream_allowed_conjunctions.end()); + + for (const auto * both_streams_push_down_allowed_conjunction_node : both_streams_push_down_conjunctions.allowed) + { + if (!left_stream_allowed_conjunctions_set.contains(both_streams_push_down_allowed_conjunction_node)) + left_stream_allowed_conjunctions.push_back(both_streams_push_down_allowed_conjunction_node); + + if (!right_stream_allowed_conjunctions_set.contains(both_streams_push_down_allowed_conjunction_node)) + right_stream_allowed_conjunctions.push_back(both_streams_push_down_allowed_conjunction_node); + } + + std::unordered_set rejected_conjunctions_set; + rejected_conjunctions_set.insert(left_stream_push_down_conjunctions.rejected.begin(), left_stream_push_down_conjunctions.rejected.end()); + rejected_conjunctions_set.insert(right_stream_push_down_conjunctions.rejected.begin(), right_stream_push_down_conjunctions.rejected.end()); + rejected_conjunctions_set.insert(both_streams_push_down_conjunctions.rejected.begin(), both_streams_push_down_conjunctions.rejected.end()); + + for (const auto & left_stream_allowed_conjunction : left_stream_allowed_conjunctions) + rejected_conjunctions_set.erase(left_stream_allowed_conjunction); + + for (const auto & right_stream_allowed_conjunction : right_stream_allowed_conjunctions) + rejected_conjunctions_set.erase(right_stream_allowed_conjunction); + + NodeRawConstPtrs rejected_conjunctions(rejected_conjunctions_set.begin(), rejected_conjunctions_set.end()); + + if (rejected_conjunctions.size() == 1) + { + chassert(rejected_conjunctions.front()->result_type); + + bool left_stream_push_constant = !left_stream_allowed_conjunctions.empty() && left_stream_allowed_conjunctions[0]->type == ActionType::COLUMN; + bool right_stream_push_constant = !right_stream_allowed_conjunctions.empty() && right_stream_allowed_conjunctions[0]->type == ActionType::COLUMN; + + if ((left_stream_push_constant || right_stream_push_constant) && !rejected_conjunctions.front()->result_type->equals(*predicate->result_type)) + { + /// No further optimization can be done + return {}; + } + } + + auto left_stream_filter_to_push_down = createActionsForConjunction(left_stream_allowed_conjunctions, left_stream_header.getColumnsWithTypeAndName()); + auto right_stream_filter_to_push_down = createActionsForConjunction(right_stream_allowed_conjunctions, right_stream_header.getColumnsWithTypeAndName()); + + auto replace_equivalent_columns_in_filter = [](const ActionsDAGPtr & filter, + const Block & stream_header, + const std::unordered_map & columns_to_replace) + { + auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter->getOutputs()[0]}, columns_to_replace); + chassert(updated_filter->getOutputs().size() == 1); + + /** If result filter to left or right stream has column that is one of the stream inputs, we need distinguish filter column from + * actual input column. It is necessary because after filter step, filter column became constant column with value 1, and + * not all JOIN algorithms properly work with constants. + * + * Example: SELECT key FROM ( SELECT key FROM t1 ) AS t1 JOIN ( SELECT key FROM t1 ) AS t2 ON t1.key = t2.key WHERE key; + */ + const auto * stream_filter_node = updated_filter->getOutputs()[0]; + if (stream_header.has(stream_filter_node->result_name)) + { + const auto & alias_node = updated_filter->addAlias(*stream_filter_node, "__filter" + stream_filter_node->result_name); + updated_filter->getOutputs()[0] = &alias_node; + } + + std::unordered_map> updated_filter_inputs; + + for (const auto & input : updated_filter->getInputs()) + updated_filter_inputs[input->result_name].push_back(input); + + for (const auto & input : filter->getInputs()) + { + if (updated_filter_inputs.contains(input->result_name)) + continue; + + const Node * updated_filter_input_node = nullptr; + + auto it = columns_to_replace.find(input->result_name); + if (it != columns_to_replace.end()) + updated_filter_input_node = &updated_filter->addInput(it->second); + else + updated_filter_input_node = &updated_filter->addInput({input->column, input->result_type, input->result_name}); + + updated_filter_inputs[input->result_name].push_back(updated_filter_input_node); + } + + for (const auto & input_column : stream_header.getColumnsWithTypeAndName()) + { + const Node * input; + auto & list = updated_filter_inputs[input_column.name]; + if (list.empty()) + { + input = &updated_filter->addInput(input_column); + } + else + { + input = list.front(); + list.pop_front(); + } + + if (input != updated_filter->getOutputs()[0]) + updated_filter->outputs.push_back(input); + } + + return updated_filter; + }; + + if (left_stream_filter_to_push_down) + left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(left_stream_filter_to_push_down, + left_stream_header, + equivalent_right_stream_column_to_left_stream_column); + + if (right_stream_filter_to_push_down) + right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(right_stream_filter_to_push_down, + right_stream_header, + equivalent_left_stream_column_to_right_stream_column); + + /* + * We should check the presence of a split filter column name in stream columns to avoid removing the required column. + * + * Example: + * A filter expression is `a AND b = c`, but `b` and `c` belong to another side of the join and not in allowed columns to push down, + * so the final split filter is just `a`. + * In this case `a` can be in stream columns but not `and(a, equals(b, c))`. + */ + + bool left_stream_filter_removes_filter = true; + bool right_stream_filter_removes_filter = true; + + if (left_stream_filter_to_push_down) + { + const auto & left_stream_filter_column_name = left_stream_filter_to_push_down->getOutputs()[0]->result_name; + left_stream_filter_removes_filter = !left_stream_header.has(left_stream_filter_column_name); + } + + if (right_stream_filter_to_push_down) + { + const auto & right_stream_filter_column_name = right_stream_filter_to_push_down->getOutputs()[0]->result_name; + right_stream_filter_removes_filter = !right_stream_header.has(right_stream_filter_column_name); + } + + ActionsDAG::ActionsForJOINFilterPushDown result + { + .left_stream_filter_to_push_down = std::move(left_stream_filter_to_push_down), + .left_stream_filter_removes_filter = left_stream_filter_removes_filter, + .right_stream_filter_to_push_down = std::move(right_stream_filter_to_push_down), + .right_stream_filter_removes_filter = right_stream_filter_removes_filter + }; + + if (!result.left_stream_filter_to_push_down && !result.right_stream_filter_to_push_down) + return result; + + /// Now, when actions are created, update the current DAG. + removeUnusedConjunctions(std::move(rejected_conjunctions), predicate, removes_filter); + + return result; +} + +void ActionsDAG::removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter) +{ + if (rejected_conjunctions.empty()) { /// The whole predicate was split. - if (can_remove_filter) + if (removes_filter) { /// If filter column is not needed, remove it from output nodes. std::erase_if(outputs, [&](const Node * node) { return node == predicate; }); @@ -2362,7 +2571,7 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( { /// Predicate is conjunction, where both allowed and rejected sets are not empty. - NodeRawConstPtrs new_children = std::move(conjunction.rejected); + NodeRawConstPtrs new_children = std::move(rejected_conjunctions); if (new_children.size() == 1 && new_children.front()->result_type->equals(*predicate->result_type)) { @@ -2403,13 +2612,12 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( std::unordered_set used_inputs; for (const auto * input : inputs) { - if (can_remove_filter && input == predicate) + if (removes_filter && input == predicate) continue; used_inputs.insert(input); } removeUnusedActions(used_inputs); - return actions; } static bool isColumnSortingPreserved(const ActionsDAG::Node * start_node, const String & sorted_column) @@ -2557,8 +2765,11 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( auto input_node_it = node_name_to_input_node_column.find(node->result_name); if (input_node_it != node_name_to_input_node_column.end()) { - result_node = &result_dag->addInput(input_node_it->second); - node_to_result_node.emplace(node, result_node); + auto & result_input = result_inputs[input_node_it->second.name]; + if (!result_input) + result_input = &result_dag->addInput(input_node_it->second); + + node_to_result_node.emplace(node, result_input); nodes_to_process.pop_back(); continue; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 469fe9ea7f1..a8a377866d3 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -372,12 +372,46 @@ public: /// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`. /// To avoid it, add inputs from `all_inputs` list, /// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order. - ActionsDAGPtr cloneActionsForFilterPushDown( + ActionsDAGPtr splitActionsForFilterPushDown( const std::string & filter_name, - bool can_remove_filter, + bool removes_filter, const Names & available_inputs, const ColumnsWithTypeAndName & all_inputs); + struct ActionsForJOINFilterPushDown + { + ActionsDAGPtr left_stream_filter_to_push_down; + bool left_stream_filter_removes_filter; + ActionsDAGPtr right_stream_filter_to_push_down; + bool right_stream_filter_removes_filter; + }; + + /** Split actions for JOIN filter push down. + * + * @param filter_name - name of filter node in current DAG. + * @param removes_filter - if filter is removed after it is applied. + * @param left_stream_available_columns_to_push_down - columns from left stream that are safe to use in push down conditions + * to left stream. + * @param left_stream_header - left stream header. + * @param right_stream_available_columns_to_push_down - columns from right stream that are safe to use in push down conditions + * to right stream. + * @param right_stream_header - right stream header. + * @param equivalent_columns_to_push_down - columns from left and right streams that are safe to use in push down conditions + * to left and right streams. + * @param equivalent_left_stream_column_to_right_stream_column - equivalent left stream column name to right stream column map. + * @param equivalent_right_stream_column_to_left_stream_column - equivalent right stream column name to left stream column map. + */ + ActionsForJOINFilterPushDown splitActionsForJOINFilterPushDown( + const std::string & filter_name, + bool removes_filter, + const Names & left_stream_available_columns_to_push_down, + const Block & left_stream_header, + const Names & right_stream_available_columns_to_push_down, + const Block & right_stream_header, + const Names & equivalent_columns_to_push_down, + const std::unordered_map & equivalent_left_stream_column_to_right_stream_column, + const std::unordered_map & equivalent_right_stream_column_to_left_stream_column); + bool isSortingPreserved(const Block & input_header, const SortDescription & sort_description, const String & ignore_output_column = "") const; @@ -429,7 +463,9 @@ private: void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); + static ActionsDAGPtr createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); + + void removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter); }; class FindOriginalNodeForOutputName diff --git a/src/Interpreters/BackupLog.cpp b/src/Interpreters/BackupLog.cpp index af6c7cf6234..a22c6e322bf 100644 --- a/src/Interpreters/BackupLog.cpp +++ b/src/Interpreters/BackupLog.cpp @@ -24,6 +24,7 @@ ColumnsDescription BackupLogElement::getColumnsDescription() { {"hostname", std::make_shared(std::make_shared()), "Hostname of the server executing the query."}, {"event_date", std::make_shared(), "Date of the entry."}, + {"event_time", std::make_shared(), "Time of the entry."}, {"event_time_microseconds", std::make_shared(6), "Time of the entry with microseconds precision."}, {"id", std::make_shared(), "Identifier of the backup or restore operation."}, {"name", std::make_shared(), "Name of the backup storage (the contents of the FROM or TO clause)."}, @@ -48,6 +49,7 @@ void BackupLogElement::appendToBlock(MutableColumns & columns) const size_t i = 0; columns[i++]->insert(getFQDNOrHostName()); columns[i++]->insert(DateLUT::instance().toDayNum(std::chrono::system_clock::to_time_t(event_time)).toUnderType()); + columns[i++]->insert(std::chrono::system_clock::to_time_t(event_time)); columns[i++]->insert(event_time_usec); columns[i++]->insert(info.id); columns[i++]->insert(info.name); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 656b6cdaa6e..02c1c56fae2 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -39,6 +39,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int INCORRECT_QUERY; } namespace @@ -500,7 +501,41 @@ public: ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); private: - using NodeNameAndNodeMinLevel = std::pair; + + class Levels + { + public: + explicit Levels(size_t level) { set(level); } + + void set(size_t level) + { + check(level); + if (level) + mask |= (uint64_t(1) << (level - 1)); + } + + void reset(size_t level) + { + check(level); + if (level) + mask &= ~(uint64_t(1) << (level - 1)); + } + + void add(Levels levels) { mask |= levels.mask; } + + size_t max() const { return 64 - getLeadingZeroBits(mask); } + + private: + uint64_t mask = 0; + + void check(size_t level) + { + if (level > 64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Maximum lambda depth exceeded. Maximum 64."); + } + }; + + using NodeNameAndNodeMinLevel = std::pair; NodeNameAndNodeMinLevel visitImpl(QueryTreeNodePtr node); @@ -586,11 +621,11 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi column_source->getNodeType() == QueryTreeNodeType::LAMBDA && actions_stack[i].getScopeNode().get() == column_source.get()) { - return {column_node_name, i}; + return {column_node_name, Levels(i)}; } } - return {column_node_name, 0}; + return {column_node_name, Levels(0)}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstant(const QueryTreeNodePtr & node) @@ -660,7 +695,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); } - return {constant_node_name, 0}; + return {constant_node_name, Levels(0)}; } @@ -688,7 +723,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi auto lambda_actions_dag = std::make_shared(); actions_stack.emplace_back(lambda_actions_dag, node); - auto [lambda_expression_node_name, level] = visitImpl(lambda_node.getExpression()); + auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression()); lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); @@ -699,8 +734,9 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi ActionsDAG::NodeRawConstPtrs lambda_children; Names required_column_names = lambda_actions->getRequiredColumns(); - if (level == actions_stack.size() - 1) - --level; + actions_stack.pop_back(); + levels.reset(actions_stack.size()); + size_t level = levels.max(); const auto & lambda_argument_names = lambda_node.getArgumentNames(); @@ -718,7 +754,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi auto lambda_node_name = calculateActionNodeName(node, *planner_context); auto function_capture = std::make_shared( lambda_actions, captured_column_names, lambda_arguments_names_and_types, lambda_node.getExpression()->getResultType(), lambda_expression_node_name); - actions_stack.pop_back(); // TODO: Pass IFunctionBase here not FunctionCaptureOverloadResolver. const auto * actions_node = actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), function_capture); @@ -735,7 +770,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi actions_stack_node.addInputColumnIfNecessary(lambda_node_name, result_type); } - return {lambda_node_name, level}; + return {lambda_node_name, levels}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::makeSetForInFunction(const QueryTreeNodePtr & node) @@ -799,7 +834,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); } - return {column.name, 0}; + return {column.name, Levels(0)}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node) @@ -833,7 +868,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi size_t index_hint_function_level = actions_stack.size() - 1; actions_stack[index_hint_function_level].addFunctionIfNecessary(function_node_name, {}, index_hint_function_overload_resolver); - return {function_node_name, index_hint_function_level}; + return {function_node_name, Levels(index_hint_function_level)}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitFunction(const QueryTreeNodePtr & node) @@ -868,7 +903,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType()); } - return {function_node_name, 0}; + return {function_node_name, Levels(0)}; } const auto & function_arguments = function_node.getArguments().getNodes(); @@ -877,14 +912,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi Names function_arguments_node_names; function_arguments_node_names.reserve(function_arguments_size); - size_t level = 0; + Levels levels(0); for (size_t function_argument_index = 0; function_argument_index < function_arguments_size; ++function_argument_index) { if (in_function_second_argument_node_name_with_level && function_argument_index == 1) { - auto & [node_name, node_min_level] = *in_function_second_argument_node_name_with_level; + auto & [node_name, node_levels] = *in_function_second_argument_node_name_with_level; function_arguments_node_names.push_back(std::move(node_name)); - level = std::max(level, node_min_level); + levels.add(node_levels); continue; } @@ -892,20 +927,21 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi if (argument->getNodeType() == QueryTreeNodeType::LAMBDA) { - auto [node_name, node_min_level] = visitLambda(argument); + auto [node_name, node_levels] = visitLambda(argument); function_arguments_node_names.push_back(std::move(node_name)); - level = std::max(level, node_min_level); + levels.add(node_levels); continue; } - auto [node_name, node_min_level] = visitImpl(argument); + auto [node_name, node_levels] = visitImpl(argument); function_arguments_node_names.push_back(std::move(node_name)); - level = std::max(level, node_min_level); + levels.add(node_levels); } ActionsDAG::NodeRawConstPtrs children; children.reserve(function_arguments_size); + size_t level = levels.max(); for (auto & function_argument_node_name : function_arguments_node_names) children.push_back(actions_stack[level].getNodeOrThrow(function_argument_node_name)); @@ -930,7 +966,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType()); } - return {function_node_name, level}; + return {function_node_name, levels}; } } diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp index 2382b3cf27a..c643ae060d6 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp @@ -86,7 +86,7 @@ size_t ProtobufListInputFormat::countRows(size_t max_block_size) ProtobufListSchemaReader::ProtobufListSchemaReader(const FormatSettings & format_settings) : schema_info( format_settings.schema.format_schema, "Protobuf", true, format_settings.schema.is_server, format_settings.schema.format_schema_path) - , skip_unsopported_fields(format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference) + , skip_unsupported_fields(format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference) , google_protos_path(format_settings.protobuf.google_protos_path) { } @@ -95,7 +95,7 @@ NamesAndTypesList ProtobufListSchemaReader::readSchema() { const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::Yes, google_protos_path); - return protobufSchemaToCHSchema(message_descriptor, skip_unsopported_fields); + return protobufSchemaToCHSchema(message_descriptor, skip_unsupported_fields); } void registerInputFormatProtobufList(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.h b/src/Processors/Formats/Impl/ProtobufListInputFormat.h index 947696bba82..8305af28506 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.h @@ -56,7 +56,7 @@ public: private: const FormatSchemaInfo schema_info; - bool skip_unsopported_fields; + bool skip_unsupported_fields; const String google_protos_path; }; diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index e020cd3c4d3..939d0900c86 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -24,6 +24,7 @@ public: void describeActions(FormatSettings & settings) const override; const ActionsDAGPtr & getExpression() const { return actions_dag; } + ActionsDAGPtr & getExpression() { return actions_dag; } const String & getFilterColumnName() const { return filter_column_name; } bool removesFilterColumn() const { return remove_filter_column; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index e71bcc5602a..ebf780bb692 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -100,7 +100,7 @@ static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node) return res; } -static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & allowed_inputs, size_t child_idx = 0) +static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0) { QueryPlan::Node * child_node = parent_node->children.front(); checkChildrenSize(child_node, child_idx + 1); @@ -114,14 +114,12 @@ static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & al bool removes_filter = filter->removesFilterColumn(); const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName(); - - auto split_filter = expression->cloneActionsForFilterPushDown(filter_column_name, removes_filter, allowed_inputs, all_inputs); - return split_filter; + return expression->splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs); } static size_t -tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const ActionsDAGPtr & split_filter, - bool can_remove_filter = true, size_t child_idx = 0) +addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const ActionsDAGPtr & split_filter, + bool can_remove_filter = true, size_t child_idx = 0, bool update_parent_filter = true) { QueryPlan::Node * child_node = parent_node->children.front(); checkChildrenSize(child_node, child_idx + 1); @@ -134,21 +132,18 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con const auto & filter_column_name = filter->getFilterColumnName(); const auto * filter_node = expression->tryFindInOutputs(filter_column_name); - if (!filter_node && !filter->removesFilterColumn()) + if (update_parent_filter && !filter_node && !filter->removesFilterColumn()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", filter_column_name, expression->dumpDAG()); - /// Filter column was replaced to constant. - const bool filter_is_constant = filter_node && filter_node->column && isColumnConst(*filter_node->column); - - /// Add new Filter step before Aggregating. - /// Expression/Filter -> Aggregating -> Something + /// Add new Filter step before Child. + /// Expression/Filter -> Child -> Something auto & node = nodes.emplace_back(); node.children.emplace_back(&node); std::swap(node.children[0], child_node->children[child_idx]); - /// Expression/Filter -> Aggregating -> Filter -> Something + /// Expression/Filter -> Child -> Filter -> Something /// New filter column is the first one. String split_filter_column_name = split_filter->getOutputs().front()->result_name; @@ -171,12 +166,22 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con ErrorCodes::LOGICAL_ERROR, "We are trying to push down a filter through a step for which we cannot update input stream"); } - if (!filter_node || filter_is_constant) - /// This means that all predicates of filter were pushed down. - /// Replace current actions to expression, as we don't need to filter anything. - parent = std::make_unique(child->getOutputStream(), expression); - else - filter->updateInputStream(child->getOutputStream()); + if (update_parent_filter) + { + /// Filter column was replaced to constant. + const bool filter_is_constant = filter_node && filter_node->column && isColumnConst(*filter_node->column); + + if (!filter_node || filter_is_constant) + { + /// This means that all predicates of filter were pushed down. + /// Replace current actions to expression, as we don't need to filter anything. + parent = std::make_unique(child->getOutputStream(), expression); + } + else + { + filter->updateInputStream(child->getOutputStream()); + } + } return 3; } @@ -186,7 +191,7 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con bool can_remove_filter = true, size_t child_idx = 0) { if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx)) - return tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx); + return addNewFilterStepOrThrow(parent_node, nodes, split_filter, can_remove_filter, child_idx); return 0; } @@ -204,6 +209,204 @@ static size_t simplePushDownOverStep(QueryPlan::Node * parent_node, QueryPlan::N return 0; } +static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, QueryPlanStepPtr & child) +{ + auto & parent = parent_node->step; + auto * filter = assert_cast(parent.get()); + + auto * join = typeid_cast(child.get()); + auto * filled_join = typeid_cast(child.get()); + + if (!join && !filled_join) + return 0; + + /** For equivalent JOIN with condition `ON lhs.x_1 = rhs.y_1 AND lhs.x_2 = rhs.y_2 ...`, we can build equivalent sets of columns and this + * will allow to push conditions that only use columns from equivalent sets to both sides of JOIN, without considering JOIN type. + * + * For example: `FROM lhs INNER JOIN rhs ON lhs.id = rhs.id AND lhs.value = rhs.value` + * In this example columns `id` and `value` from both tables are equivalent. + * + * During filter push down for different JOIN types filter push down logic is different: + * + * 1. For INNER JOIN we can push all valid conditions to both sides of JOIN. We also can push all valid conditions that use columns from + * equivalent sets to both sides of JOIN. + * 2. For LEFT/RIGHT JOIN we can push conditions that use columns from LEFT/RIGHT stream to LEFT/RIGHT JOIN side. We can also push conditions + * that use columns from LEFT/RIGHT equivalent sets to RIGHT/LEFT JOIN side. + * + * Additional filter push down optimizations: + * 1. TODO: Support building equivalent sets for more than 2 JOINS. It is possible, but will require more complex analysis step. + * 2. TODO: Support building equivalent sets for JOINs with more than 1 clause. + * 3. TODO: For LEFT/RIGHT JOIN, we can assume that RIGHT/LEFT columns used in filter will be default/NULL constants and + * check if filter will always be false, in those scenario we can transform LEFT/RIGHT JOIN into INNER JOIN and push conditions to both tables. + * 4. TODO: It is possible to pull up filter conditions from LEFT/RIGHT stream and push conditions that use columns from LEFT/RIGHT equivalent sets + * to RIGHT/LEFT JOIN side. + */ + + const auto & join_header = child->getOutputStream().header; + const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); + const auto & left_stream_input_header = child->getInputStreams().front().header; + const auto & right_stream_input_header = child->getInputStreams().back().header; + + if (table_join.kind() == JoinKind::Full) + return 0; + + std::unordered_map equivalent_left_stream_column_to_right_stream_column; + std::unordered_map equivalent_right_stream_column_to_left_stream_column; + + bool has_single_clause = table_join.getClauses().size() == 1; + + if (has_single_clause) + { + const auto & join_clause = table_join.getClauses()[0]; + size_t key_names_size = join_clause.key_names_left.size(); + + for (size_t i = 0; i < key_names_size; ++i) + { + const auto & left_table_key_name = join_clause.key_names_left[i]; + const auto & right_table_key_name = join_clause.key_names_right[i]; + + if (!join_header.has(left_table_key_name) || !join_header.has(right_table_key_name)) + continue; + + const auto & left_table_column = left_stream_input_header.getByName(left_table_key_name); + const auto & right_table_column = right_stream_input_header.getByName(right_table_key_name); + + if (!left_table_column.type->equals(*right_table_column.type)) + continue; + + equivalent_left_stream_column_to_right_stream_column[left_table_key_name] = right_table_column; + equivalent_right_stream_column_to_left_stream_column[right_table_key_name] = left_table_column; + } + } + + auto get_available_columns_for_filter = [&](bool push_to_left_stream, bool filter_push_down_input_columns_available) + { + Names available_input_columns_for_filter; + + if (!filter_push_down_input_columns_available) + return available_input_columns_for_filter; + + const auto & input_header = push_to_left_stream ? left_stream_input_header : right_stream_input_header; + const auto & input_columns_names = input_header.getNames(); + + for (const auto & name : input_columns_names) + { + if (!join_header.has(name)) + continue; + + /// Skip if type is changed. Push down expression expect equal types. + if (!input_header.getByName(name).type->equals(*join_header.getByName(name).type)) + continue; + + available_input_columns_for_filter.push_back(name); + } + + return available_input_columns_for_filter; + }; + + bool left_stream_filter_push_down_input_columns_available = true; + bool right_stream_filter_push_down_input_columns_available = true; + + if (table_join.kind() == JoinKind::Left) + right_stream_filter_push_down_input_columns_available = false; + else if (table_join.kind() == JoinKind::Right) + left_stream_filter_push_down_input_columns_available = false; + + /** We disable push down to right table in cases: + * 1. Right side is already filled. Example: JOIN with Dictionary. + * 2. ASOF Right join is not supported. + */ + bool allow_push_down_to_right = join && join->allowPushDownToRight() && table_join.strictness() != JoinStrictness::Asof; + if (!allow_push_down_to_right) + right_stream_filter_push_down_input_columns_available = false; + + Names equivalent_columns_to_push_down; + + if (left_stream_filter_push_down_input_columns_available) + { + for (const auto & [name, _] : equivalent_left_stream_column_to_right_stream_column) + equivalent_columns_to_push_down.push_back(name); + } + + if (right_stream_filter_push_down_input_columns_available) + { + for (const auto & [name, _] : equivalent_right_stream_column_to_left_stream_column) + equivalent_columns_to_push_down.push_back(name); + } + + Names left_stream_available_columns_to_push_down = get_available_columns_for_filter(true /*push_to_left_stream*/, left_stream_filter_push_down_input_columns_available); + Names right_stream_available_columns_to_push_down = get_available_columns_for_filter(false /*push_to_left_stream*/, right_stream_filter_push_down_input_columns_available); + + auto join_filter_push_down_actions = filter->getExpression()->splitActionsForJOINFilterPushDown(filter->getFilterColumnName(), + filter->removesFilterColumn(), + left_stream_available_columns_to_push_down, + left_stream_input_header.getColumnsWithTypeAndName(), + right_stream_available_columns_to_push_down, + right_stream_input_header.getColumnsWithTypeAndName(), + equivalent_columns_to_push_down, + equivalent_left_stream_column_to_right_stream_column, + equivalent_right_stream_column_to_left_stream_column); + + size_t updated_steps = 0; + + if (join_filter_push_down_actions.left_stream_filter_to_push_down) + { + updated_steps += addNewFilterStepOrThrow(parent_node, + nodes, + join_filter_push_down_actions.left_stream_filter_to_push_down, + join_filter_push_down_actions.left_stream_filter_removes_filter, + 0 /*child_idx*/, + false /*update_parent_filter*/); + LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), + "Pushed down filter {} to the {} side of join", + join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name, + JoinKind::Left); + } + + if (join_filter_push_down_actions.right_stream_filter_to_push_down) + { + updated_steps += addNewFilterStepOrThrow(parent_node, + nodes, + join_filter_push_down_actions.right_stream_filter_to_push_down, + join_filter_push_down_actions.right_stream_filter_removes_filter, + 1 /*child_idx*/, + false /*update_parent_filter*/); + LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), + "Pushed down filter {} to the {} side of join", + join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name, + JoinKind::Right); + } + + if (updated_steps > 0) + { + const auto & filter_column_name = filter->getFilterColumnName(); + const auto & filter_expression = filter->getExpression(); + + const auto * filter_node = filter_expression->tryFindInOutputs(filter_column_name); + if (!filter_node && !filter->removesFilterColumn()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", + filter_column_name, filter_expression->dumpDAG()); + + + /// Filter column was replaced to constant. + const bool filter_is_constant = filter_node && filter_node->column && isColumnConst(*filter_node->column); + + if (!filter_node || filter_is_constant) + { + /// This means that all predicates of filter were pushed down. + /// Replace current actions to expression, as we don't need to filter anything. + parent = std::make_unique(child->getOutputStream(), filter_expression); + } + else + { + filter->updateInputStream(child->getOutputStream()); + } + } + + return updated_steps; +} + size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) { if (parent_node->children.size() != 1) @@ -317,9 +520,6 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (!keys.contains(column.name)) allowed_inputs.push_back(column.name); - // for (const auto & name : allowed_inputs) - // std::cerr << name << std::endl; - if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) return updated_steps; } @@ -327,77 +527,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; - auto * join = typeid_cast(child.get()); - auto * filled_join = typeid_cast(child.get()); - - if (join || filled_join) - { - auto join_push_down = [&](JoinKind kind) -> size_t - { - const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); - - /// Only inner, cross and left(/right) join are supported. Other types may generate default values for left table keys. - /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) - return 0; - - /// There is no ASOF Right join, so we're talking about pushing to the right side - if (kind == JoinKind::Right && table_join.strictness() == JoinStrictness::Asof) - return 0; - - bool is_left = kind == JoinKind::Left; - const auto & input_header = is_left ? child->getInputStreams().front().header : child->getInputStreams().back().header; - const auto & res_header = child->getOutputStream().header; - Names allowed_keys; - const auto & source_columns = input_header.getNames(); - for (const auto & name : source_columns) - { - /// Skip key if it is renamed. - /// I don't know if it is possible. Just in case. - if (!input_header.has(name) || !res_header.has(name)) - continue; - - /// Skip if type is changed. Push down expression expect equal types. - if (!input_header.getByName(name).type->equals(*res_header.getByName(name).type)) - continue; - - allowed_keys.push_back(name); - } - - /// For left JOIN, push down to the first child; for right - to the second one. - const auto child_idx = is_left ? 0 : 1; - ActionsDAGPtr split_filter = splitFilter(parent_node, allowed_keys, child_idx); - if (!split_filter) - return 0; - /* - * We should check the presence of a split filter column name in `source_columns` to avoid removing the required column. - * - * Example: - * A filter expression is `a AND b = c`, but `b` and `c` belong to another side of the join and not in `allowed_keys`, so the final split filter is just `a`. - * In this case `a` can be in `source_columns` but not `and(a, equals(b, c))`. - * - * New filter column is the first one. - */ - const String & split_filter_column_name = split_filter->getOutputs().front()->result_name; - bool can_remove_filter = source_columns.end() == std::find(source_columns.begin(), source_columns.end(), split_filter_column_name); - const size_t updated_steps = tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx); - if (updated_steps > 0) - { - LOG_DEBUG(getLogger("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", split_filter_column_name, kind); - } - return updated_steps; - }; - - if (size_t updated_steps = join_push_down(JoinKind::Left)) - return updated_steps; - - /// For full sorting merge join we push down both to the left and right tables, because left and right streams are not independent. - if (join && join->allowPushDownToRight()) - { - if (size_t updated_steps = join_push_down(JoinKind::Right)) - return updated_steps; - } - } + if (auto updated_steps = tryPushDownOverJoinStep(parent_node, nodes, child)) + return updated_steps; /// TODO. /// We can filter earlier if expression does not depend on WITH FILL columns. diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index b40fea47b3c..64111602458 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -432,13 +432,12 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( { const auto & keys = aggregating.getParams().keys; const auto & aggregates = aggregating.getParams().aggregates; - Block key_virtual_columns = reading.getMergeTreeData().getHeaderWithVirtualsForFilter(); + const auto metadata = reading.getStorageMetadata(); + Block key_virtual_columns = reading.getMergeTreeData().getHeaderWithVirtualsForFilter(metadata); AggregateProjectionCandidates candidates; const auto & parts = reading.getParts(); - - const auto metadata = reading.getStorageMetadata(); ContextPtr context = reading.getContext(); const auto & projections = metadata->projections; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index bee42c3ddde..6bdd060513c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1415,7 +1415,8 @@ static void buildIndexes( indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); + indexes->part_values + = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context); MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(indexes->part_offset_condition, filter_actions_dag, context); indexes->use_skip_indexes = settings.use_skip_indexes; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5d4c3ab078e..4a70e3ec87c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1031,19 +1031,26 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat const Names MergeTreeData::virtuals_useful_for_filter = {"_part", "_partition_id", "_part_uuid", "_partition_value", "_part_data_version"}; -Block MergeTreeData::getHeaderWithVirtualsForFilter() const +Block MergeTreeData::getHeaderWithVirtualsForFilter(const StorageMetadataPtr & metadata) const { + const auto columns = metadata->getColumns().getAllPhysical(); Block header; auto virtuals_desc = getVirtualsPtr(); for (const auto & name : virtuals_useful_for_filter) + { + if (columns.contains(name)) + continue; if (auto column = virtuals_desc->tryGet(name)) header.insert({column->type->createColumn(), column->type, name}); + } + return header; } -Block MergeTreeData::getBlockWithVirtualsForFilter(const MergeTreeData::DataPartsVector & parts, bool ignore_empty) const +Block MergeTreeData::getBlockWithVirtualsForFilter( + const StorageMetadataPtr & metadata, const MergeTreeData::DataPartsVector & parts, bool ignore_empty) const { - auto block = getHeaderWithVirtualsForFilter(); + auto block = getHeaderWithVirtualsForFilter(metadata); for (const auto & part_or_projection : parts) { @@ -1072,7 +1079,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( return 0; auto metadata_snapshot = getInMemoryMetadataPtr(); - auto virtual_columns_block = getBlockWithVirtualsForFilter({parts[0]}); + auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr); if (!filter_dag) @@ -1091,7 +1098,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( std::unordered_set part_values; if (valid) { - virtual_columns_block = getBlockWithVirtualsForFilter(parts); + virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, parts); VirtualColumnUtils::filterBlockWithDAG(filter_dag, virtual_columns_block, local_context); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) @@ -6694,11 +6701,11 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( }; Block virtual_columns_block; - auto virtual_block = getHeaderWithVirtualsForFilter(); + auto virtual_block = getHeaderWithVirtualsForFilter(metadata_snapshot); 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 || filter_dag) { - virtual_columns_block = getBlockWithVirtualsForFilter(parts, /*ignore_empty=*/ true); + virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, parts, /*ignore_empty=*/true); if (virtual_columns_block.rows() == 0) return {}; } @@ -7074,7 +7081,7 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData( return checkStructureAndGetMergeTreeData(*source_table, src_snapshot, my_snapshot); } -std::pair MergeTreeData::cloneAndLoadDataPart( +std::pair MergeTreeData::cloneAndLoadDataPartOnSameDisk( const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, @@ -7084,23 +7091,28 @@ std::pair MergeTreeData::cloneAn const WriteSettings & write_settings) { chassert(!isStaticStorage()); - bool on_same_disk = false; - for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) + + /// Check that the storage policy contains the disk where the src_part is located. + bool does_storage_policy_allow_same_disk = false; + for (const DiskPtr & disk : getStoragePolicy()->getDisks()) { if (disk->getName() == src_part->getDataPartStorage().getDiskName()) { - on_same_disk = true; + does_storage_policy_allow_same_disk = true; break; } } - + if (!does_storage_policy_allow_same_disk) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Could not clone and load part {} because disk does not belong to storage policy", + quoteString(src_part->getDataPartStorage().getFullPath())); String dst_part_name = src_part->getNewName(dst_part_info); String tmp_dst_part_name = tmp_part_prefix + dst_part_name; auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name); /// Why it is needed if we only hardlink files? - /// Answer: In issue #59377, add copy when attach from different disk. auto reservation = src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); auto src_part_storage = src_part->getDataPartStoragePtr(); @@ -7108,30 +7120,16 @@ std::pair MergeTreeData::cloneAn MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; String with_copy; - if (params.copy_instead_of_hardlink || !on_same_disk) + if (params.copy_instead_of_hardlink) with_copy = " (copying data)"; - - std::shared_ptr dst_part_storage{}; - if (on_same_disk && !params.copy_instead_of_hardlink) - { - dst_part_storage = src_part_storage->freeze( - relative_data_path, - tmp_dst_part_name, - read_settings, - write_settings, - /* save_metadata_callback= */ {}, - params); - } - else - { - auto reservation_on_dst = getStoragePolicy()->reserve(src_part->getBytesOnDisk()); - if (!reservation_on_dst) - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on disk."); - dst_part_storage = src_part_storage->clonePart( - this->getRelativeDataPath(), tmp_dst_part_name, reservation_on_dst->getDisk(), read_settings, write_settings, {}, {}); - } - + auto dst_part_storage = src_part_storage->freeze( + relative_data_path, + tmp_dst_part_name, + read_settings, + write_settings, + /* save_metadata_callback= */ {}, + params); if (params.metadata_version_to_write.has_value()) { @@ -7153,7 +7151,7 @@ std::pair MergeTreeData::cloneAn .withPartFormatFromDisk() .build(); - if (on_same_disk && !params.copy_instead_of_hardlink && params.hardlinked_files) + if (!params.copy_instead_of_hardlink && params.hardlinked_files) { params.hardlinked_files->source_part_name = src_part->name; params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); @@ -7197,7 +7195,6 @@ std::pair MergeTreeData::cloneAn return std::make_pair(dst_data_part, std::move(temporary_directory_lock)); } - String MergeTreeData::getFullPathOnDisk(const DiskPtr & disk) const { return disk->getPath() + relative_data_path; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d21f87c337e..0d68d3a4ecb 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -839,7 +839,7 @@ public: MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; - std::pair cloneAndLoadDataPart( + std::pair cloneAndLoadDataPartOnSameDisk( const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, @@ -990,10 +990,11 @@ public: static const Names virtuals_useful_for_filter; /// Construct a sample block of virtual columns. - Block getHeaderWithVirtualsForFilter() const; + Block getHeaderWithVirtualsForFilter(const StorageMetadataPtr & metadata) const; /// Construct a block consisting only of possible virtual columns for part pruning. - Block getBlockWithVirtualsForFilter(const MergeTreeData::DataPartsVector & parts, bool ignore_empty = false) const; + Block getBlockWithVirtualsForFilter( + const StorageMetadataPtr & metadata, const MergeTreeData::DataPartsVector & parts, bool ignore_empty = false) const; /// In merge tree we do inserts with several steps. One of them: /// X. write part to temporary directory with some temp name diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index bcc936c5739..345872efddf 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -473,6 +473,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( } std::optional> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns( + const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, const ActionsDAGPtr & filter_dag, @@ -481,12 +482,12 @@ std::optional> MergeTreeDataSelectExecutor::filterPar if (!filter_dag) return {}; - auto sample = data.getHeaderWithVirtualsForFilter(); + auto sample = data.getHeaderWithVirtualsForFilter(metadata_snapshot); auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), &sample); if (!dag) return {}; - auto virtual_columns_block = data.getBlockWithVirtualsForFilter(parts); + auto virtual_columns_block = data.getBlockWithVirtualsForFilter(metadata_snapshot, parts); VirtualColumnUtils::filterBlockWithDAG(dag, virtual_columns_block, context); return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index b1afd7e6668..ecccd6d55e3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -166,6 +166,7 @@ public: /// 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 StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, const ActionsDAGPtr & filter_dag, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a971c4fda1c..5e388d6a8ac 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2146,7 +2146,7 @@ bool MutateTask::prepare() scope_guard lock; { - std::tie(part, lock) = ctx->data->cloneAndLoadDataPart( + std::tie(part, lock) = ctx->data->cloneAndLoadDataPartOnSameDisk( ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, clone_params, ctx->context->getReadSettings(), ctx->context->getWriteSettings()); part->getDataPartStorage().beginTransaction(); ctx->temporary_directory_lock = std::move(lock); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 86af02be899..0235a74400c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2096,7 +2096,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; - auto [dst_part, part_lock] = cloneAndLoadDataPart( + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( src_part, TMP_PREFIX, dst_part_info, @@ -2207,7 +2207,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const .copy_instead_of_hardlink = getSettings()->always_use_copy_instead_of_hardlinks, }; - auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPart( + auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk( src_part, TMP_PREFIX, dst_part_info, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 15d1b7f4010..fcb946c089c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2788,7 +2788,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry) auto obtain_part = [&] (PartDescriptionPtr & part_desc) { - /// Fetches with zero-copy-replication are cheap, but cloneAndLoadDataPart(OnSameDisk) will do full copy. + /// Fetches with zero-copy-replication are cheap, but cloneAndLoadDataPartOnSameDisk will do full copy. /// It's okay to check the setting for current table and disk for the source table, because src and dst part are on the same disk. bool prefer_fetch_from_other_replica = !part_desc->replica.empty() && storage_settings_ptr->allow_remote_fs_zero_copy_replication && part_desc->src_table_part && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(); @@ -2807,7 +2807,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry) .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || ((our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - auto [res_part, temporary_part_lock] = cloneAndLoadDataPart( + auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, @@ -4888,7 +4888,7 @@ bool StorageReplicatedMergeTree::fetchPart( .keep_metadata_version = true, }; - auto [cloned_part, lock] = cloneAndLoadDataPart( + auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk( part_to_clone, "tmp_clone_", part_info, @@ -8078,14 +8078,12 @@ void StorageReplicatedMergeTree::replacePartitionFrom( bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; - IDataPartStorage::ClonePartParams clone_params { .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - - auto [dst_part, part_lock] = cloneAndLoadDataPart( + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( src_part, TMP_PREFIX, dst_part_info, @@ -8093,10 +8091,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom( clone_params, query_context->getReadSettings(), query_context->getWriteSettings()); - - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); src_parts.emplace_back(src_part); + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); ephemeral_locks.emplace_back(std::move(*lock)); block_id_paths.emplace_back(block_id_path); part_checksums.emplace_back(hash_hex); @@ -8349,7 +8346,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = dest_metadata_snapshot->getMetadataVersion() }; - auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPart( + auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk( src_part, TMP_PREFIX, dst_part_info, diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 8434355ce46..f11d62e9136 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -2121,11 +2121,13 @@ def main() -> int: pr_info, dump_to_file=True, ) - update_mergeable_check( - commit, - pr_info, - job_report.check_name or _get_ext_check_name(args.job_name), - ) + if not pr_info.is_merge_queue(): + # in the merge queue mergeable status must be set only in FinishCheck (last job in wf) + update_mergeable_check( + commit, + pr_info, + job_report.check_name or _get_ext_check_name(args.job_name), + ) print(f"Job report url: [{check_url}]") prepared_events = prepare_tests_results_for_clickhouse( diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index eebc846f4b1..617f4c9d88c 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -28,21 +28,22 @@ def main(): statuses = get_commit_filtered_statuses(commit) trigger_mergeable_check(commit, statuses) - statuses = [s for s in statuses if s.context == CI_STATUS_NAME] - if not statuses: - return - # Take the latest status - status = statuses[-1] - if status.state == PENDING: - post_commit_status( - commit, - SUCCESS, - status.target_url, - "All checks finished", - CI_STATUS_NAME, - pr_info, - dump_to_file=True, - ) + if not pr_info.is_merge_queue(): + statuses = [s for s in statuses if s.context == CI_STATUS_NAME] + if not statuses: + return + # Take the latest status + status = statuses[-1] + if status.state == PENDING: + post_commit_status( + commit, + SUCCESS, + status.target_url, + "All checks finished", + CI_STATUS_NAME, + pr_info, + dump_to_file=True, + ) if __name__ == "__main__": diff --git a/tests/ci/sync_pr.py b/tests/ci/sync_pr.py index f33f6122f30..acff7ba541b 100644 --- a/tests/ci/sync_pr.py +++ b/tests/ci/sync_pr.py @@ -2,14 +2,70 @@ """Script for automatic sync PRs handling in private repos""" +import argparse import sys from get_robot_token import get_best_robot_token from pr_info import PRInfo from github_helper import GitHub +from commit_status_helper import get_commit, post_commit_status +from report import FAILURE, SUCCESS + + +def parse_args() -> argparse.Namespace: + parser = argparse.ArgumentParser( + formatter_class=argparse.ArgumentDefaultsHelpFormatter, + description="Script for handling sync PRs", + ) + parser.add_argument( + "--merge", + action="store_true", + help="merge sync pr", + ) + parser.add_argument( + "--status", + action="store_true", + help="check and set sync pr status", + ) + args = parser.parse_args() + return args + + +def merge_sync_pr(gh, sync_pr): + if not sync_pr: + print("Sync PR not found - exiting") + return + + if sync_pr.state == "closed": + print(f"Sync PR [{sync_pr.number}] already closed - exiting") + sys.exit(0) + + if sync_pr.state != "open": + print( + f"WARNING: Unknown Sync PR [{sync_pr.number}] state [{sync_pr.state}] - exiting" + ) + sys.exit(0) + + print(f"Trying to merge Sync PR [{sync_pr.number}]") + if sync_pr.draft: + gh.toggle_pr_draft(sync_pr) + sync_pr.merge() + + +def set_sync_status(gh, pr_info, sync_pr): + if not sync_pr or not sync_pr.mergeable: + post_commit_status( + get_commit(gh, pr_info.sha), FAILURE, "", "Sync PR failure", "A Sync" + ) + else: + post_commit_status(get_commit(gh, pr_info.sha), SUCCESS, "", "", "A Sync") def main(): + args = parse_args() + + assert args.merge ^ args.status + gh = GitHub(get_best_robot_token()) pr_info = PRInfo() @@ -19,27 +75,20 @@ def main(): query=f"head:sync-upstream/pr/{pr_info.merged_pr} org:ClickHouse type:pr", repo="ClickHouse/clickhouse-private", ) + + sync_pr = None + if len(prs) > 1: print(f"WARNING: More than one PR found [{prs}] - exiting") - sys.exit(0) - if len(prs) == 0: + elif len(prs) == 0: print("WARNING: No Sync PR found") - sys.exit(0) + else: + sync_pr = prs[0] - pr = prs[0] - - if pr.state == "closed": - print(f"Sync PR [{pr.number}] already closed - exiting") - sys.exit(0) - - if pr.state != "open": - print(f"WARNING: Unknown Sync PR [{pr.number}] state [{pr.state}] - exiting") - sys.exit(0) - - print(f"Trying to merge Sync PR [{pr.number}]") - if pr.draft: - gh.toggle_pr_draft(pr) - pr.merge() + if args.merge: + merge_sync_pr(gh, sync_pr) + elif args.status: + set_sync_status(gh, pr_info, sync_pr) if __name__ == "__main__": diff --git a/tests/integration/test_attach_partition_using_copy/configs/remote_servers.xml b/tests/integration/test_attach_partition_using_copy/configs/remote_servers.xml deleted file mode 100644 index b40730e9f7d..00000000000 --- a/tests/integration/test_attach_partition_using_copy/configs/remote_servers.xml +++ /dev/null @@ -1,17 +0,0 @@ - - - - - true - - replica1 - 9000 - - - replica2 - 9000 - - - - - diff --git a/tests/integration/test_attach_partition_using_copy/test.py b/tests/integration/test_attach_partition_using_copy/test.py deleted file mode 100644 index df5378742ae..00000000000 --- a/tests/integration/test_attach_partition_using_copy/test.py +++ /dev/null @@ -1,187 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry - -cluster = ClickHouseCluster(__file__) - -replica1 = cluster.add_instance( - "replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] -) -replica2 = cluster.add_instance( - "replica2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - except Exception as ex: - print(ex) - finally: - cluster.shutdown() - - -def cleanup(nodes): - for node in nodes: - node.query("DROP TABLE IF EXISTS source SYNC") - node.query("DROP TABLE IF EXISTS destination SYNC") - - -def create_source_table(node, table_name, replicated): - replica = node.name - engine = ( - f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')" - if replicated - else "MergeTree()" - ) - node.query_with_retry( - """ - ATTACH TABLE {table_name} UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7' - ( - price UInt32, - date Date, - postcode1 LowCardinality(String), - postcode2 LowCardinality(String), - type Enum8('other' = 0, 'terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4), - is_new UInt8, - duration Enum8('unknown' = 0, 'freehold' = 1, 'leasehold' = 2), - addr1 String, - addr2 String, - street LowCardinality(String), - locality LowCardinality(String), - town LowCardinality(String), - district LowCardinality(String), - county LowCardinality(String) - ) - ENGINE = {engine} - ORDER BY (postcode1, postcode2, addr1, addr2) - SETTINGS disk = disk(type = web, endpoint = 'https://raw.githubusercontent.com/ClickHouse/web-tables-demo/main/web/') - """.format( - table_name=table_name, engine=engine - ) - ) - - -def create_destination_table(node, table_name, replicated): - replica = node.name - engine = ( - f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')" - if replicated - else "MergeTree()" - ) - node.query_with_retry( - """ - CREATE TABLE {table_name} - ( - price UInt32, - date Date, - postcode1 LowCardinality(String), - postcode2 LowCardinality(String), - type Enum8('other' = 0, 'terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4), - is_new UInt8, - duration Enum8('unknown' = 0, 'freehold' = 1, 'leasehold' = 2), - addr1 String, - addr2 String, - street LowCardinality(String), - locality LowCardinality(String), - town LowCardinality(String), - district LowCardinality(String), - county LowCardinality(String) - ) - ENGINE = {engine} - ORDER BY (postcode1, postcode2, addr1, addr2) - """.format( - table_name=table_name, engine=engine - ) - ) - - -def test_both_mergtree(start_cluster): - create_source_table(replica1, "source", False) - create_destination_table(replica1, "destination", False) - - replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") - - assert_eq_with_retry( - replica1, - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", - replica1.query( - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC" - ), - ) - - assert_eq_with_retry( - replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" - ) - - cleanup([replica1]) - - -def test_all_replicated(start_cluster): - create_source_table(replica1, "source", True) - create_destination_table(replica1, "destination", True) - create_destination_table(replica2, "destination", True) - - replica1.query("SYSTEM SYNC REPLICA destination") - replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") - - assert_eq_with_retry( - replica1, - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", - replica1.query( - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC" - ), - ) - assert_eq_with_retry( - replica1, - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC", - replica2.query( - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC" - ), - ) - - assert_eq_with_retry( - replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" - ) - - assert_eq_with_retry( - replica2, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" - ) - - cleanup([replica1, replica2]) - - -def test_only_destination_replicated(start_cluster): - create_source_table(replica1, "source", False) - create_destination_table(replica1, "destination", True) - create_destination_table(replica2, "destination", True) - - replica1.query("SYSTEM SYNC REPLICA destination") - replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") - - assert_eq_with_retry( - replica1, - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", - replica1.query( - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC" - ), - ) - assert_eq_with_retry( - replica1, - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC", - replica2.query( - f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC" - ), - ) - - assert_eq_with_retry( - replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" - ) - - assert_eq_with_retry( - replica2, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" - ) - - cleanup([replica1, replica2]) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 9584ace7f45..fdd81284b2a 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -5,7 +5,6 @@ import string import threading import time from multiprocessing.dummy import Pool -from helpers.test_tools import assert_eq_with_retry import pytest from helpers.client import QueryRuntimeException @@ -1746,9 +1745,9 @@ def test_move_while_merge(start_cluster): node1.query(f"DROP TABLE IF EXISTS {name} SYNC") -def test_move_across_policies_work_for_attach_not_work_for_move(start_cluster): +def test_move_across_policies_does_not_work(start_cluster): try: - name = "test_move_across_policies_work_for_attach_not_work_for_move" + name = "test_move_across_policies_does_not_work" node1.query( """ @@ -1784,18 +1783,25 @@ def test_move_across_policies_work_for_attach_not_work_for_move(start_cluster): except QueryRuntimeException: """All parts of partition 'all' are already on disk 'jbod2'.""" - node1.query( - """ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format( - name=name - ) - ) - assert_eq_with_retry( - node1, - """SELECT * FROM {name}2""".format(name=name), + with pytest.raises( + QueryRuntimeException, + match=".*because disk does not belong to storage policy.*", + ): node1.query( - """SELECT * FROM {name}""".format(name=name), - ), - ) + """ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format( + name=name + ) + ) + + with pytest.raises( + QueryRuntimeException, + match=".*because disk does not belong to storage policy.*", + ): + node1.query( + """ALTER TABLE {name}2 REPLACE PARTITION tuple() FROM {name}""".format( + name=name + ) + ) with pytest.raises( QueryRuntimeException, @@ -1807,6 +1813,10 @@ def test_move_across_policies_work_for_attach_not_work_for_move(start_cluster): ) ) + assert node1.query( + """SELECT * FROM {name}""".format(name=name) + ).splitlines() == ["1"] + finally: node1.query(f"DROP TABLE IF EXISTS {name} SYNC") node1.query(f"DROP TABLE IF EXISTS {name}2 SYNC") diff --git a/tests/performance/join_filter_pushdown_equivalent_sets.xml b/tests/performance/join_filter_pushdown_equivalent_sets.xml new file mode 100644 index 00000000000..caddcb295c9 --- /dev/null +++ b/tests/performance/join_filter_pushdown_equivalent_sets.xml @@ -0,0 +1,16 @@ + + CREATE TABLE test_table_1(id UInt64, value String) ENGINE=MergeTree ORDER BY id + CREATE TABLE test_table_2(id UInt64, value String) ENGINE=MergeTree ORDER BY id + + INSERT INTO test_table_1 SELECT number, number FROM numbers(5000000) + INSERT INTO test_table_2 SELECT number, number FROM numbers(5000000) + + SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 FORMAT Null + SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 FORMAT Null + SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 5 FORMAT Null + SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 FORMAT Null + SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 FORMAT Null + + DROP TABLE test_table_1 + DROP TABLE test_table_2 + diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 436d06c5076..1b9755a74d5 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -180,12 +180,14 @@ Filter column: notEquals(__table1.number, 1_UInt8) > one condition of filter is pushed down before INNER JOIN Join Join -Filter column: notEquals(number, 1) +Filter column: and(notEquals(number, 1), notEquals(number, 2)) Join +Filter column: and(notEquals(b, 2), notEquals(b, 1)) > (analyzer) one condition of filter is pushed down before INNER JOIN Join Join -Filter column: notEquals(__table1.number, 1_UInt8) +Filter column: and(notEquals(__table1.number, 1_UInt8), notEquals(__table1.number, 2_UInt8)) +Filter column: and(notEquals(__table2.b, 2_UInt8), notEquals(__table2.b, 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 5a517264243..864dd69412a 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -248,14 +248,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -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, 1)" + grep -o "Join\|Filter column: and(notEquals(number, 1), notEquals(number, 2))\|Filter column: and(notEquals(b, 2), notEquals(b, 1))" echo "> (analyzer) one condition of filter is pushed down before INNER JOIN" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 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(__table1.number, 1_UInt8)" + grep -o "Join\|Filter column: and(notEquals(__table1.number, 1_UInt8), notEquals(__table1.number, 2_UInt8))\|Filter column: and(notEquals(__table2.b, 2_UInt8), notEquals(__table2.b, 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/01676_clickhouse_client_autocomplete.python b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python new file mode 100644 index 00000000000..13160d4e561 --- /dev/null +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.python @@ -0,0 +1,147 @@ +import pty +import os +import shlex +import time +import multiprocessing + +COMPLETION_TIMEOUT_SECONDS = 30 +DEBUG_LOG = os.path.join( + os.environ["CLICKHOUSE_TMP"], + os.path.basename(os.path.abspath(__file__)).strip(".python") + ".debuglog", +) + + +def run_with_timeout(func, args, timeout): + process = multiprocessing.Process(target=func, args=args) + process.start() + process.join(timeout) + + if process.is_alive(): + process.terminate() + print("Timeout") + + +def test_completion(program, argv, comp_word): + comp_begin = comp_word[:-3] + + shell_pid, master = pty.fork() + if shell_pid == 0: + os.execv(program, argv) + else: + try: + debug_log_fd = open(DEBUG_LOG, "a") + + output_b = os.read(master, 4096) + output = output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + while not ":)" in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + os.write(master, b"SET " + bytes(comp_begin.encode())) + output_b = os.read(master, 4096) + output = output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + while not comp_begin in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + time.sleep(0.01) + os.write(master, b"\t") + + output_b = os.read(master, 4096) + output = output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + # fail fast if there is a bell character in the output, + # meaning no concise completion is found + if "\x07" in output: + print(f"{comp_word}: FAIL") + return + + while not comp_word in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + print(f"{comp_word}: OK") + finally: + os.close(master) + debug_log_fd.close() + + +client_compwords_positive = [ + # system.functions + "concatAssumeInjective", + # system.table_engines + "ReplacingMergeTree", + # system.formats + "JSONEachRow", + # system.table_functions + "clusterAllReplicas", + # system.data_type_families + "SimpleAggregateFunction", + # system.settings + "max_concurrent_queries_for_all_users", + # system.clusters + "test_shard_localhost", + # system.macros + "default_path_test", + # system.storage_policies, egh not uniq + "default", + # system.aggregate_function_combinators + "uniqCombined64ForEach", + # FIXME: one may add separate case for suggestion_limit + # system.databases + "system", + # system.tables + "aggregate_function_combinators", + # system.columns + "primary_key_bytes_in_memory_allocated", + # system.dictionaries + # FIXME: none +] + +local_compwords_positive = [ + # system.functions + "concatAssumeInjective", + # system.table_engines + "ReplacingMergeTree", + # system.formats + "JSONEachRow", + # system.table_functions + "clusterAllReplicas", + # system.data_type_families + "SimpleAggregateFunction", +] + + +if __name__ == "__main__": + print("# clickhouse-client") + clickhouse_client = os.environ["CLICKHOUSE_CLIENT"] + args = shlex.split(clickhouse_client) + args.append("--wait_for_suggestions_to_load") + [ + run_with_timeout( + test_completion, [args[0], args, comp_word], COMPLETION_TIMEOUT_SECONDS + ) + for comp_word in client_compwords_positive + ] + + print("# clickhouse-local") + clickhouse_local = os.environ["CLICKHOUSE_LOCAL"] + args = shlex.split(clickhouse_local) + args.append("--wait_for_suggestions_to_load") + [ + run_with_timeout( + test_completion, [args[0], args, comp_word], COMPLETION_TIMEOUT_SECONDS + ) + for comp_word in local_compwords_positive + ] diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh index ebd6490077e..713a6cf5186 100755 --- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh @@ -1,142 +1,8 @@ #!/usr/bin/env bash -# Tags: long, no-ubsan +# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -SCRIPT_PATH="$CURDIR/$CLICKHOUSE_TEST_UNIQUE_NAME.generated-expect" - -# NOTE: database = $CLICKHOUSE_DATABASE is superfluous - -function test_completion_word() -{ - local w=$1 && shift - - local w_len=${#w} - local compword_begin=${w:0:$((w_len-3))} - local compword_end=${w:$((w_len-3))} - - # NOTE: - # - here and below you should escape variables of the expect. - # - you should not use "expect <<..." since in this case timeout/eof will - # not work (I guess due to attached stdin) - - # TODO: get build sanitizer and debug/release info to dynamically change test - # like here timeout 120 seconds is too big for release build - # but ok for sanitizer builds - cat > "$SCRIPT_PATH" << EOF -# NOTE: log will be appended -exp_internal -f $CLICKHOUSE_TMP/$(basename "${BASH_SOURCE[0]}").debuglog 0 - -# NOTE: when expect have EOF on stdin it also closes stdout, so let's reopen it -# again for logging -set stdout_channel [open "/dev/stdout" w] - -log_user 0 -set timeout 120 -match_max 100000 -expect_after { - # Do not ignore eof from expect - -i \$any_spawn_id eof { exp_continue } - # A default timeout action is to do nothing, change it to fail - -i \$any_spawn_id timeout { exit 1 } -} - -spawn bash -c "$*" -expect ":) " - -# Make a query -send -- "SET $compword_begin" -expect "SET $compword_begin" - -# Wait for suggestions to load, they are loaded in background -set is_done 0 -set timeout 1 -while {\$is_done == 0} { - send -- "\\t" - expect { - "$compword_begin$compword_end" { - puts \$stdout_channel "$compword_begin$compword_end: OK" - set is_done 1 - } - default { - sleep 1 - } - } -} - -close \$stdout_channel - -send -- "\\3\\4" -expect eof -EOF - - # NOTE: run expect under timeout since there is while loop that is not - # limited with timeout. - # - # NOTE: cat is required to serialize stdout for expect (without this pipe - # it will reopen the file again, and the output will be mixed). - timeout 2m expect -f "$SCRIPT_PATH" | cat -} - -# last 3 bytes will be completed, -# so take this in mind when you will update the list. -client_compwords_positive=( - # system.functions - concatAssumeInjective - # system.table_engines - ReplacingMergeTree - # system.formats - JSONEachRow - # system.table_functions - clusterAllReplicas - # system.data_type_families - SimpleAggregateFunction - # system.settings - max_concurrent_queries_for_all_users - # system.clusters - test_shard_localhost - # system.macros - default_path_test - # system.storage_policies, egh not uniq - default - # system.aggregate_function_combinators - uniqCombined64ForEach - - # FIXME: one may add separate case for suggestion_limit - # system.databases - system - # system.tables - aggregate_function_combinators - # system.columns - primary_key_bytes_in_memory_allocated - # system.dictionaries - # FIXME: none -) - -local_compwords_positive=( - # system.functions - concatAssumeInjective - # system.table_engines - ReplacingMergeTree - # system.formats - JSONEachRow - # system.table_functions - clusterAllReplicas - # system.data_type_families - SimpleAggregateFunction -) - -echo "# clickhouse-client" -for w in "${client_compwords_positive[@]}"; do - test_completion_word "$w" "$CLICKHOUSE_CLIENT" -done -echo "# clickhouse-local" -for w in "${local_compwords_positive[@]}"; do - test_completion_word "$w" "$CLICKHOUSE_LOCAL" -done - -rm -f "${SCRIPT_PATH:?}" - -exit 0 +python3 "$CURDIR"/01676_clickhouse_client_autocomplete.python \ No newline at end of file diff --git a/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql b/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql index 48e84246d1c..be4b64888ca 100644 --- a/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql +++ b/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql @@ -127,3 +127,14 @@ SELECT arrayMap(x -> splitByChar(toString(id), arrayMap(x -> toString(1), [NULL] DROP TABLE test_table; -- { echoOff } + +SELECT + groupArray(number) AS counts, + arraySum(arrayMap(x -> (x + 1), counts)) as hello, + arrayMap(x -> (x / hello), counts) AS res +FROM numbers(1000000) FORMAT Null; + +SELECT + arrayWithConstant(pow(10,6), 1) AS nums, + arrayMap(x -> x, nums) AS m, + arrayMap(x -> x + arraySum(m), m) AS res FORMAT Null; diff --git a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.reference b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.reference index df8198bc856..866d6cb7ec3 100644 --- a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.reference +++ b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.reference @@ -7,4 +7,4 @@ 1 1 1 1 1 -1 1 +1 2 diff --git a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql index a299e50984f..ea52df5d4b4 100644 --- a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql +++ b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS t1; CREATE TABLE t1 (key UInt8) ENGINE = Memory; diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh index d3252b29eb7..fe26784dab4 100755 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh @@ -5,33 +5,38 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Copy backups/with_broken_part.zip into the disk named "backups". -SRC_BACKUP_DIR=$CURDIR/backups -SRC_BACKUP_FILENAME=with_broken_part.zip +# Copies a test predefined backup from "/tests/queries/0_stateless/backups/" folder to the "backups" disk, +# returns the path to the backup relative to that disk. +function install_test_backup() +{ + local test_backup_filename="$1" + local test_backup_path="$CURDIR/backups/${test_backup_filename}" -BACKUPS_DISK=backups -BACKUPS_DIR=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='$BACKUPS_DISK'") + local backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'") -if [ -z "$BACKUPS_DIR" ]; then - echo Disk \'$BACKUPS_DISK\' not found - exit 1 -fi + if [ -z "${backups_disk_root}" ]; then + echo Disk \'${backups_disk_root}\' not found + exit 1 + fi -BACKUP_FILENAME=$CLICKHOUSE_DATABASE/${SRC_BACKUP_FILENAME} -BACKUP_NAME="Disk('$BACKUPS_DISK', '$BACKUP_FILENAME')" + local install_path=${backups_disk_root}/${CLICKHOUSE_DATABASE}/${test_backup_filename} + mkdir -p "$(dirname "${install_path}")" + ln -s "${test_backup_path}" "${install_path}" -mkdir -p "$(dirname "$BACKUPS_DIR/$BACKUP_FILENAME")" -ln -s "$SRC_BACKUP_DIR/$SRC_BACKUP_FILENAME" "$BACKUPS_DIR/$BACKUP_FILENAME" + echo "${CLICKHOUSE_DATABASE}/${test_backup_filename}" +} + +backup_name="$(install_test_backup with_broken_part.zip)" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" # First try to restore with the setting `restore_broken_parts_as_detached` set to false. -$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME" 2>&1 | tr -d \\n | grep "data.bin doesn't exist" | grep "while restoring part all_2_2_0" > /dev/null && echo "OK" || echo "FAILED" +$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM Disk('backups', '${backup_name}')" 2>&1 | tr -d \\n | grep "data.bin doesn't exist" | grep "while restoring part all_2_2_0" > /dev/null && echo "OK" || echo "FAILED" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" # Then try to restore with the setting `restore_broken_parts_as_detached` set to true. -$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME SETTINGS restore_broken_parts_as_detached = true" 2>/dev/null | awk -F '\t' '{print $2}' +$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM Disk('backups', '${backup_name}') SETTINGS restore_broken_parts_as_detached = true" 2>/dev/null | awk -F '\t' '{print $2}' $CLICKHOUSE_CLIENT --multiquery < t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND 2 2 2 2 3 3 3 33 \N \N \N \N +SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) ORDER BY t1.x; +2 2 2 2 +3 3 3 33 +\N \N \N \N +SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND (t1.x = t1.y OR t1.x IS NULL AND t1.y IS NULL) ORDER BY t1.x; +2 2 2 2 +3 3 3 33 +\N \N \N \N SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) AND t1.y <=> t2.y ORDER BY t1.x NULLS LAST; 2 2 2 2 \N \N \N \N diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index 6a98a7bb57b..5458370db8c 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -14,6 +14,9 @@ SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) O SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND t2.x IS NULL) ORDER BY t1.x NULLS LAST; +SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) ORDER BY t1.x; +SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND (t1.x = t1.y OR t1.x IS NULL AND t1.y IS NULL) ORDER BY t1.x; + SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) AND t1.y <=> t2.y ORDER BY t1.x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND t1.y IS NULL AND t2.x IS NULL AND t2.y IS NULL)) ORDER BY t1.x NULLS LAST; diff --git a/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh index d49f1c41c69..f857358a5ea 100755 --- a/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh +++ b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: no-ordinary-database, no-replicated-database -# Tag no-ordinary-database: TO DO +# Tags: no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.reference b/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.reference new file mode 100644 index 00000000000..04ceb193415 --- /dev/null +++ b/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.reference @@ -0,0 +1,4 @@ +RESTORED +2024-02-22 07:00:00 00 +2024-02-22 07:00:01 11 +2024-02-22 07:00:02 22 diff --git a/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh b/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh new file mode 100755 index 00000000000..3a3d0edc38f --- /dev/null +++ b/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Copies a test predefined backup from "/tests/queries/0_stateless/backups/" folder to the "backups" disk, +# returns the path to the backup relative to that disk. +function install_test_backup() +{ + local test_backup_filename="$1" + local test_backup_path="$CURDIR/backups/${test_backup_filename}" + + local backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'") + + if [ -z "${backups_disk_root}" ]; then + echo Disk \'${backups_disk_root}\' not found + exit 1 + fi + + local install_path=${backups_disk_root}/${CLICKHOUSE_DATABASE}/${test_backup_filename} + mkdir -p "$(dirname "${install_path}")" + ln -s "${test_backup_path}" "${install_path}" + + echo "${CLICKHOUSE_DATABASE}/${test_backup_filename}" +} + +backup_name="$(install_test_backup old_backup_with_matview_inner_table_metadata.zip)" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src" + +db="$CLICKHOUSE_DATABASE" +${CLICKHOUSE_CLIENT} -q "RESTORE DATABASE mydb AS ${db} FROM Disk('backups', '${backup_name}') SETTINGS allow_different_database_def=true" | grep -o "RESTORED" + +${CLICKHOUSE_CLIENT} -q "SELECT toDateTime(timestamp, 'UTC') AS ts, c12 FROM mv ORDER BY ts" + +$CLICKHOUSE_CLIENT --query "DROP TABLE mv" +$CLICKHOUSE_CLIENT --query "DROP TABLE src" diff --git a/tests/queries/0_stateless/03033_virtual_column_override.reference b/tests/queries/0_stateless/03033_virtual_column_override.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03033_virtual_column_override.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03033_virtual_column_override.sql b/tests/queries/0_stateless/03033_virtual_column_override.sql new file mode 100644 index 00000000000..49258bbb533 --- /dev/null +++ b/tests/queries/0_stateless/03033_virtual_column_override.sql @@ -0,0 +1,3 @@ +DROP TABLE IF EXISTS override_test; +CREATE TABLE override_test (_part UInt32) ENGINE = MergeTree ORDER BY tuple() AS SELECT 1; +SELECT _part FROM override_test; diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference new file mode 100644 index 00000000000..00740e6380f --- /dev/null +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -0,0 +1,710 @@ +-- { echoOn } + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; +Expression ((Project names + (Projection + ))) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: INNER + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table1.id UInt64 + __table1.value String + Filter column: equals(__table1.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + ALIAS id :: 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 0 + FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table2.id UInt64 + __table2.value String + Filter column: equals(__table2.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + ALIAS id :: 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 0 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; +5 5 5 5 +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; +Expression ((Project names + (Projection + ))) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: INNER + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table1.id UInt64 + __table1.value String + Filter column: equals(__table1.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + ALIAS id :: 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 0 + FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table2.id UInt64 + __table2.value String + Filter column: equals(__table2.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + ALIAS id :: 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 0 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; +5 5 5 5 +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5 AND rhs.id = 6; +Expression ((Project names + (Projection + ))) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: INNER + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table1.id UInt64 + __table1.value String + Filter column: and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 3 + ALIAS id :: 0 -> __table1.id UInt64 : 4 + ALIAS value :: 1 -> __table1.value String : 0 + FUNCTION equals(__table1.id : 4, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1 + FUNCTION equals(__table1.id : 4, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2 + FUNCTION and(equals(__table1.id, 5_UInt8) :: 2, equals(__table1.id, 6_UInt8) :: 1) -> and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) UInt8 : 3 + Positions: 3 4 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table2.id UInt64 + __table2.value String + Filter column: and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 3 + ALIAS id :: 0 -> __table2.id UInt64 : 4 + ALIAS value :: 1 -> __table2.value String : 0 + FUNCTION equals(__table2.id : 4, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + FUNCTION equals(__table2.id : 4, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2 + FUNCTION and(equals(__table2.id, 6_UInt8) :: 2, equals(__table2.id, 5_UInt8) :: 1) -> and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) UInt8 : 3 + Positions: 3 4 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5 AND rhs.id = 6; +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; +Expression ((Project names + (Projection + ))) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: LEFT + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table1.id UInt64 + __table1.value String + Filter column: equals(__table1.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + ALIAS id :: 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 0 + FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table2.id UInt64 + __table2.value String + Filter column: equals(__table2.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + ALIAS id :: 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 0 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; +5 5 5 5 +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; +Expression ((Project names + Projection)) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Filter column: equals(__table2.id, 5_UInt8) (removed) + Actions: INPUT :: 0 -> __table1.id UInt64 : 0 + INPUT :: 1 -> __table1.value String : 1 + INPUT :: 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + Positions: 5 0 1 2 3 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: LEFT + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table1.id UInt64 + __table1.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value :: 1 -> __table1.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table2.id UInt64 + __table2.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table2.id UInt64 : 2 + ALIAS value :: 1 -> __table2.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; +5 5 5 5 +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; +Expression ((Project names + Projection)) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Filter column: equals(__table1.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT :: 1 -> __table1.value String : 1 + INPUT :: 2 -> __table2.value String : 2 + INPUT :: 3 -> __table2.id UInt64 : 3 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 + FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 + Positions: 5 0 1 2 3 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: RIGHT + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table1.id UInt64 + __table1.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value :: 1 -> __table1.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table2.id UInt64 + __table2.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table2.id UInt64 : 2 + ALIAS value :: 1 -> __table2.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; +5 5 5 5 +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; +Expression ((Project names + (Projection + ))) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: RIGHT + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table1.id UInt64 + __table1.value String + Filter column: equals(__table1.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + ALIAS id :: 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 0 + FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: __table2.id UInt64 + __table2.value String + Filter column: equals(__table2.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 + ALIAS id :: 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 0 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; +5 5 5 5 +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; +Expression ((Project names + Projection)) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Filter column: equals(__table1.id, 5_UInt8) (removed) + Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT :: 1 -> __table1.value String : 1 + INPUT :: 2 -> __table2.value String : 2 + INPUT :: 3 -> __table2.id UInt64 : 3 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 + FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 + Positions: 5 0 1 2 3 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: FULL + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table1.id UInt64 + __table1.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value :: 1 -> __table1.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table2.id UInt64 + __table2.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table2.id UInt64 : 2 + ALIAS value :: 1 -> __table2.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; +5 5 5 5 +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; +Expression ((Project names + Projection)) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Filter column: equals(__table2.id, 5_UInt8) (removed) + Actions: INPUT :: 0 -> __table1.id UInt64 : 0 + INPUT :: 1 -> __table1.value String : 1 + INPUT :: 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + Positions: 5 0 1 2 3 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: FULL + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table1.id UInt64 + __table1.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value :: 1 -> __table1.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table2.id UInt64 + __table2.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table2.id UInt64 : 2 + ALIAS value :: 1 -> __table2.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; +5 5 5 5 +SELECT '--'; +-- +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5 AND rhs.id = 6; +Expression ((Project names + Projection)) +Header: id UInt64 + rhs.id UInt64 + value String + rhs.value String +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value :: 1 -> value String : 0 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed) + Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT :: 1 -> __table1.value String : 1 + INPUT :: 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 + COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5 + FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 6 + FUNCTION equals(__table2.id : 3, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4 + FUNCTION and(equals(__table1.id, 5_UInt8) :: 6, equals(__table2.id, 6_UInt8) :: 4) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 5 + Positions: 5 0 1 2 3 + Join (JOIN FillRightFirst) + Header: __table1.id UInt64 + __table1.value String + __table2.value String + __table2.id UInt64 + Type: FULL + Strictness: ALL + Algorithm: HashJoin + Clauses: [(__table1.id) = (__table2.id)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table1.id UInt64 + __table1.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value :: 1 -> __table1.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: __table2.id UInt64 + __table2.value String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + ALIAS id :: 0 -> __table2.id UInt64 : 2 + ALIAS value :: 1 -> __table2.value String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +SELECT '--'; +-- +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5 AND rhs.id = 6; diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql new file mode 100644 index 00000000000..9627b55e633 --- /dev/null +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -0,0 +1,131 @@ +SET allow_experimental_analyzer = 1; +SET optimize_move_to_prewhere = 0; + +DROP TABLE IF EXISTS test_table_1; +CREATE TABLE test_table_1 +( + id UInt64, + value String +) ENGINE=MergeTree ORDER BY id; + +CREATE TABLE test_table_2 +( + id UInt64, + value String +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table_1 SELECT number, number FROM numbers(10); +INSERT INTO test_table_2 SELECT number, number FROM numbers(10); + +-- { echoOn } + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5 AND rhs.id = 6; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5 AND rhs.id = 6; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE rhs.id = 5; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5 AND rhs.id = 6; + +SELECT '--'; + +SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id +WHERE lhs.id = 5 AND rhs.id = 6; + +-- { echoOff } + +DROP TABLE test_table_1; +DROP TABLE test_table_2; diff --git a/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql b/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql index 4985d3abfb6..7682e6ce866 100644 --- a/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql +++ b/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql @@ -1,5 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/29838 SET allow_experimental_analyzer=1; +SET distributed_foreground_insert=1; CREATE TABLE first_table_lr ( diff --git a/tests/queries/0_stateless/03093_analyzer_column_alias.reference b/tests/queries/0_stateless/03093_analyzer_column_alias.reference new file mode 100644 index 00000000000..4d9ef9832dd --- /dev/null +++ b/tests/queries/0_stateless/03093_analyzer_column_alias.reference @@ -0,0 +1 @@ +1 0 10 9 diff --git a/tests/queries/0_stateless/03093_analyzer_column_alias.sql b/tests/queries/0_stateless/03093_analyzer_column_alias.sql new file mode 100644 index 00000000000..9ff0f78ba24 --- /dev/null +++ b/tests/queries/0_stateless/03093_analyzer_column_alias.sql @@ -0,0 +1,21 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/26674 +SET allow_experimental_analyzer = true; + +SELECT + Carrier, + sum(toFloat64(C3)) AS C1, + sum(toFloat64(C1)) AS C2, + sum(toFloat64(C2)) AS C3 +FROM + ( + SELECT + 1 AS Carrier, + count(CAST(1, 'Nullable(Int32)')) AS C1, + max(number) AS C2, + min(number) AS C3 + FROM numbers(10) + GROUP BY Carrier + ) AS ITBL +GROUP BY Carrier +LIMIT 1000001 +SETTINGS prefer_column_name_to_alias=1; diff --git a/tests/integration/test_attach_partition_using_copy/__init__.py b/tests/queries/0_stateless/03093_bug37909_query_does_not_finish.reference similarity index 100% rename from tests/integration/test_attach_partition_using_copy/__init__.py rename to tests/queries/0_stateless/03093_bug37909_query_does_not_finish.reference diff --git a/tests/queries/0_stateless/03093_bug37909_query_does_not_finish.sql b/tests/queries/0_stateless/03093_bug37909_query_does_not_finish.sql new file mode 100644 index 00000000000..62fa3f437af --- /dev/null +++ b/tests/queries/0_stateless/03093_bug37909_query_does_not_finish.sql @@ -0,0 +1,78 @@ +-- Bug 37909 + +SELECT + v_date AS vDate, + round(sum(v_share)) AS v_sum +FROM +( + WITH + ( + SELECT rand() % 10000 + ) AS dummy_1, + ( + SELECT rand() % 10000 + ) AS dummy_2, + ( + SELECT rand() % 10000 + ) AS dummy_3, + _v AS + ( + SELECT + xxHash64(rand()) % 100000 AS d_id, + toDate(parseDateTimeBestEffort('2022-01-01') + (rand() % 2600000)) AS v_date + FROM numbers(1000000) + ORDER BY d_id ASC + ), + _i AS + ( + SELECT xxHash64(rand()) % 40000 AS d_id + FROM numbers(1000000) + ), + not_i AS + ( + SELECT + NULL AS v_date, + d_id, + 0 AS v_share + FROM _i + LIMIT 100 + ) + SELECT * + FROM + ( + SELECT + d_id, + v_date, + v_share + FROM not_i + UNION ALL + SELECT + d_id, + v_date, + 1 AS v_share + FROM + ( + SELECT + d_id, + arrayJoin(groupArray(v_date)) AS v_date + FROM + ( + SELECT + v_date, + d_id + FROM _v + UNION ALL + SELECT + NULL AS v_date, + d_id + FROM _i + ) + GROUP BY d_id + ) + ) + WHERE (v_date >= '2022-05-08') AND (v_date <= '2022-06-07') +) +/* WHERE (v_date >= '2022-05-08') AND (v_date <= '2022-06-07') placing condition has same effect */ +GROUP BY vDate +ORDER BY vDate ASC +SETTINGS allow_experimental_analyzer = 1; -- the query times out if allow_experimental_analyzer = 0 diff --git a/tests/queries/0_stateless/03093_filter_push_down_crash.reference b/tests/queries/0_stateless/03093_filter_push_down_crash.reference new file mode 100644 index 00000000000..bf98540f4b3 --- /dev/null +++ b/tests/queries/0_stateless/03093_filter_push_down_crash.reference @@ -0,0 +1,5 @@ +1 \N 1 +1 \N 1 +1 \N 1 +1 \N 1 +1 \N 1 diff --git a/tests/queries/0_stateless/03093_filter_push_down_crash.sql.j2 b/tests/queries/0_stateless/03093_filter_push_down_crash.sql.j2 new file mode 100644 index 00000000000..2cbbd89ca0c --- /dev/null +++ b/tests/queries/0_stateless/03093_filter_push_down_crash.sql.j2 @@ -0,0 +1,11 @@ +{% for join_algorithm in ['default', 'full_sorting_merge', 'hash', 'partial_merge', 'grace_hash'] -%} + +SET join_algorithm = '{{ join_algorithm }}'; + +SELECT * +FROM (SELECT 1 AS key) AS t1 +JOIN (SELECT NULL, 1 AS key) AS t2 +ON t1.key = t2.key +WHERE t1.key ORDER BY key; + +{% endfor -%} diff --git a/tests/queries/0_stateless/03093_virtual_column_override_group_by.reference b/tests/queries/0_stateless/03093_virtual_column_override_group_by.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03093_virtual_column_override_group_by.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03093_virtual_column_override_group_by.sql b/tests/queries/0_stateless/03093_virtual_column_override_group_by.sql new file mode 100644 index 00000000000..168d38a15b5 --- /dev/null +++ b/tests/queries/0_stateless/03093_virtual_column_override_group_by.sql @@ -0,0 +1,2 @@ +CREATE TABLE override_test__fuzz_45 (`_part` Float32) ENGINE = MergeTree ORDER BY tuple() AS SELECT 1; +SELECT _part FROM override_test__fuzz_45 GROUP BY materialize(6), 1; diff --git a/tests/queries/0_stateless/03094_named_tuple_bug24607.reference b/tests/queries/0_stateless/03094_named_tuple_bug24607.reference new file mode 100644 index 00000000000..fb6ca6c5c3a --- /dev/null +++ b/tests/queries/0_stateless/03094_named_tuple_bug24607.reference @@ -0,0 +1 @@ +(1,'test') 1 diff --git a/tests/queries/0_stateless/03094_named_tuple_bug24607.sql b/tests/queries/0_stateless/03094_named_tuple_bug24607.sql new file mode 100644 index 00000000000..e3c97f3fe41 --- /dev/null +++ b/tests/queries/0_stateless/03094_named_tuple_bug24607.sql @@ -0,0 +1,4 @@ +SELECT + JSONExtract('{"a":1, "b":"test"}', 'Tuple(a UInt8, b String)') AS x, + x.a +SETTINGS allow_experimental_analyzer = 1; diff --git a/tests/queries/0_stateless/03094_recursive_type_proto.reference b/tests/queries/0_stateless/03094_recursive_type_proto.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03094_recursive_type_proto.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03094_recursive_type_proto.sh b/tests/queries/0_stateless/03094_recursive_type_proto.sh new file mode 100755 index 00000000000..98a1b54ff9e --- /dev/null +++ b/tests/queries/0_stateless/03094_recursive_type_proto.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +SCHEMADIR=$CURDIR/format_schemas +$CLICKHOUSE_LOCAL -q "DESCRIBE TABLE file('nonexist', 'Protobuf') SETTINGS format_schema='$SCHEMADIR/03094_recursive_type.proto:Struct'" |& grep -c CANNOT_PARSE_PROTOBUF_SCHEMA diff --git a/tests/queries/0_stateless/backups/old_backup_with_matview_inner_table_metadata.zip b/tests/queries/0_stateless/backups/old_backup_with_matview_inner_table_metadata.zip new file mode 100644 index 00000000000..a2476da7ded Binary files /dev/null and b/tests/queries/0_stateless/backups/old_backup_with_matview_inner_table_metadata.zip differ diff --git a/tests/queries/0_stateless/format_schemas/03094_recursive_type.proto b/tests/queries/0_stateless/format_schemas/03094_recursive_type.proto new file mode 100644 index 00000000000..97b2c9480a1 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/03094_recursive_type.proto @@ -0,0 +1,17 @@ +syntax = "proto3"; + +message Struct { + map fields = 1; +} + +message Value { + // The kind of value. + oneof kind { + string string_value = 1; + ListValue list_value = 2; + } +} + +message ListValue { + repeated Value values = 1; +} diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 30c2de2b507..9f7776f5201 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -183,6 +183,8 @@ CompiledExpressionCacheCount ComplexKeyCache ComplexKeyDirect ComplexKeyHashed +Composable +composable Config ConnectionDetails Const @@ -697,6 +699,7 @@ PCRE PRCP PREWHERE PROCESSLIST +PROXYv PSUN PagerDuty ParallelFormattingOutputFormatThreads