Merge branch 'master' into ineq_join_v2

This commit is contained in:
vdimir 2024-04-30 08:15:51 +00:00
commit 23525ea3d5
No known key found for this signature in database
GPG Key ID: 6EE4CE2BEDC51862
88 changed files with 4140 additions and 177 deletions

View File

@ -1,9 +1,10 @@
#include <Analyzer/Passes/QueryAnalysisPass.h>
#include <boost/algorithm/string.hpp>
#include <Common/checkStackSize.h>
#include <Common/NamePrompter.h>
#include <Common/ProfileEvents.h>
#include <Analyzer/FunctionSecretArgumentsFinderTreeNode.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
@ -81,8 +82,8 @@
#include <Analyzer/QueryTreeBuilder.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/Identifier.h>
#include <boost/algorithm/string.hpp>
#include <Analyzer/FunctionSecretArgumentsFinderTreeNode.h>
#include <Analyzer/RecursiveCTE.h>
namespace ProfileEvents
{
@ -740,7 +741,7 @@ struct IdentifierResolveScope
/// Identifier lookup to result
std::unordered_map<IdentifierLookup, IdentifierResolveState, IdentifierLookupHash> identifier_lookup_to_resolve_state;
/// Lambda argument can be expression like constant, column, or it can be function
/// Argument can be expression like constant, column, function or table expression
std::unordered_map<std::string, QueryTreeNodePtr> expression_argument_name_to_node;
/// Alias name to query expression node
@ -1464,7 +1465,8 @@ private:
/// Lambdas that are currently in resolve process
std::unordered_set<IQueryTreeNode *> lambdas_in_resolve_process;
std::unordered_set<std::string_view> cte_in_resolve_process;
/// CTEs that are currently in resolve process
std::unordered_set<std::string_view> ctes_in_resolve_process;
/// Function name to user defined lambda map
std::unordered_map<std::string, QueryTreeNodePtr> function_name_to_user_defined_lambda;
@ -2148,9 +2150,9 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
else
{
/** Make unique column names for tuple.
*
* Example: SELECT (SELECT 2 AS x, x)
*/
*
* Example: SELECT (SELECT 2 AS x, x)
*/
makeUniqueColumnNamesInBlock(block);
scalar_block.insert({
@ -3981,6 +3983,9 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const
auto * union_node = resolved_identifier->as<UnionNode>();
bool is_cte = (subquery_node && subquery_node->isCTE()) || (union_node && union_node->isCTE());
bool is_table_from_expression_arguments = lookup_result.resolve_place == IdentifierResolvePlace::EXPRESSION_ARGUMENTS &&
resolved_identifier->getNodeType() == QueryTreeNodeType::TABLE;
bool is_valid_table_expression = is_cte || is_table_from_expression_arguments;
/** From parent scopes we can resolve table identifiers only as CTE.
* Example: SELECT (SELECT 1 FROM a) FROM test_table AS a;
@ -3988,14 +3993,10 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const
* During child scope table identifier resolve a, table node test_table with alias a from parent scope
* is invalid.
*/
if (identifier_lookup.isTableExpressionLookup() && !is_cte)
if (identifier_lookup.isTableExpressionLookup() && !is_valid_table_expression)
continue;
if (is_cte)
{
return lookup_result;
}
else if (resolved_identifier->as<ConstantNode>())
if (is_valid_table_expression || resolved_identifier->as<ConstantNode>())
{
return lookup_result;
}
@ -4071,13 +4072,9 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook
if (it->second.resolve_result.isResolved() &&
scope.use_identifier_lookup_to_result_cache &&
!scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup))
{
if (!it->second.resolve_result.isResolvedFromCTEs() || !cte_in_resolve_process.contains(identifier_lookup.identifier.getFullName()))
{
return it->second.resolve_result;
}
}
!scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup) &&
(!it->second.resolve_result.isResolvedFromCTEs() || !ctes_in_resolve_process.contains(identifier_lookup.identifier.getFullName())))
return it->second.resolve_result;
}
else
{
@ -4150,7 +4147,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook
/// To accomplish this behaviour it's not allowed to resolve identifiers to
/// CTE that is being resolved.
if (cte_query_node_it != scope.cte_name_to_query_node.end()
&& !cte_in_resolve_process.contains(full_name))
&& !ctes_in_resolve_process.contains(full_name))
{
resolve_result.resolved_identifier = cte_query_node_it->second;
resolve_result.resolve_place = IdentifierResolvePlace::CTE;
@ -6296,14 +6293,14 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id
///
/// In this example argument of function `in` is being resolve here. If CTE `test1` is not forbidden,
/// `test1` is resolved to CTE (not to the table) in `initializeQueryJoinTreeNode` function.
cte_in_resolve_process.insert(cte_name);
ctes_in_resolve_process.insert(cte_name);
if (subquery_node)
resolveQuery(resolved_identifier_node, subquery_scope);
else
resolveUnion(resolved_identifier_node, subquery_scope);
cte_in_resolve_process.erase(cte_name);
ctes_in_resolve_process.erase(cte_name);
}
}
}
@ -7874,7 +7871,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
auto & query_node_typed = query_node->as<QueryNode &>();
if (query_node_typed.isCTE())
cte_in_resolve_process.insert(query_node_typed.getCTEName());
ctes_in_resolve_process.insert(query_node_typed.getCTEName());
bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube();
@ -7956,7 +7953,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
auto * union_node = node->as<UnionNode>();
bool subquery_is_cte = (subquery_node && subquery_node->isCTE()) || (union_node && union_node->isCTE());
if (!subquery_is_cte)
continue;
@ -8213,7 +8209,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
query_node_typed.resolveProjectionColumns(std::move(projection_columns));
if (query_node_typed.isCTE())
cte_in_resolve_process.erase(query_node_typed.getCTEName());
ctes_in_resolve_process.erase(query_node_typed.getCTEName());
}
void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope)
@ -8221,13 +8217,56 @@ void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, Identifier
auto & union_node_typed = union_node->as<UnionNode &>();
if (union_node_typed.isCTE())
cte_in_resolve_process.insert(union_node_typed.getCTEName());
ctes_in_resolve_process.insert(union_node_typed.getCTEName());
auto & queries_nodes = union_node_typed.getQueries().getNodes();
for (auto & query_node : queries_nodes)
std::optional<RecursiveCTETable> recursive_cte_table;
TableNodePtr recursive_cte_table_node;
if (union_node_typed.isCTE() && union_node_typed.isRecursiveCTE())
{
auto & non_recursive_query = queries_nodes[0];
bool non_recursive_query_is_query_node = non_recursive_query->getNodeType() == QueryTreeNodeType::QUERY;
auto & non_recursive_query_mutable_context = non_recursive_query_is_query_node ? non_recursive_query->as<QueryNode &>().getMutableContext()
: non_recursive_query->as<UnionNode &>().getMutableContext();
IdentifierResolveScope non_recursive_subquery_scope(non_recursive_query, &scope /*parent_scope*/);
non_recursive_subquery_scope.subquery_depth = scope.subquery_depth + 1;
if (non_recursive_query_is_query_node)
resolveQuery(non_recursive_query, non_recursive_subquery_scope);
else
resolveUnion(non_recursive_query, non_recursive_subquery_scope);
auto temporary_table_columns = non_recursive_query_is_query_node
? non_recursive_query->as<QueryNode &>().getProjectionColumns()
: non_recursive_query->as<UnionNode &>().computeProjectionColumns();
auto temporary_table_holder = std::make_shared<TemporaryTableHolder>(
non_recursive_query_mutable_context,
ColumnsDescription{NamesAndTypesList{temporary_table_columns.begin(), temporary_table_columns.end()}},
ConstraintsDescription{},
nullptr /*query*/,
true /*create_for_global_subquery*/);
auto temporary_table_storage = temporary_table_holder->getTable();
recursive_cte_table_node = std::make_shared<TableNode>(temporary_table_storage, non_recursive_query_mutable_context);
recursive_cte_table_node->setTemporaryTableName(union_node_typed.getCTEName());
recursive_cte_table.emplace(std::move(temporary_table_holder), std::move(temporary_table_storage), std::move(temporary_table_columns));
}
size_t queries_nodes_size = queries_nodes.size();
for (size_t i = recursive_cte_table.has_value(); i < queries_nodes_size; ++i)
{
auto & query_node = queries_nodes[i];
IdentifierResolveScope subquery_scope(query_node, &scope /*parent_scope*/);
if (recursive_cte_table_node)
subquery_scope.expression_argument_name_to_node[union_node_typed.getCTEName()] = recursive_cte_table_node;
auto query_node_type = query_node->getNodeType();
if (query_node_type == QueryTreeNodeType::QUERY)
@ -8247,8 +8286,19 @@ void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, Identifier
}
}
if (recursive_cte_table && isStorageUsedInTree(recursive_cte_table->storage, union_node.get()))
{
if (union_node_typed.getUnionMode() != SelectUnionMode::UNION_ALL)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Recursive CTE subquery {} with {} union mode is unsupported, only UNION ALL union mode is supported",
union_node_typed.formatASTForErrorMessage(),
toString(union_node_typed.getUnionMode()));
union_node_typed.setRecursiveCTETable(std::move(*recursive_cte_table));
}
if (union_node_typed.isCTE())
cte_in_resolve_process.erase(union_node_typed.getCTEName());
ctes_in_resolve_process.erase(union_node_typed.getCTEName());
}
}

View File

