Fixed tests

This commit is contained in:
Maksim Kita 2023-03-05 21:45:53 +01:00
parent 0358cb36d8
commit 3d34247998
32 changed files with 277 additions and 146 deletions

View File

@ -69,8 +69,7 @@ private:
for (auto it = function_arguments.rbegin(); it != function_arguments.rend(); ++it)
candidates.push_back({ *it, is_deterministic });
// Using DFS we traverse function tree and try to find if it uses other keys as function arguments.
// TODO: Also process CONSTANT here. We can simplify GROUP BY x, x + 1 to GROUP BY x.
/// Using DFS we traverse function tree and try to find if it uses other keys as function arguments.
while (!candidates.empty())
{
auto [candidate, parents_are_only_deterministic] = candidates.back();
@ -108,6 +107,7 @@ private:
return false;
}
}
return true;
}

View File

@ -193,13 +193,9 @@ namespace ErrorCodes
* lookup should not be continued, and exception must be thrown because if lookup continues identifier can be resolved from parent scope.
*
* TODO: Update exception messages
* TODO: JOIN TREE subquery constant columns
* TODO: Table identifiers with optional UUID.
* TODO: Lookup functions arrayReduce(sum, [1, 2, 3]);
* TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level.
* TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level.
* TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns.
* TODO: Scalar subqueries cache.
*/
namespace
@ -1336,6 +1332,9 @@ private:
/// Global resolve expression node to projection names map
std::unordered_map<QueryTreeNodePtr, ProjectionNames> resolved_expressions;
/// Global resolve expression node to tree size
std::unordered_map<QueryTreeNodePtr, size_t> node_to_tree_size;
/// Global scalar subquery to scalar value map
std::unordered_map<QueryTreeNodePtrWithHash, Block> scalar_subquery_to_scalar_value;
@ -1864,7 +1863,10 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
Block scalar_block;
QueryTreeNodePtrWithHash node_with_hash(node);
auto node_without_alias = node->clone();
node_without_alias->removeAlias();
QueryTreeNodePtrWithHash node_with_hash(node_without_alias);
auto scalar_value_it = scalar_subquery_to_scalar_value.find(node_with_hash);
if (scalar_value_it != scalar_subquery_to_scalar_value.end())
@ -2334,7 +2336,13 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con
storage_id = context->resolveStorageID(storage_id);
bool is_temporary_table = storage_id.getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE;
auto storage = DatabaseCatalog::instance().tryGetTable(storage_id, context);
StoragePtr storage;
if (is_temporary_table)
storage = DatabaseCatalog::instance().getTable(storage_id, context);
else
storage = DatabaseCatalog::instance().tryGetTable(storage_id, context);
if (!storage)
return {};
@ -3007,11 +3015,39 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo
resolved_identifier = std::move(result_column_node);
}
else if (scope.joins_count == 1 && scope.context->getSettingsRef().single_join_prefer_left_table)
else if (left_resolved_identifier->isEqual(*right_resolved_identifier, IQueryTreeNode::CompareOptions{.compare_aliases = false}))
{
const auto & identifier_path_part = identifier_lookup.identifier.front();
auto * left_resolved_identifier_column = left_resolved_identifier->as<ColumnNode>();
auto * right_resolved_identifier_column = right_resolved_identifier->as<ColumnNode>();
if (left_resolved_identifier_column && right_resolved_identifier_column)
{
const auto & left_column_source_alias = left_resolved_identifier_column->getColumnSource()->getAlias();
const auto & right_column_source_alias = right_resolved_identifier_column->getColumnSource()->getAlias();
/** If column from right table was resolved using alias, we prefer column from right table.
*
* Example: SELECT dummy FROM system.one JOIN system.one AS A ON A.dummy = system.one.dummy;
*
* If alias is specified for left table, and alias is not specified for right table and identifier was resolved
* without using left table alias, we prefer column from right table.
*
* Example: SELECT dummy FROM system.one AS A JOIN system.one ON A.dummy = system.one.dummy;
*
* Otherwise we prefer column from left table.
*/
if (identifier_path_part == right_column_source_alias)
return right_resolved_identifier;
else if (!left_column_source_alias.empty() &&
right_column_source_alias.empty() &&
identifier_path_part != left_column_source_alias)
return right_resolved_identifier;
}
return left_resolved_identifier;
}
else if (left_resolved_identifier->isEqual(*right_resolved_identifier, IQueryTreeNode::CompareOptions{.compare_aliases = false}))
else if (scope.joins_count == 1 && scope.context->getSettingsRef().single_join_prefer_left_table)
{
return left_resolved_identifier;
}
@ -4455,6 +4491,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
bool is_special_function_dict_get = false;
bool is_special_function_join_get = false;
bool is_special_function_exists = false;
bool is_special_function_if = false;
if (!lambda_expression_untyped)
{
@ -4462,6 +4499,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
is_special_function_dict_get = functionIsDictGet(function_name);
is_special_function_join_get = functionIsJoinGet(function_name);
is_special_function_exists = function_name == "exists";
is_special_function_if = function_name == "if";
auto function_name_lowercase = Poco::toLower(function_name);
@ -4560,6 +4598,38 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
is_special_function_in = true;
}
if (is_special_function_if && !function_node_ptr->getArguments().getNodes().empty())
{
/** Handle special case with constant If function, even if some of the arguments are invalid.
*
* SELECT if(hasColumnInTable('system', 'numbers', 'not_existing_column'), not_existing_column, 5) FROM system.numbers;
*/
auto & if_function_arguments = function_node_ptr->getArguments().getNodes();
auto if_function_condition = if_function_arguments[0];
resolveExpressionNode(if_function_condition, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
QueryTreeNodePtr constant_if_result_node;
auto constant_condition = tryExtractConstantFromConditionNode(if_function_condition);
if (constant_condition.has_value() && if_function_arguments.size() == 3)
{
if (*constant_condition)
constant_if_result_node = if_function_arguments[1];
else
constant_if_result_node = if_function_arguments[2];
}
if (constant_if_result_node)
{
auto result_projection_names = resolveExpressionNode(constant_if_result_node,
scope,
false /*allow_lambda_expression*/,
false /*allow_table_expression*/);
node = std::move(constant_if_result_node);
return result_projection_names;
}
}
/// Resolve function arguments
bool allow_table_expressions = is_special_function_in;
@ -5422,9 +5492,9 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id
}
}
if (node
&& scope.nullable_group_by_keys.contains(node)
&& !scope.expressions_in_resolve_process_stack.hasAggregateFunction())
validateTreeSize(node, scope.context->getSettingsRef().max_expanded_ast_elements, node_to_tree_size);
if (scope.nullable_group_by_keys.contains(node) && !scope.expressions_in_resolve_process_stack.hasAggregateFunction())
{
node = node->clone();
node->convertToNullable();
@ -6746,6 +6816,15 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
validateAggregates(query_node, { .group_by_use_nulls = scope.group_by_use_nulls });
for (const auto & column : projection_columns)
{
if (isNotCreatable(column.type->getTypeId()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Invalid projection column with type {}. In scope {}",
column.type->getName(),
scope.scope_node->formatASTForErrorMessage());
}
/** WITH section can be safely removed, because WITH section only can provide aliases to query expressions
* and CTE for other sections to use.
*

View File

@ -8,6 +8,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
@ -172,6 +173,27 @@ QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression,
return cast_function_node;
}
std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node)
{
const auto * constant_node = condition_node->as<ConstantNode>();
if (!constant_node)
return {};
const auto & value = constant_node->getValue();
auto constant_type = constant_node->getResultType();
constant_type = removeNullable(removeLowCardinality(constant_type));
auto which_constant_type = WhichDataType(constant_type);
if (!which_constant_type.isUInt8() && !which_constant_type.isNothing())
return {};
if (value.isNull())
return false;
UInt8 predicate_value = value.safeGet<UInt8>();
return predicate_value > 0;
}
static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expression_node)
{
ASTPtr table_expression_node_ast;

View File

@ -19,7 +19,7 @@ bool isNameOfLocalInFunction(const std::string & function_name);
/// Returns true if function name is name of global IN function or its variations, false otherwise
bool isNameOfGlobalInFunction(const std::string & function_name);
/// Returns global in function name for local in function name
/// Returns global IN function name for local IN function name
std::string getGlobalInFunctionNameForLocalInFunctionName(const std::string & function_name);
/// Add unique suffix to names of duplicate columns in block
@ -34,6 +34,9 @@ QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression,
const ContextPtr & context,
bool resolve = true);
/// Try extract boolean constant from condition node
std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node);
/** Add table expression in tables in select query children.
* If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception.
*/

View File

@ -16,6 +16,7 @@ namespace ErrorCodes
{
extern const int NOT_AN_AGGREGATE;
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
}
class ValidateGroupByColumnsVisitor : public ConstInDepthQueryTreeVisitor<ValidateGroupByColumnsVisitor>
@ -283,4 +284,52 @@ void assertNoFunctionNodes(const QueryTreeNodePtr & node,
visitor.visit(node);
}
void validateTreeSize(const QueryTreeNodePtr & node,
size_t max_size,
std::unordered_map<QueryTreeNodePtr, size_t> & node_to_tree_size)
{
size_t tree_size = 0;
std::vector<std::pair<QueryTreeNodePtr, bool>> nodes_to_process;
nodes_to_process.emplace_back(node, false);
while (!nodes_to_process.empty())
{
const auto [node_to_process, processed_children] = nodes_to_process.back();
nodes_to_process.pop_back();
if (processed_children)
{
++tree_size;
node_to_tree_size.emplace(node_to_process, tree_size);
continue;
}
auto node_to_size_it = node_to_tree_size.find(node_to_process);
if (node_to_size_it != node_to_tree_size.end())
{
tree_size += node_to_size_it->second;
continue;
}
nodes_to_process.emplace_back(node_to_process, true);
for (const auto & node_to_process_child : node_to_process->getChildren())
{
if (!node_to_process_child)
continue;
nodes_to_process.emplace_back(node_to_process_child, false);
}
auto * constant_node = node_to_process->as<ConstantNode>();
if (constant_node && constant_node->hasSourceExpression())
nodes_to_process.emplace_back(constant_node->getSourceExpression(), false);
}
if (tree_size > max_size)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Query tree is too big. Maximum: {}",
max_size);
}
}

