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