@ -14,12 +14,14 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTWithElement.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Analyzer/Utils.h>
#include <Analyzer/UnionNode.h>
namespace DB
{
@ -107,6 +109,9 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
if (is_cte)
buffer << ", is_cte: " << is_cte;
if (is_recursive_with)
buffer << ", is_recursive_with: " << is_recursive_with;
if (is_distinct)
buffer << ", is_distinct: " << is_distinct;
@ -259,6 +264,7 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs, CompareOptions) const
return is_subquery == rhs_typed.is_subquery &&
is_cte == rhs_typed.is_cte &&
is_recursive_with == rhs_typed.is_recursive_with &&
is_distinct == rhs_typed.is_distinct &&
is_limit_with_ties == rhs_typed.is_limit_with_ties &&
is_group_by_with_totals == rhs_typed.is_group_by_with_totals &&
@ -291,6 +297,7 @@ void QueryNode::updateTreeHashImpl(HashState & state, CompareOptions) const
state.update(projection_column_type_name);
}
state.update(is_recursive_with);
state.update(is_distinct);
state.update(is_limit_with_ties);
state.update(is_group_by_with_totals);
@ -317,19 +324,20 @@ QueryTreeNodePtr QueryNode::cloneImpl() const
{
auto result_query_node = std::make_shared<QueryNode>(context);
result_query_node->is_subquery = is_subquery;
result_query_node->is_cte = is_cte;
result_query_node->is_distinct = is_distinct;
result_query_node->is_limit_with_ties = is_limit_with_ties;
result_query_node->is_group_by_with_totals = is_group_by_with_totals;
result_query_node->is_group_by_with_rollup = is_group_by_with_rollup;
result_query_node->is_group_by_with_cube = is_group_by_with_cube;
result_query_node->is_subquery = is_subquery;
result_query_node->is_cte = is_cte;
result_query_node->is_recursive_with = is_recursive_with;
result_query_node->is_distinct = is_distinct;
result_query_node->is_limit_with_ties = is_limit_with_ties;
result_query_node->is_group_by_with_totals = is_group_by_with_totals;
result_query_node->is_group_by_with_rollup = is_group_by_with_rollup;
result_query_node->is_group_by_with_cube = is_group_by_with_cube;
result_query_node->is_group_by_with_grouping_sets = is_group_by_with_grouping_sets;
result_query_node->is_group_by_all = is_group_by_all;
result_query_node->is_order_by_all = is_order_by_all;
result_query_node->cte_name = cte_name;
result_query_node->projection_columns = projection_columns;
result_query_node->settings_changes = settings_changes;
result_query_node->is_group_by_all = is_group_by_all;
result_query_node->is_order_by_all = is_order_by_all;
result_query_node->cte_name = cte_name;
result_query_node->projection_columns = projection_columns;
result_query_node->settings_changes = settings_changes;
return result_query_node;
}
@ -337,6 +345,7 @@ QueryTreeNodePtr QueryNode::cloneImpl() const
ASTPtr QueryNode::toASTImpl(const ConvertToASTOptions & options) const
{
auto select_query = std::make_shared<ASTSelectQuery>();
select_query->recursive_with = is_recursive_with;
select_query->distinct = is_distinct;
select_query->limit_with_ties = is_limit_with_ties;
select_query->group_by_with_totals = is_group_by_with_totals;
@ -347,7 +356,41 @@ ASTPtr QueryNode::toASTImpl(const ConvertToASTOptions & options) const
select_query->order_by_all = is_order_by_all;
if (hasWith())
select_query->setExpression(ASTSelectQuery::Expression::WITH, getWith().toAST(options));
{
const auto & with = getWith();
auto expression_list_ast = std::make_shared<ASTExpressionList>();
expression_list_ast->children.reserve(with.getNodes().size());
for (const auto & with_node : with)
{
auto with_node_ast = with_node->toAST(options);
expression_list_ast->children.push_back(with_node_ast);
const auto * with_query_node = with_node->as<QueryNode>();
const auto * with_union_node = with_node->as<UnionNode>();
if (!with_query_node && !with_union_node)
continue;
bool is_with_node_cte = with_query_node ? with_query_node->isCTE() : with_union_node->isCTE();
if (!is_with_node_cte)
continue;
const auto & with_node_cte_name = with_query_node ? with_query_node->cte_name : with_union_node->getCTEName();
auto * with_node_ast_subquery = with_node_ast->as<ASTSubquery>();
if (with_node_ast_subquery)
with_node_ast_subquery->cte_name = "";
auto with_element_ast = std::make_shared<ASTWithElement>();
with_element_ast->name = with_node_cte_name;
with_element_ast->subquery = std::move(with_node_ast);
with_element_ast->children.push_back(with_element_ast->subquery);
expression_list_ast->children.back() = std::move(with_element_ast);
}
select_query->setExpression(ASTSelectQuery::Expression::WITH, std::move(expression_list_ast));
}
auto projection_ast = getProjection().toAST(options);
auto & projection_expression_list_ast = projection_ast->as<ASTExpressionList &>();

View File

@ -140,6 +140,18 @@ public:
cte_name = std::move(cte_name_value);
}
/// Returns true if query node has RECURSIVE WITH, false otherwise
bool isRecursiveWith() const
{
return is_recursive_with;
}
/// Set query node RECURSIVE WITH value
void setIsRecursiveWith(bool is_recursive_with_value)
{
is_recursive_with = is_recursive_with_value;
}
/// Returns true if query node has DISTINCT, false otherwise
bool isDistinct() const
{
@ -618,6 +630,7 @@ protected:
private:
bool is_subquery = false;
bool is_cte = false;
bool is_recursive_with = false;
bool is_distinct = false;
bool is_limit_with_ties = false;
bool is_group_by_with_totals = false;

View File

@ -271,6 +271,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q
current_query_tree->setIsSubquery(is_subquery);
current_query_tree->setIsCTE(!cte_name.empty());
current_query_tree->setCTEName(cte_name);
current_query_tree->setIsRecursiveWith(select_query_typed.recursive_with);
current_query_tree->setIsDistinct(select_query_typed.distinct);
current_query_tree->setIsLimitWithTies(select_query_typed.limit_with_ties);
current_query_tree->setIsGroupByWithTotals(select_query_typed.group_by_with_totals);
@ -287,8 +288,22 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q
auto select_with_list = select_query_typed.with();
if (select_with_list)
{
current_query_tree->getWithNode() = buildExpressionList(select_with_list, current_context);
if (select_query_typed.recursive_with)
{
for (auto & with_node : current_query_tree->getWith().getNodes())
{
auto * with_union_node = with_node->as<UnionNode>();
if (!with_union_node)
continue;
with_union_node->setIsRecursiveCTE(true);
}
}
}
auto select_expression_list = select_query_typed.select();
if (select_expression_list)
current_query_tree->getProjectionNode() = buildExpressionList(select_expression_list, current_context);

View File

@ -0,0 +1,21 @@
#include <Analyzer/RecursiveCTE.h>
#include <Storages/IStorage.h>
namespace DB
{
RecursiveCTETable::RecursiveCTETable(TemporaryTableHolderPtr holder_,
StoragePtr storage_,
NamesAndTypes columns_)
: holder(std::move(holder_))
, storage(std::move(storage_))
, columns(std::move(columns_))
{}
StorageID RecursiveCTETable::getStorageID() const
{
return storage->getStorageID();
}
}

View File

@ -0,0 +1,51 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/TableNode.h>
namespace DB
{
/** Recursive CTEs allow to recursively evaluate UNION subqueries.
*
* Overview:
* https://www.postgresql.org/docs/current/queries-with.html#QUERIES-WITH-RECURSIVE
*
* Current implementation algorithm:
*
* During query analysis, when we resolve UNION node that is inside WITH RECURSIVE section of parent query we:
* 1. First resolve non recursive subquery.
* 2. Create temporary table using projection columns of resolved subquery from step 1.
* 3. Create temporary table expression node using storage from step 2.
* 4. Create resolution scope for recursive subquery. In that scope we add node from step 3 as expression argument with UNION node CTE name.
* 5. Resolve recursive subquery.
* 6. If in resolved UNION node temporary table expression storage from step 2 is used, we update UNION query with recursive CTE table.
*
* During query planning if UNION node contains recursive CTE table, we add ReadFromRecursiveCTEStep to query plan. That step is responsible for whole
* recursive CTE query execution.
*
* TODO: Improve locking in ReadFromRecursiveCTEStep.
* TODO: Improve query analysis if query contains aggregates, JOINS, GROUP BY, ORDER BY, LIMIT, OFFSET.
* TODO: Support SEARCH DEPTH FIRST BY, SEARCH BREADTH FIRST BY syntax.
* TODO: Support CYCLE syntax.
* TODO: Support UNION DISTINCT recursive CTE mode.
*/
class RecursiveCTETable
{
public:
RecursiveCTETable(TemporaryTableHolderPtr holder_,
StoragePtr storage_,
NamesAndTypes columns_);
StorageID getStorageID() const;
TemporaryTableHolderPtr holder;
StoragePtr storage;
NamesAndTypes columns;
};
}

View File

@ -33,6 +33,14 @@ TableNode::TableNode(StoragePtr storage_, const ContextPtr & context)
{
}
void TableNode::updateStorage(StoragePtr storage_value, const ContextPtr & context)
{
storage = std::move(storage_value);
storage_id = storage->getStorageID();
storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context);
}
void TableNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const
{
buffer << std::string(indent, ' ') << "TABLE id: " << format_state.getNodeId(this);

View File

@ -32,6 +32,11 @@ public:
/// Construct table node with storage, context
explicit TableNode(StoragePtr storage_, const ContextPtr & context);
/** Update table node storage.
* After this call storage, storage_id, storage_lock, storage_snapshot will be updated using new storage.
*/
void updateStorage(StoragePtr storage_value, const ContextPtr & context);
/// Get storage
const StoragePtr & getStorage() const
{

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTWithElement.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
@ -20,6 +21,8 @@
#include <DataTypes/getLeastSupertype.h>
#include <Storages/IStorage.h>
#include <Interpreters/Context.h>
#include <Analyzer/QueryNode.h>
@ -49,6 +52,9 @@ UnionNode::UnionNode(ContextMutablePtr context_, SelectUnionMode union_mode_)
NamesAndTypes UnionNode::computeProjectionColumns() const
{
if (recursive_cte_table)
return recursive_cte_table->columns;
std::vector<NamesAndTypes> projections;
NamesAndTypes query_node_projection;
@ -90,6 +96,9 @@ NamesAndTypes UnionNode::computeProjectionColumns() const
void UnionNode::removeUnusedProjectionColumns(const std::unordered_set<std::string> & used_projection_columns)
{
if (recursive_cte_table)
return;
auto projection_columns = computeProjectionColumns();
size_t projection_columns_size = projection_columns.size();
std::unordered_set<size_t> used_projection_column_indexes;
@ -113,6 +122,9 @@ void UnionNode::removeUnusedProjectionColumns(const std::unordered_set<std::stri
void UnionNode::removeUnusedProjectionColumns(const std::unordered_set<size_t> & used_projection_columns_indexes)
{
if (recursive_cte_table)
return;
auto & query_nodes = getQueries().getNodes();
for (auto & query_node : query_nodes)
{
@ -136,6 +148,12 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
if (is_cte)
buffer << ", is_cte: " << is_cte;
if (is_recursive_cte)
buffer << ", is_recursive_cte: " << is_recursive_cte;
if (recursive_cte_table)
buffer << ", recursive_cte_table: " << recursive_cte_table->storage->getStorageID().getNameForLogs();
if (!cte_name.empty())
buffer << ", cte_name: " << cte_name;
@ -149,14 +167,28 @@ bool UnionNode::isEqualImpl(const IQueryTreeNode & rhs, CompareOptions) const
{
const auto & rhs_typed = assert_cast<const UnionNode &>(rhs);
return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name &&
union_mode == rhs_typed.union_mode;
if (recursive_cte_table && rhs_typed.recursive_cte_table &&
recursive_cte_table->getStorageID() != rhs_typed.recursive_cte_table->getStorageID())
return false;
else if ((recursive_cte_table && !rhs_typed.recursive_cte_table) || (!recursive_cte_table && rhs_typed.recursive_cte_table))
return false;
return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && is_recursive_cte == rhs_typed.is_recursive_cte
&& cte_name == rhs_typed.cte_name && union_mode == rhs_typed.union_mode;
}
void UnionNode::updateTreeHashImpl(HashState & state, CompareOptions) const
{
state.update(is_subquery);
state.update(is_cte);
state.update(is_recursive_cte);
if (recursive_cte_table)
{
auto full_name = recursive_cte_table->getStorageID().getFullNameNotQuoted();
state.update(full_name.size());
state.update(full_name);
}
state.update(cte_name.size());
state.update(cte_name);
@ -170,6 +202,8 @@ QueryTreeNodePtr UnionNode::cloneImpl() const
result_union_node->is_subquery = is_subquery;
result_union_node->is_cte = is_cte;
result_union_node->is_recursive_cte = is_recursive_cte;
result_union_node->recursive_cte_table = recursive_cte_table;
result_union_node->cte_name = cte_name;
return result_union_node;
@ -183,14 +217,64 @@ ASTPtr UnionNode::toASTImpl(const ConvertToASTOptions & options) const
select_with_union_query->children.push_back(getQueriesNode()->toAST(options));
select_with_union_query->list_of_selects = select_with_union_query->children.back();
if (is_subquery)
ASTPtr result_query = std::move(select_with_union_query);
bool set_subquery_cte_name = true;
if (recursive_cte_table)
{
auto subquery = std::make_shared<ASTSubquery>(std::move(select_with_union_query));
subquery->cte_name = cte_name;
return subquery;
auto recursive_select_query = std::make_shared<ASTSelectQuery>();
recursive_select_query->recursive_with = true;
auto with_element_ast = std::make_shared<ASTWithElement>();
with_element_ast->name = cte_name;
with_element_ast->subquery = std::make_shared<ASTSubquery>(std::move(result_query));
with_element_ast->children.push_back(with_element_ast->subquery);
auto with_expression_list_ast = std::make_shared<ASTExpressionList>();
with_expression_list_ast->children.push_back(std::move(with_element_ast));
recursive_select_query->setExpression(ASTSelectQuery::Expression::WITH, std::move(with_expression_list_ast));
auto select_expression_list_ast = std::make_shared<ASTExpressionList>();
select_expression_list_ast->children.reserve(recursive_cte_table->columns.size());
for (const auto & recursive_cte_table_column : recursive_cte_table->columns)
select_expression_list_ast->children.push_back(std::make_shared<ASTIdentifier>(recursive_cte_table_column.name));
recursive_select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list_ast));
auto table_expression_ast = std::make_shared<ASTTableExpression>();
table_expression_ast->children.push_back(std::make_shared<ASTTableIdentifier>(cte_name));
table_expression_ast->database_and_table_name = table_expression_ast->children.back();
auto tables_in_select_query_element_ast = std::make_shared<ASTTablesInSelectQueryElement>();
tables_in_select_query_element_ast->children.push_back(std::move(table_expression_ast));
tables_in_select_query_element_ast->table_expression = tables_in_select_query_element_ast->children.back();
ASTPtr tables_in_select_query_ast = std::make_shared<ASTTablesInSelectQuery>();
tables_in_select_query_ast->children.push_back(std::move(tables_in_select_query_element_ast));
recursive_select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast));
auto recursive_select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
auto recursive_select_with_union_query_list_of_selects = std::make_shared<ASTExpressionList>();
recursive_select_with_union_query_list_of_selects->children.push_back(std::move(recursive_select_query));
recursive_select_with_union_query->children.push_back(std::move(recursive_select_with_union_query_list_of_selects));
recursive_select_with_union_query->list_of_selects = recursive_select_with_union_query->children.back();
result_query = std::move(recursive_select_with_union_query);
set_subquery_cte_name = false;
}
return select_with_union_query;
if (is_subquery)
{
auto subquery = std::make_shared<ASTSubquery>(std::move(result_query));
if (set_subquery_cte_name)
subquery->cte_name = cte_name;
result_query = std::move(subquery);
}
return result_query;
}
}

View File

@ -9,6 +9,7 @@
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ListNode.h>
#include <Analyzer/TableExpressionModifiers.h>
#include <Analyzer/RecursiveCTE.h>
#include <Interpreters/Context_fwd.h>
@ -84,6 +85,42 @@ public:
is_cte = is_cte_value;
}
/// Returns true if union node CTE is specified in WITH RECURSIVE, false otherwise
bool isRecursiveCTE() const
{
return is_recursive_cte;
}
/// Set union node is recursive CTE value
void setIsRecursiveCTE(bool is_recursive_cte_value)
{
is_recursive_cte = is_recursive_cte_value;
}
/// Returns true if union node has recursive CTE table, false otherwise
bool hasRecursiveCTETable() const
{
return recursive_cte_table.has_value();
}
/// Returns optional recursive CTE table
const std::optional<RecursiveCTETable> & getRecursiveCTETable() const
{
return recursive_cte_table;
}
/// Returns optional recursive CTE table
std::optional<RecursiveCTETable> & getRecursiveCTETable()
{
return recursive_cte_table;
}
/// Set union node recursive CTE table value
void setRecursiveCTETable(RecursiveCTETable recursive_cte_table_value)
{
recursive_cte_table.emplace(std::move(recursive_cte_table_value));
}
/// Get union node CTE name
const std::string & getCTEName() const
{
@ -154,6 +191,8 @@ protected:
private:
bool is_subquery = false;
bool is_cte = false;
bool is_recursive_cte = false;
std::optional<RecursiveCTETable> recursive_cte_table;
std::string cte_name;
ContextMutablePtr context;
SelectUnionMode union_mode;

View File

@ -5,6 +5,7 @@
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTFunction.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
@ -15,6 +16,8 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <Storages/IStorage.h>
#include <Interpreters/Context.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
@ -61,6 +64,36 @@ bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root)
return false;
}
bool isStorageUsedInTree(const StoragePtr & storage, const IQueryTreeNode * root)
{
std::vector<const IQueryTreeNode *> nodes_to_process;
nodes_to_process.push_back(root);
while (!nodes_to_process.empty())
{
const auto * subtree_node = nodes_to_process.back();
nodes_to_process.pop_back();
const auto * table_node = subtree_node->as<TableNode>();
const auto * table_function_node = subtree_node->as<TableFunctionNode>();
if (table_node || table_function_node)
{
const auto & table_storage = table_node ? table_node->getStorage() : table_function_node->getStorage();
if (table_storage->getStorageID() == storage->getStorageID())
return true;
}
for (const auto & child : subtree_node->getChildren())
{
if (child)
nodes_to_process.push_back(child.get());
}
}
return false;
}
bool isNameOfInFunction(const std::string & function_name)
{
bool is_special_function_in = function_name == "in" ||
@ -808,26 +841,87 @@ QueryTreeNodePtr getExpressionSource(const QueryTreeNodePtr & node)
return source;
}
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(QueryTreeNodePtr table_node, const ContextPtr & context)
/** There are no limits on the maximum size of the result for the subquery.
* Since the result of the query is not the result of the entire query.
*/
void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context)
{
/** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result.
* Because the result of this query is not the result of the entire query.
* Constraints work instead
* max_rows_in_set, max_bytes_in_set, set_overflow_mode,
* max_rows_in_join, max_bytes_in_join, join_overflow_mode,
* which are checked separately (in the Set, Join objects).
*/
Settings subquery_settings = mutable_context->getSettings();
subquery_settings.max_result_rows = 0;
subquery_settings.max_result_bytes = 0;
/// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query).
subquery_settings.extremes = false;
mutable_context->setSettings(subquery_settings);
}
QueryTreeNodePtr buildQueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
ContextMutablePtr & context)
{
auto projection_columns = columns;
QueryTreeNodes subquery_projection_nodes;
subquery_projection_nodes.reserve(projection_columns.size());
for (const auto & column : projection_columns)
subquery_projection_nodes.push_back(std::make_shared<ColumnNode>(column, table_expression));
if (subquery_projection_nodes.empty())
{
auto constant_data_type = std::make_shared<DataTypeUInt64>();
subquery_projection_nodes.push_back(std::make_shared<ConstantNode>(1UL, constant_data_type));
projection_columns.push_back({"1", std::move(constant_data_type)});
}
updateContextForSubqueryExecution(context);
auto query_node = std::make_shared<QueryNode>(std::move(context));
query_node->getProjection().getNodes() = std::move(subquery_projection_nodes);
query_node->resolveProjectionColumns(projection_columns);
query_node->getJoinTree() = table_expression;
return query_node;
}
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
ContextMutablePtr & context)
{
auto result = buildQueryToReadColumnsFromTableExpression(columns, table_expression, context);
result->as<QueryNode &>().setIsSubquery(true);
return result;
}
QueryTreeNodePtr buildQueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
const ContextPtr & context)
{
auto context_copy = Context::createCopy(context);
return buildQueryToReadColumnsFromTableExpression(columns, table_expression, context_copy);
}
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
const ContextPtr & context)
{
auto context_copy = Context::createCopy(context);
return buildSubqueryToReadColumnsFromTableExpression(columns, table_expression, context_copy);
}
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const QueryTreeNodePtr & table_node, const ContextPtr & context)
{
const auto & storage_snapshot = table_node->as<TableNode>()->getStorageSnapshot();
auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary));
size_t columns_to_select_size = columns_to_select.size();
auto column_nodes_to_select = std::make_shared<ListNode>();
column_nodes_to_select->getNodes().reserve(columns_to_select_size);
NamesAndTypes projection_columns;
projection_columns.reserve(columns_to_select_size);
for (auto & column : columns_to_select)
{
column_nodes_to_select->getNodes().emplace_back(std::make_shared<ColumnNode>(column, table_node));
projection_columns.emplace_back(column.name, column.type);
}
auto subquery_for_table = std::make_shared<QueryNode>(Context::createCopy(context));
subquery_for_table->setIsSubquery(true);
subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select);
subquery_for_table->getJoinTree() = std::move(table_node);
subquery_for_table->resolveProjectionColumns(std::move(projection_columns));
return subquery_for_table;
auto columns_to_select_list = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary));
NamesAndTypes columns_to_select(columns_to_select_list.begin(), columns_to_select_list.end());
return buildSubqueryToReadColumnsFromTableExpression(columns_to_select, table_node, context);
}
}

View File

@ -1,9 +1,13 @@
#pragma once
#include <Analyzer/IQueryTreeNode.h>
#include <Core/NamesAndTypes.h>
#include <Storages/IStorage_fwd.h>
#include <Interpreters/Context_fwd.h>
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
@ -12,6 +16,9 @@ class FunctionNode;
/// Returns true if node part of root tree, false otherwise
bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root);
/// Returns true if storage is used in tree, false otherwise
bool isStorageUsedInTree(const StoragePtr & storage, const IQueryTreeNode * root);
/// Returns true if function name is name of IN function or its variations, false otherwise
bool isNameOfInFunction(const std::string & function_name);
@ -108,7 +115,41 @@ QueryTreeNodePtr createCastFunction(QueryTreeNodePtr node, DataTypePtr result_ty
/// Checks that node has only one source and returns it
QueryTreeNodePtr getExpressionSource(const QueryTreeNodePtr & node);
/// Build subquery which we execute for `IN table` function.
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(QueryTreeNodePtr table_node, const ContextPtr & context);
/// Update mutable context for subquery execution
void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context);
/** Build query to read specified columns from table expression.
* Specified mutable context will be used as query context.
*/
QueryTreeNodePtr buildQueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
ContextMutablePtr & context);
/** Build subquery to read specified columns from table expression.
* Specified mutable context will be used as query context.
*/
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
ContextMutablePtr & context);
/** Build query to read specified columns from table expression.
* Specified context will be copied and used as query context.
*/
QueryTreeNodePtr buildQueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
const ContextPtr & context);
/** Build subquery to read specified columns from table expression.
* Specified context will be copied and used as query context.
*/
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
const ContextPtr & context);
/** Build subquery to read all columns from table expression.
* Specified context will be copied and used as query context.
*/
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const QueryTreeNodePtr & table_node, const ContextPtr & context);
}

View File

@ -39,6 +39,7 @@ static struct InitFiu
REGULAR(replicated_merge_tree_commit_zk_fail_when_recovering_from_hw_fault) \
REGULAR(use_delayed_remote_source) \
REGULAR(cluster_discovery_faults) \
REGULAR(replicated_sends_failpoint) \
ONCE(smt_commit_merge_mutate_zk_fail_after_op) \
ONCE(smt_commit_merge_mutate_zk_fail_before_op) \
ONCE(smt_commit_write_zk_fail_after_op) \

View File

@ -67,6 +67,9 @@ static constexpr auto DBMS_DEFAULT_MAX_PARSER_DEPTH = 1000;
/// Default limit on the amount of backtracking of recursive descend parser.
static constexpr auto DBMS_DEFAULT_MAX_PARSER_BACKTRACKS = 1000000;
/// Default limit on recursive CTE evaluation depth.
static constexpr auto DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH = 1000;
/// Default limit on query size.
static constexpr auto DBMS_DEFAULT_MAX_QUERY_SIZE = 262144;

View File