View File

@ -31,4 +31,11 @@ void assertNoFunctionNodes(const QueryTreeNodePtr & node,
std::string_view exception_function_name,
std::string_view exception_place_message);
/** Validate tree size. If size of tree is greater than max size throws exception.
* Additionally for each node in tree, update node to tree size map.
*/
void validateTreeSize(const QueryTreeNodePtr & node,
size_t max_size,
std::unordered_map<QueryTreeNodePtr, size_t> & node_to_tree_size);
}

View File

@ -38,6 +38,8 @@ public:
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool isDeterministic() const override { return false; }
bool isDeterministicInScopeOfQuery() const override { return false; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{

View File

@ -18,6 +18,7 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Analyzer/Utils.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/TableNode.h>
@ -61,6 +62,8 @@ void JoinClause::dump(WriteBuffer & buffer) const
for (const auto & dag_node : dag_nodes)
{
dag_nodes_dump += dag_node->result_name;
dag_nodes_dump += " ";
dag_nodes_dump += dag_node->result_type->getName();
dag_nodes_dump += ", ";
}

View File

@ -343,27 +343,6 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con
return function_node;
}
std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node)
{
const auto * constant_node = condition_node->as<ConstantNode>();
if (!constant_node)
return {};
const auto & value = constant_node->getValue();
auto constant_type = constant_node->getResultType();
constant_type = removeNullable(removeLowCardinality(constant_type));
auto which_constant_type = WhichDataType(constant_type);
if (!which_constant_type.isUInt8() && !which_constant_type.isNothing())
return {};
if (value.isNull())
return false;
UInt8 predicate_value = value.safeGet<UInt8>();
return predicate_value > 0;
}
QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node,
const ContextPtr & context,
ResultReplacementMap * result_replacement_map)

