diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4e4785892c3..e22377e2332 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -29,7 +29,7 @@
* Add two new functions which allow for user-defined keys/seeds with SipHash{64,128}. [#45513](https://github.com/ClickHouse/ClickHouse/pull/45513) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Allow a three-argument version for table function `format`. close [#45808](https://github.com/ClickHouse/ClickHouse/issues/45808). [#45873](https://github.com/ClickHouse/ClickHouse/pull/45873) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)).
* Add `JodaTime` format support for 'x','w','S'. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. [#46073](https://github.com/ClickHouse/ClickHouse/pull/46073) ([zk_kiger](https://github.com/zk-kiger)).
-* Support window function `ntile`.
+* Support window function `ntile`. ([lgbo](https://github.com/lgbo-ustc)).
* Add setting `final` to implicitly apply the `FINAL` modifier to every table. [#40945](https://github.com/ClickHouse/ClickHouse/pull/40945) ([Arthur Passos](https://github.com/arthurpassos)).
* Added `arrayPartialSort` and `arrayPartialReverseSort` functions. [#46296](https://github.com/ClickHouse/ClickHouse/pull/46296) ([Joanna Hulboj](https://github.com/jh0x)).
* The new http parameter `client_protocol_version` allows setting a client protocol version for HTTP responses using the Native format. [#40397](https://github.com/ClickHouse/ClickHouse/issues/40397). [#46360](https://github.com/ClickHouse/ClickHouse/pull/46360) ([Geoff Genz](https://github.com/genzgd)).
diff --git a/README.md b/README.md
index 1974de3ce94..fcbe65e8223 100644
--- a/README.md
+++ b/README.md
@@ -14,7 +14,7 @@ curl https://clickhouse.com/ | sh
* [Tutorial](https://clickhouse.com/docs/en/getting_started/tutorial/) shows how to set up and query a small ClickHouse cluster.
* [Documentation](https://clickhouse.com/docs/en/) provides more in-depth information.
* [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format.
-* [Slack](https://clickhousedb.slack.com/) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time.
+* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-1gh9ds7f4-PgDhJAaF8ad5RbWBAAjzFg) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time.
* [Blog](https://clickhouse.com/blog/) contains various ClickHouse-related articles, as well as announcements and reports about events.
* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/ClickHouse/index.html) with syntax highlight and navigation.
* [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlight, powered by github.dev.
diff --git a/contrib/libunwind b/contrib/libunwind
index 5022f30f3e0..e48aa13f67d 160000
--- a/contrib/libunwind
+++ b/contrib/libunwind
@@ -1 +1 @@
-Subproject commit 5022f30f3e092a54a7c101c335ce5e08769db366
+Subproject commit e48aa13f67dc722511b5af33a32ba9b7748176b5
diff --git a/contrib/zstd-cmake/CMakeLists.txt b/contrib/zstd-cmake/CMakeLists.txt
index f44d5db12c4..a6262178dc7 100644
--- a/contrib/zstd-cmake/CMakeLists.txt
+++ b/contrib/zstd-cmake/CMakeLists.txt
@@ -30,25 +30,10 @@
# - zstd homepage : http://www.zstd.net/
# ################################################################
-# Get library version based on information from input content (use regular exp)
-function(GetLibraryVersion _content _outputVar1 _outputVar2 _outputVar3)
- string(REGEX MATCHALL ".*define ZSTD_VERSION_MAJOR+.* ([0-9]+).*define ZSTD_VERSION_MINOR+.* ([0-9]+).*define ZSTD_VERSION_RELEASE+.* ([0-9]+)" VERSION_REGEX "${_content}")
- SET(${_outputVar1} ${CMAKE_MATCH_1} PARENT_SCOPE)
- SET(${_outputVar2} ${CMAKE_MATCH_2} PARENT_SCOPE)
- SET(${_outputVar3} ${CMAKE_MATCH_3} PARENT_SCOPE)
-endfunction()
-
# Define library directory, where sources and header files are located
SET(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib")
INCLUDE_DIRECTORIES(BEFORE ${LIBRARY_DIR} "${LIBRARY_DIR}/common")
-# Read file content
-FILE(READ "${LIBRARY_DIR}/zstd.h" HEADER_CONTENT)
-
-# Parse version
-GetLibraryVersion("${HEADER_CONTENT}" LIBVER_MAJOR LIBVER_MINOR LIBVER_RELEASE)
-MESSAGE(STATUS "ZSTD VERSION ${LIBVER_MAJOR}.${LIBVER_MINOR}.${LIBVER_RELEASE}")
-
# cd contrib/zstd/lib
# find . -name '*.c' -or -name '*.S' | grep -vP 'deprecated|legacy' | sort | sed 's/^\./ "${LIBRARY_DIR}/"'
SET(Sources
diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md
index 404c6c6f227..6a15c32e9dd 100644
--- a/docs/en/getting-started/install.md
+++ b/docs/en/getting-started/install.md
@@ -5,6 +5,10 @@ description: Install ClickHouse
slug: /en/install
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+import CodeBlock from '@theme/CodeBlock';
+
# Install ClickHouse
You have three options for getting up and running with ClickHouse:
@@ -19,17 +23,23 @@ The quickest and easiest way to get up and running with ClickHouse is to create
## Self-Managed Install
+
+
+
1. The simplest way to download ClickHouse locally is to run the following command. If your operating system is supported, an appropriate ClickHouse binary will be downloaded and made runnable:
+
```bash
curl https://clickhouse.com/ | sh
```
1. Run the `install` command, which defines a collection of useful symlinks along with the files and folders used by ClickHouse - all of which you can see in the output of the install script:
+
```bash
sudo ./clickhouse install
```
1. At the end of the install script, you are prompted for a password for the `default` user. Feel free to enter a password, or you can optionally leave it blank:
+
```response
Creating log directory /var/log/clickhouse-server.
Creating data directory /var/lib/clickhouse.
@@ -40,6 +50,7 @@ The quickest and easiest way to get up and running with ClickHouse is to create
Enter password for default user:
```
You should see the following output:
+
```response
ClickHouse has been successfully installed.
@@ -51,10 +62,45 @@ The quickest and easiest way to get up and running with ClickHouse is to create
```
1. Run the following command to start the ClickHouse server:
+ ```bash
+ sudo clickhouse start
+ ```
+
+
+
+
+1. The simplest way to download ClickHouse locally is to run the following command. If your operating system is supported, an appropriate ClickHouse binary will be downloaded and made runnable:
```bash
- sudo clickhouse start
+ curl https://clickhouse.com/ | sh
```
+1. Run the ClickHouse server:
+
+ ```bash
+ ./clickhouse server
+ ```
+
+1. Open a new terminal and use the **clickhouse-client** to connect to your service:
+
+ ```bash
+ ./clickhouse client
+ ```
+
+ ```response
+ ./clickhouse client
+ ClickHouse client version 23.2.1.1501 (official build).
+ Connecting to localhost:9000 as user default.
+ Connected to ClickHouse server version 23.2.1 revision 54461.
+
+ local-host :)
+ ```
+
+ You are ready to start sending DDL and SQL commands to ClickHouse!
+
+
+
+
+
:::tip
The [Quick Start](/docs/en/quick-start.mdx/#step-1-get-clickhouse) walks through the steps to download and run ClickHouse, connect to it, and insert data.
:::
diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md
index 75ae6f3d2bc..17d03dfa4ec 100644
--- a/docs/en/operations/server-configuration-parameters/settings.md
+++ b/docs/en/operations/server-configuration-parameters/settings.md
@@ -467,7 +467,7 @@ Port for exchanging data between ClickHouse servers.
The hostname that can be used by other servers to access this server.
-If omitted, it is defined in the same way as the `hostname-f` command.
+If omitted, it is defined in the same way as the `hostname -f` command.
Useful for breaking away from a specific network interface.
diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md
index 4c2054274e4..bde37241ed0 100644
--- a/docs/en/sql-reference/syntax.md
+++ b/docs/en/sql-reference/syntax.md
@@ -83,9 +83,13 @@ Examples: `1`, `10_000_000`, `0xffff_ffff`, `18446744073709551615`, `0xDEADBEEF`
### String
-Only string literals in single quotes are supported. The enclosed characters can be backslash-escaped. The following escape sequences have a corresponding special value: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. In all other cases, escape sequences in the format `\c`, where `c` is any character, are converted to `c`. It means that you can use the sequences `\'`and`\\`. The value will have the [String](../sql-reference/data-types/string.md) type.
+String literals must be enclosed in single quotes, double quotes are not supported.
+Escaping works either
-In string literals, you need to escape at least `'` and `\`. Single quotes can be escaped with the single quote, literals `'It\'s'` and `'It''s'` are equal.
+- in SQL-style based on a preceding single quote where the single-quote character `'` (and only this character) can be escaped as `''`, or
+- in C-style based on a preceding backslash with the following supported escape sequences: `\\`, `\'`, `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. The backslash loses its special meaning and will be interpreted literally if it precedes characters different than the listed ones.
+
+In string literals, you need to escape at least `'` and `\` using escape codes `''` (or: `\'`) and `\\`.
### Compound
diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp
index 28a0c4ad7e0..fe4dd2c5016 100644
--- a/src/Analyzer/JoinNode.cpp
+++ b/src/Analyzer/JoinNode.cpp
@@ -15,6 +15,11 @@
namespace DB
{
+namespace ErrorCodes
+{
+ extern const int LOGICAL_ERROR;
+}
+
JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_,
QueryTreeNodePtr right_table_expression_,
QueryTreeNodePtr join_expression_,
@@ -113,4 +118,18 @@ ASTPtr JoinNode::toASTImpl() const
return tables_in_select_query_ast;
}
+void JoinNode::crossToInner(const QueryTreeNodePtr & join_expression_)
+{
+ if (kind != JoinKind::Cross && kind != JoinKind::Comma)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rewrite {} to INNER JOIN, expected CROSS", toString(kind));
+
+ if (children[join_expression_child_index])
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Join expression is expected to be empty for CROSS JOIN, got '{}'",
+ children[join_expression_child_index]->formatConvertedASTForErrorMessage());
+
+ kind = JoinKind::Inner;
+ strictness = JoinStrictness::All;
+ children[join_expression_child_index] = join_expression_;
+}
+
}
diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h
index 15ba11a0122..0d856985794 100644
--- a/src/Analyzer/JoinNode.h
+++ b/src/Analyzer/JoinNode.h
@@ -126,6 +126,13 @@ public:
return QueryTreeNodeType::JOIN;
}
+ /*
+ * Convert CROSS to INNER JOIN - changes JOIN kind and sets a new join expression
+ * (that was moved from WHERE clause).
+ * Expects the current kind to be CROSS (and join expression to be null because of that).
+ */
+ void crossToInner(const QueryTreeNodePtr & join_expression_);
+
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
protected:
diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp
index 10efebe0731..fdf818681d7 100644
--- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp
+++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp
@@ -1,8 +1,11 @@
#include "AutoFinalOnQueryPass.h"
-#include
-#include
#include
+
+#include
+#include
+#include
+#include
#include
namespace DB
@@ -10,52 +13,64 @@ namespace DB
namespace
{
- class AutoFinalOnQueryPassVisitor : public InDepthQueryTreeVisitorWithContext
+
+class AutoFinalOnQueryPassVisitor : public InDepthQueryTreeVisitorWithContext
+{
+public:
+ using Base = InDepthQueryTreeVisitorWithContext;
+ using Base::Base;
+
+ void visitImpl(QueryTreeNodePtr & node)
{
- public:
- using Base = InDepthQueryTreeVisitorWithContext;
- using Base::Base;
+ const auto & context = getContext();
+ if (!context->getSettingsRef().final)
+ return;
- void visitImpl(QueryTreeNodePtr & node)
+ const auto * query_node = node->as();
+ if (!query_node)
+ return;
+
+ auto table_expressions = extractTableExpressions(query_node->getJoinTree());
+ for (auto & table_expression : table_expressions)
+ applyFinalIfNeeded(table_expression);
+ }
+private:
+ static void applyFinalIfNeeded(QueryTreeNodePtr & node)
+ {
+ auto * table_node = node->as();
+ auto * table_function_node = node->as();
+ if (!table_node && !table_function_node)
+ return;
+
+ const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage();
+ bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote();
+ if (!is_final_supported)
+ return;
+
+ TableExpressionModifiers table_expression_modifiers_with_final(true /*has_final*/, {}, {});
+
+ if (table_node)
{
- if (auto * table_node = node->as())
- {
- if (autoFinalOnQuery(*table_node, table_node->getStorage(), getContext()))
- {
- auto modifier = TableExpressionModifiers(true, std::nullopt, std::nullopt);
- table_node->setTableExpressionModifiers(modifier);
- }
- }
+ if (table_node->hasTableExpressionModifiers())
+ table_node->getTableExpressionModifiers()->setHasFinal(true);
+ else
+ table_node->setTableExpressionModifiers(table_expression_modifiers_with_final);
}
-
- private:
- static bool autoFinalOnQuery(TableNode & table_node, StoragePtr storage, ContextPtr context)
+ else if (table_function_node)
{
- bool is_auto_final_setting_on = context->getSettingsRef().final;
- bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote();
- bool is_query_already_final = table_node.hasTableExpressionModifiers() ? table_node.getTableExpressionModifiers().has_value() : false;
-
- return is_auto_final_setting_on && !is_query_already_final && is_final_supported;
+ if (table_function_node->hasTableExpressionModifiers())
+ table_function_node->getTableExpressionModifiers()->setHasFinal(true);
+ else
+ table_function_node->setTableExpressionModifiers(table_expression_modifiers_with_final);
}
+ }
+};
- };
-
-}
-
-String AutoFinalOnQueryPass::getName()
-{
- return "AutoFinalOnQueryPass";
-}
-
-String AutoFinalOnQueryPass::getDescription()
-{
- return "Automatically applies final modifier to queries if it is supported and if user level final setting is set.";
}
void AutoFinalOnQueryPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
auto visitor = AutoFinalOnQueryPassVisitor(std::move(context));
-
visitor.visit(query_tree_node);
}
diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.h b/src/Analyzer/Passes/AutoFinalOnQueryPass.h
index eacbe0f8235..3489597108c 100644
--- a/src/Analyzer/Passes/AutoFinalOnQueryPass.h
+++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.h
@@ -7,13 +7,23 @@
namespace DB
{
-
+/** Automatically applies final modifier to table expressions in queries if it is supported and if user level final setting is set.
+ *
+ * Example: SELECT id, value FROM test_table;
+ * Result: SELECT id, value FROM test_table FINAL;
+ */
class AutoFinalOnQueryPass final : public IQueryTreePass
{
public:
- String getName() override;
+ String getName() override
+ {
+ return "AutoFinalOnQueryPass";
+ }
- String getDescription() override;
+ String getDescription() override
+ {
+ return "Automatically applies final modifier to table expressions in queries if it is supported and if user level final setting is set";
+ }
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
diff --git a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp
new file mode 100644
index 00000000000..4f62d6ef068
--- /dev/null
+++ b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp
@@ -0,0 +1,262 @@
+#include
+
+#include
+
+#include
+#include
+#include
+#include
+#include
+
+#include
+#include
+
+#include
+
+
+namespace DB
+{
+
+namespace ErrorCodes
+{
+ extern const int INCORRECT_QUERY;
+}
+
+namespace
+{
+
+void exctractJoinConditions(const QueryTreeNodePtr & node, QueryTreeNodes & equi_conditions, QueryTreeNodes & other)
+{
+ auto * func = node->as();
+ if (!func)
+ {
+ other.push_back(node);
+ return;
+ }
+
+ const auto & args = func->getArguments().getNodes();
+
+ if (args.size() == 2 && func->getFunctionName() == "equals")
+ {
+ equi_conditions.push_back(node);
+ }
+ else if (func->getFunctionName() == "and")
+ {
+ for (const auto & arg : args)
+ exctractJoinConditions(arg, equi_conditions, other);
+ }
+ else
+ {
+ other.push_back(node);
+ }
+}
+
+const QueryTreeNodePtr & getEquiArgument(const QueryTreeNodePtr & cond, size_t index)
+{
+ const auto * func = cond->as();
+ chassert(func && func->getFunctionName() == "equals" && func->getArguments().getNodes().size() == 2);
+ 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)
+{
+ if (!source)
+ return true;
+
+ if (source->isEqual(*table_expression))
+ return true;
+
+ if (const auto * join_node = table_expression->as())
+ {
+ return findInTableExpression(source, join_node->getLeftTableExpression())
+ || findInTableExpression(source, join_node->getRightTableExpression());
+ }
+
+
+ return false;
+}
+
+void getJoinNodes(QueryTreeNodePtr & join_tree_node, std::vector & join_nodes)
+{
+ auto * join_node = join_tree_node->as();
+ if (!join_node)
+ return;
+
+ if (!isCrossOrComma(join_node->getKind()))
+ return;
+
+ join_nodes.push_back(join_node);
+ getJoinNodes(join_node->getLeftTableExpression(), join_nodes);
+ getJoinNodes(join_node->getRightTableExpression(), join_nodes);
+}
+
+class CrossToInnerJoinVisitor : public InDepthQueryTreeVisitorWithContext
+{
+public:
+ using Base = InDepthQueryTreeVisitorWithContext;
+ using Base::Base;
+
+ /// Returns false if can't rewrite cross to inner join
+ bool tryRewrite(JoinNode & join_node, QueryTreeNodePtr & where_condition)
+ {
+ if (!isCrossOrComma(join_node.getKind()))
+ return false;
+
+ if (!where_condition)
+ return false;
+
+ const auto & left_table = join_node.getLeftTableExpression();
+ const auto & right_table = join_node.getRightTableExpression();
+
+ QueryTreeNodes equi_conditions;
+ QueryTreeNodes other_conditions;
+ exctractJoinConditions(where_condition, equi_conditions, other_conditions);
+ bool can_convert_cross_to_inner = false;
+ for (auto & cond : equi_conditions)
+ {
+ auto left_src = getExpressionSource(getEquiArgument(cond, 0));
+ auto right_src = getExpressionSource(getEquiArgument(cond, 1));
+ if (left_src.second && right_src.second && left_src.first && right_src.first)
+ {
+ bool can_join_on = (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 (can_join_on)
+ {
+ can_convert_cross_to_inner = true;
+ continue;
+ }
+ }
+
+ /// Can't join on this condition, move it to other conditions
+ other_conditions.push_back(cond);
+ cond = nullptr;
+ }
+
+ if (!can_convert_cross_to_inner)
+ return false;
+
+ equi_conditions.erase(std::remove(equi_conditions.begin(), equi_conditions.end(), nullptr), equi_conditions.end());
+ join_node.crossToInner(makeConjunction(equi_conditions));
+ where_condition = makeConjunction(other_conditions);
+ return true;
+ }
+
+ void visitImpl(QueryTreeNodePtr & node)
+ {
+ if (!isEnabled())
+ return;
+
+ auto * query_node = node->as();
+ if (!query_node)
+ return;
+
+ auto & where_node = query_node->getWhere();
+ if (!where_node)
+ return;
+
+ auto & join_tree_node = query_node->getJoinTree();
+ if (!join_tree_node || join_tree_node->getNodeType() != QueryTreeNodeType::JOIN)
+ return;
+
+ /// In case of multiple joins, we can try to rewrite all of them
+ /// Example: SELECT * FROM t1, t2, t3 WHERE t1.a = t2.a AND t2.a = t3.a
+ std::vector join_nodes;
+ getJoinNodes(join_tree_node, join_nodes);
+
+ for (auto * join_node : join_nodes)
+ {
+ bool is_rewritten = tryRewrite(*join_node, where_node);
+
+ if (!is_rewritten && forceRewrite(join_node->getKind()))
+ {
+ throw Exception(ErrorCodes::INCORRECT_QUERY,
+ "Failed to rewrite '{}' to INNER JOIN: "
+ "no equi-join conditions found in WHERE clause. "
+ "You may set setting `cross_to_inner_join_rewrite` to `1` to allow slow CROSS JOIN for this case",
+ join_node->formatASTForErrorMessage());
+ }
+ }
+ }
+
+private:
+ bool isEnabled() const
+ {
+ return getSettings().cross_to_inner_join_rewrite;
+ }
+
+ bool forceRewrite(JoinKind kind) const
+ {
+ if (kind == JoinKind::Cross)
+ return false;
+ /// Comma join can be forced to rewrite
+ return getSettings().cross_to_inner_join_rewrite >= 2;
+ }
+
+ QueryTreeNodePtr makeConjunction(const QueryTreeNodes & nodes)
+ {
+ if (nodes.empty())
+ return nullptr;
+
+ if (nodes.size() == 1)
+ return nodes.front();
+
+ auto function_node = std::make_shared("and");
+ for (const auto & node : nodes)
+ function_node->getArguments().getNodes().push_back(node);
+
+ const auto & function = FunctionFactory::instance().get("and", getContext());
+ function_node->resolveAsFunction(function->build(function_node->getArgumentColumns()));
+ return function_node;
+ }
+};
+
+}
+
+void CrossToInnerJoinPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
+{
+ CrossToInnerJoinVisitor visitor(std::move(context));
+ visitor.visit(query_tree_node);
+}
+
+}
diff --git a/src/Analyzer/Passes/CrossToInnerJoinPass.h b/src/Analyzer/Passes/CrossToInnerJoinPass.h
new file mode 100644
index 00000000000..127d26dc41d
--- /dev/null
+++ b/src/Analyzer/Passes/CrossToInnerJoinPass.h
@@ -0,0 +1,28 @@
+#pragma once
+
+#include
+
+namespace DB
+{
+
+
+/** Replace CROSS JOIN with INNER JOIN.
+ * Example:
+ * SELECT * FROM t1 CROSS JOIN t2 WHERE t1.a = t2.a AND t1.b > 10 AND t2.b = t2.c
+ * We can move equality condition to ON section of INNER JOIN:
+ * SELECT * FROM t1 INNER JOIN t2 ON t1.a = t2.a WHERE t1.b > 10 AND t2.b = t2.c
+ */
+class CrossToInnerJoinPass final : public IQueryTreePass
+{
+public:
+ String getName() override { return "CrossToInnerJoin"; }
+
+ String getDescription() override
+ {
+ return "Replace CROSS JOIN with INNER JOIN";
+ }
+
+ void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
+};
+
+}
diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp
index 218e47d973f..9ba18e27f73 100644
--- a/src/Analyzer/QueryTreePassManager.cpp
+++ b/src/Analyzer/QueryTreePassManager.cpp
@@ -39,6 +39,7 @@
#include
#include
#include
+#include
namespace DB
@@ -268,6 +269,7 @@ void addQueryTreePasses(QueryTreePassManager & manager)
manager.addPass(std::make_unique());
+ manager.addPass(std::make_unique());
}
}
diff --git a/src/Analyzer/TableExpressionModifiers.h b/src/Analyzer/TableExpressionModifiers.h
index f61c2a61610..9b76c9bc0fd 100644
--- a/src/Analyzer/TableExpressionModifiers.h
+++ b/src/Analyzer/TableExpressionModifiers.h
@@ -28,6 +28,12 @@ public:
return has_final;
}
+ /// Set has final value
+ void setHasFinal(bool value)
+ {
+ has_final = value;
+ }
+
/// Returns true if sample size ratio is specified, false otherwise
bool hasSampleSizeRatio() const
{
diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h
index 292ab740c5b..a88630ffd00 100644
--- a/src/Analyzer/TableFunctionNode.h
+++ b/src/Analyzer/TableFunctionNode.h
@@ -116,6 +116,12 @@ public:
return table_expression_modifiers;
}
+ /// Get table expression modifiers
+ std::optional & getTableExpressionModifiers()
+ {
+ return table_expression_modifiers;
+ }
+
/// Set table expression modifiers
void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value)
{
diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h
index 4965de535df..6d47f87c78b 100644
--- a/src/Analyzer/TableNode.h
+++ b/src/Analyzer/TableNode.h
@@ -68,6 +68,12 @@ public:
return table_expression_modifiers;
}
+ /// Get table expression modifiers
+ std::optional & getTableExpressionModifiers()
+ {
+ return table_expression_modifiers;
+ }
+
/// Set table expression modifiers
void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value)
{
diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp
index 96f76b70f31..3ed3ed73328 100644
--- a/src/Columns/ColumnVector.cpp
+++ b/src/Columns/ColumnVector.cpp
@@ -747,7 +747,7 @@ namespace
*/
template
requires (std::is_same_v || std::is_same_v)
- void replicateSSE42Int32(const IntType * __restrict data, IntType * __restrict result_data, const IColumn::Offsets & offsets)
+ void replicateSSE2Int32(const IntType * __restrict data, IntType * __restrict result_data, const IColumn::Offsets & offsets)
{
const IntType * data_copy_begin_ptr = nullptr;
size_t offsets_size = offsets.size();
@@ -842,7 +842,7 @@ ColumnPtr ColumnVector::replicate(const IColumn::Offsets & offsets) const
#ifdef __SSE2__
if constexpr (std::is_same_v)
{
- replicateSSE42Int32(getData().data(), res->getData().data(), offsets);
+ replicateSSE2Int32(getData().data(), res->getData().data(), offsets);
return res;
}
#endif
diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h
index b9c03aae0ca..78535a751c3 100644
--- a/src/Core/NamesAndTypes.h
+++ b/src/Core/NamesAndTypes.h
@@ -53,7 +53,17 @@ private:
/// This needed to use structured bindings for NameAndTypePair
/// const auto & [name, type] = name_and_type
template
-decltype(auto) get(const NameAndTypePair & name_and_type)
+const std::tuple_element_t & get(const NameAndTypePair & name_and_type)
+{
+ if constexpr (I == 0)
+ return name_and_type.name;
+ else if constexpr (I == 1)
+ return name_and_type.type;
+}
+
+/// auto & [name, type] = name_and_type
+template
+std::tuple_element_t & get(NameAndTypePair & name_and_type)
{
if constexpr (I == 0)
return name_and_type.name;
diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp
index c233060e646..435401796a0 100644
--- a/src/Interpreters/executeQuery.cpp
+++ b/src/Interpreters/executeQuery.cpp
@@ -1276,6 +1276,12 @@ void executeQuery(
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr);
auto & pipeline = streams.pipeline;
+ QueryResultDetails result_details
+ {
+ .query_id = context->getClientInfo().current_query_id,
+ .timezone = DateLUT::instance().getTimeZone(),
+ };
+
std::unique_ptr compressed_buffer;
try
{
@@ -1334,9 +1340,8 @@ void executeQuery(
out->onProgress(progress);
});
- if (set_result_details)
- set_result_details(
- context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone());
+ result_details.content_type = out->getContentType();
+ result_details.format = format_name;
pipeline.complete(std::move(out));
}
@@ -1345,6 +1350,9 @@ void executeQuery(
pipeline.setProgressCallback(context->getProgressCallback());
}
+ if (set_result_details)
+ set_result_details(result_details);
+
if (pipeline.initialized())
{
CompletedPipelineExecutor executor(pipeline);
diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h
index 9c561d8b88c..93152cc1de6 100644
--- a/src/Interpreters/executeQuery.h
+++ b/src/Interpreters/executeQuery.h
@@ -11,7 +11,15 @@ namespace DB
class ReadBuffer;
class WriteBuffer;
-using SetResultDetailsFunc = std::function;
+struct QueryResultDetails
+{
+ String query_id;
+ std::optional content_type;
+ std::optional format;
+ std::optional timezone;
+};
+
+using SetResultDetailsFunc = std::function;
/// Parse and execute a query.
void executeQuery(
diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp
index 2c5c8ab5e62..614e5c03bc1 100644
--- a/src/Planner/PlannerActionsVisitor.cpp
+++ b/src/Planner/PlannerActionsVisitor.cpp
@@ -65,6 +65,15 @@ public:
return node_name_to_node.find(node_name) != node_name_to_node.end();
}
+ [[maybe_unused]] bool containsInputNode(const std::string & node_name)
+ {
+ const auto * node = tryGetNode(node_name);
+ if (node && node->type == ActionsDAG::ActionType::INPUT)
+ return true;
+
+ return false;
+ }
+
[[maybe_unused]] const ActionsDAG::Node * tryGetNode(const std::string & node_name)
{
auto it = node_name_to_node.find(node_name);
@@ -421,7 +430,16 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
auto function_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name);
- if (function_node.isAggregateFunction() || function_node.isWindowFunction())
+ /* Aggregate functions, window functions, and GROUP BY expressions were already analyzed in the previous steps.
+ * If we have already visited some expression, we don't need to revisit it or its arguments again.
+ * For example, the expression from the aggregation step is also present in the projection:
+ * SELECT foo(a, b, c) as x FROM table GROUP BY foo(a, b, c)
+ * In this case we should not analyze `a`, `b`, `c` again.
+ * Moreover, it can lead to an error if we have arrayJoin in the arguments because it will be calculated twice.
+ */
+ bool is_input_node = function_node.isAggregateFunction() || function_node.isWindowFunction()
+ || actions_stack.front().containsInputNode(function_node_name);
+ if (is_input_node)
{
size_t actions_stack_size = actions_stack.size();
diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp
index bea2fe87e6d..d898049209e 100644
--- a/src/Server/HTTPHandler.cpp
+++ b/src/Server/HTTPHandler.cpp
@@ -831,12 +831,20 @@ void HTTPHandler::processQuery(
customizeContext(request, context);
executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context,
- [&response, this] (const String & current_query_id, const String & content_type, const String & format, const String & timezone)
+ [&response, this] (const QueryResultDetails & details)
{
- response.setContentType(content_type_override.value_or(content_type));
- response.add("X-ClickHouse-Query-Id", current_query_id);
- response.add("X-ClickHouse-Format", format);
- response.add("X-ClickHouse-Timezone", timezone);
+ response.add("X-ClickHouse-Query-Id", details.query_id);
+
+ if (content_type_override)
+ response.setContentType(*content_type_override);
+ else if (details.content_type)
+ response.setContentType(*details.content_type);
+
+ if (details.format)
+ response.add("X-ClickHouse-Format", *details.format);
+
+ if (details.timezone)
+ response.add("X-ClickHouse-Timezone", *details.timezone);
}
);
diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp
index 3715d658730..d8ea359ce5f 100644
--- a/src/Server/MySQLHandler.cpp
+++ b/src/Server/MySQLHandler.cpp
@@ -352,11 +352,15 @@ void MySQLHandler::comQuery(ReadBuffer & payload)
format_settings.mysql_wire.max_packet_size = max_packet_size;
format_settings.mysql_wire.sequence_id = &sequence_id;
- auto set_result_details = [&with_output](const String &, const String &, const String &format, const String &)
+ auto set_result_details = [&with_output](const QueryResultDetails & details)
{
- if (format != "MySQLWire")
- throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats");
- with_output = true;
+ if (details.format)
+ {
+ if (*details.format != "MySQLWire")
+ throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats");
+
+ with_output = true;
+ }
};
executeQuery(should_replace ? replacement : payload, *out, false, query_context, set_result_details, format_settings);
diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp
index 5ff5d255372..e3b87d48ce6 100644
--- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp
+++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp
@@ -1076,7 +1076,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
read_result.checkInternalConsistency();
- if (!read_result.can_return_prewhere_column_without_filtering)
+ if (!read_result.can_return_prewhere_column_without_filtering && last_reader_in_chain)
{
if (!read_result.filterWasApplied())
{
@@ -1380,17 +1380,14 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r
current_step_filter = result.columns[prewhere_column_pos];
}
+ /// In case when we are returning prewhere column the caller expects it to serve as a final filter:
+ /// it must contain 0s not only from the current step but also from all the previous steps.
+ /// One way to achieve this is to apply the final_filter if we know that the final_filter was not applied at
+ /// several previous steps but was accumulated instead.
+ result.can_return_prewhere_column_without_filtering = result.filterWasApplied();
+
if (prewhere_info->remove_column)
result.columns.erase(result.columns.begin() + prewhere_column_pos);
- else
- {
- /// In case when we are not removing prewhere column the caller expects it to serve as a final filter:
- /// it must contain 0s not only from the current step but also from all the previous steps.
- /// One way to achieve this is to apply the final_filter if we know that the final _filter was not applied at
- /// several previous steps but was accumulated instead.
- result.can_return_prewhere_column_without_filtering =
- (!result.final_filter.present() || result.final_filter.countBytesInFilter() == result.num_rows);
- }
FilterWithCachedCount current_filter(current_step_filter);
diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp
index 31fd99f0aa1..035cbdac55e 100644
--- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp
+++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp
@@ -985,7 +985,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart(
}
else if (Coordination::isUserError(multi_code))
{
- String failed_op_path = zkutil::KeeperMultiException(multi_code, ops, responses).getPathForFirstFailedOp();
+ String failed_op_path = ops[zkutil::getFailedOpIndex(multi_code, responses)]->getPath();
auto contains = [](const auto & block_ids, const String & path)
{
@@ -1002,7 +1002,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart(
if (multi_code == Coordination::Error::ZNODEEXISTS && deduplicate_block && contains(block_id_path, failed_op_path))
{
- /// Block with the same id have just appeared in table (or other replica), rollback thee insertion.
+ /// Block with the same id have just appeared in table (or other replica), rollback the insertion.
LOG_INFO(log, "Block with ID {} already exists (it was just appeared). Renaming part {} back to {}. Will retry write.",
toString(block_id), part->name, temporary_part_relative_path);
diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py
index f37b2656be3..ecc36b1c4e3 100644
--- a/tests/ci/build_check.py
+++ b/tests/ci/build_check.py
@@ -9,6 +9,10 @@ import time
from shutil import rmtree
from typing import List, Tuple
+from ccache_utils import get_ccache_if_not_exists, upload_ccache
+from ci_config import CI_CONFIG, BuildConfig
+from commit_status_helper import get_commit_filtered_statuses, get_commit
+from docker_pull_helper import get_image_with_version
from env_helper import (
CACHES_PATH,
GITHUB_JOB,
@@ -18,18 +22,17 @@ from env_helper import (
S3_DOWNLOAD,
TEMP_PATH,
)
-from s3_helper import S3Helper
+from get_robot_token import get_best_robot_token
+from github_helper import GitHub
from pr_info import PRInfo
+from s3_helper import S3Helper
+from tee_popen import TeePopen
from version_helper import (
ClickHouseVersion,
Git,
get_version_from_repo,
update_version_local,
)
-from ccache_utils import get_ccache_if_not_exists, upload_ccache
-from ci_config import CI_CONFIG, BuildConfig
-from docker_pull_helper import get_image_with_version
-from tee_popen import TeePopen
IMAGE_NAME = "clickhouse/binary-builder"
BUILD_LOG_NAME = "build_log.log"
@@ -122,8 +125,7 @@ def check_for_success_run(
logged_prefix = os.path.join(S3_BUILDS_BUCKET, s3_prefix, "")
logging.info("Checking for artifacts in %s", logged_prefix)
try:
- # TODO: theoretically, it would miss performance artifact for pr==0,
- # but luckily we rerun only really failed tasks now, so we're safe
+ # Performance artifacts are now part of regular build, so we're safe
build_results = s3_helper.list_prefix(s3_prefix)
except Exception as ex:
logging.info("Got exception while listing %s: %s\nRerun", logged_prefix, ex)
@@ -231,6 +233,29 @@ def upload_master_static_binaries(
print(f"::notice ::Binary static URL: {url}")
+def mark_failed_reports_pending(build_name: str, sha: str) -> None:
+ try:
+ gh = GitHub(get_best_robot_token())
+ commit = get_commit(gh, sha)
+ statuses = get_commit_filtered_statuses(commit)
+ report_status = [
+ name
+ for name, builds in CI_CONFIG["builds_report_config"].items()
+ if build_name in builds
+ ][0]
+ for status in statuses:
+ if status.context == report_status and status.state in ["failure", "error"]:
+ logging.info(
+ "Commit already have failed status for '%s', setting it to 'pending'",
+ report_status,
+ )
+ commit.create_status(
+ "pending", status.url, "Set to pending on rerun", report_status
+ )
+ except: # we do not care about any exception here
+ logging.info("Failed to get or mark the reports status as pending, continue")
+
+
def main():
logging.basicConfig(level=logging.INFO)
@@ -260,6 +285,9 @@ def main():
# put them as github actions artifact (result)
check_for_success_run(s3_helper, s3_path_prefix, build_name, build_config)
+ # If it's a latter running, we need to mark possible failed status
+ mark_failed_reports_pending(build_name, pr_info.sha)
+
docker_image = get_image_with_version(IMAGES_PATH, IMAGE_NAME)
image_version = docker_image.version
diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py
index 147b49d910e..7987ea2643f 100644
--- a/tests/ci/cherry_pick.py
+++ b/tests/ci/cherry_pick.py
@@ -508,7 +508,7 @@ def main():
logging.getLogger("git_helper").setLevel(logging.DEBUG)
token = args.token or get_best_robot_token()
- gh = GitHub(token, create_cache_dir=False, per_page=100)
+ gh = GitHub(token, create_cache_dir=False)
bp = Backport(gh, args.repo, args.dry_run)
# https://github.com/python/mypy/issues/3004
bp.gh.cache_path = f"{TEMP_PATH}/gh_cache" # type: ignore
diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py
index 785250c3904..c3f8215b220 100644
--- a/tests/ci/commit_status_helper.py
+++ b/tests/ci/commit_status_helper.py
@@ -6,11 +6,12 @@ import time
from typing import List
import logging
-from ci_config import CI_CONFIG, REQUIRED_CHECKS
-from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL
from github import Github
from github.Commit import Commit
from github.CommitStatus import CommitStatus
+
+from ci_config import CI_CONFIG, REQUIRED_CHECKS
+from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL
from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL
RETRY = 5
diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py
index 1b8861b92a6..eae0bc1c33b 100644
--- a/tests/ci/github_helper.py
+++ b/tests/ci/github_helper.py
@@ -35,6 +35,8 @@ class GitHub(github.Github):
self._cache_path = Path(CACHE_PATH)
if create_cache_dir:
self.cache_path = self.cache_path
+ if not kwargs.get("per_page"):
+ kwargs["per_page"] = 100
# And set Path
super().__init__(*args, **kwargs)
self._retries = 0
diff --git a/tests/ci/mark_release_ready.py b/tests/ci/mark_release_ready.py
index 0ef134b9280..b103dd053bb 100755
--- a/tests/ci/mark_release_ready.py
+++ b/tests/ci/mark_release_ready.py
@@ -43,7 +43,7 @@ def main():
description = "the release can be created from the commit"
args.token = args.token or get_best_robot_token()
- gh = GitHub(args.token, create_cache_dir=False, per_page=100)
+ gh = GitHub(args.token, create_cache_dir=False)
# Get the rate limits for a quick fail
gh.get_rate_limit()
commit = get_commit(gh, args.commit)
diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py
index 4a21bfcdd70..fedac48f24d 100644
--- a/tests/ci/merge_pr.py
+++ b/tests/ci/merge_pr.py
@@ -217,7 +217,7 @@ def main():
args = parse_args()
logging.info("Going to process PR #%s in repo %s", args.pr, args.repo)
token = args.token or get_best_robot_token()
- gh = GitHub(token, per_page=100)
+ gh = GitHub(token)
repo = gh.get_repo(args.repo)
# An ugly and not nice fix to patch the wrong organization URL,
# see https://github.com/PyGithub/PyGithub/issues/2395#issuecomment-1378629710
diff --git a/tests/ci/release.py b/tests/ci/release.py
index a4fe4046572..f7b5155e750 100755
--- a/tests/ci/release.py
+++ b/tests/ci/release.py
@@ -199,6 +199,10 @@ class Release:
raise ValueError(
"The relese type must be 'major' for minor versions>=12"
)
+ if self._version.minor < 12 and self.release_type == "major":
+ raise ValueError(
+ "The relese type must be 'minor' for minor versions<12"
+ )
with self._checkout(self.release_commit, True):
# Checkout to the commit, it will provide the correct current version
@@ -265,7 +269,11 @@ class Release:
f"for {self.release_type} release"
)
- def _commit_cmake_contributors(self, version: ClickHouseVersion) -> None:
+ def _update_cmake_contributors(
+ self, version: ClickHouseVersion, reset_tweak: bool = True
+ ) -> None:
+ if reset_tweak:
+ version = version.reset_tweak()
update_cmake_version(version)
update_contributors(raise_error=True)
if self.dry_run:
@@ -274,9 +282,15 @@ class Release:
self.run(f"git diff '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'"),
)
self.run(f"git checkout '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'")
+
+ def _commit_cmake_contributors(
+ self, version: ClickHouseVersion, reset_tweak: bool = True
+ ) -> None:
+ if reset_tweak:
+ version = version.reset_tweak()
self.run(
- f"git commit -m 'Update version to {version.string}' "
- f"'{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'",
+ f"git commit '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}' "
+ f"-m 'Update autogenerated version to {version.string} and contributors'",
dry_run=self.dry_run,
)
@@ -321,27 +335,12 @@ class Release:
with self._create_gh_release(False):
self.version = self.version.update(self.release_type)
self.version.with_description(version_type)
- update_cmake_version(self.version)
- update_contributors(raise_error=True)
- if self.dry_run:
- logging.info(
- "Dry running, resetting the following changes in the repo:\n%s",
- self.run(
- f"git diff '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'"
- ),
- )
- self.run(f"git checkout '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'")
-
+ self._update_cmake_contributors(self.version)
# Checkouting the commit of the branch and not the branch itself,
# then we are able to skip rollback
with self._checkout(f"{self.release_branch}^0", False):
current_commit = self.run("git rev-parse HEAD")
- self.run(
- f"git commit -m "
- f"'Update version to {self.version.string}' "
- f"'{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'",
- dry_run=self.dry_run,
- )
+ self._commit_cmake_contributors(self.version)
with self._push(
"HEAD", with_rollback_on_fail=False, remote_ref=self.release_branch
):
@@ -406,6 +405,7 @@ class Release:
if version_type == VersionType.LTS:
pr_labels += " --label release-lts"
new_version.with_description(version_type)
+ self._update_cmake_contributors(new_version)
self._commit_cmake_contributors(new_version)
with self._push(self.release_branch):
with self._create_gh_label(
@@ -434,6 +434,7 @@ class Release:
self.read_version()
self.version = self.version.update(self.release_type)
self.version.with_description(VersionType.TESTING)
+ self._update_cmake_contributors(self.version)
self._commit_cmake_contributors(self.version)
with self._push(helper_branch):
body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md")
diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py
index 9350785b33b..89878990c2c 100644
--- a/tests/ci/style_check.py
+++ b/tests/ci/style_check.py
@@ -148,7 +148,7 @@ def main():
if args.push:
checkout_head(pr_info)
- gh = GitHub(get_best_robot_token(), per_page=100, create_cache_dir=False)
+ gh = GitHub(get_best_robot_token(), create_cache_dir=False)
atexit.register(update_mergeable_check, gh, pr_info, NAME)
diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py
index 69cfba64be3..1ede2c90b55 100755
--- a/tests/ci/version_helper.py
+++ b/tests/ci/version_helper.py
@@ -88,6 +88,13 @@ class ClickHouseVersion:
self.major, self.minor, self.patch + 1, self.revision, self._git
)
+ def reset_tweak(self) -> "ClickHouseVersion":
+ if self._git is not None:
+ self._git.update()
+ return ClickHouseVersion(
+ self.major, self.minor, self.patch, self.revision, self._git, "1"
+ )
+
@property
def major(self) -> int:
return self._major
diff --git a/tests/config/users.d/prefetch_settings.xml b/tests/config/users.d/prefetch_settings.xml
index 33ac0a4eb01..d2b6d8a9386 100644
--- a/tests/config/users.d/prefetch_settings.xml
+++ b/tests/config/users.d/prefetch_settings.xml
@@ -1,7 +1,7 @@
- 1
+ 0
0
1Gi
diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.reference b/tests/queries/0_stateless/00849_multiple_comma_join_2.reference
index 2652a82ab54..16f228a5569 100644
--- a/tests/queries/0_stateless/00849_multiple_comma_join_2.reference
+++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.reference
@@ -1,205 +1,33 @@
-SELECT a
-FROM t1
-CROSS JOIN t2
-SELECT a
-FROM t1
-ALL INNER JOIN t2 ON a = t2.a
-WHERE a = t2.a
-SELECT a
-FROM t1
-ALL INNER JOIN t2 ON b = t2.b
-WHERE b = t2.b
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- a AS `--t1.a`,
- t2.a AS `--t2.a`
- FROM t1
- ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`
-) AS `--.s`
-ALL INNER JOIN t3 ON `--t1.a` = a
-WHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = a)
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- b AS `--t1.b`,
- a AS `--t1.a`,
- t2.b AS `--t2.b`
- FROM t1
- ALL INNER JOIN t2 ON `--t1.b` = `--t2.b`
-) AS `--.s`
-ALL INNER JOIN t3 ON `--t1.b` = b
-WHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = b)
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- `--t1.a`,
- `--t2.a`,
- a AS `--t3.a`
- FROM
- (
- SELECT
- a AS `--t1.a`,
- t2.a AS `--t2.a`
- FROM t1
- ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`
- ) AS `--.s`
- ALL INNER JOIN t3 ON `--t1.a` = `--t3.a`
-) AS `--.s`
-ALL INNER JOIN t4 ON `--t1.a` = a
-WHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = `--t3.a`) AND (`--t1.a` = a)
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- `--t1.b`,
- `--t1.a`,
- `--t2.b`,
- b AS `--t3.b`
- FROM
- (
- SELECT
- b AS `--t1.b`,
- a AS `--t1.a`,
- t2.b AS `--t2.b`
- FROM t1
- ALL INNER JOIN t2 ON `--t1.b` = `--t2.b`
- ) AS `--.s`
- ALL INNER JOIN t3 ON `--t1.b` = `--t3.b`
-) AS `--.s`
-ALL INNER JOIN t4 ON `--t1.b` = b
-WHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = `--t3.b`) AND (`--t1.b` = b)
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- `--t1.a`,
- `--t2.a`,
- a AS `--t3.a`
- FROM
- (
- SELECT
- a AS `--t1.a`,
- t2.a AS `--t2.a`
- FROM t1
- ALL INNER JOIN t2 ON `--t2.a` = `--t1.a`
- ) AS `--.s`
- ALL INNER JOIN t3 ON `--t2.a` = `--t3.a`
-) AS `--.s`
-ALL INNER JOIN t4 ON `--t2.a` = a
-WHERE (`--t2.a` = `--t1.a`) AND (`--t2.a` = `--t3.a`) AND (`--t2.a` = a)
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- `--t1.a`,
- `--t2.a`,
- a AS `--t3.a`
- FROM
- (
- SELECT
- a AS `--t1.a`,
- t2.a AS `--t2.a`
- FROM t1
- CROSS JOIN t2
- ) AS `--.s`
- ALL INNER JOIN t3 ON (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`)
-) AS `--.s`
-ALL INNER JOIN t4 ON `--t3.a` = a
-WHERE (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`) AND (`--t3.a` = a)
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- `--t1.a`,
- `--t2.a`,
- a AS `--t3.a`
- FROM
- (
- SELECT
- a AS `--t1.a`,
- t2.a AS `--t2.a`
- FROM t1
- CROSS JOIN t2
- ) AS `--.s`
- CROSS JOIN t3
-) AS `--.s`
-ALL INNER JOIN t4 ON (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`)
-WHERE (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`)
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- `--t1.a`,
- `--t2.a`,
- a AS `--t3.a`
- FROM
- (
- SELECT
- a AS `--t1.a`,
- t2.a AS `--t2.a`
- FROM t1
- ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`
- ) AS `--.s`
- ALL INNER JOIN t3 ON `--t2.a` = `--t3.a`
-) AS `--.s`
-ALL INNER JOIN t4 ON `--t3.a` = a
-WHERE (`--t1.a` = `--t2.a`) AND (`--t2.a` = `--t3.a`) AND (`--t3.a` = a)
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT `--t1.a`
- FROM
- (
- SELECT a AS `--t1.a`
- FROM t1
- CROSS JOIN t2
- ) AS `--.s`
- CROSS JOIN t3
-) AS `--.s`
-CROSS JOIN t4
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT `--t1.a`
- FROM
- (
- SELECT a AS `--t1.a`
- FROM t1
- CROSS JOIN t2
- ) AS `--.s`
- CROSS JOIN t3
-) AS `--.s`
-CROSS JOIN t4
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT a AS `--t1.a`
- FROM t1
- CROSS JOIN t2
-) AS `--.s`
-CROSS JOIN t3
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT a AS `--t1.a`
- FROM t1
- ALL INNER JOIN t2 USING (a)
-) AS `--.s`
-CROSS JOIN t3
-SELECT `--t1.a` AS `t1.a`
-FROM
-(
- SELECT
- a AS `--t1.a`,
- t2.a AS `--t2.a`
- FROM t1
- ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`
-) AS `--.s`
-CROSS JOIN t3
+0 1
+0 1
+0 2
+0 2
+0 3
+0 3
+0 3
+1 2
+2 1
+0 3
+3 0
+3 0
+2 0
+1 1
+1 1
+0 1
+0 1
+0 2
+0 2
+0 3
+0 3
+0 3
+1 2
+2 1
+0 3
+3 0
+3 0
+2 0
+1 1
+1 1
SELECT * FROM t1, t2
1 1 1 1
1 1 1 \N
diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql
index eb803450ff7..db8b27c4d4d 100644
--- a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql
+++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql
@@ -12,31 +12,107 @@ CREATE TABLE t2 (a UInt32, b Nullable(Int32)) ENGINE = Memory;
CREATE TABLE t3 (a UInt32, b Nullable(Int32)) ENGINE = Memory;
CREATE TABLE t4 (a UInt32, b Nullable(Int32)) ENGINE = Memory;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2 WHERE t1.a = t2.a;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2 WHERE t1.b = t2.b;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b;
+SET allow_experimental_analyzer = 0;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t2.a = t1.a AND t2.a = t3.a AND t2.a = t4.a;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t3.a = t1.a AND t3.a = t2.a AND t3.a = t4.a;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t4.a = t1.a AND t4.a = t2.a AND t4.a = t3.a;
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a;
+--- EXPLAIN SYNTAX (old AST based optimization)
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2 WHERE t1.a = t2.a);
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4;
-EXPLAIN SYNTAX SELECT t1.a FROM t1 CROSS JOIN t2 CROSS JOIN t3 CROSS JOIN t4;
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2 WHERE t1.b = t2.b);
-EXPLAIN SYNTAX SELECT t1.a FROM t1, t2 CROSS JOIN t3;
-EXPLAIN SYNTAX SELECT t1.a FROM t1 JOIN t2 USING a CROSS JOIN t3;
-EXPLAIN SYNTAX SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a CROSS JOIN t3;
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t2.a = t1.a AND t2.a = t3.a AND t2.a = t4.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t3.a = t1.a AND t3.a = t2.a AND t3.a = t4.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t4.a = t1.a AND t4.a = t2.a AND t4.a = t3.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2, t3, t4);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1 CROSS JOIN t2 CROSS JOIN t3 CROSS JOIN t4);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1, t2 CROSS JOIN t3);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1 JOIN t2 USING a CROSS JOIN t3);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN SYNTAX SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a CROSS JOIN t3);
+
+--- EXPLAIN QUERY TREE
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.a = t2.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.b = t2.b);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t2.a = t1.a AND t2.a = t3.a AND t2.a = t4.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t3.a = t1.a AND t3.a = t2.a AND t3.a = t4.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t4.a = t1.a AND t4.a = t2.a AND t4.a = t3.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1 CROSS JOIN t2 CROSS JOIN t3 CROSS JOIN t4);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 CROSS JOIN t3);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 USING a CROSS JOIN t3);
+
+SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM (
+ EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a CROSS JOIN t3);
INSERT INTO t1 values (1,1), (2,2), (3,3), (4,4);
INSERT INTO t2 values (1,1), (1, Null);
INSERT INTO t3 values (1,1), (1, Null);
INSERT INTO t4 values (1,1), (1, Null);
+SET allow_experimental_analyzer = 1;
+
SELECT 'SELECT * FROM t1, t2';
SELECT * FROM t1, t2
ORDER BY t1.a, t2.b;
diff --git a/tests/queries/0_stateless/01176_mysql_client_interactive.expect b/tests/queries/0_stateless/01176_mysql_client_interactive.expect
index 8d23b3bef60..2bb6ba8726b 100755
--- a/tests/queries/0_stateless/01176_mysql_client_interactive.expect
+++ b/tests/queries/0_stateless/01176_mysql_client_interactive.expect
@@ -12,9 +12,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$MYSQL_CLIENT_BINARY \$MYSQL_CLIENT_OPT"
diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect
index a5f127ec19e..4784fc38a84 100755
--- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect
+++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect
@@ -40,5 +40,5 @@ expect "2020-01-01 00:00:00.000\t2"
send -- "DROP TABLE test_01179\r"
expect "Ok."
-send -- "\4"
+send -- "exit\r"
expect eof
diff --git a/tests/queries/0_stateless/01180_client_syntax_errors.expect b/tests/queries/0_stateless/01180_client_syntax_errors.expect
index c1fd0f93510..d5c2ab220d5 100755
--- a/tests/queries/0_stateless/01180_client_syntax_errors.expect
+++ b/tests/queries/0_stateless/01180_client_syntax_errors.expect
@@ -10,9 +10,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect
index 629698b4565..3bfd454bb1f 100755
--- a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect
+++ b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect
@@ -11,9 +11,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
# useful debugging configuration
diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect
index 6b11b1eee15..1ded43d3fed 100755
--- a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect
+++ b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect
@@ -10,9 +10,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
diff --git a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect
index de485383024..4091d9fec91 100755
--- a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect
+++ b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect
@@ -11,9 +11,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
diff --git a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect
index 8547be839d4..90e19e077ec 100755
--- a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect
+++ b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect
@@ -10,9 +10,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=$history_file"
diff --git a/tests/queries/0_stateless/01520_client_print_query_id.expect b/tests/queries/0_stateless/01520_client_print_query_id.expect
index cbeacc6a4ec..cec1695c433 100755
--- a/tests/queries/0_stateless/01520_client_print_query_id.expect
+++ b/tests/queries/0_stateless/01520_client_print_query_id.expect
@@ -10,9 +10,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh
index 056613c11b5..617148de5a3 100755
--- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh
+++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh
@@ -34,9 +34,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i \$any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i \$any_spawn_id timeout { exit 1 }
}
spawn bash -c "$*"
diff --git a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect
index 223690f1f8b..a7e4b45eb44 100755
--- a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect
+++ b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect
@@ -10,9 +10,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
diff --git a/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect
index 0e06c2f99df..e5b6bc65e9b 100755
--- a/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect
+++ b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect
@@ -10,9 +10,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
# history file is not required, in-memory history is enough
diff --git a/tests/queries/0_stateless/01933_client_replxx_convert_history.expect b/tests/queries/0_stateless/01933_client_replxx_convert_history.expect
index 0c95b630742..69c5ff0118e 100755
--- a/tests/queries/0_stateless/01933_client_replxx_convert_history.expect
+++ b/tests/queries/0_stateless/01933_client_replxx_convert_history.expect
@@ -10,9 +10,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
exec bash -c "echo select 1 > $history_file.txt"
diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect
index c93635b3b27..4e6dd3e1b0f 100755
--- a/tests/queries/0_stateless/01945_show_debug_warning.expect
+++ b/tests/queries/0_stateless/01945_show_debug_warning.expect
@@ -14,9 +14,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
set Debug_type 0
diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect
index 4f28fafc1e6..b74238c7477 100755
--- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect
+++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect
@@ -15,9 +15,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
#
diff --git a/tests/queries/0_stateless/02047_client_exception.expect b/tests/queries/0_stateless/02047_client_exception.expect
index 69f468907a3..4dfdf211ba2 100755
--- a/tests/queries/0_stateless/02047_client_exception.expect
+++ b/tests/queries/0_stateless/02047_client_exception.expect
@@ -6,14 +6,14 @@ exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0
set history_file $env(CLICKHOUSE_TMP)/$basename.history
log_user 0
-set timeout 20
+set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
@@ -31,5 +31,5 @@ expect "Received exception from server"
send -- "DROP TABLE test_02047\r"
expect "Ok."
-send -- "\4"
+send -- "exit\r"
expect eof
diff --git a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect
index a9905128ad5..a1454696253 100755
--- a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect
+++ b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect
@@ -5,14 +5,14 @@ set basename [file tail $argv0]
exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0
log_user 0
-set timeout 20
+set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion"
@@ -50,5 +50,5 @@ expect "Ok."
send -- "drop table t\r"
expect "Ok."
-send -- "\4"
+send -- "exit\r"
expect eof
diff --git a/tests/queries/0_stateless/02105_backslash_letter_commands.expect b/tests/queries/0_stateless/02105_backslash_letter_commands.expect
index 8f8ec1f5abd..984e6f6d2eb 100755
--- a/tests/queries/0_stateless/02105_backslash_letter_commands.expect
+++ b/tests/queries/0_stateless/02105_backslash_letter_commands.expect
@@ -10,9 +10,9 @@ set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
@@ -20,15 +20,15 @@ expect ":) "
# Send a command
send -- "\\ld;\r"
-expect "Syntax error: *"
+expect "Syntax error: "
expect ":) "
send -- "\\c;\r"
-expect "Syntax error: *"
+expect "Syntax error: "
expect ":) "
send -- " \\l ; \\d; \r"
-expect "Syntax error (Multi-statements are not allowed): *"
+expect "Syntax error (Multi-statements are not allowed): "
expect ":) "
send -- " \\l ;\r"
diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect
index 4fd430a4a69..5318c7eac32 100755
--- a/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect
+++ b/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect
@@ -11,9 +11,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
system "$basedir/helpers/02112_prepare.sh"
diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect
index a90e85d1069..3413651fe68 100755
--- a/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect
+++ b/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect
@@ -10,9 +10,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion --interactive --query 'create table t(i Int32) engine=Memory; insert into t select 1'"
diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect
index 34eac360132..404daeda88e 100755
--- a/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect
+++ b/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect
@@ -11,9 +11,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
system "$basedir/helpers/02112_prepare.sh"
diff --git a/tests/queries/0_stateless/02116_interactive_hello.expect b/tests/queries/0_stateless/02116_interactive_hello.expect
index 7e895196304..92722e203ec 100755
--- a/tests/queries/0_stateless/02116_interactive_hello.expect
+++ b/tests/queries/0_stateless/02116_interactive_hello.expect
@@ -12,9 +12,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
diff --git a/tests/queries/0_stateless/02132_client_history_navigation.expect b/tests/queries/0_stateless/02132_client_history_navigation.expect
index 3316f26d552..dc7e44b41ee 100755
--- a/tests/queries/0_stateless/02132_client_history_navigation.expect
+++ b/tests/queries/0_stateless/02132_client_history_navigation.expect
@@ -11,9 +11,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
# useful debugging configuration
@@ -23,16 +23,20 @@ spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \
expect ":) "
# Make a query
-send -- "SELECT 1\r"
+send -- "SELECT 'unique1'\r"
expect "1"
expect ":) "
-send -- "SELECT 2"
+send -- "SELECT 'unique2'"
# NOTE: it does not work for alacritty with TERM=xterm
send -- "\033\[A"
-expect "SELECT 1"
+expect "SELECT 'unique1'"
send -- "\033\[B"
-expect "SELECT 2"
+expect "SELECT 'unique2'"
send -- "\r"
-expect "2"
+# First is echoed command
+expect "'unique2'"
+# Second is the response from the server
+expect "'unique2'"
+
send -- "exit\r"
expect eof
diff --git a/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect b/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect
index 41d32891e98..68544a4f014 100755
--- a/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect
+++ b/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect
@@ -11,9 +11,9 @@ set uuid ""
match_max 100000
expect_after {
# Do not ignore eof from read.
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=$history_file"
diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect
index f70b699c71f..0c31efc02e9 100755
--- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect
+++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect
@@ -9,15 +9,15 @@ set timeout 20
match_max 100000
expect_after {
- eof { exp_continue }
- timeout { exit 1 }
+ -i $any_spawn_id eof { exp_continue }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion"
expect ":) "
send -- "insert into table function null() format TSV some trash here 123 \n 456\r"
-expect -re ".*DB::Exception: Table function 'null' requires 'structure'.*\r"
+expect "EMPTY_LIST_OF_COLUMNS_PASSED"
expect ":) "
send -- ""
diff --git a/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect b/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect
index 07815e57610..6e545cf43cd 100755
--- a/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect
+++ b/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect
@@ -17,8 +17,8 @@ match_max 100000
set stty_init "rows 25 cols 120"
expect_after {
- eof { exp_continue }
- timeout { exit 1 }
+ -i $any_spawn_id eof { exp_continue }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash
diff --git a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect
index 3333ee93468..f144735c320 100755
--- a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect
+++ b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect
@@ -17,8 +17,8 @@ match_max 100000
set stty_init "rows 25 cols 120"
expect_after {
- eof { exp_continue }
- timeout { exit 1 }
+ -i $any_spawn_id eof { exp_continue }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash
diff --git a/tests/queries/0_stateless/02352_interactive_queries_from_file.expect b/tests/queries/0_stateless/02352_interactive_queries_from_file.expect
index a34fc9909f8..e28fb38862c 100755
--- a/tests/queries/0_stateless/02352_interactive_queries_from_file.expect
+++ b/tests/queries/0_stateless/02352_interactive_queries_from_file.expect
@@ -11,9 +11,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "echo 'select 1;\nselect 2;\nselect 3' > queries_02352"
diff --git a/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql b/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql
index cdbac93937e..86a8414e799 100644
--- a/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql
+++ b/tests/queries/0_stateless/02364_setting_cross_to_inner_rewrite.sql
@@ -1,5 +1,3 @@
-
-
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
diff --git a/tests/queries/0_stateless/02417_repeat_input_commands.expect b/tests/queries/0_stateless/02417_repeat_input_commands.expect
index 3658d5d8494..8070200c55c 100755
--- a/tests/queries/0_stateless/02417_repeat_input_commands.expect
+++ b/tests/queries/0_stateless/02417_repeat_input_commands.expect
@@ -11,9 +11,9 @@ match_max 100000
expect_after {
# Do not ignore eof from expect
- eof { exp_continue }
+ -i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
- timeout { exit 1 }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
diff --git a/tests/queries/0_stateless/02456_progress_tty.expect b/tests/queries/0_stateless/02456_progress_tty.expect
index ba6cc0537eb..3d1d92e5400 100755
--- a/tests/queries/0_stateless/02456_progress_tty.expect
+++ b/tests/queries/0_stateless/02456_progress_tty.expect
@@ -10,8 +10,8 @@ match_max 100000
set stty_init "rows 25 cols 120"
expect_after {
- eof { exp_continue }
- timeout { exit 1 }
+ -i $any_spawn_id eof { exp_continue }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash
diff --git a/tests/queries/0_stateless/02473_multistep_split_prewhere.python b/tests/queries/0_stateless/02473_multistep_split_prewhere.python
new file mode 100644
index 00000000000..41d8a746e11
--- /dev/null
+++ b/tests/queries/0_stateless/02473_multistep_split_prewhere.python
@@ -0,0 +1,131 @@
+#!/usr/bin/env python3
+import requests
+import os
+import sys
+
+CURDIR = os.path.dirname(os.path.realpath(__file__))
+sys.path.insert(0, os.path.join(CURDIR, 'helpers'))
+
+from pure_http_client import ClickHouseClient
+
+
+class Tester:
+ '''
+ - Creates test table with multiple integer columns
+ - Runs read queries with multiple range conditions on different columns in PREWHERE and check that the result is correct
+ '''
+ def __init__(self, session, url, index_granularity, total_rows):
+ self.session = session
+ self.url = url
+ self.index_granularity = index_granularity
+ self.total_rows = total_rows
+ self.reported_errors = set()
+ self.repro_queries = []
+
+ def report_error(self):
+ print('Repro steps:', '\n\n\t'.join(self.repro_queries))
+ exit(1)
+
+ def query(self, query_text, include_in_repro_steps = True, expected_data = None):
+ self.repro_queries.append(query_text)
+ resp = self.session.post(self.url, data=query_text)
+ if resp.status_code != 200:
+ # Group similar errors
+ error = resp.text[0:40]
+ if error not in self.reported_errors:
+ self.reported_errors.add(error)
+ print('Code:', resp.status_code)
+ print('Result:', resp.text)
+ self.report_error()
+
+ result = resp.text
+ # Check that the result is as expected
+ if ((not expected_data is None) and (int(result) != len(expected_data))):
+ print('Expected {} rows, got {}'.format(len(expected_data), result))
+ print('Expected data:' + str(expected_data))
+ self.report_error()
+
+ if not include_in_repro_steps:
+ self.repro_queries.pop()
+
+
+ def check_data(self, all_data, c_range_start, c_range_end, d_range_start, d_range_end):
+ for to_select in ['count()', 'sum(e)']: # Test reading with and without column with default value
+ self.query('SELECT {} FROM tab_02473;'.format(to_select), False, all_data)
+
+ delta = 10
+ for b_range_start in [0, delta]:
+ for b_range_end in [self.total_rows - delta]: #, self.total_rows]:
+ expected = all_data[
+ (all_data.a == 0) &
+ (all_data.b > b_range_start) &
+ (all_data.b <= b_range_end)]
+ self.query('SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} WHERE a == 0;'.format(
+ to_select, b_range_start, b_range_end), False, expected)
+
+ expected = all_data[
+ (all_data.a == 0) &
+ (all_data.b > b_range_start) &
+ (all_data.b <= b_range_end) &
+ (all_data.c > c_range_start) &
+ (all_data.c <= c_range_end)]
+ self.query('SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} AND c > {} AND c <= {} WHERE a == 0;'.format(
+ to_select, b_range_start, b_range_end, c_range_start, c_range_end), False, expected)
+
+ expected = all_data[
+ (all_data.a == 0) &
+ (all_data.b > b_range_start) &
+ (all_data.b <= b_range_end) &
+ (all_data.c > c_range_start) &
+ (all_data.c <= c_range_end) &
+ (all_data.d > d_range_start) &
+ (all_data.d <= d_range_end)]
+ self.query('SELECT {} from tab_02473 PREWHERE b > {} AND b <= {} AND c > {} AND c <= {} AND d > {} AND d <= {} WHERE a == 0;'.format(
+ to_select, b_range_start, b_range_end, c_range_start, c_range_end, d_range_start, d_range_end), False, expected)
+
+
+ def run_test(self, c_range_start, c_range_end, d_range_start, d_range_end):
+ self.repro_queries = []
+
+ self.query('''
+ CREATE TABLE tab_02473 (a Int8, b Int32, c Int32, d Int32, PRIMARY KEY (a))
+ ENGINE = MergeTree() ORDER BY (a, b)
+ SETTINGS min_bytes_for_wide_part = 0, index_granularity = {};'''.format(self.index_granularity))
+
+ self.query('INSERT INTO tab_02473 select 0, number+1, number+1, number+1 FROM numbers({});'.format(self.total_rows))
+
+ client = ClickHouseClient()
+ all_data = client.query_return_df("SELECT a, b, c, d, 1 as e FROM tab_02473 FORMAT TabSeparatedWithNames;")
+
+ self.query('OPTIMIZE TABLE tab_02473 FINAL SETTINGS mutations_sync=2;')
+
+ # After all data has been written add a column with default value
+ self.query('ALTER TABLE tab_02473 ADD COLUMN e Int64 DEFAULT 1;')
+
+ self.check_data(all_data, c_range_start, c_range_end, d_range_start, d_range_end)
+
+ self.query('DROP TABLE tab_02473;')
+
+
+
+def main():
+ # Enable multiple prewhere read steps
+ url = os.environ['CLICKHOUSE_URL'] + '&enable_multiple_prewhere_read_steps=1&move_all_conditions_to_prewhere=0&max_threads=1'
+
+ default_index_granularity = 10;
+ total_rows = 8 * default_index_granularity
+ step = default_index_granularity
+ session = requests.Session()
+ for index_granularity in [default_index_granularity-1, default_index_granularity]:
+ tester = Tester(session, url, index_granularity, total_rows)
+ # Test combinations of ranges of columns c and d
+ for c_range_start in range(0, total_rows, int(2.3 * step)):
+ for c_range_end in range(c_range_start + 3 * step, total_rows, int(2.1 * step)):
+ for d_range_start in range(int(0.5 * step), total_rows, int(2.7 * step)):
+ for d_range_end in range(d_range_start + 3 * step, total_rows, int(2.2 * step)):
+ tester.run_test(c_range_start, c_range_end, d_range_start, d_range_end)
+
+
+if __name__ == "__main__":
+ main()
+
diff --git a/tests/queries/0_stateless/02473_multistep_split_prewhere.reference b/tests/queries/0_stateless/02473_multistep_split_prewhere.reference
new file mode 100644
index 00000000000..e69de29bb2d
diff --git a/tests/queries/0_stateless/02473_multistep_split_prewhere.sh b/tests/queries/0_stateless/02473_multistep_split_prewhere.sh
new file mode 100755
index 00000000000..c5482c42028
--- /dev/null
+++ b/tests/queries/0_stateless/02473_multistep_split_prewhere.sh
@@ -0,0 +1,11 @@
+#!/usr/bin/env bash
+# Tags: long
+
+CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
+# shellcheck source=../shell_config.sh
+. "$CURDIR"/../shell_config.sh
+
+# We should have correct env vars from shell_config.sh to run this test
+
+python3 "$CURDIR"/02473_multistep_split_prewhere.python
+
diff --git a/tests/queries/0_stateless/02480_client_option_print_num_processed_rows.expect b/tests/queries/0_stateless/02480_client_option_print_num_processed_rows.expect
index 77e219e804e..dd3c9309b2d 100755
--- a/tests/queries/0_stateless/02480_client_option_print_num_processed_rows.expect
+++ b/tests/queries/0_stateless/02480_client_option_print_num_processed_rows.expect
@@ -10,8 +10,8 @@ match_max 100000
set stty_init "rows 25 cols 120"
expect_after {
- eof { exp_continue }
- timeout { exit 1 }
+ -i $any_spawn_id eof { exp_continue }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash
diff --git a/tests/queries/0_stateless/02493_inconsistent_hex_and_binary_number.expect b/tests/queries/0_stateless/02493_inconsistent_hex_and_binary_number.expect
index a391756ba22..2d595b0f492 100755
--- a/tests/queries/0_stateless/02493_inconsistent_hex_and_binary_number.expect
+++ b/tests/queries/0_stateless/02493_inconsistent_hex_and_binary_number.expect
@@ -10,8 +10,8 @@ match_max 100000
set stty_init "rows 25 cols 120"
expect_after {
- eof { exp_continue }
- timeout { exit 1 }
+ -i $any_spawn_id eof { exp_continue }
+ -i $any_spawn_id timeout { exit 1 }
}
spawn bash
diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns_2.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns_2.reference
new file mode 100644
index 00000000000..e69de29bb2d
diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns_2.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns_2.sql
new file mode 100644
index 00000000000..0791d3a539e
--- /dev/null
+++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns_2.sql
@@ -0,0 +1,26 @@
+DROP TABLE IF EXISTS t_02559;
+
+CREATE TABLE t_02559 (
+ key UInt64,
+ value Array(String))
+ENGINE = MergeTree
+ORDER BY key
+SETTINGS index_granularity=400, min_bytes_for_wide_part=0;
+
+INSERT INTO t_02559 SELECT number,
+if (number < 100 OR number > 1000,
+ [toString(number)],
+ emptyArrayString())
+ FROM numbers(2000);
+
+SET enable_multiple_prewhere_read_steps=1, move_all_conditions_to_prewhere=1;
+
+SELECT * FROM t_02559
+WHERE (key < 5 OR key > 500)
+ AND NOT has(value, toString(key))
+ AND length(value) == 1
+LIMIT 10
+SETTINGS max_block_size = 81,
+ max_threads = 1;
+
+DROP TABLE IF EXISTS t_02559;
diff --git a/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference
new file mode 100644
index 00000000000..e4d7ff55b86
--- /dev/null
+++ b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference
@@ -0,0 +1,205 @@
+5 6 5 6 5
+3 4 3 4 5
+3 4 3 4 7
+3 4 3 4 9
+5 6 5 6 5
+5 6 5 6 7
+5 6 5 6 9
+-- { echoOn }
+
+EXPLAIN QUERY TREE
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0) AND t2.a = t3.x AND 1;
+QUERY id: 0
+ PROJECTION COLUMNS
+ t1.a UInt64
+ t1.b UInt64
+ t2.a UInt64
+ t2.b UInt64
+ x UInt64
+ PROJECTION
+ LIST id: 1, nodes: 5
+ COLUMN id: 2, column_name: a, result_type: UInt64, source_id: 3
+ COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 3
+ COLUMN id: 5, column_name: a, result_type: UInt64, source_id: 6
+ COLUMN id: 7, column_name: b, result_type: UInt64, source_id: 6
+ COLUMN id: 8, column_name: x, result_type: UInt64, source_id: 9
+ JOIN TREE
+ JOIN id: 10, strictness: ALL, kind: INNER
+ LEFT TABLE EXPRESSION
+ JOIN id: 11, strictness: ALL, kind: INNER
+ LEFT TABLE EXPRESSION
+ TABLE id: 3, table_name: default.t1
+ RIGHT TABLE EXPRESSION
+ TABLE id: 6, table_name: default.t2
+ JOIN EXPRESSION
+ FUNCTION id: 12, function_name: equals, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 13, nodes: 2
+ COLUMN id: 14, column_name: a, result_type: UInt64, source_id: 3
+ FUNCTION id: 15, function_name: if, function_type: ordinary, result_type: UInt64
+ ARGUMENTS
+ LIST id: 16, nodes: 3
+ FUNCTION id: 17, function_name: greater, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 18, nodes: 2
+ COLUMN id: 19, column_name: b, result_type: UInt64, source_id: 6
+ CONSTANT id: 20, constant_value: UInt64_0, constant_value_type: UInt8
+ COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6
+ CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8
+ RIGHT TABLE EXPRESSION
+ QUERY id: 9, alias: t3, is_subquery: 1
+ PROJECTION COLUMNS
+ x UInt64
+ PROJECTION
+ LIST id: 23, nodes: 1
+ COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25
+ JOIN TREE
+ TABLE id: 25, table_name: default.t3
+ WHERE
+ FUNCTION id: 26, function_name: equals, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 27, nodes: 2
+ FUNCTION id: 28, function_name: plus, function_type: ordinary, result_type: UInt64
+ ARGUMENTS
+ LIST id: 29, nodes: 2
+ COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25
+ CONSTANT id: 30, constant_value: UInt64_1, constant_value_type: UInt8
+ COLUMN id: 31, column_name: b, result_type: UInt64, source_id: 25
+ JOIN EXPRESSION
+ FUNCTION id: 32, function_name: equals, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 33, nodes: 2
+ COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6
+ COLUMN id: 34, column_name: x, result_type: UInt64, source_id: 9
+ WHERE
+ CONSTANT id: 35, constant_value: UInt64_1, constant_value_type: UInt8
+EXPLAIN QUERY TREE
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0) AND t2.a = t3.x AND 1
+SETTINGS cross_to_inner_join_rewrite = 0;
+QUERY id: 0
+ PROJECTION COLUMNS
+ t1.a UInt64
+ t1.b UInt64
+ t2.a UInt64
+ t2.b UInt64
+ x UInt64
+ PROJECTION
+ LIST id: 1, nodes: 5
+ COLUMN id: 2, column_name: a, result_type: UInt64, source_id: 3
+ COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 3
+ COLUMN id: 5, column_name: a, result_type: UInt64, source_id: 6
+ COLUMN id: 7, column_name: b, result_type: UInt64, source_id: 6
+ COLUMN id: 8, column_name: x, result_type: UInt64, source_id: 9
+ JOIN TREE
+ JOIN id: 10, kind: COMMA
+ LEFT TABLE EXPRESSION
+ JOIN id: 11, kind: COMMA
+ LEFT TABLE EXPRESSION
+ TABLE id: 3, table_name: default.t1
+ RIGHT TABLE EXPRESSION
+ TABLE id: 6, table_name: default.t2
+ RIGHT TABLE EXPRESSION
+ QUERY id: 9, alias: t3, is_subquery: 1
+ PROJECTION COLUMNS
+ x UInt64
+ PROJECTION
+ LIST id: 12, nodes: 1
+ COLUMN id: 13, column_name: a, result_type: UInt64, source_id: 14
+ JOIN TREE
+ TABLE id: 14, table_name: default.t3
+ WHERE
+ FUNCTION id: 15, function_name: equals, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 16, nodes: 2
+ FUNCTION id: 17, function_name: plus, function_type: ordinary, result_type: UInt64
+ ARGUMENTS
+ LIST id: 18, nodes: 2
+ COLUMN id: 13, column_name: a, result_type: UInt64, source_id: 14
+ CONSTANT id: 19, constant_value: UInt64_1, constant_value_type: UInt8
+ COLUMN id: 20, column_name: b, result_type: UInt64, source_id: 14
+ WHERE
+ FUNCTION id: 21, function_name: and, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 22, nodes: 3
+ FUNCTION id: 23, function_name: equals, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 24, nodes: 2
+ COLUMN id: 25, column_name: a, result_type: UInt64, source_id: 3
+ FUNCTION id: 26, function_name: if, function_type: ordinary, result_type: UInt64
+ ARGUMENTS
+ LIST id: 27, nodes: 3
+ FUNCTION id: 28, function_name: greater, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 29, nodes: 2
+ COLUMN id: 30, column_name: b, result_type: UInt64, source_id: 6
+ CONSTANT id: 31, constant_value: UInt64_0, constant_value_type: UInt8
+ COLUMN id: 32, column_name: a, result_type: UInt64, source_id: 6
+ CONSTANT id: 33, constant_value: UInt64_0, constant_value_type: UInt8
+ FUNCTION id: 34, function_name: equals, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 35, nodes: 2
+ COLUMN id: 32, column_name: a, result_type: UInt64, source_id: 6
+ COLUMN id: 36, column_name: x, result_type: UInt64, source_id: 9
+ CONSTANT id: 37, constant_value: UInt64_1, constant_value_type: UInt8
+ SETTINGS cross_to_inner_join_rewrite=0
+EXPLAIN QUERY TREE
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0);
+QUERY id: 0
+ PROJECTION COLUMNS
+ t1.a UInt64
+ t1.b UInt64
+ t2.a UInt64
+ t2.b UInt64
+ x UInt64
+ PROJECTION
+ LIST id: 1, nodes: 5
+ COLUMN id: 2, column_name: a, result_type: UInt64, source_id: 3
+ COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 3
+ COLUMN id: 5, column_name: a, result_type: UInt64, source_id: 6
+ COLUMN id: 7, column_name: b, result_type: UInt64, source_id: 6
+ COLUMN id: 8, column_name: x, result_type: UInt64, source_id: 9
+ JOIN TREE
+ JOIN id: 10, kind: COMMA
+ LEFT TABLE EXPRESSION
+ JOIN id: 11, strictness: ALL, kind: INNER
+ LEFT TABLE EXPRESSION
+ TABLE id: 3, table_name: default.t1
+ RIGHT TABLE EXPRESSION
+ TABLE id: 6, table_name: default.t2
+ JOIN EXPRESSION
+ FUNCTION id: 12, function_name: equals, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 13, nodes: 2
+ COLUMN id: 14, column_name: a, result_type: UInt64, source_id: 3
+ FUNCTION id: 15, function_name: if, function_type: ordinary, result_type: UInt64
+ ARGUMENTS
+ LIST id: 16, nodes: 3
+ FUNCTION id: 17, function_name: greater, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 18, nodes: 2
+ COLUMN id: 19, column_name: b, result_type: UInt64, source_id: 6
+ CONSTANT id: 20, constant_value: UInt64_0, constant_value_type: UInt8
+ COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6
+ CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8
+ RIGHT TABLE EXPRESSION
+ QUERY id: 9, alias: t3, is_subquery: 1
+ PROJECTION COLUMNS
+ x UInt64
+ PROJECTION
+ LIST id: 23, nodes: 1
+ COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25
+ JOIN TREE
+ TABLE id: 25, table_name: default.t3
+ WHERE
+ FUNCTION id: 26, function_name: equals, function_type: ordinary, result_type: UInt8
+ ARGUMENTS
+ LIST id: 27, nodes: 2
+ FUNCTION id: 28, function_name: plus, function_type: ordinary, result_type: UInt64
+ ARGUMENTS
+ LIST id: 29, nodes: 2
+ COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25
+ CONSTANT id: 30, constant_value: UInt64_1, constant_value_type: UInt8
+ COLUMN id: 31, column_name: b, result_type: UInt64, source_id: 25
diff --git a/tests/queries/0_stateless/02564_analyzer_cross_to_inner.sql b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.sql
new file mode 100644
index 00000000000..a83cd238982
--- /dev/null
+++ b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.sql
@@ -0,0 +1,50 @@
+SET allow_experimental_analyzer = 1;
+
+DROP TABLE IF EXISTS t1;
+DROP TABLE IF EXISTS t2;
+DROP TABLE IF EXISTS t3;
+
+CREATE TABLE t1 (a UInt64, b UInt64) ENGINE = Memory;
+INSERT INTO t1 VALUES (1, 2), (3, 4), (5, 6);
+
+CREATE TABLE t2 (a UInt64, b UInt64) ENGINE = Memory;
+INSERT INTO t2 VALUES (3, 4), (5, 6), (7, 8);
+
+CREATE TABLE t3 (a UInt64, b UInt64) ENGINE = Memory;
+INSERT INTO t3 VALUES (5, 6), (7, 8), (9, 10);
+
+SET cross_to_inner_join_rewrite = 1;
+
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0) AND t2.a = t3.x AND 1
+;
+
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0)
+ORDER BY t1.a, t2.a, t3.x
+;
+
+-- { echoOn }
+
+EXPLAIN QUERY TREE
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0) AND t2.a = t3.x AND 1;
+
+EXPLAIN QUERY TREE
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0) AND t2.a = t3.x AND 1
+SETTINGS cross_to_inner_join_rewrite = 0;
+
+EXPLAIN QUERY TREE
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0);
+
+-- { echoOff }
+
+SELECT * FROM t1, t2, (SELECT a as x from t3 where a + 1 = b ) as t3
+WHERE t1.a = if(t2.b > 0, t2.a, 0)
+SETTINGS cross_to_inner_join_rewrite = 2; -- { serverError INCORRECT_QUERY }
+
+DROP TABLE IF EXISTS t1;
+DROP TABLE IF EXISTS t2;
+DROP TABLE IF EXISTS t3;
diff --git a/tests/queries/0_stateless/02564_query_id_header.reference b/tests/queries/0_stateless/02564_query_id_header.reference
new file mode 100644
index 00000000000..413e8929f36
--- /dev/null
+++ b/tests/queries/0_stateless/02564_query_id_header.reference
@@ -0,0 +1,22 @@
+CREATE TABLE t_query_id_header (a UInt64) ENGINE = Memory
+< Content-Type: text/plain; charset=UTF-8
+< X-ClickHouse-Query-Id: query_id
+< X-ClickHouse-Timezone: timezone
+INSERT INTO t_query_id_header VALUES (1)
+< Content-Type: text/plain; charset=UTF-8
+< X-ClickHouse-Query-Id: query_id
+< X-ClickHouse-Timezone: timezone
+EXISTS TABLE t_query_id_header
+< Content-Type: text/tab-separated-values; charset=UTF-8
+< X-ClickHouse-Format: TabSeparated
+< X-ClickHouse-Query-Id: query_id
+< X-ClickHouse-Timezone: timezone
+SELECT * FROM t_query_id_header
+< Content-Type: text/tab-separated-values; charset=UTF-8
+< X-ClickHouse-Format: TabSeparated
+< X-ClickHouse-Query-Id: query_id
+< X-ClickHouse-Timezone: timezone
+DROP TABLE t_query_id_header
+< Content-Type: text/plain; charset=UTF-8
+< X-ClickHouse-Query-Id: query_id
+< X-ClickHouse-Timezone: timezone
diff --git a/tests/queries/0_stateless/02564_query_id_header.sh b/tests/queries/0_stateless/02564_query_id_header.sh
new file mode 100755
index 00000000000..67ddbcfcc46
--- /dev/null
+++ b/tests/queries/0_stateless/02564_query_id_header.sh
@@ -0,0 +1,30 @@
+#!/usr/bin/env bash
+
+CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
+# shellcheck source=../shell_config.sh
+. "$CURDIR"/../shell_config.sh
+
+CLICKHOUSE_TIMEZONE_ESCAPED=$($CLICKHOUSE_CLIENT --query="SELECT timezone()" | sed 's/[]\/$*.^+:()[]/\\&/g')
+
+function run_and_check_headers()
+{
+ query=$1
+ query_id="${CLICKHOUSE_DATABASE}_${RANDOM}"
+
+ echo "$query"
+
+ ${CLICKHOUSE_CURL} -sS -v "${CLICKHOUSE_URL}&query_id=$query_id" -d "$1" 2>&1 \
+ | grep -e "< X-ClickHouse-Query-Id" -e "< X-ClickHouse-Timezone" -e "< X-ClickHouse-Format" -e "< Content-Type" \
+ | sed "s/$CLICKHOUSE_TIMEZONE_ESCAPED/timezone/" \
+ | sed "s/$query_id/query_id/" \
+ | sed "s/\r$//" \
+ | sort
+}
+
+${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_query_id_header"
+
+run_and_check_headers "CREATE TABLE t_query_id_header (a UInt64) ENGINE = Memory"
+run_and_check_headers "INSERT INTO t_query_id_header VALUES (1)"
+run_and_check_headers "EXISTS TABLE t_query_id_header"
+run_and_check_headers "SELECT * FROM t_query_id_header"
+run_and_check_headers "DROP TABLE t_query_id_header"
diff --git a/tests/queries/0_stateless/02669_local_desc_abort_on_twitter_json.reference b/tests/queries/0_stateless/02669_local_desc_abort_on_twitter_json.reference
new file mode 100644
index 00000000000..d00491fd7e5
--- /dev/null
+++ b/tests/queries/0_stateless/02669_local_desc_abort_on_twitter_json.reference
@@ -0,0 +1 @@
+1
diff --git a/tests/queries/0_stateless/02669_local_desc_abort_on_twitter_json.sh b/tests/queries/0_stateless/02669_local_desc_abort_on_twitter_json.sh
new file mode 100755
index 00000000000..e4f738f18ff
--- /dev/null
+++ b/tests/queries/0_stateless/02669_local_desc_abort_on_twitter_json.sh
@@ -0,0 +1,8 @@
+#!/usr/bin/env bash
+
+CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
+# shellcheck source=../shell_config.sh
+. "$CUR_DIR"/../shell_config.sh
+
+$CLICKHOUSE_LOCAL -q "desc file('$CUR_DIR/data_json/twitter.jsonl')" 2>&1 | grep -c "ONLY_NULLS_WHILE_READING_SCHEMA"
+
diff --git a/tests/queries/0_stateless/25338_analyzer_array_join_calc_twice.reference b/tests/queries/0_stateless/25338_analyzer_array_join_calc_twice.reference
new file mode 100644
index 00000000000..bc42121fb39
--- /dev/null
+++ b/tests/queries/0_stateless/25338_analyzer_array_join_calc_twice.reference
@@ -0,0 +1,6 @@
+2
+3
+4
+2
+3
+4
diff --git a/tests/queries/0_stateless/25338_analyzer_array_join_calc_twice.sql b/tests/queries/0_stateless/25338_analyzer_array_join_calc_twice.sql
new file mode 100644
index 00000000000..b6bb258db28
--- /dev/null
+++ b/tests/queries/0_stateless/25338_analyzer_array_join_calc_twice.sql
@@ -0,0 +1,5 @@
+SET allow_experimental_analyzer = 1;
+
+SELECT 1 + arrayJoin(a) AS m FROM (SELECT [1, 2, 3] AS a) GROUP BY m;
+
+SELECT 1 + arrayJoin(a) AS m FROM (SELECT [1, 2, 3] AS a) GROUP BY 1 + arrayJoin(a);
diff --git a/tests/queries/0_stateless/data_json/twitter.jsonl b/tests/queries/0_stateless/data_json/twitter.jsonl
new file mode 100644
index 00000000000..e498cb13e3f
Binary files /dev/null and b/tests/queries/0_stateless/data_json/twitter.jsonl differ
diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt
index 00477f0fb8f..ab03da8cc80 100644
--- a/utils/check-style/aspell-ignore/en/aspell-dict.txt
+++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt
@@ -24,6 +24,7 @@ CapnProto
CentOS
ClickHouse
ClickHouse's
+CodeBlock
Config
ConnectionDetails
Contrib
@@ -151,6 +152,7 @@ Submodules
Subqueries
TSVRaw
TSan
+TabItem
TabSeparated
TabSeparatedRaw
TabSeparatedRawWithNames
diff --git a/utils/check-style/check-style b/utils/check-style/check-style
index 8436d3378d9..53165d14f96 100755
--- a/utils/check-style/check-style
+++ b/utils/check-style/check-style
@@ -338,6 +338,13 @@ for test_case in "${expect_tests[@]}"; do
pattern="^spawn.*CLICKHOUSE_CLIENT_BINARY.*--history_file$"
grep -q "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'"
fi
+
+ # Otherwise expect_after/expect_before will not bail without stdin attached
+ # (and actually this is a hack anyway, correct way is to use $any_spawn_id)
+ pattern="-i \$any_spawn_id timeout"
+ grep -q -- "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'"
+ pattern="-i \$any_spawn_id eof"
+ grep -q -- "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'"
done
# Conflict markers