@ -623,6 +623,7 @@ class IColumn;
M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
M(UInt64, max_parser_backtracks, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS, "Maximum parser backtracking (how many times it tries different alternatives in the recursive descend parsing process).", 0) \
M(UInt64, max_recursive_cte_evaluation_depth, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth", 0) \
M(Bool, allow_settings_after_format_in_insert, false, "Allow SETTINGS after FORMAT, but note, that this is not always safe (note: this is a compatibility setting).", 0) \
M(Seconds, periodic_live_view_refresh, 60, "Interval after which periodically refreshed live view is forced to refresh.", 0) \
M(Bool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
@ -739,6 +740,7 @@ class IColumn;
M(Bool, query_plan_split_filter, true, "Allow to split filters in the query plan", 0) \
M(Bool, query_plan_merge_expressions, true, "Allow to merge expressions in the query plan", 0) \
M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \
M(Bool, query_plan_convert_outer_join_to_inner_join, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values", 0) \
M(Bool, query_plan_optimize_prewhere, true, "Allow to push down filter to PREWHERE expression for supported storages", 0) \
M(Bool, query_plan_execute_functions_after_sorting, true, "Allow to re-order functions after sorting", 0) \
M(Bool, query_plan_reuse_storage_ordering_for_window_functions, true, "Allow to use the storage sorting for window functions", 0) \

View File

@ -96,6 +96,8 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"},
{"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."},
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},
{"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"},
{"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"},
}},
{"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"},
{"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"},

View File

@ -109,4 +109,9 @@ void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, co
copyDataImpl(from, to, true, bytes, &is_cancelled, throttler);
}
void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, std::function<void()> cancellation_hook, ThrottlerPtr throttler)
{
copyDataImpl(from, to, false, std::numeric_limits<size_t>::max(), cancellation_hook, throttler);
}
}

View File

@ -33,5 +33,6 @@ void copyDataMaxBytes(ReadBuffer & from, WriteBuffer & to, size_t max_bytes);
/// Same as above but also use throttler to limit maximum speed
void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, const std::atomic<int> & is_cancelled, ThrottlerPtr throttler);
void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic<int> & is_cancelled, ThrottlerPtr throttler);
void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, std::function<void()> cancellation_hook, ThrottlerPtr throttler);
}

View File

@ -2013,6 +2013,63 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameS
return res;
}
bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header)
{
const auto * filter_node = tryFindInOutputs(filter_name);
if (!filter_node)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Outputs for ActionsDAG does not contain filter column name {}. DAG:\n{}",
filter_name,
dumpDAG());
std::unordered_map<std::string, ColumnWithTypeAndName> input_node_name_to_default_input_column;
for (const auto * input : inputs)
{
if (!input_stream_header.has(input->result_name))
continue;
if (input->column)
continue;
auto constant_column = input->result_type->createColumnConst(0, input->result_type->getDefault());
auto constant_column_with_type_and_name = ColumnWithTypeAndName{constant_column, input->result_type, input->result_name};
input_node_name_to_default_input_column.emplace(input->result_name, std::move(constant_column_with_type_and_name));
}
ActionsDAGPtr filter_with_default_value_inputs;
try
{
filter_with_default_value_inputs = buildFilterActionsDAG({filter_node}, input_node_name_to_default_input_column);
}
catch (const Exception &)
{
/** It is possible that duing DAG construction, some functions cannot be executed for constant default value inputs
* and exception will be thrown.
*/
return false;
}
const auto * filter_with_default_value_inputs_filter_node = filter_with_default_value_inputs->getOutputs()[0];
if (!filter_with_default_value_inputs_filter_node->column || !isColumnConst(*filter_with_default_value_inputs_filter_node->column))
return false;
const auto & constant_type = filter_with_default_value_inputs_filter_node->result_type;
auto which_constant_type = WhichDataType(constant_type);
if (!which_constant_type.isUInt8() && !which_constant_type.isNothing())
return false;
Field value;
filter_with_default_value_inputs_filter_node->column->get(0, value);
if (value.isNull())
return true;
UInt8 predicate_value = value.safeGet<UInt8>();
return predicate_value == 0;
}
ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & column_name) const
{
const auto * node = tryFindInOutputs(column_name);

View File

@ -355,6 +355,13 @@ public:
/// The second contains the rest.
SplitResult splitActionsBySortingDescription(const NameSet & sort_columns) const;
/** Returns true if filter DAG is always false for inputs with default values.
*
* @param filter_name - name of filter node in current DAG.
* @param input_stream_header - input stream header.
*/
bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header);
/// Create actions which may calculate part of filter using only available_inputs.
/// If nothing may be calculated, returns nullptr.
/// Otherwise, return actions which inputs are from available_inputs.

View File

@ -1618,6 +1618,33 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder &
external_tables_mapping.emplace(table_name, std::make_shared<TemporaryTableHolder>(std::move(temporary_table)));
}
void Context::updateExternalTable(const String & table_name, TemporaryTableHolder && temporary_table)
{
if (isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables");
auto temporary_table_ptr = std::make_shared<TemporaryTableHolder>(std::move(temporary_table));
std::lock_guard lock(mutex);
auto it = external_tables_mapping.find(table_name);
if (it == external_tables_mapping.end())
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary table {} does not exists", backQuoteIfNeed(table_name));
it->second = std::move(temporary_table_ptr);
}
void Context::addOrUpdateExternalTable(const String & table_name, TemporaryTableHolder && temporary_table)
{
if (isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables");
auto temporary_table_ptr = std::make_shared<TemporaryTableHolder>(std::move(temporary_table));
std::lock_guard lock(mutex);
auto [it, inserted] = external_tables_mapping.emplace(table_name, temporary_table_ptr);
if (!inserted)
it->second = std::move(temporary_table_ptr);
}
std::shared_ptr<TemporaryTableHolder> Context::findExternalTable(const String & table_name) const
{
if (isGlobalContext())

View File

@ -677,6 +677,8 @@ public:
Tables getExternalTables() const;
void addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table);
void updateExternalTable(const String & table_name, TemporaryTableHolder && temporary_table);
void addOrUpdateExternalTable(const String & table_name, TemporaryTableHolder && temporary_table);
std::shared_ptr<TemporaryTableHolder> findExternalTable(const String & table_name) const;
std::shared_ptr<TemporaryTableHolder> removeExternalTable(const String & table_name);

View File

@ -113,8 +113,10 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext
FutureSetFromSubqueryPtr future_set;
};
using TemporaryTableHolderPtr = std::shared_ptr<TemporaryTableHolder>;
///TODO maybe remove shared_ptr from here?
using TemporaryTablesMapping = std::map<String, std::shared_ptr<TemporaryTableHolder>>;
using TemporaryTablesMapping = std::map<String, TemporaryTableHolderPtr>;
class BackgroundSchedulePoolTaskHolder;

View File

@ -31,8 +31,21 @@ public:
}
std::string getName() const override { return "FullSortingMergeJoin"; }
const TableJoin & getTableJoin() const override { return *table_join; }
bool isCloneSupported() const override
{
return true;
}
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,
const Block &,
const Block & right_sample_block_) const override
{
return std::make_shared<FullSortingMergeJoin>(table_join_, right_sample_block_, null_direction);
}
int getNullDirection() const { return null_direction; }
bool addBlockToJoin(const Block & /* block */, bool /* check_limits */) override

View File

@ -240,11 +240,13 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla
}
HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block_,
bool any_take_last_row_, size_t reserve_num, const String & instance_id_)
bool any_take_last_row_, size_t reserve_num_, const String & instance_id_)
: table_join(table_join_)
, kind(table_join->kind())
, strictness(table_join->strictness())
, any_take_last_row(any_take_last_row_)
, reserve_num(reserve_num_)
, instance_id(instance_id_)
, asof_inequality(table_join->getAsofInequality())
, data(std::make_shared<RightTableData>())
, right_sample_block(right_sample_block_)
@ -331,7 +333,7 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
}
for (auto & maps : data->maps)
dataMapInit(maps, reserve_num);
dataMapInit(maps);
}
HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_columns, Sizes & key_sizes)
@ -493,9 +495,8 @@ struct KeyGetterForType
using Type = typename KeyGetterForTypeImpl<type, Value, Mapped>::Type;
};
void HashJoin::dataMapInit(MapsVariant & map, size_t reserve_num)
void HashJoin::dataMapInit(MapsVariant & map)
{
if (kind == JoinKind::Cross)
return;
joinDispatchInit(kind, strictness, map);

View File

@ -149,13 +149,26 @@ class HashJoin : public IJoin
public:
HashJoin(
std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block,
bool any_take_last_row_ = false, size_t reserve_num = 0, const String & instance_id_ = "");
bool any_take_last_row_ = false, size_t reserve_num_ = 0, const String & instance_id_ = "");
~HashJoin() override;
std::string getName() const override { return "HashJoin"; }
const TableJoin & getTableJoin() const override { return *table_join; }
bool isCloneSupported() const override
{
return true;
}
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,
const Block &,
const Block & right_sample_block_) const override
{
return std::make_shared<HashJoin>(table_join_, right_sample_block_, any_take_last_row, reserve_num, instance_id);
}
/** Add block of data from right hand of JOIN to the map.
* Returns false, if some limit was exceeded and you should not insert more data.
*/
@ -413,7 +426,9 @@ private:
/// This join was created from StorageJoin and it is already filled.
bool from_storage_join = false;
bool any_take_last_row; /// Overwrite existing values when encountering the same key again
const bool any_take_last_row; /// Overwrite existing values when encountering the same key again
const size_t reserve_num;
const String instance_id;
std::optional<TypeIndex> asof_type;
const ASOFJoinInequality asof_inequality;
@ -455,7 +470,7 @@ private:
/// If set HashJoin instance is not available for modification (addBlockToJoin)
TableLockHolder storage_join_lock = nullptr;
void dataMapInit(MapsVariant &, size_t);
void dataMapInit(MapsVariant & map);
void initRightBlockStructure(Block & saved_block_sample);

View File

@ -11,6 +11,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
struct ExtraBlock;
using ExtraBlockPtr = std::shared_ptr<ExtraBlock>;
@ -52,6 +57,23 @@ public:
virtual const TableJoin & getTableJoin() const = 0;
/// Returns true if clone is supported
virtual bool isCloneSupported() const
{
return false;
}
/// Clone underlyhing JOIN algorithm using table join, left sample block, right sample block
virtual std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,
const Block & left_sample_block_,
const Block & right_sample_block_) const
{
(void)(table_join_);
(void)(left_sample_block_);
(void)(right_sample_block_);
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Clone method is not supported for {}", getName());
}
/// Add block of data from right hand of JOIN.
/// @returns false, if some limit was exceeded and you should not insert more data.
virtual bool addBlockToJoin(const Block & block, bool check_limits = true) = 0; /// NOLINT

View File

@ -27,7 +27,9 @@ public:
const StoragePtr & storage_,
const SelectQueryOptions & select_query_options_);
/// Initialize interpreter with query tree
/** Initialize interpreter with query tree.
* No query tree passes are applied.
*/
InterpreterSelectQueryAnalyzer(const QueryTreeNodePtr & query_tree_,
const ContextPtr & context_,
const SelectQueryOptions & select_query_options_);

View File

@ -508,6 +508,7 @@ static void validateUpdateColumns(
/// because their sizes couldn't change, since sizes of all nested subcolumns must be consistent.
static std::optional<std::vector<ASTPtr>> getExpressionsOfUpdatedNestedSubcolumns(
const String & column_name,
NameSet affected_materialized,
const NamesAndTypesList & all_columns,
const std::unordered_map<String, ASTPtr> & column_to_update_expression)
{
@ -520,6 +521,10 @@ static std::optional<std::vector<ASTPtr>> getExpressionsOfUpdatedNestedSubcolumn
auto split = Nested::splitName(column.name);
if (isArray(column.type) && split.first == source_name && !split.second.empty())
{
// Materialized nested columns shall never be part of the update expression
if (affected_materialized.contains(column.name))
continue;
auto it = column_to_update_expression.find(column.name);
if (it == column_to_update_expression.end())
return {};
@ -655,7 +660,10 @@ void MutationsInterpreter::prepare(bool dry_run)
if (materialized_it != column_to_affected_materialized.end())
for (const auto & mat_column : materialized_it->second)
affected_materialized.emplace(mat_column);
}
for (const auto & [column_name, update_expr] : command.column_to_update_expression)
{
/// When doing UPDATE column = expression WHERE condition
/// we will replace column to the result of the following expression:
///
@ -689,7 +697,7 @@ void MutationsInterpreter::prepare(bool dry_run)
{
std::shared_ptr<ASTFunction> function = nullptr;
auto nested_update_exprs = getExpressionsOfUpdatedNestedSubcolumns(column_name, all_columns, command.column_to_update_expression);
auto nested_update_exprs = getExpressionsOfUpdatedNestedSubcolumns(column_name, affected_materialized, all_columns, command.column_to_update_expression);
if (!nested_update_exprs)
{
function = makeASTFunction("validateNestedArraySizes",

View File

@ -56,6 +56,14 @@ struct SelectQueryOptions
std::optional<UInt32> shard_num;
std::optional<UInt32> shard_count;
/** During read from MergeTree parts will be removed from snapshot after they are not needed.
* This optimization will break subsequent execution of the same query tree, because table node
* will no more have valid snapshot.
*
* TODO: Implement this functionality in safer way
*/
bool merge_tree_enable_remove_parts_from_snapshot_optimization = true;
SelectQueryOptions( /// NOLINT(google-explicit-constructor)
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete,
size_t depth = 0,

View File

@ -246,7 +246,10 @@ public:
table_join.strictness = strictness;
}
TableJoin(const TableJoin & rhs) = default;
JoinKind kind() const { return table_join.kind; }
void setKind(JoinKind kind) { table_join.kind = kind; }
JoinStrictness strictness() const { return table_join.strictness; }
bool sameStrictnessAndKind(JoinStrictness, JoinKind) const;
const SizeLimits & sizeLimits() const { return size_limits; }

View File

@ -44,6 +44,7 @@ ASTPtr ASTSelectQuery::clone() const
void ASTSelectQuery::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{
hash_state.update(recursive_with);
hash_state.update(distinct);
hash_state.update(group_by_with_totals);
hash_state.update(group_by_with_rollup);
@ -64,6 +65,10 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
if (with())
{
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH" << (s.hilite ? hilite_none : "");
if (recursive_with)
s.ostr << (s.hilite ? hilite_keyword : "") << " RECURSIVE" << (s.hilite ? hilite_none : "");
s.one_line
? with()->formatImpl(s, state, frame)
: with()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);

View File

@ -83,6 +83,7 @@ public:
ASTPtr clone() const override;
bool recursive_with = false;
bool distinct = false;
bool group_by_all = false;
bool group_by_with_totals = false;

View File

@ -13,7 +13,7 @@ class ASTSubquery : public ASTWithAlias
{
public:
// Stored the name when the subquery is defined in WITH clause. For example:
// WITH (SELECT 1) AS a SELECT * FROM a AS b; cte_name will be `a`.
// WITH a AS (SELECT 1) SELECT * FROM a AS b; cte_name will be `a`.
String cte_name;
/** Get the text that identifies this element. */

View File

@ -516,6 +516,7 @@ namespace DB
MR_MACROS(WITH_REPLACE_OPTION, "WITH REPLACE OPTION") \
MR_MACROS(WITH_TIES, "WITH TIES") \
MR_MACROS(WITH, "WITH") \
MR_MACROS(RECURSIVE, "RECURSIVE") \
MR_MACROS(WK, "WK") \
MR_MACROS(WRITABLE, "WRITABLE") \
MR_MACROS(WW, "WW") \

View File

@ -46,6 +46,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_where(Keyword::WHERE);
ParserKeyword s_group_by(Keyword::GROUP_BY);
ParserKeyword s_with(Keyword::WITH);
ParserKeyword s_recursive(Keyword::RECURSIVE);
ParserKeyword s_totals(Keyword::TOTALS);
ParserKeyword s_having(Keyword::HAVING);
ParserKeyword s_window(Keyword::WINDOW);
@ -103,6 +104,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (s_with.ignore(pos, expected))
{
select_query->recursive_with = s_recursive.ignore(pos, expected);
if (!ParserList(std::make_unique<ParserWithElement>(), std::make_unique<ParserToken>(TokenType::Comma))
.parse(pos, with_expression_list, expected))
return false;

View File

@ -90,7 +90,7 @@ public:
auto subquery_to_execute = in_second_argument;
if (in_second_argument->as<TableNode>())
subquery_to_execute = buildSubqueryToReadColumnsFromTableExpression(std::move(subquery_to_execute), planner_context.getQueryContext());
subquery_to_execute = buildSubqueryToReadColumnsFromTableExpression(subquery_to_execute, planner_context.getQueryContext());
sets.addFromSubquery(set_key, std::move(subquery_to_execute), settings);
}

View File

@ -35,6 +35,7 @@
#include <Processors/QueryPlan/LimitByStep.h>
#include <Processors/QueryPlan/WindowStep.h>
#include <Processors/QueryPlan/ReadNothingStep.h>
#include <Processors/QueryPlan/ReadFromRecursiveCTEStep.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Interpreters/Context.h>
@ -1249,6 +1250,21 @@ void Planner::buildPlanForUnionNode()
|| union_mode == SelectUnionMode::INTERSECT_DEFAULT)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION mode must be initialized");
if (union_node.hasRecursiveCTETable())
{
const auto & recursive_cte_table = *union_node.getRecursiveCTETable();
ColumnsWithTypeAndName recursive_cte_columns;
recursive_cte_columns.reserve(recursive_cte_table.columns.size());
for (const auto & recursive_cte_table_column : recursive_cte_table.columns)
recursive_cte_columns.emplace_back(recursive_cte_table_column.type, recursive_cte_table_column.name);
auto read_from_recursive_cte_step = std::make_unique<ReadFromRecursiveCTEStep>(Block(std::move(recursive_cte_columns)), query_tree);
read_from_recursive_cte_step->setStepDescription(query_tree->toAST()->formatForErrorMessage());
query_plan.addStep(std::move(read_from_recursive_cte_step));
return;
}
const auto & union_queries_nodes = union_node.getQueries().getNodes();
size_t queries_size = union_queries_nodes.size();
@ -1366,6 +1382,7 @@ void Planner::buildPlanForQueryNode()
select_query_info.has_window = hasWindowFunctionNodes(query_tree);
select_query_info.has_aggregates = hasAggregateFunctionNodes(query_tree);
select_query_info.need_aggregate = query_node.hasGroupBy() || select_query_info.has_aggregates;
select_query_info.merge_tree_enable_remove_parts_from_snapshot_optimization = select_query_options.merge_tree_enable_remove_parts_from_snapshot_optimization;
if (!select_query_info.has_window && query_node.hasQualify())
{

View File

@ -1089,18 +1089,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
auto right_plan = std::move(right_join_tree_query_plan.query_plan);
auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
// {
// WriteBufferFromOwnString buf;
// left_plan.explainPlan(buf, {.header = true, .actions = true});
// std::cerr << "left plan \n "<< buf.str() << std::endl;
// }
// {
// WriteBufferFromOwnString buf;
// right_plan.explainPlan(buf, {.header = true, .actions = true});
// std::cerr << "right plan \n "<< buf.str() << std::endl;
// }
JoinClausesAndActions join_clauses_and_actions;
JoinKind join_kind = join_node.getKind();
JoinStrictness join_strictness = join_node.getStrictness();

View File

@ -160,26 +160,6 @@ ASTPtr queryNodeToDistributedSelectQuery(const QueryTreeNodePtr & query_node)
return ast;
}
/** There are no limits on the maximum size of the result for the subquery.
* Since the result of the query is not the result of the entire query.
*/
void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context)
{
/** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result.
* Because the result of this query is not the result of the entire query.
* Constraints work instead
* max_rows_in_set, max_bytes_in_set, set_overflow_mode,
* max_rows_in_join, max_bytes_in_join, join_overflow_mode,
* which are checked separately (in the Set, Join objects).
*/
Settings subquery_settings = mutable_context->getSettings();
subquery_settings.max_result_rows = 0;
subquery_settings.max_result_bytes = 0;
/// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query).
subquery_settings.extremes = false;
mutable_context->setSettings(subquery_settings);
}
namespace
{
@ -422,38 +402,6 @@ QueryTreeNodePtr replaceTableExpressionsWithDummyTables(
return query_node->cloneAndReplace(replacement_map);
}
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
const ContextPtr & context)
{
auto projection_columns = columns;
QueryTreeNodes subquery_projection_nodes;
subquery_projection_nodes.reserve(projection_columns.size());
for (const auto & column : projection_columns)
subquery_projection_nodes.push_back(std::make_shared<ColumnNode>(column, table_expression));
if (subquery_projection_nodes.empty())
{
auto constant_data_type = std::make_shared<DataTypeUInt64>();
subquery_projection_nodes.push_back(std::make_shared<ConstantNode>(1UL, constant_data_type));
projection_columns.push_back({"1", std::move(constant_data_type)});
}
auto context_copy = Context::createCopy(context);
updateContextForSubqueryExecution(context_copy);
auto query_node = std::make_shared<QueryNode>(std::move(context_copy));
query_node->getProjection().getNodes() = std::move(subquery_projection_nodes);
query_node->resolveProjectionColumns(projection_columns);
query_node->getJoinTree() = table_expression;
query_node->setIsSubquery(true);
return query_node;
}
SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context)
{
SelectQueryInfo select_query_info;

View File

@ -40,9 +40,6 @@ ASTPtr queryNodeToDistributedSelectQuery(const QueryTreeNodePtr & query_node);
/// Build context for subquery execution
ContextPtr buildSubqueryContext(const ContextPtr & context);
/// Update mutable context for subquery execution
void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context);
/// Build limits for storage
StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptions & options);
@ -76,11 +73,6 @@ QueryTreeNodePtr replaceTableExpressionsWithDummyTables(
const ContextPtr & context,
ResultReplacementMap * result_replacement_map = nullptr);
/// Build subquery to read specified columns from table expression
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTypes & columns,
const QueryTreeNodePtr & table_expression,
const ContextPtr & context);
SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context);
/// Build filter for specific table_expression

View File

@ -64,11 +64,12 @@ static void executeJob(ExecutingGraph::Node * node, ReadProgressCallback * read_
}
}
}
catch (Exception & exception)
catch (Exception exception) /// NOLINT
{
/// Copy exception before modifying it because multiple threads can rethrow the same exception
if (checkCanAddAdditionalInfoToException(exception))
exception.addMessage("While executing " + node->processor->getName());
throw;
throw exception;
}
}