View File

@ -63,9 +63,6 @@ bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_no
/// Returns `and` function node that has condition nodes as its arguments
QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, const ContextPtr & context);
/// Try extract boolean constant from condition node
std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node);
/// Replace tables nodes and table function nodes with dummy table nodes
using ResultReplacementMap = std::unordered_map<QueryTreeNodePtr, QueryTreeNodePtr>;
QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node,

View File

@ -201,6 +201,7 @@ MergeTreeConditionInverted::MergeTreeConditionInverted(
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
rpn = std::move(
RPNBuilder<RPNElement>(
query_info.filter_actions_dag->getOutputs().at(0), context_,
@ -208,10 +209,10 @@ MergeTreeConditionInverted::MergeTreeConditionInverted(
{
return this->traverseAtomAST(node, out);
}).extractRPN());
return;
}
ASTPtr filter_node = buildFilterNode(query_info.query);
if (!filter_node)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
@ -226,7 +227,6 @@ MergeTreeConditionInverted::MergeTreeConditionInverted(
query_info.prepared_sets,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return traverseAtomAST(node, out); });
rpn = std::move(builder).extractRPN();
}
/// Keep in-sync with MergeTreeConditionFullText::alwaysUnknownOrTrue

View File

@ -10,13 +10,13 @@ l \N \N String Nullable(String)
\N \N
\N \N
using
l \N String Nullable(String)
\N String Nullable(String)
l \N String Nullable(String)
l \N Nullable(String) Nullable(String)
l \N Nullable(String) Nullable(String)
\N \N Nullable(String) Nullable(String)
\N \N Nullable(String) Nullable(String)
l \N Nullable(String) Nullable(String)
l \N Nullable(String) Nullable(String)
\N \N Nullable(String) Nullable(String)
l \N String Nullable(String)
\N String Nullable(String)
l \N String Nullable(String)
\N \N Nullable(String) Nullable(String)
\N \N
\N \N
@ -32,13 +32,13 @@ l \N \N Nullable(String) Nullable(String)
\N \N
\N \N
using + join_use_nulls
l \N String Nullable(String)
l \N Nullable(String) Nullable(String)
\N \N Nullable(String) Nullable(String)
\N \N Nullable(String) Nullable(String)
l \N String Nullable(String)
l \N Nullable(String) Nullable(String)
\N \N Nullable(String) Nullable(String)
\N \N Nullable(String) Nullable(String)
r \N Nullable(String) Nullable(String)
r \N Nullable(String) Nullable(String)
l \N Nullable(String) Nullable(String)
l \N Nullable(String) Nullable(String)
r \N Nullable(String) Nullable(String)
r \N Nullable(String) Nullable(String)
\N \N
\N \N

View File

@ -1,4 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS t1_00848;
DROP TABLE IF EXISTS t2_00848;
@ -53,16 +54,16 @@ SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 LEFT JOIN t3_00848 t3 ON t1
SELECT 'using + join_use_nulls';
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 ANY LEFT JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id;
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 ANY FULL JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id;
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2_00848 t2 ANY FULL JOIN t3_00848 t3 USING(id) ORDER BY t2.id, t3.id;
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 ANY LEFT JOIN t3_00848 t3 USING(id) ORDER BY id;
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 ANY FULL JOIN t3_00848 t3 USING(id) ORDER BY id;
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2_00848 t2 ANY FULL JOIN t3_00848 t3 USING(id) ORDER BY id;
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 LEFT JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id;
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 FULL JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id;
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2_00848 t2 FULL JOIN t3_00848 t3 USING(id) ORDER BY t2.id, t3.id;
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 LEFT JOIN t3_00848 t3 USING(id) ORDER BY id;
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 FULL JOIN t3_00848 t3 USING(id) ORDER BY id;
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2_00848 t2 FULL JOIN t3_00848 t3 USING(id) ORDER BY id;
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 ANY LEFT JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id;
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 LEFT JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id;
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 ANY LEFT JOIN t3_00848 t3 USING(id) ORDER BY id;
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 LEFT JOIN t3_00848 t3 USING(id) ORDER BY id;
DROP TABLE t1_00848;
DROP TABLE t2_00848;