View File

@ -90,11 +90,8 @@ struct PushingAsyncPipelineExecutor::Data
void rethrowExceptionIfHas()
{
if (has_exception)
{
has_exception = false;
if (has_exception.exchange(false))
std::rethrow_exception(exception);
}
}
};

View File

@ -31,6 +31,7 @@ public:
void describeActions(FormatSettings & settings) const override;
const JoinPtr & getJoin() const { return join; }
void setJoin(JoinPtr join_) { join = std::move(join_); }
bool allowPushDownToRight() const;
bool canUpdateInputStream() const override { return true; }

View File

@ -50,6 +50,9 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &);
/// May split FilterStep and push down only part of it.
size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes);
/// Convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values
size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes);
/// Move ExpressionStep after SortingStep if possible.
/// May split ExpressionStep and lift up only a part of it.
size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes);
@ -78,12 +81,13 @@ size_t tryAggregatePartitionsIndependently(QueryPlan::Node * node, QueryPlan::No
inline const auto & getOptimizations()
{
static const std::array<Optimization, 10> optimizations = {{
static const std::array<Optimization, 11> optimizations = {{
{tryLiftUpArrayJoin, "liftUpArrayJoin", &QueryPlanOptimizationSettings::lift_up_array_join},
{tryPushDownLimit, "pushDownLimit", &QueryPlanOptimizationSettings::push_down_limit},
{trySplitFilter, "splitFilter", &QueryPlanOptimizationSettings::split_filter},
{tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::merge_expressions},
{tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down},
{tryConvertOuterJoinToInnerJoin, "convertOuterJoinToInnerJoin", &QueryPlanOptimizationSettings::convert_outer_join_to_inner_join},
{tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::execute_functions_after_sorting},
{tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::reuse_storage_ordering_for_window_functions},
{tryLiftUpUnion, "liftUpUnion", &QueryPlanOptimizationSettings::lift_up_union},

View File

@ -22,6 +22,8 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const
settings.filter_push_down = from.query_plan_enable_optimizations && from.query_plan_filter_push_down;
settings.convert_outer_join_to_inner_join = from.query_plan_enable_optimizations && from.query_plan_convert_outer_join_to_inner_join;
settings.optimize_prewhere = from.query_plan_enable_optimizations && from.query_plan_optimize_prewhere;
settings.execute_functions_after_sorting = from.query_plan_enable_optimizations && from.query_plan_execute_functions_after_sorting;

View File

@ -34,6 +34,9 @@ struct QueryPlanOptimizationSettings
/// If filter push down optimization is enabled.
bool filter_push_down = true;
/// If convert OUTER JOIN to INNER JOIN optimization is enabled.
bool convert_outer_join_to_inner_join = true;
/// If reorder-functions-after-sorting optimization is enabled.
bool execute_functions_after_sorting = true;

View File

@ -0,0 +1,65 @@
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/FilterStep.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/JoinStep.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/IJoin.h>
#include <Interpreters/TableJoin.h>
namespace DB::QueryPlanOptimizations
{
size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
{
auto & parent = parent_node->step;
auto * filter = typeid_cast<FilterStep *>(parent.get());
if (!filter)
return 0;
QueryPlan::Node * child_node = parent_node->children.front();
auto & child = child_node->step;
auto * join = typeid_cast<JoinStep *>(child.get());
if (!join || !join->getJoin()->isCloneSupported())
return 0;
const auto & table_join = join->getJoin()->getTableJoin();
if (table_join.strictness() == JoinStrictness::Asof)
return 0;
/// TODO: Support join_use_nulls
if (table_join.joinUseNulls())
return 0;
bool check_left_stream = table_join.kind() == JoinKind::Right || table_join.kind() == JoinKind::Full;
bool check_right_stream = table_join.kind() == JoinKind::Left || table_join.kind() == JoinKind::Full;
if (!check_left_stream && !check_right_stream)
return 0;
const auto & filter_dag = filter->getExpression();
const auto & filter_column_name = filter->getFilterColumnName();
const auto & left_stream_input_header = join->getInputStreams().front().header;
const auto & right_stream_input_header = join->getInputStreams().back().header;
bool left_stream_safe = true;
bool right_stream_safe = true;
if (check_left_stream)
left_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header);
if (check_right_stream)
right_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header);
if (!left_stream_safe || !right_stream_safe)
return 0;
auto updated_table_join = std::make_shared<TableJoin>(table_join);
updated_table_join->setKind(JoinKind::Inner);
auto updated_join = join->getJoin()->clone(updated_table_join, left_stream_input_header, right_stream_input_header);
join->setJoin(std::move(updated_join));
return 1;
}
}

View File

@ -236,9 +236,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
* Additional filter push down optimizations:
* 1. TODO: Support building equivalent sets for more than 2 JOINS. It is possible, but will require more complex analysis step.
* 2. TODO: Support building equivalent sets for JOINs with more than 1 clause.
* 3. TODO: For LEFT/RIGHT JOIN, we can assume that RIGHT/LEFT columns used in filter will be default/NULL constants and
* check if filter will always be false, in those scenario we can transform LEFT/RIGHT JOIN into INNER JOIN and push conditions to both tables.
* 4. TODO: It is possible to pull up filter conditions from LEFT/RIGHT stream and push conditions that use columns from LEFT/RIGHT equivalent sets
* 3. TODO: It is possible to pull up filter conditions from LEFT/RIGHT stream and push conditions that use columns from LEFT/RIGHT equivalent sets
* to RIGHT/LEFT JOIN side.
*/

View File

@ -292,6 +292,7 @@ ReadFromMergeTree::ReadFromMergeTree(
, log(std::move(log_))
, analyzed_result_ptr(analyzed_result_ptr_)
, is_parallel_reading_from_replicas(enable_parallel_reading)
, enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization)
{
if (is_parallel_reading_from_replicas)
{
@ -1970,9 +1971,12 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
{
auto result = getAnalysisResult();
/// Do not keep data parts in snapshot.
/// They are stored separately, and some could be released after PK analysis.
storage_snapshot->data = std::make_unique<MergeTreeData::SnapshotData>();
if (enable_remove_parts_from_snapshot_optimization)
{
/// Do not keep data parts in snapshot.
/// They are stored separately, and some could be released after PK analysis.
storage_snapshot->data = std::make_unique<MergeTreeData::SnapshotData>();
}
result.checkLimits(context->getSettingsRef(), query_info);
shared_virtual_fields.emplace("_sample_factor", result.sampling.used_sample_factor);

View File

@ -280,6 +280,7 @@ private:
std::optional<MergeTreeAllRangesCallback> all_ranges_callback;
std::optional<MergeTreeReadTaskCallback> read_task_callback;
bool enable_vertical_final = false;
bool enable_remove_parts_from_snapshot_optimization = true;
};
}

View File

@ -0,0 +1,20 @@
#include <Processors/QueryPlan/ReadFromRecursiveCTEStep.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Processors/Sources/RecursiveCTESource.h>
namespace DB
{
ReadFromRecursiveCTEStep::ReadFromRecursiveCTEStep(Block output_header, QueryTreeNodePtr recursive_cte_union_node_)
: ISourceStep(DataStream{.header = std::move(output_header)})
, recursive_cte_union_node(std::move(recursive_cte_union_node_))
{
}
void ReadFromRecursiveCTEStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
pipeline.init(Pipe(std::make_shared<RecursiveCTESource>(getOutputStream().header, recursive_cte_union_node)));
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <Processors/QueryPlan/ISourceStep.h>
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
class ReadFromRecursiveCTEStep : public ISourceStep
{
public:
explicit ReadFromRecursiveCTEStep(Block output_header, QueryTreeNodePtr recursive_cte_union_node_);
String getName() const override { return "ReadFromRecursiveCTEStep"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
private:
QueryTreeNodePtr recursive_cte_union_node;
};
}

View File

@ -0,0 +1,253 @@
#include <Processors/Sources/RecursiveCTESource.h>
#include <Storages/IStorage.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Transforms/SquashingChunksTransform.h>
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/Context.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/UnionNode.h>
#include <Analyzer/TableNode.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TOO_DEEP_RECURSION;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
namespace
{
std::vector<TableNode *> collectTableNodesWithStorage(const StoragePtr & storage, IQueryTreeNode * root)
{
std::vector<TableNode *> result;
std::vector<IQueryTreeNode *> nodes_to_process;
nodes_to_process.push_back(root);
while (!nodes_to_process.empty())
{
auto * subtree_node = nodes_to_process.back();
nodes_to_process.pop_back();
auto * table_node = subtree_node->as<TableNode>();
if (table_node && table_node->getStorageID() == storage->getStorageID())
result.push_back(table_node);
for (auto & child : subtree_node->getChildren())
{
if (child)
nodes_to_process.push_back(child.get());
}
}
return result;
}
}
class RecursiveCTEChunkGenerator
{
public:
RecursiveCTEChunkGenerator(Block header_, QueryTreeNodePtr recursive_cte_union_node_)
: header(std::move(header_))
, recursive_cte_union_node(std::move(recursive_cte_union_node_))
{
auto & recursive_cte_union_node_typed = recursive_cte_union_node->as<UnionNode &>();
chassert(recursive_cte_union_node_typed.hasRecursiveCTETable());
auto & recursive_cte_table = recursive_cte_union_node_typed.getRecursiveCTETable();
recursive_table_nodes = collectTableNodesWithStorage(recursive_cte_table->storage, recursive_cte_union_node.get());
if (recursive_table_nodes.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION query {} is not recursive", recursive_cte_union_node->formatASTForErrorMessage());
size_t recursive_cte_union_node_queries_size = recursive_cte_union_node_typed.getQueries().getNodes().size();
chassert(recursive_cte_union_node_queries_size > 1);
non_recursive_query = recursive_cte_union_node_typed.getQueries().getNodes()[0];
recursive_query = recursive_cte_union_node_typed.getQueries().getNodes()[1];
if (recursive_cte_union_node_queries_size > 2)
{
auto working_union_query = std::make_shared<UnionNode>(recursive_cte_union_node_typed.getMutableContext(),
recursive_cte_union_node_typed.getUnionMode());
auto & working_union_query_subqueries = working_union_query->getQueries().getNodes();
for (size_t i = 1; i < recursive_cte_union_node_queries_size; ++i)
working_union_query_subqueries.push_back(recursive_cte_union_node_typed.getQueries().getNodes()[i]);
recursive_query = std::move(working_union_query);
}
recursive_query_context = recursive_query->as<QueryNode>() ? recursive_query->as<QueryNode &>().getMutableContext() :
recursive_query->as<UnionNode &>().getMutableContext();
const auto & recursive_query_projection_columns = recursive_query->as<QueryNode>() ? recursive_query->as<QueryNode &>().getProjectionColumns() :
recursive_query->as<UnionNode &>().computeProjectionColumns();
if (recursive_cte_table->columns.size() != recursive_query_projection_columns.size())
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH,
"Recursive CTE subquery {}. Expected projection columns to have same size in recursive and non recursive subquery.",
recursive_cte_union_node->formatASTForErrorMessage());
working_temporary_table_storage = recursive_cte_table->storage;
intermediate_temporary_table_holder = std::make_shared<TemporaryTableHolder>(
recursive_query_context,
ColumnsDescription{NamesAndTypesList{recursive_cte_table->columns.begin(), recursive_cte_table->columns.end()}},
ConstraintsDescription{},
nullptr /*query*/,
true /*create_for_global_subquery*/);
intermediate_temporary_table_storage = intermediate_temporary_table_holder->getTable();
}
Chunk generate()
{
Chunk current_chunk;
while (!finished)
{
if (!executor.has_value())
buildStepExecutor();
while (current_chunk.getNumRows() == 0 && executor->pull(current_chunk))
{
}
read_rows_during_recursive_step += current_chunk.getNumRows();
if (current_chunk.getNumRows() > 0)
break;
executor.reset();
if (read_rows_during_recursive_step == 0)
{
finished = true;
truncateTemporaryTable(intermediate_temporary_table_storage);
continue;
}
read_rows_during_recursive_step = 0;
for (auto & recursive_table_node : recursive_table_nodes)
recursive_table_node->updateStorage(intermediate_temporary_table_storage, recursive_query_context);
truncateTemporaryTable(working_temporary_table_storage);
std::swap(intermediate_temporary_table_storage, working_temporary_table_storage);
}
return current_chunk;
}
private:
void buildStepExecutor()
{
const auto & recursive_subquery_settings = recursive_query_context->getSettingsRef();
if (recursive_step > recursive_subquery_settings.max_recursive_cte_evaluation_depth)
throw Exception(
ErrorCodes::TOO_DEEP_RECURSION,
"Maximum recursive CTE evaluation depth ({}) exceeded, during evaluation of {}. Consider raising "
"max_recursive_cte_evaluation_depth setting.",
recursive_subquery_settings.max_recursive_cte_evaluation_depth,
recursive_cte_union_node->formatASTForErrorMessage());
auto & query_to_execute = recursive_step > 0 ? recursive_query : non_recursive_query;
++recursive_step;
SelectQueryOptions select_query_options;
select_query_options.merge_tree_enable_remove_parts_from_snapshot_optimization = false;
auto interpreter = std::make_unique<InterpreterSelectQueryAnalyzer>(query_to_execute, recursive_query_context, select_query_options);
auto pipeline_builder = interpreter->buildQueryPipeline();
pipeline_builder.addSimpleTransform([&](const Block & in_header)
{
return std::make_shared<MaterializingTransform>(in_header);
});
auto convert_to_temporary_tables_header_actions_dag = ActionsDAG::makeConvertingActions(
pipeline_builder.getHeader().getColumnsWithTypeAndName(),
header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Position);
auto convert_to_temporary_tables_header_actions = std::make_shared<ExpressionActions>(std::move(convert_to_temporary_tables_header_actions_dag));
pipeline_builder.addSimpleTransform([&](const Block & input_header)
{
return std::make_shared<ExpressionTransform>(input_header, convert_to_temporary_tables_header_actions);
});
/// TODO: Support squashing transform
auto intermediate_temporary_table_storage_sink = intermediate_temporary_table_storage->write(
{},
intermediate_temporary_table_storage->getInMemoryMetadataPtr(),
recursive_query_context,
false /*async_insert*/);
pipeline_builder.addChain(Chain(std::move(intermediate_temporary_table_storage_sink)));
pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline_builder));
pipeline.setProgressCallback(recursive_query_context->getProgressCallback());
pipeline.setProcessListElement(recursive_query_context->getProcessListElement());
executor.emplace(pipeline);
}
void truncateTemporaryTable(StoragePtr & temporary_table)
{
/// TODO: Support proper locking
TableExclusiveLockHolder table_exclusive_lock;
temporary_table->truncate({},
temporary_table->getInMemoryMetadataPtr(),
recursive_query_context,
table_exclusive_lock);
}
Block header;
QueryTreeNodePtr recursive_cte_union_node;
std::vector<TableNode *> recursive_table_nodes;
QueryTreeNodePtr non_recursive_query;
QueryTreeNodePtr recursive_query;
ContextMutablePtr recursive_query_context;
StoragePtr working_temporary_table_storage;
TemporaryTableHolderPtr intermediate_temporary_table_holder;
StoragePtr intermediate_temporary_table_storage;
QueryPipeline pipeline;
std::optional<PullingAsyncPipelineExecutor> executor;
size_t recursive_step = 0;
size_t read_rows_during_recursive_step = 0;
bool finished = false;
};
RecursiveCTESource::RecursiveCTESource(Block header, QueryTreeNodePtr recursive_cte_union_node_)
: ISource(header)
, generator(std::make_unique<RecursiveCTEChunkGenerator>(std::move(header), std::move(recursive_cte_union_node_)))
{}
RecursiveCTESource::~RecursiveCTESource() = default;
Chunk RecursiveCTESource::generate()
{
return generator->generate();
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Processors/ISource.h>
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
class RecursiveCTEChunkGenerator;
class RecursiveCTESource : public ISource
{
public:
explicit RecursiveCTESource(Block header, QueryTreeNodePtr recursive_cte_union_node_);
~RecursiveCTESource() override;
String getName() const override { return "RecursiveCTESource"; }
Chunk generate() override;
private:
std::unique_ptr<RecursiveCTEChunkGenerator> generator;
};
}

View File

@ -62,6 +62,7 @@ std::shared_ptr<WriteBufferFromPocoSocket> HTTPServerResponse::send()
stream = std::make_shared<WriteBufferFromPocoSocket>(session.socket(), write_event);
}
send_started = true;
return stream;
}
@ -96,9 +97,16 @@ std::pair<std::shared_ptr<WriteBufferFromPocoSocket>, std::shared_ptr<WriteBuffe
else
stream = std::make_shared<WriteBufferFromPocoSocket>(session.socket(), write_event);
send_started = true;
return std::make_pair(header_stream, stream);
}
void HTTPServerResponse::beginWrite(std::ostream & ostr) const
{
HTTPResponse::beginWrite(ostr);
send_started = true;
}
void HTTPServerResponse::sendBuffer(const void * buffer, std::size_t length)
{
setContentLength(static_cast<int>(length));

View File

@ -209,6 +209,10 @@ public:
/// or redirect() has been called.
std::pair<std::shared_ptr<WriteBufferFromPocoSocket>, std::shared_ptr<WriteBufferFromPocoSocket>> beginSend();
/// Override to correctly mark that the data send had been started for
/// zero-copy response (i.e. replicated fetches).
void beginWrite(std::ostream & ostr) const;
/// Sends the response header to the client, followed
/// by the contents of the given buffer.
///
@ -229,7 +233,7 @@ public:
/// according to the given realm.
/// Returns true if the response (header) has been sent.
bool sent() const { return !!stream; }
bool sent() const { return send_started; }
/// Sets the status code, which must be one of
/// HTTP_MOVED_PERMANENTLY (301), HTTP_FOUND (302),
@ -251,6 +255,7 @@ private:
ProfileEvents::Event write_event;
std::shared_ptr<WriteBufferFromPocoSocket> stream;
std::shared_ptr<WriteBufferFromPocoSocket> header_stream;
mutable bool send_started = false;
};
}