View File

@ -1 +1,3 @@
SELECT 1 AS a, a + a AS b, b + b AS c, c + c AS d, d + d AS e, e + e AS f, f + f AS g, g + g AS h, h + h AS i, i + i AS j, j + j AS k, k + k AS l, l + l AS m, m + m AS n, n + n AS o, o + o AS p, p + p AS q, q + q AS r, r + r AS s, s + s AS t, t + t AS u, u + u AS v, v + v AS w, w + w AS x, x + x AS y, y + y AS z; -- { serverError 168 }
SET allow_experimental_analyzer = 1;
SELECT 1 AS a, a + a AS b, b + b AS c, c + c AS d, d + d AS e, e + e AS f, f + f AS g, g + g AS h, h + h AS i, i + i AS j, j + j AS k, k + k AS l, l + l AS m, m + m AS n, n + n AS o, o + o AS p, p + p AS q, q + q AS r, r + r AS s, s + s AS t, t + t AS u, u + u AS v, v + v AS w, w + w AS x, x + x AS y, y + y AS z; -- { serverError 36 }

View File

@ -1,12 +1,15 @@
0 0
0 0
0 0
0
0
0
0
┌─one.dummy─┬─A.dummy─┬─B.dummy─┐
│ 0 │ 0 │ 0 │
└───────────┴─────────┴─────────┘
0
0
┌─system.one.dummy─┬─A.dummy─┬─B.dummy─┐
│ 0 │ 0 │ 0 │
└──────────────────┴─────────┴─────────┘
┌─A.dummy─┬─one.dummy─┬─two.dummy─┐
│ 0 │ 0 │ 0 │
└─────────┴───────────┴───────────┘

View File

@ -1,4 +1,6 @@
select * from system.one cross join system.one; -- { serverError 352 }
SET allow_experimental_analyzer = 1;
select * from system.one cross join system.one;
select * from system.one cross join system.one r;
select * from system.one l cross join system.one;
select * from system.one left join system.one using dummy;
@ -8,10 +10,10 @@ USE system;
SELECT dummy FROM one AS A JOIN one ON A.dummy = one.dummy;
SELECT dummy FROM one JOIN one AS A ON A.dummy = one.dummy;
SELECT dummy FROM one l JOIN one r ON dummy = r.dummy; -- { serverError 352 }
SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; -- { serverError 352 }
SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy; -- { serverError 352 }
SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; -- { serverError 352 }
SELECT dummy FROM one l JOIN one r ON dummy = r.dummy;
SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; -- { serverError 403 }
SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy;
SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; -- { serverError 403 }
SELECT * from one
JOIN one A ON one.dummy = A.dummy

View File

@ -1,5 +1,5 @@
122
Table dictdb_01041_01040.dict_invalidate doesn\'t exist
1
133

View File

@ -53,7 +53,7 @@ function check_exception_detected()
export -f check_exception_detected;
timeout 30 bash -c check_exception_detected 2> /dev/null
$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1 | grep -Eo "Table dictdb_01041_01040.dict_invalidate .* exist"
$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1 | grep -Eo "dictdb_01041_01040.dict_invalidate.*UNKNOWN_TABLE" | wc -l
$CLICKHOUSE_CLIENT --query "
CREATE TABLE dictdb_01041_01040.dict_invalidate

View File

@ -2,13 +2,13 @@ execute: default
"foo"
1
execute: --stage fetch_columns
"dummy"
"system.one.dummy_0"
0
execute: --stage with_mergeable_state
"1"
"1_UInt8"
1
execute: --stage with_mergeable_state_after_aggregation
"1"
"1_UInt8"
1
execute: --stage complete
"foo"

View File

@ -915,12 +915,12 @@ from
(select number, intDiv(number, 3) p, mod(number, 5) o
from numbers(16)) t
;
Expression ((Projection + Before ORDER BY))
Expression ((Project names + Projection))
Window (Window step for window \'\')
Window (Window step for window \'PARTITION BY p\')
Window (Window step for window \'PARTITION BY p ORDER BY o ASC\')
Sorting (Sorting for window \'PARTITION BY p ORDER BY o ASC\')
Expression ((Before window functions + (Projection + Before ORDER BY)))
Window (Window step for window \'PARTITION BY t.p_0\')
Window (Window step for window \'PARTITION BY t.p_0 ORDER BY t.o_1 ASC\')
Sorting (Sorting for window \'PARTITION BY t.p_0 ORDER BY t.o_1 ASC\')
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
ReadFromStorage (SystemNumbers)
explain select
count(*) over (order by o, number),
@ -929,13 +929,13 @@ from
(select number, intDiv(number, 3) p, mod(number, 5) o
from numbers(16)) t
;
Expression ((Projection + Before ORDER BY))
Window (Window step for window \'ORDER BY o ASC, number ASC\')
Sorting (Sorting for window \'ORDER BY o ASC, number ASC\')
Window (Window step for window \'ORDER BY number ASC\')
Expression ((Before window functions + (Projection + Before ORDER BY)) [lifted up part])
Sorting (Sorting for window \'ORDER BY number ASC\')
Expression ((Before window functions + (Projection + Before ORDER BY)))
Expression ((Project names + Projection))
Window (Window step for window \'ORDER BY t.o_0 ASC, t.number_1 ASC\')
Sorting (Sorting for window \'ORDER BY t.o_0 ASC, t.number_1 ASC\')
Window (Window step for window \'ORDER BY t.number_1 ASC\')
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part])
Sorting (Sorting for window \'ORDER BY t.number_1 ASC\')
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
ReadFromStorage (SystemNumbers)
-- A test case for the sort comparator found by fuzzer.
SELECT

View File

@ -37,63 +37,59 @@
"Node Type": "Aggregating",
"Header": [
{
"Name": "number",
"Name": "number_0",
"Type": "UInt64"
},
{
"Name": "plus(number, 1)",
"Type": "UInt64"
},
{
"Name": "quantile(0.2)(number)",
"Name": "quantile(0.2_Float64)(number_0)",
"Type": "Float64"
},
{
"Name": "sumIf(number, greater(number, 0))",
"Name": "sumIf(number_0, greater(number_0, 0_UInt8))",
"Type": "UInt64"
}
],
"Keys": ["number", "plus(number, 1)"],
"Keys": ["number_0"],
"Aggregates": [
{
"Name": "quantile(0.2)(number)",
"Name": "quantile(0.2_Float64)(number_0)",
"Function": {
"Name": "quantile",
"Parameters": ["0.2"],
"Argument Types": ["UInt64"],
"Result Type": "Float64"
},
"Arguments": ["number"]
"Arguments": ["number_0"]
},
{
"Name": "sumIf(number, greater(number, 0))",
"Name": "sumIf(number_0, greater(number_0, 0_UInt8))",
"Function": {
"Name": "sumIf",
"Argument Types": ["UInt64", "UInt8"],
"Result Type": "UInt64"
},
"Arguments": ["number", "greater(number, 0)"]
"Arguments": ["number_0", "greater(number_0, 0_UInt8)"]
}
],
--------
"Node Type": "ArrayJoin",
"Left": false,
"Columns": ["x", "y"],
"Columns": ["x_0", "y_1"],
--------
"Node Type": "Distinct",
"Columns": ["intDiv(number, 3)", "intDiv(number, 2)"],
"Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"],
--
"Node Type": "Distinct",
"Columns": ["intDiv(number, 3)", "intDiv(number, 2)"],
"Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"],
--------
"Sort Description": [
{
"Column": "number",
"Column": "number_0",
"Ascending": false,
"With Fill": false
},
{
"Column": "plus(number, 1)",
"Column": "plus(number_0, 1_UInt8)",
"Ascending": true,
"With Fill": false
}

View File

@ -5,26 +5,29 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, description = 0 SELECT 1 UNION ALL SELECT 2 FORMAT TSVRaw"
opts=(
"--allow_experimental_analyzer=1"
)
$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, description = 0 SELECT 1 UNION ALL SELECT 2 FORMAT TSVRaw"
echo "--------"
$CLICKHOUSE_CLIENT -q "explain json = 1, description = 0, header = 1 select 1, 2 + dummy FORMAT TSVRaw" 2> /dev/null | grep Header -m 1 -A 8
$CLICKHOUSE_CLIENT "${opts[@]}" -q "explain json = 1, description = 0, header = 1 select 1, 2 + dummy FORMAT TSVRaw" 2> /dev/null | grep Header -m 1 -A 8
echo "--------"
$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, actions = 1, header = 1, description = 0
$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, actions = 1, header = 1, description = 0
SELECT quantile(0.2)(number), sumIf(number, number > 0) from numbers(2) group by number, number + 1 FORMAT TSVRaw
" | grep Aggregating -A 40
" | grep Aggregating -A 36
echo "--------"
$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, actions = 1, description = 0
$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, actions = 1, description = 0
SELECT x, y from numbers(2) array join [number, 1] as x, [number + 1] as y FORMAT TSVRaw
" | grep ArrayJoin -A 2
echo "--------"
$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, actions = 1, description = 0
$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, actions = 1, description = 0
SELECT distinct intDiv(number, 2), intDiv(number, 3) from numbers(10) FORMAT TSVRaw
" | grep Distinct -A 1
echo "--------"
$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, actions = 1, description = 0
$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, actions = 1, description = 0
SELECT number + 1 from numbers(10) order by number desc, number + 1 limit 3 FORMAT TSVRaw
" | grep "Sort Description" -A 12

View File

@ -6,9 +6,9 @@ insert into test values (0);
select if(0, y, 42) from test;
select if(1, 42, y) from test;
select if(toUInt8(0), y, 42) from test;
select if(toInt8(0), y, 42) from test;
select if(toUInt8(0), y, 42) from test;
select if(toUInt8(1), 42, y) from test;
select if(toUInt8(1), 42, y) from test;
select if(toInt8(1), 42, y) from test;
select if(toUInt8(toUInt8(0)), y, 42) from test;
select if(cast(cast(0, 'UInt8'), 'UInt8'), y, 42) from test;
explain syntax select x, if((select hasColumnInTable(currentDatabase(), 'test', 'y')), y, x || '_') from test;

View File

@ -7,5 +7,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# Should finish in reasonable time (milliseconds).
# In previous versions this query led to exponential complexity of query analysis.
${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple((1, untuple((untuple(tuple(untuple(tuple(untuple((untuple((1, 1, 1, 1)), 1, 1, 1)))))), 1, 1))))))" 2>&1 | grep -cF 'TOO_BIG_AST'
${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple((1, 1, 1, 1, 1))))))))))))))))))))))))))" 2>&1 | grep -cF 'TOO_BIG_AST'
${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple((1, untuple((untuple(tuple(untuple(tuple(untuple((untuple((1, 1, 1, 1)), 1, 1, 1)))))), 1, 1))))))" 2>&1 | grep -cF 'too big'
${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple((1, 1, 1, 1, 1))))))))))))))))))))))))))" 2>&1 | grep -cF 'too big'