View File

@ -18,12 +18,14 @@
#include <Common/CurrentMetrics.h>
#include <Common/NetException.h>
#include <Common/randomDelay.h>
#include <Common/FailPoint.h>
#include <Common/thread_local_rng.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <base/scope_guard.h>
#include <Poco/Net/HTTPRequest.h>
#include <boost/algorithm/string/join.hpp>
#include <base/sort.h>
#include <random>
namespace fs = std::filesystem;
@ -47,6 +49,12 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int S3_ERROR;
extern const int ZERO_COPY_REPLICATION_ERROR;
extern const int FAULT_INJECTED;
}
namespace FailPoints
{
extern const char replicated_sends_failpoint[];
}
namespace DataPartsExchange
@ -298,11 +306,23 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk(
auto file_in = desc.input_buffer_getter();
HashingWriteBuffer hashing_out(out);
copyDataWithThrottler(*file_in, hashing_out, blocker.getCounter(), data.getSendsThrottler());
hashing_out.finalize();
if (blocker.isCancelled())
throw Exception(ErrorCodes::ABORTED, "Transferring part to replica was cancelled");
const auto & is_cancelled = blocker.getCounter();
auto cancellation_hook = [&]()
{
if (is_cancelled)
throw Exception(ErrorCodes::ABORTED, "Transferring part to replica was cancelled");
fiu_do_on(FailPoints::replicated_sends_failpoint,
{
std::bernoulli_distribution fault(0.1);
if (fault(thread_local_rng))
throw Exception(ErrorCodes::FAULT_INJECTED, "Failpoint replicated_sends_failpoint is triggered");
});
};
copyDataWithThrottler(*file_in, hashing_out, cancellation_hook, data.getSendsThrottler());
hashing_out.finalize();
if (hashing_out.count() != desc.file_size)
throw Exception(
@ -742,9 +762,11 @@ void Fetcher::downloadBaseOrProjectionPartToDisk(
if (expected_hash != hashing_out.getHash())
throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH,
"Checksum mismatch for file {} transferred from {}",
"Checksum mismatch for file {} transferred from {} (0x{} vs 0x{})",
(fs::path(data_part_storage->getFullPath()) / file_name).string(),
replica_path);
replica_path,
getHexUIntLowercase(expected_hash),
getHexUIntLowercase(hashing_out.getHash()));
if (file_name != "checksums.txt" &&
file_name != "columns.txt" &&

View File

@ -235,6 +235,9 @@ struct SelectQueryInfo
/// For IStorageSystemOneBlock
std::vector<UInt8> columns_mask;
/// During read from MergeTree parts will be removed from snapshot after they are not needed
bool merge_tree_enable_remove_parts_from_snapshot_optimization = true;
bool isFinal() const;
};
}

View File