View File

@ -1,3 +1,3 @@
02177_CTE_GLOBAL_ON 5 500 11 0 5
02177_CTE_GLOBAL_OFF 1 100 5 0 1
02177_CTE_NEW_ANALYZER 2 200 3 0 2
02177_CTE_GLOBAL_ON 1 100 4 0 1
02177_CTE_GLOBAL_OFF 1 100 4 0 1
02177_CTE_NEW_ANALYZER 1 100 4 0 1

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
WITH
( SELECT sleep(0.0001) FROM system.one ) as a1,
( SELECT sleep(0.0001) FROM system.one ) as a2,

View File

@ -30,7 +30,7 @@ SELECT
FORMAT Vertical;
Row 1:
──────
cutURLParameter('http://bigmir.net/?a=b&c=d', []): http://bigmir.net/?a=b&c=d
cutURLParameter('http://bigmir.net/?a=b&c=d', array()): http://bigmir.net/?a=b&c=d
cutURLParameter('http://bigmir.net/?a=b&c=d', ['a']): http://bigmir.net/?c=d
cutURLParameter('http://bigmir.net/?a=b&c=d', ['a', 'c']): http://bigmir.net/?
cutURLParameter('http://bigmir.net/?a=b&c=d', ['c']): http://bigmir.net/?a=b
@ -43,7 +43,7 @@ cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', ['c', 'g']): http:
cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', ['e', 'g']): http://bigmir.net/?a=b&c=d#e
cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', ['test', 'e']): http://bigmir.net/?a=b&c=d#test?g=h
cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', ['test', 'g']): http://bigmir.net/?a=b&c=d#test?e=f
cutURLParameter('//bigmir.net/?a=b&c=d', []): //bigmir.net/?a=b&c=d
cutURLParameter('//bigmir.net/?a=b&c=d', array()): //bigmir.net/?a=b&c=d
cutURLParameter('//bigmir.net/?a=b&c=d', ['a']): //bigmir.net/?c=d
cutURLParameter('//bigmir.net/?a=b&c=d', ['a', 'c']): //bigmir.net/?
cutURLParameter('//bigmir.net/?a=b&c=d#e=f', ['a', 'e']): //bigmir.net/?c=d#
@ -88,7 +88,7 @@ SELECT
FORMAT Vertical;
Row 1:
──────
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), []): http://bigmir.net/?a=b&c=d
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), array()): http://bigmir.net/?a=b&c=d
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['a']): http://bigmir.net/?c=d
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['a', 'c']): http://bigmir.net/?
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['c']): http://bigmir.net/?a=b
@ -101,7 +101,7 @@ cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), ['c', 'g']):
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), ['e', 'g']): http://bigmir.net/?a=b&c=d#e
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), ['test', 'e']): http://bigmir.net/?a=b&c=d#test?g=h
cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), ['test', 'g']): http://bigmir.net/?a=b&c=d#test?e=f
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), []): //bigmir.net/?a=b&c=d
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), array()): //bigmir.net/?a=b&c=d
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), ['a']): //bigmir.net/?c=d
cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), ['a', 'c']): //bigmir.net/?
cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e=f'), ['a', 'e']): //bigmir.net/?c=d#