@ -366,7 +366,7 @@ QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_contex
auto subquery_to_execute = in_function_subquery_node;
if (subquery_to_execute->as<TableNode>())
subquery_to_execute = buildSubqueryToReadColumnsFromTableExpression(std::move(subquery_to_execute), planner_context->getQueryContext());
subquery_to_execute = buildSubqueryToReadColumnsFromTableExpression(subquery_to_execute, planner_context->getQueryContext());
auto temporary_table_expression_node = executeSubqueryNode(subquery_to_execute,
planner_context->getMutableQueryContext(),

View File

@ -0,0 +1,22 @@
<test>
<settings>
<join_algorithm>hash</join_algorithm>
</settings>
<create_query>CREATE TABLE test_table_1(id UInt64, value String) ENGINE=MergeTree ORDER BY id</create_query>
<create_query>CREATE TABLE test_table_2(id UInt64, value String) ENGINE=MergeTree ORDER BY id</create_query>
<fill_query>INSERT INTO test_table_1 SELECT (rand64() % 5000000) AS key, toString(key) FROM numbers(5000000)</fill_query>
<fill_query>INSERT INTO test_table_2 SELECT (rand64() % 5000000) AS key, toString(key) FROM numbers(5000000)</fill_query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.value != '' FORMAT Null</query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.value != '' AND rhs.id = 5 FORMAT Null</query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.value != '' FORMAT Null</query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.value != '' AND lhs.id = 5 FORMAT Null</query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.value != '' AND rhs.value != ''</query>
<drop_query>DROP TABLE test_table_1</drop_query>
<drop_query>DROP TABLE test_table_2</drop_query>
</test>

View File

@ -0,0 +1,98 @@
-- { echoOn }
SET allow_experimental_analyzer = 1;
WITH RECURSIVE recursive_cte AS (SELECT 1 AS n UNION ALL SELECT n + 1 FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
1
2
3
4
5
6
7
8
9
10
SELECT '--';
--
WITH RECURSIVE recursive_cte AS (SELECT toUInt8(1) AS n UNION ALL SELECT toUInt8(n + 1) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
1
2
3
4
5
6
7
8
9
10
SELECT '--';
--
WITH RECURSIVE recursive_cte AS (SELECT toUInt16(1) AS n UNION ALL SELECT toUInt8(n + 1) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
1
2
3
4
5
6
7
8
9
10
SELECT '--';
--
WITH RECURSIVE recursive_cte AS (SELECT materialize(toUInt16(1)) AS n UNION ALL SELECT toUInt8(n + 1) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
1
2
3
4
5
6
7
8
9
10
SELECT '--';
--
WITH RECURSIVE recursive_cte AS (SELECT toUInt16(1) AS n UNION ALL SELECT materialize(toUInt8(n + 1)) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
1
2
3
4
5
6
7
8
9
10
SELECT '--';
--
WITH RECURSIVE recursive_cte AS (SELECT toUInt16(1) AS n, '1' AS concat UNION ALL SELECT materialize(toUInt8(n + 1)), concat || toString(n + 1) FROM recursive_cte WHERE n < 10)
SELECT n, concat FROM recursive_cte;
1 1
2 12
3 123
4 1234
5 12345
6 123456
7 1234567
8 12345678
9 123456789
10 12345678910
SELECT '--';
--
WITH RECURSIVE recursive_cte AS (SELECT 1 AS n UNION ALL SELECT n + 1 FROM recursive_cte)
SELECT n FROM recursive_cte LIMIT 5;
1
2
3
4
5
SELECT '--';
--
WITH RECURSIVE recursive_cte AS (SELECT materialize(toUInt8(1)) AS n UNION ALL SELECT materialize(toUInt8(n + 1)) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte FORMAT Null SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }

View File

@ -0,0 +1,43 @@
-- { echoOn }
SET allow_experimental_analyzer = 1;
WITH RECURSIVE recursive_cte AS (SELECT 1 AS n UNION ALL SELECT n + 1 FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
SELECT '--';
WITH RECURSIVE recursive_cte AS (SELECT toUInt8(1) AS n UNION ALL SELECT toUInt8(n + 1) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
SELECT '--';
WITH RECURSIVE recursive_cte AS (SELECT toUInt16(1) AS n UNION ALL SELECT toUInt8(n + 1) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
SELECT '--';
WITH RECURSIVE recursive_cte AS (SELECT materialize(toUInt16(1)) AS n UNION ALL SELECT toUInt8(n + 1) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
SELECT '--';
WITH RECURSIVE recursive_cte AS (SELECT toUInt16(1) AS n UNION ALL SELECT materialize(toUInt8(n + 1)) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte;
SELECT '--';
WITH RECURSIVE recursive_cte AS (SELECT toUInt16(1) AS n, '1' AS concat UNION ALL SELECT materialize(toUInt8(n + 1)), concat || toString(n + 1) FROM recursive_cte WHERE n < 10)
SELECT n, concat FROM recursive_cte;
SELECT '--';
WITH RECURSIVE recursive_cte AS (SELECT 1 AS n UNION ALL SELECT n + 1 FROM recursive_cte)
SELECT n FROM recursive_cte LIMIT 5;
SELECT '--';
WITH RECURSIVE recursive_cte AS (SELECT materialize(toUInt8(1)) AS n UNION ALL SELECT materialize(toUInt8(n + 1)) FROM recursive_cte WHERE n < 10)
SELECT n FROM recursive_cte FORMAT Null SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- { echoOff }

View File

@ -0,0 +1,9 @@
0 \N ROOT
1 0 Child_1
2 0 Child_2
3 1 Child_1_1
--
0 \N ROOT [0]
1 0 Child_1 [0,1]
2 0 Child_2 [0,2]
3 1 Child_1_1 [0,1,3]

View File

@ -0,0 +1,37 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS tree;
CREATE TABLE tree
(
id UInt64,
link Nullable(UInt64),
data String
) ENGINE=TinyLog;
INSERT INTO tree VALUES (0, NULL, 'ROOT'), (1, 0, 'Child_1'), (2, 0, 'Child_2'), (3, 1, 'Child_1_1');
WITH RECURSIVE search_tree AS (
SELECT id, link, data
FROM tree t
WHERE t.id = 0
UNION ALL
SELECT t.id, t.link, t.data
FROM tree t, search_tree st
WHERE t.link = st.id
)
SELECT * FROM search_tree;
SELECT '--';
WITH RECURSIVE search_tree AS (
SELECT id, link, data, [t.id] AS path
FROM tree t
WHERE t.id = 0
UNION ALL
SELECT t.id, t.link, t.data, arrayConcat(path, [t.id])
FROM tree t, search_tree st
WHERE t.link = st.id
)
SELECT * FROM search_tree;
DROP TABLE tree;

View File

@ -0,0 +1,242 @@
-- { echoOn }
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS tree;
CREATE TABLE tree
(
id UInt64,
link Nullable(UInt64),
data String
) ENGINE=MergeTree ORDER BY ();
INSERT INTO tree VALUES (0, NULL, 'ROOT'), (1, 0, 'Child_1'), (2, 0, 'Child_2'), (3, 1, 'Child_1_1');
WITH RECURSIVE search_tree AS (
SELECT id, link, data
FROM tree t
WHERE t.id = 0
UNION ALL
SELECT t.id, t.link, t.data
FROM tree t, search_tree st
WHERE t.link = st.id
)
SELECT * FROM search_tree;
0 \N ROOT
1 0 Child_1
2 0 Child_2
3 1 Child_1_1
SELECT '--';
--
WITH RECURSIVE search_tree AS (
SELECT id, link, data, [t.id] AS path
FROM tree t
WHERE t.id = 0
UNION ALL
SELECT t.id, t.link, t.data, arrayConcat(path, [t.id])
FROM tree t, search_tree st
WHERE t.link = st.id
)
SELECT * FROM search_tree;
0 \N ROOT [0]
1 0 Child_1 [0,1]
2 0 Child_2 [0,2]
3 1 Child_1_1 [0,1,3]
DROP TABLE tree;
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
--
-- Some examples with a tree
--
-- department structure represented here is as follows:
--
-- ROOT-+->A-+->B-+->C
-- | |
-- | +->D-+->F
-- +->E-+->G
DROP TABLE IF EXISTS department;
CREATE TABLE department (
id UInt64, -- department ID
parent_department UInt64, -- upper department ID
name String -- department name
)
ENGINE=MergeTree ORDER BY ();
INSERT INTO department VALUES (0, NULL, 'ROOT');
INSERT INTO department VALUES (1, 0, 'A');
INSERT INTO department VALUES (2, 1, 'B');
INSERT INTO department VALUES (3, 2, 'C');
INSERT INTO department VALUES (4, 2, 'D');
INSERT INTO department VALUES (5, 0, 'E');
INSERT INTO department VALUES (6, 4, 'F');
INSERT INTO department VALUES (7, 5, 'G');
-- extract all departments under 'A'. Result should be A, B, C, D and F
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT name as root_name, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment ORDER BY name;
A 1 0 A
A 2 1 B
A 3 2 C
A 4 2 D
A 6 4 F
-- extract all departments under 'A' with "level" number
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT 1 AS level, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment ORDER BY name;
1 1 0 A
2 2 1 B
3 3 2 C
3 4 2 D
4 6 4 F
-- extract all departments under 'A' with "level" number.
-- Only shows level 2 or more
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT 1 AS level, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name;
2 2 1 B
3 3 2 C
3 4 2 D
4 6 4 F
-- "RECURSIVE" is ignored if the query has no self-reference
WITH RECURSIVE subdepartment AS
(
-- note lack of recursive UNION structure
SELECT * FROM department WHERE name = 'A'
)
SELECT * FROM subdepartment ORDER BY name;
1 0 A
-- corner case in which sub-WITH gets initialized first
SELECT * FROM
(
WITH RECURSIVE q AS (
SELECT * FROM department
UNION ALL
(WITH x AS (SELECT * FROM q)
SELECT * FROM x)
)
SELECT * FROM q LIMIT 24
) ORDER BY id, parent_department, name;
0 0 ROOT
0 0 ROOT
0 0 ROOT
1 0 A
1 0 A
1 0 A
2 1 B
2 1 B
2 1 B
3 2 C
3 2 C
3 2 C
4 2 D
4 2 D
4 2 D
5 0 E
5 0 E
5 0 E
6 4 F
6 4 F
6 4 F
7 5 G
7 5 G
7 5 G
SELECT * FROM
(
WITH RECURSIVE q AS (
SELECT * FROM department
UNION ALL
(WITH RECURSIVE x AS (
SELECT * FROM department
UNION ALL
(SELECT * FROM q UNION ALL SELECT * FROM x)
)
SELECT * FROM x)
)
SELECT * FROM q LIMIT 32
) ORDER BY id, parent_department, name;
0 0 ROOT
0 0 ROOT
0 0 ROOT
0 0 ROOT
1 0 A
1 0 A
1 0 A
1 0 A
2 1 B
2 1 B
2 1 B
2 1 B
3 2 C
3 2 C
3 2 C
3 2 C
4 2 D
4 2 D
4 2 D
4 2 D
5 0 E
5 0 E
5 0 E
5 0 E
6 4 F
6 4 F
6 4 F
6 4 F
7 5 G
7 5 G
7 5 G
7 5 G

View File

@ -0,0 +1,179 @@
-- { echoOn }
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS tree;
CREATE TABLE tree
(
id UInt64,
link Nullable(UInt64),
data String
) ENGINE=MergeTree ORDER BY ();
INSERT INTO tree VALUES (0, NULL, 'ROOT'), (1, 0, 'Child_1'), (2, 0, 'Child_2'), (3, 1, 'Child_1_1');
WITH RECURSIVE search_tree AS (
SELECT id, link, data
FROM tree t
WHERE t.id = 0
UNION ALL
SELECT t.id, t.link, t.data
FROM tree t, search_tree st
WHERE t.link = st.id
)
SELECT * FROM search_tree;
SELECT '--';
WITH RECURSIVE search_tree AS (
SELECT id, link, data, [t.id] AS path
FROM tree t
WHERE t.id = 0
UNION ALL
SELECT t.id, t.link, t.data, arrayConcat(path, [t.id])
FROM tree t, search_tree st
WHERE t.link = st.id
)
SELECT * FROM search_tree;
DROP TABLE tree;
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
--
-- Some examples with a tree
--
-- department structure represented here is as follows:
--
-- ROOT-+->A-+->B-+->C
-- | |
-- | +->D-+->F
-- +->E-+->G
DROP TABLE IF EXISTS department;
CREATE TABLE department (
id UInt64, -- department ID
parent_department UInt64, -- upper department ID
name String -- department name
)
ENGINE=MergeTree ORDER BY ();
INSERT INTO department VALUES (0, NULL, 'ROOT');
INSERT INTO department VALUES (1, 0, 'A');
INSERT INTO department VALUES (2, 1, 'B');
INSERT INTO department VALUES (3, 2, 'C');
INSERT INTO department VALUES (4, 2, 'D');
INSERT INTO department VALUES (5, 0, 'E');
INSERT INTO department VALUES (6, 4, 'F');
INSERT INTO department VALUES (7, 5, 'G');
-- extract all departments under 'A'. Result should be A, B, C, D and F
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT name as root_name, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment ORDER BY name;
-- extract all departments under 'A' with "level" number
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT 1 AS level, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment ORDER BY name;
-- extract all departments under 'A' with "level" number.
-- Only shows level 2 or more
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT 1 AS level, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name;
-- "RECURSIVE" is ignored if the query has no self-reference
WITH RECURSIVE subdepartment AS
(
-- note lack of recursive UNION structure
SELECT * FROM department WHERE name = 'A'
)
SELECT * FROM subdepartment ORDER BY name;
-- corner case in which sub-WITH gets initialized first
SELECT * FROM
(
WITH RECURSIVE q AS (
SELECT * FROM department
UNION ALL
(WITH x AS (SELECT * FROM q)
SELECT * FROM x)
)
SELECT * FROM q LIMIT 24
) ORDER BY id, parent_department, name;
SELECT * FROM
(
WITH RECURSIVE q AS (
SELECT * FROM department
UNION ALL
(WITH RECURSIVE x AS (
SELECT * FROM department
UNION ALL
(SELECT * FROM q UNION ALL SELECT * FROM x)
)
SELECT * FROM x)
)
SELECT * FROM q LIMIT 32
) ORDER BY id, parent_department, name;
-- { echoOff }

View File

@ -0,0 +1,110 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
-- WITH RECURSIVE
-- sum of 1..100
WITH RECURSIVE t AS (
SELECT 1 AS n
UNION ALL
SELECT n+1 FROM t WHERE n < 100
)
SELECT sum(n) FROM t;
5050
WITH RECURSIVE t AS (
SELECT 1 AS n
UNION ALL
SELECT n+1 FROM t WHERE n < 5
)
SELECT * FROM t;
1
2
3
4
5
-- This'd be an infinite loop, but outside query reads only as much as needed
WITH RECURSIVE t AS (
SELECT 1 AS n
UNION ALL
SELECT n+1 FROM t)
SELECT * FROM t LIMIT 10;
1
2
3
4
5
6
7
8
9
10
WITH RECURSIVE t AS (
SELECT 'foo' AS n
UNION ALL
SELECT n || ' bar' FROM t WHERE length(n) < 20
)
SELECT n, toTypeName(n) FROM t;
foo String
foo bar String
foo bar bar String
foo bar bar bar String
foo bar bar bar bar String
foo bar bar bar bar bar String
WITH RECURSIVE t AS (
SELECT '7' AS n
UNION ALL
SELECT n+1 FROM t WHERE n < 10
)
SELECT n, toTypeName(n) FROM t; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Deeply nested WITH caused a list-munging problem in v13
-- Detection of cross-references and self-references
WITH RECURSIVE w1 AS
(WITH w2 AS
(WITH w3 AS
(WITH w4 AS
(WITH w5 AS
(WITH RECURSIVE w6 AS
(WITH w7 AS
(WITH w8 AS
(SELECT 1)
SELECT * FROM w8)
SELECT * FROM w7)
SELECT * FROM w6)
SELECT * FROM w5)
SELECT * FROM w4)
SELECT * FROM w3)
SELECT * FROM w2)
SELECT * FROM w1;
1

View File

@ -0,0 +1,95 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
-- WITH RECURSIVE
-- sum of 1..100
WITH RECURSIVE t AS (
SELECT 1 AS n
UNION ALL
SELECT n+1 FROM t WHERE n < 100
)
SELECT sum(n) FROM t;
WITH RECURSIVE t AS (
SELECT 1 AS n
UNION ALL
SELECT n+1 FROM t WHERE n < 5
)
SELECT * FROM t;
-- This'd be an infinite loop, but outside query reads only as much as needed
WITH RECURSIVE t AS (
SELECT 1 AS n
UNION ALL
SELECT n+1 FROM t)
SELECT * FROM t LIMIT 10;
WITH RECURSIVE t AS (
SELECT 'foo' AS n
UNION ALL
SELECT n || ' bar' FROM t WHERE length(n) < 20
)
SELECT n, toTypeName(n) FROM t;
WITH RECURSIVE t AS (
SELECT '7' AS n
UNION ALL
SELECT n+1 FROM t WHERE n < 10
)
SELECT n, toTypeName(n) FROM t; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Deeply nested WITH caused a list-munging problem in v13
-- Detection of cross-references and self-references
WITH RECURSIVE w1 AS
(WITH w2 AS
(WITH w3 AS
(WITH w4 AS
(WITH w5 AS
(WITH RECURSIVE w6 AS
(WITH w7 AS
(WITH w8 AS
(SELECT 1)
SELECT * FROM w8)
SELECT * FROM w7)
SELECT * FROM w6)
SELECT * FROM w5)
SELECT * FROM w4)
SELECT * FROM w3)
SELECT * FROM w2)
SELECT * FROM w1;
-- { echoOff }

View File

@ -1,5 +1,6 @@
SET allow_experimental_analyzer = 1;
SET optimize_move_to_prewhere = 0;
SET query_plan_convert_outer_join_to_inner_join = 0;
DROP TABLE IF EXISTS test_table_1;
CREATE TABLE test_table_1

View File

@ -0,0 +1,224 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- Some examples with a tree
--
-- department structure represented here is as follows:
--
-- ROOT-+->A-+->B-+->C
-- | |
-- | +->D-+->F
-- +->E-+->G
DROP TABLE IF EXISTS department;
CREATE TABLE department (
id UInt64, -- department ID
parent_department UInt64, -- upper department ID
name String -- department name
)
ENGINE=TinyLog;
INSERT INTO department VALUES (0, NULL, 'ROOT');
INSERT INTO department VALUES (1, 0, 'A');
INSERT INTO department VALUES (2, 1, 'B');
INSERT INTO department VALUES (3, 2, 'C');
INSERT INTO department VALUES (4, 2, 'D');
INSERT INTO department VALUES (5, 0, 'E');
INSERT INTO department VALUES (6, 4, 'F');
INSERT INTO department VALUES (7, 5, 'G');
-- extract all departments under 'A'. Result should be A, B, C, D and F
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT name as root_name, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment ORDER BY name;
A 1 0 A
A 2 1 B
A 3 2 C
A 4 2 D
A 6 4 F
-- extract all departments under 'A' with "level" number
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT 1 AS level, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment ORDER BY name;
1 1 0 A
2 2 1 B
3 3 2 C
3 4 2 D
4 6 4 F
-- extract all departments under 'A' with "level" number.
-- Only shows level 2 or more
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT 1 AS level, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name;
2 2 1 B
3 3 2 C
3 4 2 D
4 6 4 F
-- "RECURSIVE" is ignored if the query has no self-reference
WITH RECURSIVE subdepartment AS
(
-- note lack of recursive UNION structure
SELECT * FROM department WHERE name = 'A'
)
SELECT * FROM subdepartment ORDER BY name;
1 0 A
-- inside subqueries
SELECT count(*) FROM
(
WITH RECURSIVE t AS (
SELECT toUInt64(1) AS n UNION ALL SELECT n + 1 FROM t WHERE n < 500
)
SELECT * FROM t
) AS t WHERE n < (
SELECT count(*) FROM (
WITH RECURSIVE t AS (
SELECT toUInt64(1) AS n UNION ALL SELECT n + 1 FROM t WHERE n < 100
)
SELECT * FROM t WHERE n < 50000
) AS t WHERE n < 100);
98
-- corner case in which sub-WITH gets initialized first
WITH RECURSIVE q AS (
SELECT * FROM department
UNION ALL
(WITH x AS (SELECT * FROM q)
SELECT * FROM x)
)
SELECT * FROM q LIMIT 24;
0 0 ROOT
1 0 A
2 1 B
3 2 C
4 2 D
5 0 E
6 4 F
7 5 G
0 0 ROOT
1 0 A
2 1 B
3 2 C
4 2 D
5 0 E
6 4 F
7 5 G
0 0 ROOT
1 0 A
2 1 B
3 2 C
4 2 D
5 0 E
6 4 F
7 5 G
WITH RECURSIVE q AS (
SELECT * FROM department
UNION ALL
(WITH RECURSIVE x AS (
SELECT * FROM department
UNION ALL
(SELECT * FROM q UNION ALL SELECT * FROM x)
)
SELECT * FROM x)
)
SELECT * FROM q LIMIT 32;
0 0 ROOT
1 0 A
2 1 B
3 2 C
4 2 D
5 0 E
6 4 F
7 5 G
0 0 ROOT
1 0 A
2 1 B
3 2 C
4 2 D
5 0 E
6 4 F
7 5 G
0 0 ROOT
1 0 A
2 1 B
3 2 C
4 2 D
5 0 E
6 4 F
7 5 G
0 0 ROOT
1 0 A
2 1 B
3 2 C
4 2 D
5 0 E
6 4 F
7 5 G
-- recursive term has sub-UNION
WITH RECURSIVE t AS (
SELECT 1 AS i, 2 AS j
UNION ALL
SELECT t2.i, t.j+1 FROM
(SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2
JOIN t ON (t2.i = t.i+1))
SELECT * FROM t;
1 2
2 3
3 4

View File

@ -0,0 +1,162 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- Some examples with a tree
--
-- department structure represented here is as follows:
--
-- ROOT-+->A-+->B-+->C
-- | |
-- | +->D-+->F
-- +->E-+->G
DROP TABLE IF EXISTS department;
CREATE TABLE department (
id UInt64, -- department ID
parent_department UInt64, -- upper department ID
name String -- department name
)
ENGINE=TinyLog;
INSERT INTO department VALUES (0, NULL, 'ROOT');
INSERT INTO department VALUES (1, 0, 'A');
INSERT INTO department VALUES (2, 1, 'B');
INSERT INTO department VALUES (3, 2, 'C');
INSERT INTO department VALUES (4, 2, 'D');
INSERT INTO department VALUES (5, 0, 'E');
INSERT INTO department VALUES (6, 4, 'F');
INSERT INTO department VALUES (7, 5, 'G');
-- extract all departments under 'A'. Result should be A, B, C, D and F
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT name as root_name, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment ORDER BY name;
-- extract all departments under 'A' with "level" number
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT 1 AS level, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment ORDER BY name;
-- extract all departments under 'A' with "level" number.
-- Only shows level 2 or more
WITH RECURSIVE subdepartment AS
(
-- non recursive term
SELECT 1 AS level, * FROM department WHERE name = 'A'
UNION ALL
-- recursive term
SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd
WHERE d.parent_department = sd.id
)
SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name;
-- "RECURSIVE" is ignored if the query has no self-reference
WITH RECURSIVE subdepartment AS
(
-- note lack of recursive UNION structure
SELECT * FROM department WHERE name = 'A'
)
SELECT * FROM subdepartment ORDER BY name;
-- inside subqueries
SELECT count(*) FROM
(
WITH RECURSIVE t AS (
SELECT toUInt64(1) AS n UNION ALL SELECT n + 1 FROM t WHERE n < 500
)
SELECT * FROM t
) AS t WHERE n < (
SELECT count(*) FROM (
WITH RECURSIVE t AS (
SELECT toUInt64(1) AS n UNION ALL SELECT n + 1 FROM t WHERE n < 100
)
SELECT * FROM t WHERE n < 50000
) AS t WHERE n < 100);
-- corner case in which sub-WITH gets initialized first
WITH RECURSIVE q AS (
SELECT * FROM department
UNION ALL
(WITH x AS (SELECT * FROM q)
SELECT * FROM x)
)
SELECT * FROM q LIMIT 24;
WITH RECURSIVE q AS (
SELECT * FROM department
UNION ALL
(WITH RECURSIVE x AS (
SELECT * FROM department
UNION ALL
(SELECT * FROM q UNION ALL SELECT * FROM x)
)
SELECT * FROM x)
)
SELECT * FROM q LIMIT 32;
-- recursive term has sub-UNION
WITH RECURSIVE t AS (
SELECT 1 AS i, 2 AS j
UNION ALL
SELECT t2.i, t.j+1 FROM
(SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2
JOIN t ON (t2.i = t.i+1))
SELECT * FROM t;
-- { echoOff }

View File

@ -0,0 +1,113 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- different tree example
--
DROP TABLE IF EXISTS tree;
CREATE TABLE tree(
id UInt64,
parent_id Nullable(UInt64)
)
ENGINE=TinyLog;
INSERT INTO tree
VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11);
--
-- get all paths from "second level" nodes to leaf nodes
--
WITH RECURSIVE t AS (
SELECT 1 AS id, []::Array(UInt64) AS path
UNION ALL
SELECT tree.id, arrayConcat(t.path, [tree.id])
FROM tree JOIN t ON (tree.parent_id = t.id)
)
SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON
(t1.path[1] = t2.path[1] AND
length(t1.path) = 1 AND
length(t2.path) > 1)
ORDER BY t1.id, t2.id;
2 [2] 4 [2,4]
2 [2] 5 [2,5]
2 [2] 6 [2,6]
2 [2] 9 [2,4,9]
2 [2] 10 [2,4,10]
2 [2] 14 [2,4,9,14]
3 [3] 7 [3,7]
3 [3] 8 [3,8]
3 [3] 11 [3,7,11]
3 [3] 12 [3,7,12]
3 [3] 13 [3,7,13]
3 [3] 15 [3,7,11,15]
3 [3] 16 [3,7,11,16]
-- just count 'em
WITH RECURSIVE t AS (
SELECT 1 AS id, []::Array(UInt64) AS path
UNION ALL
SELECT tree.id, arrayConcat(t.path, [tree.id])
FROM tree JOIN t ON (tree.parent_id = t.id)
)
SELECT t1.id, count(t2.path) FROM t AS t1 JOIN t AS t2 ON
(t1.path[1] = t2.path[1] AND
length(t1.path) = 1 AND
length(t2.path) > 1)
GROUP BY t1.id
ORDER BY t1.id;
2 6
3 7
-- -- this variant tickled a whole-row-variable bug in 8.4devel
WITH RECURSIVE t AS (
SELECT 1 AS id, []::Array(UInt64) AS path
UNION ALL
SELECT tree.id, arrayConcat(t.path, [tree.id])
FROM tree JOIN t ON (tree.parent_id = t.id)
)
SELECT t1.id, t2.path, tuple(t2.*) FROM t AS t1 JOIN t AS t2 ON
(t1.id=t2.id);
1 [] (1,[])
2 [2] (2,[2])
3 [3] (3,[3])
4 [2,4] (4,[2,4])
5 [2,5] (5,[2,5])
6 [2,6] (6,[2,6])
7 [3,7] (7,[3,7])
8 [3,8] (8,[3,8])
9 [2,4,9] (9,[2,4,9])
10 [2,4,10] (10,[2,4,10])
11 [3,7,11] (11,[3,7,11])
12 [3,7,12] (12,[3,7,12])
13 [3,7,13] (13,[3,7,13])
14 [2,4,9,14] (14,[2,4,9,14])
15 [3,7,11,15] (15,[3,7,11,15])
16 [3,7,11,16] (16,[3,7,11,16])

View File

@ -0,0 +1,89 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- different tree example
--
DROP TABLE IF EXISTS tree;
CREATE TABLE tree(
id UInt64,
parent_id Nullable(UInt64)
)
ENGINE=TinyLog;
INSERT INTO tree
VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11);
--
-- get all paths from "second level" nodes to leaf nodes
--
WITH RECURSIVE t AS (
SELECT 1 AS id, []::Array(UInt64) AS path
UNION ALL
SELECT tree.id, arrayConcat(t.path, [tree.id])
FROM tree JOIN t ON (tree.parent_id = t.id)
)
SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON
(t1.path[1] = t2.path[1] AND
length(t1.path) = 1 AND
length(t2.path) > 1)
ORDER BY t1.id, t2.id;
-- just count 'em
WITH RECURSIVE t AS (
SELECT 1 AS id, []::Array(UInt64) AS path
UNION ALL
SELECT tree.id, arrayConcat(t.path, [tree.id])
FROM tree JOIN t ON (tree.parent_id = t.id)
)
SELECT t1.id, count(t2.path) FROM t AS t1 JOIN t AS t2 ON
(t1.path[1] = t2.path[1] AND
length(t1.path) = 1 AND
length(t2.path) > 1)
GROUP BY t1.id
ORDER BY t1.id;
-- -- this variant tickled a whole-row-variable bug in 8.4devel
WITH RECURSIVE t AS (
SELECT 1 AS id, []::Array(UInt64) AS path
UNION ALL
SELECT tree.id, arrayConcat(t.path, [tree.id])
FROM tree JOIN t ON (tree.parent_id = t.id)
)
SELECT t1.id, t2.path, tuple(t2.*) FROM t AS t1 JOIN t AS t2 ON
(t1.id=t2.id);
-- { echoOff }

View File

@ -0,0 +1,114 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- test cycle detection
--
DROP TABLE IF EXISTS graph;
CREATE TABLE graph(
f UInt64,
t UInt64,
label String
)
ENGINE = TinyLog;
INSERT INTO graph VALUES (1, 2, 'arc 1 -> 2'), (1, 3, 'arc 1 -> 3'), (2, 3, 'arc 2 -> 3'), (1, 4, 'arc 1 -> 4'), (4, 5, 'arc 4 -> 5'), (5, 1, 'arc 5 -> 1');
WITH RECURSIVE search_graph AS (
SELECT *, false AS is_cycle, [tuple(g.f, g.t)] AS path FROM graph g
UNION ALL
SELECT g.*, has(path, tuple(g.f, g.t)), arrayConcat(sg.path, [tuple(g.f, g.t)])
FROM graph g, search_graph sg
WHERE g.f = sg.t AND NOT is_cycle
)
SELECT * FROM search_graph;
1 2 arc 1 -> 2 false [(1,2)]
1 3 arc 1 -> 3 false [(1,3)]
2 3 arc 2 -> 3 false [(2,3)]
1 4 arc 1 -> 4 false [(1,4)]
4 5 arc 4 -> 5 false [(4,5)]
5 1 arc 5 -> 1 false [(5,1)]
1 2 arc 1 -> 2 false [(5,1),(1,2)]
1 3 arc 1 -> 3 false [(5,1),(1,3)]
2 3 arc 2 -> 3 false [(1,2),(2,3)]
1 4 arc 1 -> 4 false [(5,1),(1,4)]
4 5 arc 4 -> 5 false [(1,4),(4,5)]
5 1 arc 5 -> 1 false [(4,5),(5,1)]
1 2 arc 1 -> 2 false [(4,5),(5,1),(1,2)]
1 3 arc 1 -> 3 false [(4,5),(5,1),(1,3)]
2 3 arc 2 -> 3 false [(5,1),(1,2),(2,3)]
1 4 arc 1 -> 4 false [(4,5),(5,1),(1,4)]
4 5 arc 4 -> 5 false [(5,1),(1,4),(4,5)]
5 1 arc 5 -> 1 false [(1,4),(4,5),(5,1)]
1 2 arc 1 -> 2 false [(1,4),(4,5),(5,1),(1,2)]
1 3 arc 1 -> 3 false [(1,4),(4,5),(5,1),(1,3)]
2 3 arc 2 -> 3 false [(4,5),(5,1),(1,2),(2,3)]
1 4 arc 1 -> 4 true [(1,4),(4,5),(5,1),(1,4)]
4 5 arc 4 -> 5 true [(4,5),(5,1),(1,4),(4,5)]
5 1 arc 5 -> 1 true [(5,1),(1,4),(4,5),(5,1)]
2 3 arc 2 -> 3 false [(1,4),(4,5),(5,1),(1,2),(2,3)]
-- ordering by the path column has same effect as SEARCH DEPTH FIRST
WITH RECURSIVE search_graph AS (
SELECT *, false AS is_cycle, [tuple(g.f, g.t)] AS path FROM graph g
UNION ALL
SELECT g.*, has(path, tuple(g.f, g.t)), arrayConcat(sg.path, [tuple(g.f, g.t)])
FROM graph g, search_graph sg
WHERE g.f = sg.t AND NOT is_cycle
)
SELECT * FROM search_graph ORDER BY path;
1 2 arc 1 -> 2 false [(1,2)]
2 3 arc 2 -> 3 false [(1,2),(2,3)]
1 3 arc 1 -> 3 false [(1,3)]
1 4 arc 1 -> 4 false [(1,4)]
4 5 arc 4 -> 5 false [(1,4),(4,5)]
5 1 arc 5 -> 1 false [(1,4),(4,5),(5,1)]
1 2 arc 1 -> 2 false [(1,4),(4,5),(5,1),(1,2)]
2 3 arc 2 -> 3 false [(1,4),(4,5),(5,1),(1,2),(2,3)]
1 3 arc 1 -> 3 false [(1,4),(4,5),(5,1),(1,3)]
1 4 arc 1 -> 4 true [(1,4),(4,5),(5,1),(1,4)]
2 3 arc 2 -> 3 false [(2,3)]
4 5 arc 4 -> 5 false [(4,5)]
5 1 arc 5 -> 1 false [(4,5),(5,1)]
1 2 arc 1 -> 2 false [(4,5),(5,1),(1,2)]
2 3 arc 2 -> 3 false [(4,5),(5,1),(1,2),(2,3)]
1 3 arc 1 -> 3 false [(4,5),(5,1),(1,3)]
1 4 arc 1 -> 4 false [(4,5),(5,1),(1,4)]
4 5 arc 4 -> 5 true [(4,5),(5,1),(1,4),(4,5)]
5 1 arc 5 -> 1 false [(5,1)]
1 2 arc 1 -> 2 false [(5,1),(1,2)]
2 3 arc 2 -> 3 false [(5,1),(1,2),(2,3)]
1 3 arc 1 -> 3 false [(5,1),(1,3)]
1 4 arc 1 -> 4 false [(5,1),(1,4)]
4 5 arc 4 -> 5 false [(5,1),(1,4),(4,5)]
5 1 arc 5 -> 1 true [(5,1),(1,4),(4,5),(5,1)]

View File

@ -0,0 +1,70 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- test cycle detection
--
DROP TABLE IF EXISTS graph;
CREATE TABLE graph(
f UInt64,
t UInt64,
label String
)
ENGINE = TinyLog;
INSERT INTO graph VALUES (1, 2, 'arc 1 -> 2'), (1, 3, 'arc 1 -> 3'), (2, 3, 'arc 2 -> 3'), (1, 4, 'arc 1 -> 4'), (4, 5, 'arc 4 -> 5'), (5, 1, 'arc 5 -> 1');
WITH RECURSIVE search_graph AS (
SELECT *, false AS is_cycle, [tuple(g.f, g.t)] AS path FROM graph g
UNION ALL
SELECT g.*, has(path, tuple(g.f, g.t)), arrayConcat(sg.path, [tuple(g.f, g.t)])
FROM graph g, search_graph sg
WHERE g.f = sg.t AND NOT is_cycle
)
SELECT * FROM search_graph;
-- ordering by the path column has same effect as SEARCH DEPTH FIRST
WITH RECURSIVE search_graph AS (
SELECT *, false AS is_cycle, [tuple(g.f, g.t)] AS path FROM graph g
UNION ALL
SELECT g.*, has(path, tuple(g.f, g.t)), arrayConcat(sg.path, [tuple(g.f, g.t)])
FROM graph g, search_graph sg
WHERE g.f = sg.t AND NOT is_cycle
)
SELECT * FROM search_graph ORDER BY path;
-- { echoOff }

View File

@ -0,0 +1,182 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- test multiple WITH queries
--
WITH RECURSIVE
y AS (SELECT 1 AS id),
x AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5)
SELECT * FROM x ORDER BY id;
1
2
3
4
5
-- forward reference OK
WITH RECURSIVE
x AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5),
y AS (SELECT 1 AS id)
SELECT * FROM x ORDER BY id;
1
2
3
4
5
WITH RECURSIVE
x AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 5),
y AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM y WHERE id < 10)
SELECT y.*, x.* FROM y LEFT JOIN x USING (id) ORDER BY y.id;
1 1
2 2
3 3
4 4
5 5
6 0
7 0
8 0
9 0
10 0
WITH RECURSIVE
x AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 5),
y AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 10)
SELECT y.*, x.* FROM y LEFT JOIN x USING (id) ORDER BY y.id;
1 1
2 2
3 3
4 4
5 5
6 0
WITH RECURSIVE
x AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 3 ),
y AS
(SELECT * FROM x UNION ALL SELECT * FROM x),
z AS
(SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10)
SELECT * FROM z ORDER BY id;
1
2
2
3
3
3
4
4
4
5
5
5
6
6
6
7
7
7
8
8
8
9
9
9
10
10
10
WITH RECURSIVE
x AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 3 ),
y AS
(SELECT * FROM x UNION ALL SELECT * FROM x),
z AS
(SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10)
SELECT * FROM z ORDER BY id;
1
1
2
2
2
2
3
3
3
3
3
3
4
4
4
4
4
4
5
5
5
5
5
5
6
6
6
6
6
6
7
7
7
7
7
7
8
8
8
8
8
8
9
9
9
9
9
9
10
10
10
10
10
10

View File

@ -0,0 +1,84 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- test multiple WITH queries
--
WITH RECURSIVE
y AS (SELECT 1 AS id),
x AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5)
SELECT * FROM x ORDER BY id;
-- forward reference OK
WITH RECURSIVE
x AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5),
y AS (SELECT 1 AS id)
SELECT * FROM x ORDER BY id;
WITH RECURSIVE
x AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 5),
y AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM y WHERE id < 10)
SELECT y.*, x.* FROM y LEFT JOIN x USING (id) ORDER BY y.id;
WITH RECURSIVE
x AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 5),
y AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 10)
SELECT y.*, x.* FROM y LEFT JOIN x USING (id) ORDER BY y.id;
WITH RECURSIVE
x AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 3 ),
y AS
(SELECT * FROM x UNION ALL SELECT * FROM x),
z AS
(SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10)
SELECT * FROM z ORDER BY id;
WITH RECURSIVE
x AS
(SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 3 ),
y AS
(SELECT * FROM x UNION ALL SELECT * FROM x),
z AS
(SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10)
SELECT * FROM z ORDER BY id;
-- { echoOff }

View File

@ -0,0 +1,96 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- error cases
--
-- INTERSECT
WITH RECURSIVE x AS (SELECT 1 AS n INTERSECT SELECT n+1 FROM x)
SELECT * FROM x; -- {serverError UNSUPPORTED_METHOD}
WITH RECURSIVE x AS (SELECT 1 AS n INTERSECT ALL SELECT n+1 FROM x)
SELECT * FROM x; -- {serverError UNSUPPORTED_METHOD}
-- EXCEPT
WITH RECURSIVE x AS (SELECT 1 AS n EXCEPT SELECT n+1 FROM x)
SELECT * FROM x; -- {serverError UNSUPPORTED_METHOD}
WITH RECURSIVE x AS (SELECT 1 AS n EXCEPT ALL SELECT n+1 FROM x)
SELECT * FROM x; -- {serverError UNSUPPORTED_METHOD}
-- no non-recursive term
WITH RECURSIVE x AS (SELECT n FROM x)
SELECT * FROM x; -- {serverError UNKNOWN_TABLE}
-- recursive term in the left hand side (strictly speaking, should allow this)
WITH RECURSIVE x AS (SELECT n FROM x UNION ALL SELECT 1 AS n)
SELECT * FROM x; -- {serverError UNKNOWN_TABLE}
DROP TABLE IF EXISTS y;
CREATE TABLE y (a UInt64) ENGINE=TinyLog;
INSERT INTO y SELECT * FROM numbers(1, 10);
-- LEFT JOIN
WITH RECURSIVE x AS (SELECT a AS n FROM y WHERE a = 1
UNION ALL
SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- RIGHT JOIN
WITH RECURSIVE x AS (SELECT a AS n FROM y WHERE a = 1
UNION ALL
SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- FULL JOIN
WITH RECURSIVE x AS (SELECT a AS n FROM y WHERE a = 1
UNION ALL
SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- subquery
WITH RECURSIVE x AS (SELECT 1 AS n UNION ALL SELECT n+1 FROM x
WHERE n IN (SELECT * FROM x))
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- aggregate functions
WITH RECURSIVE x AS (SELECT 1 AS n UNION ALL SELECT count(*) FROM x)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
WITH RECURSIVE x AS (SELECT 1 AS n UNION ALL SELECT sum(n) FROM x)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- ORDER BY
WITH RECURSIVE x AS (SELECT 1 AS n UNION ALL SELECT n+1 FROM x ORDER BY 1)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- target list has a recursive query name
WITH RECURSIVE x AS (SELECT 1 AS id
UNION ALL
SELECT (SELECT * FROM x) FROM x WHERE id < 5
) SELECT * FROM x; -- { serverError UNKNOWN_TABLE }
-- mutual recursive query (not implemented)
WITH RECURSIVE
x AS (SELECT 1 AS id UNION ALL SELECT id+1 FROM y WHERE id < 5),
y AS (SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 5)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }

View File

@ -0,0 +1,114 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
--
-- error cases
--
-- INTERSECT
WITH RECURSIVE x AS (SELECT 1 AS n INTERSECT SELECT n+1 FROM x)
SELECT * FROM x; -- {serverError UNSUPPORTED_METHOD}
WITH RECURSIVE x AS (SELECT 1 AS n INTERSECT ALL SELECT n+1 FROM x)
SELECT * FROM x; -- {serverError UNSUPPORTED_METHOD}
-- EXCEPT
WITH RECURSIVE x AS (SELECT 1 AS n EXCEPT SELECT n+1 FROM x)
SELECT * FROM x; -- {serverError UNSUPPORTED_METHOD}
WITH RECURSIVE x AS (SELECT 1 AS n EXCEPT ALL SELECT n+1 FROM x)
SELECT * FROM x; -- {serverError UNSUPPORTED_METHOD}
-- no non-recursive term
WITH RECURSIVE x AS (SELECT n FROM x)
SELECT * FROM x; -- {serverError UNKNOWN_TABLE}
-- recursive term in the left hand side (strictly speaking, should allow this)
WITH RECURSIVE x AS (SELECT n FROM x UNION ALL SELECT 1 AS n)
SELECT * FROM x; -- {serverError UNKNOWN_TABLE}
DROP TABLE IF EXISTS y;
CREATE TABLE y (a UInt64) ENGINE=TinyLog;
INSERT INTO y SELECT * FROM numbers(1, 10);
-- LEFT JOIN
WITH RECURSIVE x AS (SELECT a AS n FROM y WHERE a = 1
UNION ALL
SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- RIGHT JOIN
WITH RECURSIVE x AS (SELECT a AS n FROM y WHERE a = 1
UNION ALL
SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- FULL JOIN
WITH RECURSIVE x AS (SELECT a AS n FROM y WHERE a = 1
UNION ALL
SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- subquery
WITH RECURSIVE x AS (SELECT 1 AS n UNION ALL SELECT n+1 FROM x
WHERE n IN (SELECT * FROM x))
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- aggregate functions
WITH RECURSIVE x AS (SELECT 1 AS n UNION ALL SELECT count(*) FROM x)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
WITH RECURSIVE x AS (SELECT 1 AS n UNION ALL SELECT sum(n) FROM x)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- ORDER BY
WITH RECURSIVE x AS (SELECT 1 AS n UNION ALL SELECT n+1 FROM x ORDER BY 1)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- target list has a recursive query name
WITH RECURSIVE x AS (SELECT 1 AS id
UNION ALL
SELECT (SELECT * FROM x) FROM x WHERE id < 5
) SELECT * FROM x; -- { serverError UNKNOWN_TABLE }
-- mutual recursive query (not implemented)
WITH RECURSIVE
x AS (SELECT 1 AS id UNION ALL SELECT id+1 FROM y WHERE id < 5),
y AS (SELECT 1 AS id UNION ALL SELECT id+1 FROM x WHERE id < 5)
SELECT * FROM x FORMAT NULL SETTINGS max_recursive_cte_evaluation_depth = 5; -- { serverError TOO_DEEP_RECURSION }
-- { echoOff }

View File

@ -0,0 +1,397 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
WITH RECURSIVE foo AS
(SELECT 1 AS i
UNION ALL
(SELECT i+1 FROM foo WHERE i < 10
UNION ALL
SELECT i+1 FROM foo WHERE i < 5)
) SELECT * FROM foo;
1
2
2
3
3
3
3
4
4
4
4
4
4
4
4
5
5
5
5
5
5
5
5
5
5
5
5
5
5
5
5
6
6
6
6
6
6
6
6
6
6
6
6
6
6
6
6
7
7
7
7
7
7
7
7
7
7
7
7
7
7
7
7
8
8
8
8
8
8
8
8
8
8
8
8
8
8
8
8
9
9
9
9
9
9
9
9
9
9
9
9
9
9
9
9
10
10
10
10
10
10
10
10
10
10
10
10
10
10
10
10
WITH RECURSIVE foo AS
(SELECT 1 AS i
UNION ALL
SELECT * FROM
(SELECT i+1 FROM foo WHERE i < 10
UNION ALL
SELECT i+1 FROM foo WHERE i < 5) AS t
) SELECT * FROM foo;
1
2
2
3
3
3
3
4
4
4
4
4
4
4
4
5
5
5
5
5
5
5
5
5
5
5
5
5
5
5
5
6
6
6
6
6
6
6
6
6
6
6
6
6
6
6
6
7
7
7
7
7
7
7
7
7
7
7
7
7
7
7
7
8
8
8
8
8
8
8
8
8
8
8
8
8
8
8
8
9
9
9
9
9
9
9
9
9
9
9
9
9
9
9
9
10
10
10
10
10
10
10
10
10
10
10
10
10
10
10
10
WITH RECURSIVE foo AS
(SELECT 1 AS i
UNION ALL
(SELECT i+1 FROM foo WHERE i < 10
EXCEPT
SELECT i+1 FROM foo WHERE i < 5)
) SELECT * FROM foo;
1
WITH RECURSIVE foo AS
(SELECT 1 AS i
UNION ALL
(SELECT i+1 FROM foo WHERE i < 10
INTERSECT
SELECT i+1 FROM foo WHERE i < 5)
) SELECT * FROM foo;
1
2
3
4
5
--
-- test for nested-recursive-WITH bug
--
WITH RECURSIVE t AS (
WITH RECURSIVE s AS (
SELECT toUInt64(1) AS i
UNION ALL
SELECT i+1 FROM s WHERE i < 10
)
SELECT i AS j FROM s
UNION ALL
SELECT j+1 FROM t WHERE j < 10
)
SELECT * FROM t;
1
2
3
4
5
6
7
8
9
10
2
3
4
5
6
7
8
9
10
3
4
5
6
7
8
9
10
4
5
6
7
8
9
10
5
6
7
8
9
10
6
7
8
9
10
7
8
9
10
8
9
10
9
10
10
--
-- Test CTEs read in non-initialization orders
--
WITH RECURSIVE
tab AS (SELECT * FROM values('id_key UInt64, link UInt64', (1,17), (2,17), (3,17), (4,17), (6,17), (5,17))),
iter AS (
SELECT 0 AS id_key, 'base' AS row_type, 17 AS link
UNION ALL (
WITH remaining AS (
SELECT tab.id_key AS id_key, 'true'::text AS row_type, iter.link AS link, MIN(tab.id_key) OVER () AS min
FROM tab INNER JOIN iter USING (link)
WHERE tab.id_key > iter.id_key
),
first_remaining AS (
SELECT id_key, row_type, link
FROM remaining
WHERE id_key=min
),
effect AS (
SELECT tab.id_key AS id_key, 'new'::text AS row_type, tab.link AS link
FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key
WHERE e.row_type = 'false'
)
SELECT * FROM first_remaining
UNION ALL SELECT * FROM effect
)
)
SELECT * FROM iter;
0 base 17
1 true 17
2 true 17
3 true 17
4 true 17
5 true 17
6 true 17

View File

@ -0,0 +1,115 @@
/**
* Based on https://github.com/postgres/postgres/blob/master/src/test/regress/sql/with.sql, license:
*
* PostgreSQL Database Management System
* (formerly known as Postgres, then as Postgres95)
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
*
* Portions Copyright (c) 1994, The Regents of the University of California
*
* Permission to use, copy, modify, and distribute this software and its
* documentation for any purpose, without fee, and without a written agreement
* is hereby granted, provided that the above copyright notice and this
* paragraph and the following two paragraphs appear in all copies.
*
* IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
* DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
* LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS
* DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*
* THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
* AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS
* ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO
*PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS.
*/
--
-- Tests for common table expressions (WITH query, ... SELECT ...)
--
-- { echoOn }
SET allow_experimental_analyzer = 1;
WITH RECURSIVE foo AS
(SELECT 1 AS i
UNION ALL
(SELECT i+1 FROM foo WHERE i < 10
UNION ALL
SELECT i+1 FROM foo WHERE i < 5)
) SELECT * FROM foo;
WITH RECURSIVE foo AS
(SELECT 1 AS i
UNION ALL
SELECT * FROM
(SELECT i+1 FROM foo WHERE i < 10
UNION ALL
SELECT i+1 FROM foo WHERE i < 5) AS t
) SELECT * FROM foo;
WITH RECURSIVE foo AS
(SELECT 1 AS i
UNION ALL
(SELECT i+1 FROM foo WHERE i < 10
EXCEPT
SELECT i+1 FROM foo WHERE i < 5)
) SELECT * FROM foo;
WITH RECURSIVE foo AS
(SELECT 1 AS i
UNION ALL
(SELECT i+1 FROM foo WHERE i < 10
INTERSECT
SELECT i+1 FROM foo WHERE i < 5)
) SELECT * FROM foo;
--
-- test for nested-recursive-WITH bug
--
WITH RECURSIVE t AS (
WITH RECURSIVE s AS (
SELECT toUInt64(1) AS i
UNION ALL
SELECT i+1 FROM s WHERE i < 10
)
SELECT i AS j FROM s
UNION ALL
SELECT j+1 FROM t WHERE j < 10
)
SELECT * FROM t;
--
-- Test CTEs read in non-initialization orders
--
WITH RECURSIVE
tab AS (SELECT * FROM values('id_key UInt64, link UInt64', (1,17), (2,17), (3,17), (4,17), (6,17), (5,17))),
iter AS (
SELECT 0 AS id_key, 'base' AS row_type, 17 AS link
UNION ALL (
WITH remaining AS (
SELECT tab.id_key AS id_key, 'true'::text AS row_type, iter.link AS link, MIN(tab.id_key) OVER () AS min
FROM tab INNER JOIN iter USING (link)
WHERE tab.id_key > iter.id_key
),
first_remaining AS (
SELECT id_key, row_type, link
FROM remaining
WHERE id_key=min
),
effect AS (
SELECT tab.id_key AS id_key, 'new'::text AS row_type, tab.link AS link
FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key
WHERE e.row_type = 'false'
)
SELECT * FROM first_remaining
UNION ALL SELECT * FROM effect
)
)
SELECT * FROM iter;
-- { echoOff }

View File

@ -0,0 +1,3 @@
5555 ['moto','hello'] ['chocolatine','croissant'] [3159487597665552601,10142490492830962361] [17245759883905314919,3957103312270590367]
5555 ['hello'] ['croissant'] [10142490492830962361] [3957103312270590367]
5555 ['hello'] ['croissant au chocolat'] [10142490492830962361] [6230099869648002788]

View File

@ -0,0 +1,33 @@
SET asterisk_include_materialized_columns = 1 ;
CREATE TABLE elements
(
`id` UInt32,
`nested.key` Array(String),
`nested.value` Array(String),
`nested.key_hashed` Array(UInt64) MATERIALIZED arrayMap(x -> sipHash64(x), nested.key),
`nested.val_hashed` Array(UInt64) MATERIALIZED arrayMap(x -> sipHash64(x), nested.value),
)
ENGINE = Memory ;
INSERT INTO elements (id,`nested.key`,`nested.value`) VALUES (5555, ['moto', 'hello'],['chocolatine', 'croissant']);
SELECT * FROM elements ;
ALTER TABLE elements
UPDATE
`nested.key` = arrayFilter((x, v) -> NOT (match(v, 'chocolatine')), `nested.key`, `nested.value` ),
`nested.value` = arrayFilter((x, v) -> NOT (match(v, 'chocolatine')), `nested.value`, `nested.value`)
WHERE id = 5555
SETTINGS mutations_sync = 1 ;
SELECT * FROM elements ;
ALTER TABLE elements
UPDATE
`nested.value` = arrayMap(x -> concat(x, ' au chocolat'), `nested.value`)
WHERE id = 5555
SETTINGS mutations_sync = 1 ;
SELECT * FROM elements ;

View File

@ -0,0 +1,225 @@
Expression ((Project names + (Projection + )))
Header: id UInt64
value String
rhs.id UInt64
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 0 2 1
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: INNER
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Expression
Header: __table1.id UInt64
__table1.value String
Actions: INPUT : 1 -> value String : 0
INPUT :: 0 -> __table1.id UInt64 : 1
INPUT :: 2 -> id UInt64 : 2
ALIAS value :: 0 -> __table1.value String : 3
Positions: 1 3
ReadFromMergeTree (default.test_table_1)
Header: __table1.id UInt64
id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Prewhere info
Need filter: 1
Prewhere filter
Prewhere filter column: notEquals(__table1.id, 0_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1
ALIAS id : 0 -> __table1.id UInt64 : 2
FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3
Positions: 2 0 3
Expression
Header: __table2.id UInt64
__table2.value String
Actions: INPUT : 1 -> value String : 0
INPUT :: 0 -> __table2.id UInt64 : 1
INPUT :: 2 -> id UInt64 : 2
ALIAS value :: 0 -> __table2.value String : 3
Positions: 1 3
ReadFromMergeTree (default.test_table_2)
Header: __table2.id UInt64
id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Prewhere info
Need filter: 1
Prewhere filter
Prewhere filter column: notEquals(__table2.id, 0_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1
ALIAS id : 0 -> __table2.id UInt64 : 2
FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3
Positions: 2 0 3
--
2 Value_2 2 Value_2
--
Expression ((Project names + (Projection + )))
Header: id UInt64
value String
rhs.id UInt64
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 0 2 1
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: INNER
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Expression
Header: __table1.id UInt64
__table1.value String
Actions: INPUT : 1 -> value String : 0
INPUT :: 0 -> __table1.id UInt64 : 1
INPUT :: 2 -> id UInt64 : 2
ALIAS value :: 0 -> __table1.value String : 3
Positions: 1 3
ReadFromMergeTree (default.test_table_1)
Header: __table1.id UInt64
id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Prewhere info
Need filter: 1
Prewhere filter
Prewhere filter column: notEquals(__table1.id, 0_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1
ALIAS id : 0 -> __table1.id UInt64 : 2
FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3
Positions: 2 0 3
Expression
Header: __table2.id UInt64
__table2.value String
Actions: INPUT : 1 -> value String : 0
INPUT :: 0 -> __table2.id UInt64 : 1
INPUT :: 2 -> id UInt64 : 2
ALIAS value :: 0 -> __table2.value String : 3
Positions: 1 3
ReadFromMergeTree (default.test_table_2)
Header: __table2.id UInt64
id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Prewhere info
Need filter: 1
Prewhere filter
Prewhere filter column: notEquals(__table2.id, 0_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1
ALIAS id : 0 -> __table2.id UInt64 : 2
FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3
Positions: 2 0 3
--
2 Value_2 2 Value_2
--
Expression ((Project names + (Projection + )))
Header: id UInt64
value String
rhs.id UInt64
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 0 2 1
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: INNER
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Expression
Header: __table1.id UInt64
__table1.value String
Actions: INPUT : 1 -> value String : 0
INPUT :: 0 -> __table1.id UInt64 : 1
INPUT :: 2 -> id UInt64 : 2
ALIAS value :: 0 -> __table1.value String : 3
Positions: 1 3
ReadFromMergeTree (default.test_table_1)
Header: __table1.id UInt64
id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Prewhere info
Need filter: 1
Prewhere filter
Prewhere filter column: and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1
ALIAS id : 0 -> __table1.id UInt64 : 2
FUNCTION notEquals(__table1.id : 2, 0_UInt8 : 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3
FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 4
FUNCTION and(notEquals(__table1.id, 0_UInt8) :: 4, notEquals(__table1.id, 0_UInt8) :: 3) -> and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) UInt8 : 1
Positions: 2 0 1
Expression
Header: __table2.id UInt64
__table2.value String
Actions: INPUT : 1 -> value String : 0
INPUT :: 0 -> __table2.id UInt64 : 1
INPUT :: 2 -> id UInt64 : 2
ALIAS value :: 0 -> __table2.value String : 3
Positions: 1 3
ReadFromMergeTree (default.test_table_2)
Header: __table2.id UInt64
id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Prewhere info
Need filter: 1
Prewhere filter
Prewhere filter column: and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1
ALIAS id : 0 -> __table2.id UInt64 : 2
FUNCTION notEquals(__table2.id : 2, 0_UInt8 : 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3
FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 4
FUNCTION and(notEquals(__table2.id, 0_UInt8) :: 4, notEquals(__table2.id, 0_UInt8) :: 3) -> and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) UInt8 : 1
Positions: 2 0 1
--
2 Value_2 2 Value_2

View File

@ -0,0 +1,44 @@
SET allow_experimental_analyzer = 1;
SET join_algorithm = 'hash';
DROP TABLE IF EXISTS test_table_1;
CREATE TABLE test_table_1
(
id UInt64,
value String
) ENGINE=MergeTree ORDER BY id;
DROP TABLE IF EXISTS test_table_2;
CREATE TABLE test_table_2
(
id UInt64,
value String
) ENGINE=MergeTree ORDER BY id;
INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2');
INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3');
EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0;
SELECT '--';
SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0;
SELECT '--';
EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0;
SELECT '--';
SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0;
SELECT '--';
EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0;
SELECT '--';
SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0;
DROP TABLE test_table_1;
DROP TABLE test_table_2;

View File

@ -0,0 +1,6 @@
-- { echoOn }
select table, errorCodeToName(error), count() from system.part_log where database = currentDatabase() and error > 0 and errorCodeToName(error) not in ('FAULT_INJECTED', 'NO_REPLICA_HAS_PART', 'ATTEMPT_TO_READ_AFTER_EOF') group by 1, 2 order by 1, 2;
select count() from data_r1;
100000
select count() from data_r2;
100000

View File

@ -0,0 +1,19 @@
-- Tags: no-parallel
-- Tag no-parallel -- due to failpoints
create table data_r1 (key Int, value String) engine=ReplicatedMergeTree('/tables/{database}/data', '{table}') order by tuple();
create table data_r2 (key Int, value String) engine=ReplicatedMergeTree('/tables/{database}/data', '{table}') order by tuple();
system enable failpoint replicated_sends_failpoint;
insert into data_r1 select number, randomPrintableASCII(100) from numbers(100_000) settings max_block_size=1000, min_insert_block_size_rows=1000;
system disable failpoint replicated_sends_failpoint;
system sync replica data_r2;
system flush logs;
select event_time_microseconds, logger_name, message from system.text_log where level = 'Error' and message like '%Malformed chunked encoding%' order by 1 format LineAsString;
-- { echoOn }
select table, errorCodeToName(error), count() from system.part_log where database = currentDatabase() and error > 0 and errorCodeToName(error) not in ('FAULT_INJECTED', 'NO_REPLICA_HAS_PART', 'ATTEMPT_TO_READ_AFTER_EOF') group by 1, 2 order by 1, 2;
select count() from data_r1;
select count() from data_r2;