View File

@ -1,4 +1,7 @@
SET allow_experimental_analyzer = 1;
-- { echoOn }
SELECT
cutURLParameter('http://bigmir.net/?a=b&c=d', []),
cutURLParameter('http://bigmir.net/?a=b&c=d', ['a']),

View File

@ -1,9 +1,9 @@
1
1
Expression ((Projection + Before ORDER BY))
Expression ((Project names + (Projection + Change column names to column identifiers)))
Limit (preliminary LIMIT (without OFFSET))
ReadFromStorage (SystemNumbers)
Expression ((Projection + Before ORDER BY))
Expression ((Project names + (Projection + Change column names to column identifiers)))
Limit (preliminary LIMIT (without OFFSET))
ReadFromStorage (SystemNumbers)
(Expression)

View File

@ -1,6 +1,7 @@
-- Tags: no-parallel
-- Tag no-parallel: Messes with internal cache
SET allow_experimental_analyzer = 1;
SET allow_experimental_query_cache = true;
SYSTEM DROP QUERY CACHE;

View File

@ -6,10 +6,8 @@ true
=====
true
=====
=====
1
=====
=====
allow_experimental_analyzer
true
#45440

View File

@ -42,31 +42,10 @@ SETTINGS enable_optimize_predicate_expression = 0;
SELECT '=====';
SELECT toBool(sin(SUM(number))) AS x
FROM
(
SELECT 1 AS number
)
GROUP BY number
HAVING 1 AND sin(sum(number))
SETTINGS enable_optimize_predicate_expression = 1; -- { serverError 59 }
SELECT '=====';
SELECT 1 and sin(1);
SELECT '=====';
SELECT toBool(sin(SUM(number))) AS x
FROM
(
SELECT 1 AS number
)
GROUP BY number
HAVING x AND sin(1)
SETTINGS enable_optimize_predicate_expression = 0; -- { serverError 59 }
SELECT '=====';
SELECT 'allow_experimental_analyzer';
SET allow_experimental_analyzer = 1;