mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' into pretty-type-names-default
This commit is contained in:
commit
10ebe99fed
@ -22,7 +22,7 @@
|
||||
* The MergeTree setting `clean_deleted_rows` is deprecated, it has no effect anymore. The `CLEANUP` keyword for the `OPTIMIZE` is not allowed by default (it can be unlocked with the `allow_experimental_replacing_merge_with_cleanup` setting). [#58267](https://github.com/ClickHouse/ClickHouse/pull/58267) ([Alexander Tokmakov](https://github.com/tavplubix)). This fixes [#57930](https://github.com/ClickHouse/ClickHouse/issues/57930). This closes [#54988](https://github.com/ClickHouse/ClickHouse/issues/54988). This closes [#54570](https://github.com/ClickHouse/ClickHouse/issues/54570). This closes [#50346](https://github.com/ClickHouse/ClickHouse/issues/50346). This closes [#47579](https://github.com/ClickHouse/ClickHouse/issues/47579). The feature has to be removed because it is not good. We have to remove it as quickly as possible, because there is no other option. [#57932](https://github.com/ClickHouse/ClickHouse/pull/57932) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
|
||||
#### New Feature
|
||||
* Implement Refreshable Materialized Views, requested in [#33919](https://github.com/ClickHouse/ClickHouse/issues/57995). [#56946](https://github.com/ClickHouse/ClickHouse/pull/56946) ([Michael Kolupaev](https://github.com/al13n321), [Michael Guzov](https://github.com/koloshmet)).
|
||||
* Implement Refreshable Materialized Views, requested in [#33919](https://github.com/ClickHouse/ClickHouse/issues/33919). [#56946](https://github.com/ClickHouse/ClickHouse/pull/56946) ([Michael Kolupaev](https://github.com/al13n321), [Michael Guzov](https://github.com/koloshmet)).
|
||||
* Introduce `PASTE JOIN`, which allows users to join tables without `ON` clause simply by row numbers. Example: `SELECT * FROM (SELECT number AS a FROM numbers(2)) AS t1 PASTE JOIN (SELECT number AS a FROM numbers(2) ORDER BY a DESC) AS t2`. [#57995](https://github.com/ClickHouse/ClickHouse/pull/57995) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
|
||||
* The `ORDER BY` clause now supports specifying `ALL`, meaning that ClickHouse sorts by all columns in the `SELECT` clause. Example: `SELECT col1, col2 FROM tab WHERE [...] ORDER BY ALL`. [#57875](https://github.com/ClickHouse/ClickHouse/pull/57875) ([zhongyuankai](https://github.com/zhongyuankai)).
|
||||
* Added a new mutation command `ALTER TABLE <table> APPLY DELETED MASK`, which allows to enforce applying of mask written by lightweight delete and to remove rows marked as deleted from disk. [#57433](https://github.com/ClickHouse/ClickHouse/pull/57433) ([Anton Popov](https://github.com/CurtizJ)).
|
||||
|
@ -1262,6 +1262,7 @@ SELECT * FROM json_each_row_nested
|
||||
|
||||
- [input_format_import_nested_json](/docs/en/operations/settings/settings-formats.md/#input_format_import_nested_json) - map nested JSON data to nested tables (it works for JSONEachRow format). Default value - `false`.
|
||||
- [input_format_json_read_bools_as_numbers](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`.
|
||||
- [input_format_json_read_bools_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_bools_as_strings) - allow to parse bools as strings in JSON input formats. Default value - `true`.
|
||||
- [input_format_json_read_numbers_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `true`.
|
||||
- [input_format_json_read_arrays_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_arrays_as_strings) - allow to parse JSON arrays as strings in JSON input formats. Default value - `true`.
|
||||
- [input_format_json_read_objects_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `true`.
|
||||
|
@ -614,6 +614,26 @@ DESC format(JSONEachRow, $$
|
||||
└───────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
|
||||
```
|
||||
|
||||
##### input_format_json_read_bools_as_strings
|
||||
|
||||
Enabling this setting allows reading Bool values as strings.
|
||||
|
||||
This setting is enabled by default.
|
||||
|
||||
**Example:**
|
||||
|
||||
```sql
|
||||
SET input_format_json_read_bools_as_strings = 1;
|
||||
DESC format(JSONEachRow, $$
|
||||
{"value" : true}
|
||||
{"value" : "Hello, World"}
|
||||
$$)
|
||||
```
|
||||
```response
|
||||
┌─name──┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
|
||||
│ value │ Nullable(String) │ │ │ │ │ │
|
||||
└───────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
|
||||
```
|
||||
##### input_format_json_read_arrays_as_strings
|
||||
|
||||
Enabling this setting allows reading JSON array values as strings.
|
||||
|
@ -377,6 +377,12 @@ Allow parsing bools as numbers in JSON input formats.
|
||||
|
||||
Enabled by default.
|
||||
|
||||
## input_format_json_read_bools_as_strings {#input_format_json_read_bools_as_strings}
|
||||
|
||||
Allow parsing bools as strings in JSON input formats.
|
||||
|
||||
Enabled by default.
|
||||
|
||||
## input_format_json_read_numbers_as_strings {#input_format_json_read_numbers_as_strings}
|
||||
|
||||
Allow parsing numbers as strings in JSON input formats.
|
||||
|
@ -27,7 +27,7 @@ $ clickhouse-format --query "select number from numbers(10) where number%2 order
|
||||
|
||||
Result:
|
||||
|
||||
```sql
|
||||
```bash
|
||||
SELECT number
|
||||
FROM numbers(10)
|
||||
WHERE number % 2
|
||||
@ -49,22 +49,20 @@ SELECT sum(number) FROM numbers(5)
|
||||
3. Multiqueries:
|
||||
|
||||
```bash
|
||||
$ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);"
|
||||
$ clickhouse-format -n <<< "SELECT min(number) FROM numbers(5); SELECT max(number) FROM numbers(5);"
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```sql
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT 1 AS x
|
||||
UNION ALL
|
||||
SELECT 1
|
||||
UNION DISTINCT
|
||||
SELECT 3
|
||||
)
|
||||
```
|
||||
SELECT min(number)
|
||||
FROM numbers(5)
|
||||
;
|
||||
|
||||
SELECT max(number)
|
||||
FROM numbers(5)
|
||||
;
|
||||
|
||||
```
|
||||
|
||||
4. Obfuscating:
|
||||
@ -75,7 +73,7 @@ $ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWE
|
||||
|
||||
Result:
|
||||
|
||||
```sql
|
||||
```
|
||||
SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END;
|
||||
```
|
||||
|
||||
@ -87,7 +85,7 @@ $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWE
|
||||
|
||||
Result:
|
||||
|
||||
```sql
|
||||
```
|
||||
SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END;
|
||||
```
|
||||
|
||||
@ -99,7 +97,7 @@ $ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELE
|
||||
|
||||
Result:
|
||||
|
||||
```sql
|
||||
```
|
||||
SELECT * \
|
||||
FROM \
|
||||
( \
|
||||
|
@ -143,9 +143,17 @@ public:
|
||||
return alias;
|
||||
}
|
||||
|
||||
const String & getOriginalAlias() const
|
||||
{
|
||||
return original_alias.empty() ? alias : original_alias;
|
||||
}
|
||||
|
||||
/// Set node alias
|
||||
void setAlias(String alias_value)
|
||||
{
|
||||
if (original_alias.empty())
|
||||
original_alias = std::move(alias);
|
||||
|
||||
alias = std::move(alias_value);
|
||||
}
|
||||
|
||||
@ -276,6 +284,9 @@ protected:
|
||||
|
||||
private:
|
||||
String alias;
|
||||
/// An alias from query. Alias can be replaced by query passes,
|
||||
/// but we need to keep the original one to support additional_table_filters.
|
||||
String original_alias;
|
||||
ASTPtr original_ast;
|
||||
};
|
||||
|
||||
|
@ -52,6 +52,7 @@
|
||||
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
|
||||
#include <Analyzer/createUniqueTableAliases.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Analyzer/SetUtils.h>
|
||||
#include <Analyzer/AggregationUtils.h>
|
||||
@ -1198,7 +1199,7 @@ private:
|
||||
|
||||
static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope);
|
||||
|
||||
static void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope);
|
||||
void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope);
|
||||
|
||||
static void convertLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope);
|
||||
|
||||
@ -2168,7 +2169,12 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_
|
||||
scope.scope_node->formatASTForErrorMessage());
|
||||
|
||||
--positional_argument_number;
|
||||
*node_to_replace = projection_nodes[positional_argument_number];
|
||||
*node_to_replace = projection_nodes[positional_argument_number]->clone();
|
||||
if (auto it = resolved_expressions.find(projection_nodes[positional_argument_number]);
|
||||
it != resolved_expressions.end())
|
||||
{
|
||||
resolved_expressions[*node_to_replace] = it->second;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -7366,6 +7372,7 @@ void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context
|
||||
{
|
||||
QueryAnalyzer analyzer;
|
||||
analyzer.resolve(query_tree_node, table_expression, context);
|
||||
createUniqueTableAliases(query_tree_node, table_expression, context);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -326,7 +326,7 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q
|
||||
}
|
||||
}
|
||||
|
||||
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node)
|
||||
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join)
|
||||
{
|
||||
QueryTreeNodes result;
|
||||
|
||||
@ -357,6 +357,8 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node)
|
||||
{
|
||||
auto & array_join_node = node_to_process->as<ArrayJoinNode &>();
|
||||
nodes_to_process.push_front(array_join_node.getTableExpression());
|
||||
if (add_array_join)
|
||||
result.push_back(std::move(node_to_process));
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::JOIN:
|
||||
|
@ -51,7 +51,7 @@ std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr &
|
||||
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options);
|
||||
|
||||
/// Extract table, table function, query, union from join tree
|
||||
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node);
|
||||
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false);
|
||||
|
||||
/// Extract left table expression from join tree
|
||||
QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node);
|
||||
|
141
src/Analyzer/createUniqueTableAliases.cpp
Normal file
141
src/Analyzer/createUniqueTableAliases.cpp
Normal file
@ -0,0 +1,141 @@
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
#include <Analyzer/createUniqueTableAliases.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/LambdaNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class CreateUniqueTableAliasesVisitor : public InDepthQueryTreeVisitorWithContext<CreateUniqueTableAliasesVisitor>
|
||||
{
|
||||
public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<CreateUniqueTableAliasesVisitor>;
|
||||
|
||||
explicit CreateUniqueTableAliasesVisitor(const ContextPtr & context)
|
||||
: Base(context)
|
||||
{
|
||||
// Insert a fake node on top of the stack.
|
||||
scope_nodes_stack.push_back(std::make_shared<LambdaNode>(Names{}, nullptr));
|
||||
}
|
||||
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto node_type = node->getNodeType();
|
||||
|
||||
switch (node_type)
|
||||
{
|
||||
case QueryTreeNodeType::QUERY:
|
||||
[[fallthrough]];
|
||||
case QueryTreeNodeType::UNION:
|
||||
{
|
||||
/// Queries like `(SELECT 1) as t` have invalid syntax. To avoid creating such queries (e.g. in StorageDistributed)
|
||||
/// we need to remove aliases for top level queries.
|
||||
/// N.B. Subquery depth starts count from 1, so the following condition checks if it's a top level.
|
||||
if (getSubqueryDepth() == 1)
|
||||
{
|
||||
node->removeAlias();
|
||||
break;
|
||||
}
|
||||
[[fallthrough]];
|
||||
}
|
||||
case QueryTreeNodeType::TABLE:
|
||||
[[fallthrough]];
|
||||
case QueryTreeNodeType::TABLE_FUNCTION:
|
||||
[[fallthrough]];
|
||||
case QueryTreeNodeType::ARRAY_JOIN:
|
||||
{
|
||||
auto & alias = table_expression_to_alias[node];
|
||||
if (alias.empty())
|
||||
{
|
||||
scope_to_nodes_with_aliases[scope_nodes_stack.back()].push_back(node);
|
||||
alias = fmt::format("__table{}", ++next_id);
|
||||
node->setAlias(alias);
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
switch (node_type)
|
||||
{
|
||||
case QueryTreeNodeType::QUERY:
|
||||
[[fallthrough]];
|
||||
case QueryTreeNodeType::UNION:
|
||||
[[fallthrough]];
|
||||
case QueryTreeNodeType::LAMBDA:
|
||||
scope_nodes_stack.push_back(node);
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
void leaveImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (scope_nodes_stack.back() == node)
|
||||
{
|
||||
if (auto it = scope_to_nodes_with_aliases.find(scope_nodes_stack.back());
|
||||
it != scope_to_nodes_with_aliases.end())
|
||||
{
|
||||
for (const auto & node_with_alias : it->second)
|
||||
{
|
||||
table_expression_to_alias.erase(node_with_alias);
|
||||
}
|
||||
scope_to_nodes_with_aliases.erase(it);
|
||||
}
|
||||
scope_nodes_stack.pop_back();
|
||||
}
|
||||
|
||||
/// Here we revisit subquery for IN function. Reasons:
|
||||
/// * For remote query execution, query tree may be traversed a few times.
|
||||
/// In such a case, it is possible to get AST like
|
||||
/// `IN ((SELECT ... FROM table AS __table4) AS __table1)` which result in
|
||||
/// `Multiple expressions for the alias` exception
|
||||
/// * Tables in subqueries could have different aliases => different three hashes,
|
||||
/// which is important to be able to find a set in PreparedSets
|
||||
/// See 01253_subquery_in_aggregate_function_JustStranger.
|
||||
///
|
||||
/// So, we revisit this subquery to make aliases stable.
|
||||
/// This should be safe cause columns from IN subquery can't be used in main query anyway.
|
||||
if (node->getNodeType() == QueryTreeNodeType::FUNCTION)
|
||||
{
|
||||
auto * function_node = node->as<FunctionNode>();
|
||||
if (isNameOfInFunction(function_node->getFunctionName()))
|
||||
{
|
||||
auto arg = function_node->getArguments().getNodes().back();
|
||||
/// Avoid aliasing IN `table`
|
||||
if (arg->getNodeType() != QueryTreeNodeType::TABLE)
|
||||
CreateUniqueTableAliasesVisitor(getContext()).visit(function_node->getArguments().getNodes().back());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
size_t next_id = 0;
|
||||
|
||||
// Stack of nodes which create scopes: QUERY, UNION and LAMBDA.
|
||||
QueryTreeNodes scope_nodes_stack;
|
||||
|
||||
std::unordered_map<QueryTreeNodePtr, QueryTreeNodes> scope_to_nodes_with_aliases;
|
||||
|
||||
// We need to use raw pointer as a key, not a QueryTreeNodePtrWithHash.
|
||||
std::unordered_map<QueryTreeNodePtr, String> table_expression_to_alias;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & /*table_expression*/, const ContextPtr & context)
|
||||
{
|
||||
CreateUniqueTableAliasesVisitor(context).visit(node);
|
||||
}
|
||||
|
||||
}
|
18
src/Analyzer/createUniqueTableAliases.h
Normal file
18
src/Analyzer/createUniqueTableAliases.h
Normal file
@ -0,0 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
class IQueryTreeNode;
|
||||
using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/*
|
||||
* For each table expression in the Query Tree generate and add a unique alias.
|
||||
* If table expression had an alias in initial query tree, override it.
|
||||
*/
|
||||
void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression, const ContextPtr & context);
|
||||
|
||||
}
|
@ -157,7 +157,7 @@ class IColumn;
|
||||
M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \
|
||||
M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \
|
||||
M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \
|
||||
M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \
|
||||
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
|
||||
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
|
||||
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \
|
||||
M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \
|
||||
@ -709,7 +709,6 @@ class IColumn;
|
||||
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) \
|
||||
M(Bool, query_plan_lift_up_union, true, "Allow to move UNIONs up so that more parts of the query plan can be optimized", 0) \
|
||||
M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \
|
||||
M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimization", 0) \
|
||||
M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimization", 0) \
|
||||
M(Bool, query_plan_remove_redundant_sorting, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries", 0) \
|
||||
@ -918,6 +917,7 @@ class IColumn;
|
||||
MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \
|
||||
MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \
|
||||
MAKE_OBSOLETE(M, Bool, allow_experimental_s3queue, true) \
|
||||
MAKE_OBSOLETE(M, Bool, query_plan_optimize_primary_key, true) \
|
||||
|
||||
/** The section above is for obsolete settings. Do not add anything there. */
|
||||
|
||||
@ -983,6 +983,7 @@ class IColumn;
|
||||
M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \
|
||||
M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \
|
||||
M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \
|
||||
M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \
|
||||
M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \
|
||||
M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \
|
||||
M(Bool, input_format_json_read_numbers_as_strings, true, "Allow to parse numbers as strings in JSON input formats", 0) \
|
||||
|
@ -81,7 +81,8 @@ namespace SettingsChangesHistory
|
||||
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
|
||||
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
|
||||
{
|
||||
{"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}}},
|
||||
{"24.1", {{"print_pretty_type_names", false, true, "Better user experience."},
|
||||
{"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}}},
|
||||
{"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."},
|
||||
{"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"},
|
||||
{"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"},
|
||||
|
@ -335,6 +335,22 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist
|
||||
{
|
||||
read(column, [&](ColumnString::Chars & data) { readJSONArrayInto(data, istr); });
|
||||
}
|
||||
else if (settings.json.read_bools_as_strings && !istr.eof() && (*istr.position() == 't' || *istr.position() == 'f'))
|
||||
{
|
||||
String str_value;
|
||||
if (*istr.position() == 't')
|
||||
{
|
||||
assertString("true", istr);
|
||||
str_value = "true";
|
||||
}
|
||||
else if (*istr.position() == 'f')
|
||||
{
|
||||
assertString("false", istr);
|
||||
str_value = "false";
|
||||
}
|
||||
|
||||
read(column, [&](ColumnString::Chars & data) { data.insert(str_value.begin(), str_value.end()); });
|
||||
}
|
||||
else if (settings.json.read_numbers_as_strings && !istr.eof() && *istr.position() != '"')
|
||||
{
|
||||
String field;
|
||||
|
@ -450,10 +450,11 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo
|
||||
break;
|
||||
case FormatSettings::EscapingRule::JSON:
|
||||
result += fmt::format(
|
||||
", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_objects_as_strings={}, read_numbers_as_strings={}, "
|
||||
", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_bools_as_strings={}, read_objects_as_strings={}, read_numbers_as_strings={}, "
|
||||
"read_arrays_as_strings={}, try_infer_objects_as_tuples={}, infer_incomplete_types_as_strings={}, try_infer_objects={}",
|
||||
settings.json.try_infer_numbers_from_strings,
|
||||
settings.json.read_bools_as_numbers,
|
||||
settings.json.read_bools_as_strings,
|
||||
settings.json.read_objects_as_strings,
|
||||
settings.json.read_numbers_as_strings,
|
||||
settings.json.read_arrays_as_strings,
|
||||
|
@ -111,6 +111,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
|
||||
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
|
||||
format_settings.json.quote_decimals = settings.output_format_json_quote_decimals;
|
||||
format_settings.json.read_bools_as_numbers = settings.input_format_json_read_bools_as_numbers;
|
||||
format_settings.json.read_bools_as_strings = settings.input_format_json_read_bools_as_strings;
|
||||
format_settings.json.read_numbers_as_strings = settings.input_format_json_read_numbers_as_strings;
|
||||
format_settings.json.read_objects_as_strings = settings.input_format_json_read_objects_as_strings;
|
||||
format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings;
|
||||
|
@ -204,6 +204,7 @@ struct FormatSettings
|
||||
bool ignore_unknown_keys_in_named_tuple = false;
|
||||
bool serialize_as_strings = false;
|
||||
bool read_bools_as_numbers = true;
|
||||
bool read_bools_as_strings = true;
|
||||
bool read_numbers_as_strings = true;
|
||||
bool read_objects_as_strings = true;
|
||||
bool read_arrays_as_strings = true;
|
||||
|
@ -377,6 +377,22 @@ namespace
|
||||
type_indexes.erase(TypeIndex::UInt8);
|
||||
}
|
||||
|
||||
/// If we have Bool and String types convert all numbers to String.
|
||||
/// It's applied only when setting input_format_json_read_bools_as_strings is enabled.
|
||||
void transformJSONBoolsAndStringsToString(DataTypes & data_types, TypeIndexesSet & type_indexes)
|
||||
{
|
||||
if (!type_indexes.contains(TypeIndex::String) || !type_indexes.contains(TypeIndex::UInt8))
|
||||
return;
|
||||
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isBool(type))
|
||||
type = std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
type_indexes.erase(TypeIndex::UInt8);
|
||||
}
|
||||
|
||||
/// If we have type Nothing/Nullable(Nothing) and some other non Nothing types,
|
||||
/// convert all Nothing/Nullable(Nothing) types to the first non Nothing.
|
||||
/// For example, when we have [Nothing, Array(Int64)] it will convert it to [Array(Int64), Array(Int64)]
|
||||
@ -628,6 +644,10 @@ namespace
|
||||
if (settings.json.read_bools_as_numbers)
|
||||
transformBoolsAndNumbersToNumbers(data_types, type_indexes);
|
||||
|
||||
/// Convert Bool to String if needed.
|
||||
if (settings.json.read_bools_as_strings)
|
||||
transformJSONBoolsAndStringsToString(data_types, type_indexes);
|
||||
|
||||
if (settings.json.try_infer_objects_as_tuples)
|
||||
mergeJSONPaths(data_types, type_indexes, settings, json_info);
|
||||
};
|
||||
|
@ -1382,8 +1382,12 @@ void skipJSONField(ReadBuffer & buf, StringRef name_of_field)
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol '{}' for key '{}'",
|
||||
std::string(*buf.position(), 1), name_of_field.toString());
|
||||
throw Exception(
|
||||
ErrorCodes::INCORRECT_DATA,
|
||||
"Cannot read JSON field here: '{}'. Unexpected symbol '{}'{}",
|
||||
String(buf.position(), std::min(buf.available(), size_t(10))),
|
||||
std::string(1, *buf.position()),
|
||||
name_of_field.empty() ? "" : " for key " + name_of_field.toString());
|
||||
}
|
||||
}
|
||||
|
||||
@ -1753,7 +1757,7 @@ void readQuotedField(String & s, ReadBuffer & buf)
|
||||
void readJSONField(String & s, ReadBuffer & buf)
|
||||
{
|
||||
s.clear();
|
||||
auto parse_func = [](ReadBuffer & in) { skipJSONField(in, "json_field"); };
|
||||
auto parse_func = [](ReadBuffer & in) { skipJSONField(in, ""); };
|
||||
readParsedValueInto(s, buf, parse_func);
|
||||
}
|
||||
|
||||
|
@ -1419,7 +1419,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool
|
||||
return set;
|
||||
}
|
||||
|
||||
FutureSetPtr external_table_set;
|
||||
FutureSetFromSubqueryPtr external_table_set;
|
||||
|
||||
/// A special case is if the name of the table is specified on the right side of the IN statement,
|
||||
/// and the table has the type Set (a previously prepared set).
|
||||
|
@ -664,26 +664,26 @@ void Aggregator::compileAggregateFunctionsIfNeeded()
|
||||
for (size_t i = 0; i < aggregate_functions.size(); ++i)
|
||||
{
|
||||
const auto * function = aggregate_functions[i];
|
||||
bool function_is_compilable = function->isCompilable();
|
||||
if (!function_is_compilable)
|
||||
continue;
|
||||
|
||||
size_t offset_of_aggregate_function = offsets_of_aggregate_states[i];
|
||||
AggregateFunctionWithOffset function_to_compile
|
||||
|
||||
if (function->isCompilable())
|
||||
{
|
||||
.function = function,
|
||||
.aggregate_data_offset = offset_of_aggregate_function
|
||||
};
|
||||
AggregateFunctionWithOffset function_to_compile
|
||||
{
|
||||
.function = function,
|
||||
.aggregate_data_offset = offset_of_aggregate_function
|
||||
};
|
||||
|
||||
functions_to_compile.emplace_back(std::move(function_to_compile));
|
||||
functions_to_compile.emplace_back(std::move(function_to_compile));
|
||||
|
||||
functions_description += function->getDescription();
|
||||
functions_description += ' ';
|
||||
functions_description += function->getDescription();
|
||||
functions_description += ' ';
|
||||
|
||||
functions_description += std::to_string(offset_of_aggregate_function);
|
||||
functions_description += ' ';
|
||||
functions_description += std::to_string(offset_of_aggregate_function);
|
||||
functions_description += ' ';
|
||||
}
|
||||
|
||||
is_aggregate_function_compiled[i] = true;
|
||||
is_aggregate_function_compiled[i] = function->isCompilable();
|
||||
}
|
||||
|
||||
if (functions_to_compile.empty())
|
||||
@ -1685,13 +1685,14 @@ bool Aggregator::executeOnBlock(Columns columns,
|
||||
/// For the case when there are no keys (all aggregate into one row).
|
||||
if (result.type == AggregatedDataVariants::Type::without_key)
|
||||
{
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_functions_instructions.data()))
|
||||
{
|
||||
executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
|
||||
}
|
||||
else
|
||||
#endif
|
||||
/// TODO: Enable compilation after investigation
|
||||
// #if USE_EMBEDDED_COMPILER
|
||||
// if (compiled_aggregate_functions_holder)
|
||||
// {
|
||||
// executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
|
||||
// }
|
||||
// else
|
||||
// #endif
|
||||
{
|
||||
executeWithoutKeyImpl<false>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
|
||||
}
|
||||
|
@ -82,8 +82,8 @@ private:
|
||||
|
||||
using DDLGuardPtr = std::unique_ptr<DDLGuard>;
|
||||
|
||||
class FutureSet;
|
||||
using FutureSetPtr = std::shared_ptr<FutureSet>;
|
||||
class FutureSetFromSubquery;
|
||||
using FutureSetFromSubqueryPtr = std::shared_ptr<FutureSetFromSubquery>;
|
||||
|
||||
/// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID.
|
||||
/// Such table can be accessed from everywhere by its ID.
|
||||
@ -116,7 +116,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext
|
||||
|
||||
IDatabase * temporary_tables = nullptr;
|
||||
UUID id = UUIDHelpers::Nil;
|
||||
FutureSetPtr future_set;
|
||||
FutureSetFromSubqueryPtr future_set;
|
||||
};
|
||||
|
||||
///TODO maybe remove shared_ptr from here?
|
||||
|
@ -2378,12 +2378,25 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
|
||||
else
|
||||
{
|
||||
// It's possible to optimize count() given only partition predicates
|
||||
SelectQueryInfo temp_query_info;
|
||||
temp_query_info.query = query_ptr;
|
||||
temp_query_info.syntax_analyzer_result = syntax_analyzer_result;
|
||||
temp_query_info.prepared_sets = query_analyzer->getPreparedSets();
|
||||
ActionsDAG::NodeRawConstPtrs filter_nodes;
|
||||
if (analysis_result.hasPrewhere())
|
||||
{
|
||||
auto & prewhere_info = analysis_result.prewhere_info;
|
||||
filter_nodes.push_back(&prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name));
|
||||
|
||||
return storage->totalRowsByPartitionPredicate(temp_query_info, context);
|
||||
if (prewhere_info->row_level_filter)
|
||||
filter_nodes.push_back(&prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name));
|
||||
}
|
||||
if (analysis_result.hasWhere())
|
||||
{
|
||||
filter_nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name));
|
||||
}
|
||||
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes, {}, context);
|
||||
if (!filter_actions_dag)
|
||||
return {};
|
||||
|
||||
return storage->totalRowsByPartitionPredicate(filter_actions_dag, context);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -67,8 +67,7 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
|
||||
{
|
||||
const auto & function_argument_types = function.getArgumentTypes();
|
||||
|
||||
auto & context = module.getContext();
|
||||
llvm::IRBuilder<> b(context);
|
||||
llvm::IRBuilder<> b(module.getContext());
|
||||
auto * size_type = b.getIntNTy(sizeof(size_t) * 8);
|
||||
auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy());
|
||||
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false);
|
||||
@ -76,8 +75,6 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
|
||||
/// Create function in module
|
||||
|
||||
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, function.getName(), module);
|
||||
func->setAttributes(llvm::AttributeList::get(context, {{2, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
|
||||
|
||||
auto * args = func->args().begin();
|
||||
llvm::Value * rows_count_arg = args++;
|
||||
llvm::Value * columns_arg = args++;
|
||||
@ -199,9 +196,6 @@ static void compileCreateAggregateStatesFunctions(llvm::Module & module, const s
|
||||
auto * create_aggregate_states_function_type = llvm::FunctionType::get(b.getVoidTy(), { aggregate_data_places_type }, false);
|
||||
auto * create_aggregate_states_function = llvm::Function::Create(create_aggregate_states_function_type, llvm::Function::ExternalLinkage, name, module);
|
||||
|
||||
create_aggregate_states_function->setAttributes(
|
||||
llvm::AttributeList::get(context, {{1, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
|
||||
|
||||
auto * arguments = create_aggregate_states_function->args().begin();
|
||||
llvm::Value * aggregate_data_place_arg = arguments++;
|
||||
|
||||
@ -247,11 +241,6 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
|
||||
auto * add_into_aggregate_states_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_type->getPointerTo(), places_type }, false);
|
||||
auto * add_into_aggregate_states_func = llvm::Function::Create(add_into_aggregate_states_func_declaration, llvm::Function::ExternalLinkage, name, module);
|
||||
|
||||
add_into_aggregate_states_func->setAttributes(llvm::AttributeList::get(
|
||||
context,
|
||||
{{3, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
|
||||
{4, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
|
||||
|
||||
auto * arguments = add_into_aggregate_states_func->args().begin();
|
||||
llvm::Value * row_start_arg = arguments++;
|
||||
llvm::Value * row_end_arg = arguments++;
|
||||
@ -307,7 +296,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
|
||||
llvm::Value * aggregation_place = nullptr;
|
||||
|
||||
if (places_argument_type == AddIntoAggregateStatesPlacesArgumentType::MultiplePlaces)
|
||||
aggregation_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateInBoundsGEP(b.getInt8Ty()->getPointerTo(), places_arg, counter_phi));
|
||||
aggregation_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateGEP(b.getInt8Ty()->getPointerTo(), places_arg, counter_phi));
|
||||
else
|
||||
aggregation_place = places_arg;
|
||||
|
||||
@ -324,7 +313,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
|
||||
auto & column = columns[previous_columns_size + column_argument_index];
|
||||
const auto & argument_type = arguments_types[column_argument_index];
|
||||
|
||||
auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateInBoundsGEP(column.data_element_type, column.data_ptr, counter_phi));
|
||||
auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateGEP(column.data_element_type, column.data_ptr, counter_phi));
|
||||
|
||||
if (!argument_type->isNullable())
|
||||
{
|
||||
@ -332,7 +321,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
|
||||
continue;
|
||||
}
|
||||
|
||||
auto * column_null_data_with_offset = b.CreateInBoundsGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi);
|
||||
auto * column_null_data_with_offset = b.CreateGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi);
|
||||
auto * is_null = b.CreateICmpNE(b.CreateLoad(b.getInt8Ty(), column_null_data_with_offset), b.getInt8(0));
|
||||
auto * nullable_unitialized = llvm::Constant::getNullValue(toNullableType(b, column.data_element_type));
|
||||
auto * first_insert = b.CreateInsertValue(nullable_unitialized, column_data_element, {0});
|
||||
@ -365,8 +354,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
|
||||
|
||||
static void compileMergeAggregatesStates(llvm::Module & module, const std::vector<AggregateFunctionWithOffset> & functions, const std::string & name)
|
||||
{
|
||||
auto & context = module.getContext();
|
||||
llvm::IRBuilder<> b(context);
|
||||
llvm::IRBuilder<> b(module.getContext());
|
||||
|
||||
auto * aggregate_data_place_type = b.getInt8Ty()->getPointerTo();
|
||||
auto * aggregate_data_places_type = aggregate_data_place_type->getPointerTo();
|
||||
@ -377,11 +365,6 @@ static void compileMergeAggregatesStates(llvm::Module & module, const std::vecto
|
||||
auto * merge_aggregates_states_func
|
||||
= llvm::Function::Create(merge_aggregates_states_func_declaration, llvm::Function::ExternalLinkage, name, module);
|
||||
|
||||
merge_aggregates_states_func->setAttributes(llvm::AttributeList::get(
|
||||
context,
|
||||
{{1, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
|
||||
{2, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
|
||||
|
||||
auto * arguments = merge_aggregates_states_func->args().begin();
|
||||
llvm::Value * aggregate_data_places_dst_arg = arguments++;
|
||||
llvm::Value * aggregate_data_places_src_arg = arguments++;
|
||||
@ -443,11 +426,6 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
|
||||
auto * insert_aggregates_into_result_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_type->getPointerTo(), aggregate_data_places_type }, false);
|
||||
auto * insert_aggregates_into_result_func = llvm::Function::Create(insert_aggregates_into_result_func_declaration, llvm::Function::ExternalLinkage, name, module);
|
||||
|
||||
insert_aggregates_into_result_func->setAttributes(llvm::AttributeList::get(
|
||||
context,
|
||||
{{3, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
|
||||
{4, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
|
||||
|
||||
auto * arguments = insert_aggregates_into_result_func->args().begin();
|
||||
llvm::Value * row_start_arg = arguments++;
|
||||
llvm::Value * row_end_arg = arguments++;
|
||||
@ -482,7 +460,7 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
|
||||
auto * counter_phi = b.CreatePHI(row_start_arg->getType(), 2);
|
||||
counter_phi->addIncoming(row_start_arg, entry);
|
||||
|
||||
auto * aggregate_data_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateInBoundsGEP(b.getInt8Ty()->getPointerTo(), aggregate_data_places_arg, counter_phi));
|
||||
auto * aggregate_data_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateGEP(b.getInt8Ty()->getPointerTo(), aggregate_data_places_arg, counter_phi));
|
||||
|
||||
for (size_t i = 0; i < functions.size(); ++i)
|
||||
{
|
||||
@ -492,11 +470,11 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
|
||||
const auto * aggregate_function_ptr = functions[i].function;
|
||||
auto * final_value = aggregate_function_ptr->compileGetResult(b, aggregation_place_with_offset);
|
||||
|
||||
auto * result_column_data_element = b.CreateInBoundsGEP(columns[i].data_element_type, columns[i].data_ptr, counter_phi);
|
||||
auto * result_column_data_element = b.CreateGEP(columns[i].data_element_type, columns[i].data_ptr, counter_phi);
|
||||
if (columns[i].null_data_ptr)
|
||||
{
|
||||
b.CreateStore(b.CreateExtractValue(final_value, {0}), result_column_data_element);
|
||||
auto * result_column_is_null_element = b.CreateInBoundsGEP(b.getInt8Ty(), columns[i].null_data_ptr, counter_phi);
|
||||
auto * result_column_is_null_element = b.CreateGEP(b.getInt8Ty(), columns[i].null_data_ptr, counter_phi);
|
||||
b.CreateStore(b.CreateSelect(b.CreateExtractValue(final_value, {1}), b.getInt8(1), b.getInt8(0)), result_column_is_null_element);
|
||||
}
|
||||
else
|
||||
|
@ -97,7 +97,7 @@ FutureSetFromSubquery::FutureSetFromSubquery(
|
||||
String key,
|
||||
std::unique_ptr<QueryPlan> source_,
|
||||
StoragePtr external_table_,
|
||||
FutureSetPtr external_table_set_,
|
||||
std::shared_ptr<FutureSetFromSubquery> external_table_set_,
|
||||
const Settings & settings,
|
||||
bool in_subquery_)
|
||||
: external_table(std::move(external_table_))
|
||||
@ -168,6 +168,24 @@ std::unique_ptr<QueryPlan> FutureSetFromSubquery::build(const ContextPtr & conte
|
||||
return plan;
|
||||
}
|
||||
|
||||
void FutureSetFromSubquery::buildSetInplace(const ContextPtr & context)
|
||||
{
|
||||
if (external_table_set)
|
||||
external_table_set->buildSetInplace(context);
|
||||
|
||||
auto plan = build(context);
|
||||
|
||||
if (!plan)
|
||||
return;
|
||||
|
||||
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
|
||||
pipeline.complete(std::make_shared<EmptySink>(Block()));
|
||||
|
||||
CompletedPipelineExecutor executor(pipeline);
|
||||
executor.execute();
|
||||
}
|
||||
|
||||
SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context)
|
||||
{
|
||||
if (!context->getSettingsRef().use_index_for_in_with_subqueries)
|
||||
@ -233,7 +251,7 @@ String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes &
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings)
|
||||
FutureSetFromTuplePtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings)
|
||||
{
|
||||
auto from_tuple = std::make_shared<FutureSetFromTuple>(std::move(block), settings);
|
||||
const auto & set_types = from_tuple->getTypes();
|
||||
@ -247,7 +265,7 @@ FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Set
|
||||
return from_tuple;
|
||||
}
|
||||
|
||||
FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_)
|
||||
FutureSetFromStoragePtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_)
|
||||
{
|
||||
auto from_storage = std::make_shared<FutureSetFromStorage>(std::move(set_));
|
||||
auto [it, inserted] = sets_from_storage.emplace(key, from_storage);
|
||||
@ -258,11 +276,11 @@ FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_)
|
||||
return from_storage;
|
||||
}
|
||||
|
||||
FutureSetPtr PreparedSets::addFromSubquery(
|
||||
FutureSetFromSubqueryPtr PreparedSets::addFromSubquery(
|
||||
const Hash & key,
|
||||
std::unique_ptr<QueryPlan> source,
|
||||
StoragePtr external_table,
|
||||
FutureSetPtr external_table_set,
|
||||
FutureSetFromSubqueryPtr external_table_set,
|
||||
const Settings & settings,
|
||||
bool in_subquery)
|
||||
{
|
||||
@ -282,7 +300,7 @@ FutureSetPtr PreparedSets::addFromSubquery(
|
||||
return from_subquery;
|
||||
}
|
||||
|
||||
FutureSetPtr PreparedSets::addFromSubquery(
|
||||
FutureSetFromSubqueryPtr PreparedSets::addFromSubquery(
|
||||
const Hash & key,
|
||||
QueryTreeNodePtr query_tree,
|
||||
const Settings & settings)
|
||||
@ -300,7 +318,7 @@ FutureSetPtr PreparedSets::addFromSubquery(
|
||||
return from_subquery;
|
||||
}
|
||||
|
||||
FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const
|
||||
FutureSetFromTuplePtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const
|
||||
{
|
||||
auto it = sets_from_tuple.find(key);
|
||||
if (it == sets_from_tuple.end())
|
||||
|
@ -69,6 +69,8 @@ private:
|
||||
SetPtr set;
|
||||
};
|
||||
|
||||
using FutureSetFromStoragePtr = std::shared_ptr<FutureSetFromStorage>;
|
||||
|
||||
/// Set from tuple is filled as well as set from storage.
|
||||
/// Additionally, it can be converted to set useful for PK.
|
||||
class FutureSetFromTuple final : public FutureSet
|
||||
@ -86,6 +88,8 @@ private:
|
||||
SetKeyColumns set_key_columns;
|
||||
};
|
||||
|
||||
using FutureSetFromTuplePtr = std::shared_ptr<FutureSetFromTuple>;
|
||||
|
||||
/// Set from subquery can be built inplace for PK or in CreatingSet step.
|
||||
/// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created,
|
||||
/// but ordinary set would be created instead.
|
||||
@ -96,7 +100,7 @@ public:
|
||||
String key,
|
||||
std::unique_ptr<QueryPlan> source_,
|
||||
StoragePtr external_table_,
|
||||
FutureSetPtr external_table_set_,
|
||||
std::shared_ptr<FutureSetFromSubquery> external_table_set_,
|
||||
const Settings & settings,
|
||||
bool in_subquery_);
|
||||
|
||||
@ -110,6 +114,7 @@ public:
|
||||
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
|
||||
|
||||
std::unique_ptr<QueryPlan> build(const ContextPtr & context);
|
||||
void buildSetInplace(const ContextPtr & context);
|
||||
|
||||
QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); }
|
||||
void setQueryPlan(std::unique_ptr<QueryPlan> source_);
|
||||
@ -119,7 +124,7 @@ public:
|
||||
private:
|
||||
SetAndKeyPtr set_and_key;
|
||||
StoragePtr external_table;
|
||||
FutureSetPtr external_table_set;
|
||||
std::shared_ptr<FutureSetFromSubquery> external_table_set;
|
||||
|
||||
std::unique_ptr<QueryPlan> source;
|
||||
QueryTreeNodePtr query_tree;
|
||||
@ -130,6 +135,8 @@ private:
|
||||
// with new analyzer it's not a case
|
||||
};
|
||||
|
||||
using FutureSetFromSubqueryPtr = std::shared_ptr<FutureSetFromSubquery>;
|
||||
|
||||
/// Container for all the sets used in query.
|
||||
class PreparedSets
|
||||
{
|
||||
@ -141,32 +148,32 @@ public:
|
||||
UInt64 operator()(const Hash & key) const { return key.low64 ^ key.high64; }
|
||||
};
|
||||
|
||||
using SetsFromTuple = std::unordered_map<Hash, std::vector<std::shared_ptr<FutureSetFromTuple>>, Hashing>;
|
||||
using SetsFromStorage = std::unordered_map<Hash, std::shared_ptr<FutureSetFromStorage>, Hashing>;
|
||||
using SetsFromSubqueries = std::unordered_map<Hash, std::shared_ptr<FutureSetFromSubquery>, Hashing>;
|
||||
using SetsFromTuple = std::unordered_map<Hash, std::vector<FutureSetFromTuplePtr>, Hashing>;
|
||||
using SetsFromStorage = std::unordered_map<Hash, FutureSetFromStoragePtr, Hashing>;
|
||||
using SetsFromSubqueries = std::unordered_map<Hash, FutureSetFromSubqueryPtr, Hashing>;
|
||||
|
||||
FutureSetPtr addFromStorage(const Hash & key, SetPtr set_);
|
||||
FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings);
|
||||
FutureSetFromStoragePtr addFromStorage(const Hash & key, SetPtr set_);
|
||||
FutureSetFromTuplePtr addFromTuple(const Hash & key, Block block, const Settings & settings);
|
||||
|
||||
FutureSetPtr addFromSubquery(
|
||||
FutureSetFromSubqueryPtr addFromSubquery(
|
||||
const Hash & key,
|
||||
std::unique_ptr<QueryPlan> source,
|
||||
StoragePtr external_table,
|
||||
FutureSetPtr external_table_set,
|
||||
FutureSetFromSubqueryPtr external_table_set,
|
||||
const Settings & settings,
|
||||
bool in_subquery = false);
|
||||
|
||||
FutureSetPtr addFromSubquery(
|
||||
FutureSetFromSubqueryPtr addFromSubquery(
|
||||
const Hash & key,
|
||||
QueryTreeNodePtr query_tree,
|
||||
const Settings & settings);
|
||||
|
||||
FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const;
|
||||
std::shared_ptr<FutureSetFromStorage> findStorage(const Hash & key) const;
|
||||
std::shared_ptr<FutureSetFromSubquery> findSubquery(const Hash & key) const;
|
||||
FutureSetFromTuplePtr findTuple(const Hash & key, const DataTypes & types) const;
|
||||
FutureSetFromStoragePtr findStorage(const Hash & key) const;
|
||||
FutureSetFromSubqueryPtr findSubquery(const Hash & key) const;
|
||||
void markAsINSubquery(const Hash & key);
|
||||
|
||||
using Subqueries = std::vector<std::shared_ptr<FutureSetFromSubquery>>;
|
||||
using Subqueries = std::vector<FutureSetFromSubqueryPtr>;
|
||||
Subqueries getSubqueries() const;
|
||||
bool hasSubqueries() const { return !sets_from_subqueries.empty(); }
|
||||
|
||||
|
@ -36,7 +36,6 @@ struct RequiredSourceColumnsData
|
||||
|
||||
bool has_table_join = false;
|
||||
bool has_array_join = false;
|
||||
bool visit_index_hint = false;
|
||||
|
||||
bool addColumnAliasIfAny(const IAST & ast);
|
||||
void addColumnIdentifier(const ASTIdentifier & node);
|
||||
|
@ -72,11 +72,6 @@ void RequiredSourceColumnsMatcher::visit(const ASTPtr & ast, Data & data)
|
||||
}
|
||||
if (auto * t = ast->as<ASTFunction>())
|
||||
{
|
||||
/// "indexHint" is a special function for index analysis.
|
||||
/// Everything that is inside it is not calculated. See KeyCondition
|
||||
if (!data.visit_index_hint && t->name == "indexHint")
|
||||
return;
|
||||
|
||||
data.addColumnAliasIfAny(*ast);
|
||||
visit(*t, ast, data);
|
||||
return;
|
||||
|
@ -995,13 +995,12 @@ void TreeRewriterResult::collectSourceColumns(bool add_special)
|
||||
/// Calculate which columns are required to execute the expression.
|
||||
/// Then, delete all other columns from the list of available columns.
|
||||
/// After execution, columns will only contain the list of columns needed to read from the table.
|
||||
bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw)
|
||||
bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool no_throw)
|
||||
{
|
||||
/// We calculate required_source_columns with source_columns modifications and swap them on exit
|
||||
required_source_columns = source_columns;
|
||||
|
||||
RequiredSourceColumnsVisitor::Data columns_context;
|
||||
columns_context.visit_index_hint = visit_index_hint;
|
||||
RequiredSourceColumnsVisitor(columns_context).visit(query);
|
||||
|
||||
NameSet source_column_names;
|
||||
@ -1385,7 +1384,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
result.window_function_asts = getWindowFunctions(query, *select_query);
|
||||
result.expressions_with_window_function = getExpressionsWithWindowFunctions(query);
|
||||
|
||||
result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key);
|
||||
result.collectUsedColumns(query, true);
|
||||
|
||||
if (!result.missed_subcolumns.empty())
|
||||
{
|
||||
@ -1422,7 +1421,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
result.aggregates = getAggregates(query, *select_query);
|
||||
result.window_function_asts = getWindowFunctions(query, *select_query);
|
||||
result.expressions_with_window_function = getExpressionsWithWindowFunctions(query);
|
||||
result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key);
|
||||
result.collectUsedColumns(query, true);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1499,7 +1498,7 @@ TreeRewriterResultPtr TreeRewriter::analyze(
|
||||
else
|
||||
assertNoAggregates(query, "in wrong place");
|
||||
|
||||
bool is_ok = result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key, no_throw);
|
||||
bool is_ok = result.collectUsedColumns(query, false, no_throw);
|
||||
if (!is_ok)
|
||||
return {};
|
||||
|
||||
|
@ -88,7 +88,7 @@ struct TreeRewriterResult
|
||||
bool add_special = true);
|
||||
|
||||
void collectSourceColumns(bool add_special);
|
||||
bool collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw = false);
|
||||
bool collectUsedColumns(const ASTPtr & query, bool is_select, bool no_throw = false);
|
||||
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
|
||||
const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; }
|
||||
NameSet getArrayJoinSourceNameSet() const;
|
||||
|
@ -1057,7 +1057,7 @@ void addBuildSubqueriesForSetsStepIfNeeded(
|
||||
Planner subquery_planner(
|
||||
query_tree,
|
||||
subquery_options,
|
||||
planner_context->getGlobalPlannerContext());
|
||||
std::make_shared<GlobalPlannerContext>()); //planner_context->getGlobalPlannerContext());
|
||||
subquery_planner.buildQueryPlanIfNeeded();
|
||||
|
||||
subquery->setQueryPlan(std::make_unique<QueryPlan>(std::move(subquery_planner).extractQueryPlan()));
|
||||
|
@ -20,12 +20,15 @@ const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const Quer
|
||||
return createColumnIdentifier(column_node_typed.getColumn(), column_source_node);
|
||||
}
|
||||
|
||||
const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & /*column_source_node*/)
|
||||
const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & column_source_node)
|
||||
{
|
||||
std::string column_identifier;
|
||||
|
||||
column_identifier += column.name;
|
||||
column_identifier += '_' + std::to_string(column_identifiers.size());
|
||||
const auto & source_alias = column_source_node->getAlias();
|
||||
if (!source_alias.empty())
|
||||
column_identifier = source_alias + "." + column.name;
|
||||
else
|
||||
column_identifier = column.name;
|
||||
|
||||
auto [it, inserted] = column_identifiers.emplace(column_identifier);
|
||||
assert(inserted);
|
||||
|
@ -817,7 +817,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
}
|
||||
}
|
||||
|
||||
const auto & table_expression_alias = table_expression->getAlias();
|
||||
const auto & table_expression_alias = table_expression->getOriginalAlias();
|
||||
auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context);
|
||||
add_filter(additional_filters_info, "additional filter");
|
||||
|
||||
@ -1058,6 +1058,18 @@ 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();
|
||||
|
@ -20,6 +20,7 @@
|
||||
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
#include <Analyzer/TableNode.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
@ -113,41 +114,96 @@ String JoinClause::dump() const
|
||||
namespace
|
||||
{
|
||||
|
||||
std::optional<JoinTableSide> extractJoinTableSideFromExpression(const ActionsDAG::Node * expression_root_node,
|
||||
const std::unordered_set<const ActionsDAG::Node *> & join_expression_dag_input_nodes,
|
||||
const NameSet & left_table_expression_columns_names,
|
||||
const NameSet & right_table_expression_columns_names,
|
||||
using TableExpressionSet = std::unordered_set<const IQueryTreeNode *>;
|
||||
|
||||
TableExpressionSet extractTableExpressionsSet(const QueryTreeNodePtr & node)
|
||||
{
|
||||
TableExpressionSet res;
|
||||
for (const auto & expr : extractTableExpressions(node, true))
|
||||
res.insert(expr.get());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
std::optional<JoinTableSide> extractJoinTableSideFromExpression(//const ActionsDAG::Node * expression_root_node,
|
||||
const IQueryTreeNode * expression_root_node,
|
||||
//const std::unordered_set<const ActionsDAG::Node *> & join_expression_dag_input_nodes,
|
||||
const TableExpressionSet & left_table_expressions,
|
||||
const TableExpressionSet & right_table_expressions,
|
||||
const JoinNode & join_node)
|
||||
{
|
||||
std::optional<JoinTableSide> table_side;
|
||||
std::vector<const ActionsDAG::Node *> nodes_to_process;
|
||||
std::vector<const IQueryTreeNode *> nodes_to_process;
|
||||
nodes_to_process.push_back(expression_root_node);
|
||||
|
||||
// std::cerr << "==== extractJoinTableSideFromExpression\n";
|
||||
// std::cerr << "inp nodes" << std::endl;
|
||||
// for (const auto * node : join_expression_dag_input_nodes)
|
||||
// std::cerr << reinterpret_cast<const void *>(node) << ' ' << node->result_name << std::endl;
|
||||
|
||||
|
||||
// std::cerr << "l names" << std::endl;
|
||||
// for (const auto & l : left_table_expression_columns_names)
|
||||
// std::cerr << l << std::endl;
|
||||
|
||||
// std::cerr << "r names" << std::endl;
|
||||
// for (const auto & r : right_table_expression_columns_names)
|
||||
// std::cerr << r << std::endl;
|
||||
|
||||
// const auto * left_table_expr = join_node.getLeftTableExpression().get();
|
||||
// const auto * right_table_expr = join_node.getRightTableExpression().get();
|
||||
|
||||
while (!nodes_to_process.empty())
|
||||
{
|
||||
const auto * node_to_process = nodes_to_process.back();
|
||||
nodes_to_process.pop_back();
|
||||
|
||||
for (const auto & child : node_to_process->children)
|
||||
nodes_to_process.push_back(child);
|
||||
//std::cerr << "... " << reinterpret_cast<const void *>(node_to_process) << ' ' << node_to_process->result_name << std::endl;
|
||||
|
||||
if (!join_expression_dag_input_nodes.contains(node_to_process))
|
||||
if (const auto * function_node = node_to_process->as<FunctionNode>())
|
||||
{
|
||||
for (const auto & child : function_node->getArguments())
|
||||
nodes_to_process.push_back(child.get());
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto * column_node = node_to_process->as<ColumnNode>();
|
||||
if (!column_node)
|
||||
continue;
|
||||
|
||||
const auto & input_name = node_to_process->result_name;
|
||||
// if (!join_expression_dag_input_nodes.contains(node_to_process))
|
||||
// continue;
|
||||
|
||||
bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name);
|
||||
bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name);
|
||||
const auto & input_name = column_node->getColumnName();
|
||||
|
||||
if (!left_table_expression_contains_input && !right_table_expression_contains_input)
|
||||
// bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name);
|
||||
// bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name);
|
||||
|
||||
// if (!left_table_expression_contains_input && !right_table_expression_contains_input)
|
||||
// throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
// "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns",
|
||||
// join_node.formatASTForErrorMessage(),
|
||||
// input_name,
|
||||
// boost::join(left_table_expression_columns_names, ", "),
|
||||
// boost::join(right_table_expression_columns_names, ", "));
|
||||
|
||||
const auto * column_source = column_node->getColumnSource().get();
|
||||
if (!column_source)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No source for column {} in JOIN {}", input_name, join_node.formatASTForErrorMessage());
|
||||
|
||||
bool is_column_from_left_expr = left_table_expressions.contains(column_source);
|
||||
bool is_column_from_right_expr = right_table_expressions.contains(column_source);
|
||||
|
||||
if (!is_column_from_left_expr && !is_column_from_right_expr)
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
"JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns",
|
||||
join_node.formatASTForErrorMessage(),
|
||||
input_name,
|
||||
boost::join(left_table_expression_columns_names, ", "),
|
||||
boost::join(right_table_expression_columns_names, ", "));
|
||||
column_source->formatASTForErrorMessage(),
|
||||
join_node.getLeftTableExpression()->formatASTForErrorMessage(),
|
||||
join_node.getRightTableExpression()->formatASTForErrorMessage());
|
||||
|
||||
auto input_table_side = left_table_expression_contains_input ? JoinTableSide::Left : JoinTableSide::Right;
|
||||
auto input_table_side = is_column_from_left_expr ? JoinTableSide::Left : JoinTableSide::Right;
|
||||
if (table_side && (*table_side) != input_table_side)
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
"JOIN {} join expression contains column from left and right table",
|
||||
@ -159,29 +215,58 @@ std::optional<JoinTableSide> extractJoinTableSideFromExpression(const ActionsDAG
|
||||
return table_side;
|
||||
}
|
||||
|
||||
void buildJoinClause(ActionsDAGPtr join_expression_dag,
|
||||
const std::unordered_set<const ActionsDAG::Node *> & join_expression_dag_input_nodes,
|
||||
const ActionsDAG::Node * join_expressions_actions_node,
|
||||
const NameSet & left_table_expression_columns_names,
|
||||
const NameSet & right_table_expression_columns_names,
|
||||
const ActionsDAG::Node * appendExpression(
|
||||
ActionsDAGPtr & dag,
|
||||
const QueryTreeNodePtr & expression,
|
||||
const PlannerContextPtr & planner_context,
|
||||
const JoinNode & join_node)
|
||||
{
|
||||
PlannerActionsVisitor join_expression_visitor(planner_context);
|
||||
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression);
|
||||
if (join_expression_dag_node_raw_pointers.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"JOIN {} ON clause contains multiple expressions",
|
||||
join_node.formatASTForErrorMessage());
|
||||
|
||||
return join_expression_dag_node_raw_pointers[0];
|
||||
}
|
||||
|
||||
void buildJoinClause(
|
||||
ActionsDAGPtr & left_dag,
|
||||
ActionsDAGPtr & right_dag,
|
||||
const PlannerContextPtr & planner_context,
|
||||
//ActionsDAGPtr join_expression_dag,
|
||||
//const std::unordered_set<const ActionsDAG::Node *> & join_expression_dag_input_nodes,
|
||||
//const ActionsDAG::Node * join_expressions_actions_node,
|
||||
const QueryTreeNodePtr & join_expression,
|
||||
const TableExpressionSet & left_table_expressions,
|
||||
const TableExpressionSet & right_table_expressions,
|
||||
const JoinNode & join_node,
|
||||
JoinClause & join_clause)
|
||||
{
|
||||
std::string function_name;
|
||||
|
||||
if (join_expressions_actions_node->function)
|
||||
function_name = join_expressions_actions_node->function->getName();
|
||||
//std::cerr << join_expression_dag->dumpDAG() << std::endl;
|
||||
auto * function_node = join_expression->as<FunctionNode>();
|
||||
if (function_node)
|
||||
function_name = function_node->getFunction()->getName();
|
||||
|
||||
// if (join_expressions_actions_node->function)
|
||||
// function_name = join_expressions_actions_node->function->getName();
|
||||
|
||||
/// For 'and' function go into children
|
||||
if (function_name == "and")
|
||||
{
|
||||
for (const auto & child : join_expressions_actions_node->children)
|
||||
for (const auto & child : function_node->getArguments())
|
||||
{
|
||||
buildJoinClause(join_expression_dag,
|
||||
join_expression_dag_input_nodes,
|
||||
buildJoinClause(//join_expression_dag,
|
||||
//join_expression_dag_input_nodes,
|
||||
left_dag,
|
||||
right_dag,
|
||||
planner_context,
|
||||
child,
|
||||
left_table_expression_columns_names,
|
||||
right_table_expression_columns_names,
|
||||
left_table_expressions,
|
||||
right_table_expressions,
|
||||
join_node,
|
||||
join_clause);
|
||||
}
|
||||
@ -194,45 +279,49 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag,
|
||||
|
||||
if (function_name == "equals" || function_name == "isNotDistinctFrom" || is_asof_join_inequality)
|
||||
{
|
||||
const auto * left_child = join_expressions_actions_node->children.at(0);
|
||||
const auto * right_child = join_expressions_actions_node->children.at(1);
|
||||
const auto left_child = function_node->getArguments().getNodes().at(0);//join_expressions_actions_node->children.at(0);
|
||||
const auto right_child = function_node->getArguments().getNodes().at(1); //join_expressions_actions_node->children.at(1);
|
||||
|
||||
auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child,
|
||||
join_expression_dag_input_nodes,
|
||||
left_table_expression_columns_names,
|
||||
right_table_expression_columns_names,
|
||||
auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child.get(),
|
||||
//join_expression_dag_input_nodes,
|
||||
left_table_expressions,
|
||||
right_table_expressions,
|
||||
join_node);
|
||||
|
||||
auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child,
|
||||
join_expression_dag_input_nodes,
|
||||
left_table_expression_columns_names,
|
||||
right_table_expression_columns_names,
|
||||
auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child.get(),
|
||||
//join_expression_dag_input_nodes,
|
||||
left_table_expressions,
|
||||
right_table_expressions,
|
||||
join_node);
|
||||
|
||||
if (!left_expression_side_optional && !right_expression_side_optional)
|
||||
{
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
"JOIN {} ON expression {} with constants is not supported",
|
||||
join_node.formatASTForErrorMessage(),
|
||||
join_expressions_actions_node->result_name);
|
||||
"JOIN {} ON expression with constants is not supported",
|
||||
join_node.formatASTForErrorMessage());
|
||||
}
|
||||
else if (left_expression_side_optional && !right_expression_side_optional)
|
||||
{
|
||||
join_clause.addCondition(*left_expression_side_optional, join_expressions_actions_node);
|
||||
auto & dag = *left_expression_side_optional == JoinTableSide::Left ? left_dag : right_dag;
|
||||
const auto * node = appendExpression(dag, join_expression, planner_context, join_node);
|
||||
join_clause.addCondition(*left_expression_side_optional, node);
|
||||
}
|
||||
else if (!left_expression_side_optional && right_expression_side_optional)
|
||||
{
|
||||
join_clause.addCondition(*right_expression_side_optional, join_expressions_actions_node);
|
||||
auto & dag = *right_expression_side_optional == JoinTableSide::Left ? left_dag : right_dag;
|
||||
const auto * node = appendExpression(dag, join_expression, planner_context, join_node);
|
||||
join_clause.addCondition(*right_expression_side_optional, node);
|
||||
}
|
||||
else
|
||||
{
|
||||
// std::cerr << "===============\n";
|
||||
auto left_expression_side = *left_expression_side_optional;
|
||||
auto right_expression_side = *right_expression_side_optional;
|
||||
|
||||
if (left_expression_side != right_expression_side)
|
||||
{
|
||||
const ActionsDAG::Node * left_key = left_child;
|
||||
const ActionsDAG::Node * right_key = right_child;
|
||||
auto left_key = left_child;
|
||||
auto right_key = right_child;
|
||||
|
||||
if (left_expression_side == JoinTableSide::Right)
|
||||
{
|
||||
@ -241,6 +330,9 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag,
|
||||
asof_inequality = reverseASOFJoinInequality(asof_inequality);
|
||||
}
|
||||
|
||||
const auto * left_node = appendExpression(left_dag, left_key, planner_context, join_node);
|
||||
const auto * right_node = appendExpression(right_dag, right_key, planner_context, join_node);
|
||||
|
||||
if (is_asof_join_inequality)
|
||||
{
|
||||
if (join_clause.hasASOF())
|
||||
@ -250,55 +342,66 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag,
|
||||
join_node.formatASTForErrorMessage());
|
||||
}
|
||||
|
||||
join_clause.addASOFKey(left_key, right_key, asof_inequality);
|
||||
join_clause.addASOFKey(left_node, right_node, asof_inequality);
|
||||
}
|
||||
else
|
||||
{
|
||||
bool null_safe_comparison = function_name == "isNotDistinctFrom";
|
||||
join_clause.addKey(left_key, right_key, null_safe_comparison);
|
||||
join_clause.addKey(left_node, right_node, null_safe_comparison);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
join_clause.addCondition(left_expression_side, join_expressions_actions_node);
|
||||
auto & dag = left_expression_side == JoinTableSide::Left ? left_dag : right_dag;
|
||||
const auto * node = appendExpression(dag, join_expression, planner_context, join_node);
|
||||
join_clause.addCondition(left_expression_side, node);
|
||||
}
|
||||
}
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
auto expression_side_optional = extractJoinTableSideFromExpression(join_expressions_actions_node,
|
||||
join_expression_dag_input_nodes,
|
||||
left_table_expression_columns_names,
|
||||
right_table_expression_columns_names,
|
||||
auto expression_side_optional = extractJoinTableSideFromExpression(//join_expressions_actions_node,
|
||||
//join_expression_dag_input_nodes,
|
||||
join_expression.get(),
|
||||
left_table_expressions,
|
||||
right_table_expressions,
|
||||
join_node);
|
||||
|
||||
if (!expression_side_optional)
|
||||
expression_side_optional = JoinTableSide::Right;
|
||||
|
||||
auto expression_side = *expression_side_optional;
|
||||
join_clause.addCondition(expression_side, join_expressions_actions_node);
|
||||
auto & dag = expression_side == JoinTableSide::Left ? left_dag : right_dag;
|
||||
const auto * node = appendExpression(dag, join_expression, planner_context, join_node);
|
||||
join_clause.addCondition(expression_side, node);
|
||||
}
|
||||
|
||||
JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & join_expression_input_columns,
|
||||
JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName & join_expression_input_columns,
|
||||
const ColumnsWithTypeAndName & left_table_expression_columns,
|
||||
const ColumnsWithTypeAndName & right_table_expression_columns,
|
||||
const JoinNode & join_node,
|
||||
const PlannerContextPtr & planner_context)
|
||||
{
|
||||
ActionsDAGPtr join_expression_actions = std::make_shared<ActionsDAG>(join_expression_input_columns);
|
||||
//ActionsDAGPtr join_expression_actions = std::make_shared<ActionsDAG>(join_expression_input_columns);
|
||||
|
||||
ActionsDAGPtr left_join_actions = std::make_shared<ActionsDAG>(left_table_expression_columns);
|
||||
ActionsDAGPtr right_join_actions = std::make_shared<ActionsDAG>(right_table_expression_columns);
|
||||
|
||||
// LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions cols {} ", left_join_actions->dumpDAG());
|
||||
// LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions cols {} ", right_join_actions->dumpDAG());
|
||||
|
||||
/** In ActionsDAG if input node has constant representation additional constant column is added.
|
||||
* That way we cannot simply check that node has INPUT type during resolution of expression join table side.
|
||||
* Put all nodes after actions dag initialization in set.
|
||||
* To check if actions dag node is input column, we check if set contains it.
|
||||
*/
|
||||
const auto & join_expression_actions_nodes = join_expression_actions->getNodes();
|
||||
// const auto & join_expression_actions_nodes = join_expression_actions->getNodes();
|
||||
|
||||
std::unordered_set<const ActionsDAG::Node *> join_expression_dag_input_nodes;
|
||||
join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size());
|
||||
for (const auto & node : join_expression_actions_nodes)
|
||||
join_expression_dag_input_nodes.insert(&node);
|
||||
// std::unordered_set<const ActionsDAG::Node *> join_expression_dag_input_nodes;
|
||||
// join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size());
|
||||
// for (const auto & node : join_expression_actions_nodes)
|
||||
// join_expression_dag_input_nodes.insert(&node);
|
||||
|
||||
/** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction.
|
||||
* If we do not ignore it, this function will be replaced by underlying constant.
|
||||
@ -308,6 +411,9 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
* ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value);
|
||||
*/
|
||||
auto join_expression = join_node.getJoinExpression();
|
||||
// LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->formatConvertedASTForErrorMessage());
|
||||
// LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->dumpTree());
|
||||
|
||||
auto * constant_join_expression = join_expression->as<ConstantNode>();
|
||||
|
||||
if (constant_join_expression && constant_join_expression->hasSourceExpression())
|
||||
@ -319,18 +425,18 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
"JOIN {} join expression expected function",
|
||||
join_node.formatASTForErrorMessage());
|
||||
|
||||
PlannerActionsVisitor join_expression_visitor(planner_context);
|
||||
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression);
|
||||
if (join_expression_dag_node_raw_pointers.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"JOIN {} ON clause contains multiple expressions",
|
||||
join_node.formatASTForErrorMessage());
|
||||
// PlannerActionsVisitor join_expression_visitor(planner_context);
|
||||
// auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression);
|
||||
// if (join_expression_dag_node_raw_pointers.size() != 1)
|
||||
// throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
// "JOIN {} ON clause contains multiple expressions",
|
||||
// join_node.formatASTForErrorMessage());
|
||||
|
||||
const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0];
|
||||
if (!join_expressions_actions_root_node->function)
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
"JOIN {} join expression expected function",
|
||||
join_node.formatASTForErrorMessage());
|
||||
// const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0];
|
||||
// if (!join_expressions_actions_root_node->function)
|
||||
// throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
// "JOIN {} join expression expected function",
|
||||
// join_node.formatASTForErrorMessage());
|
||||
|
||||
size_t left_table_expression_columns_size = left_table_expression_columns.size();
|
||||
|
||||
@ -360,21 +466,27 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
join_right_actions_names_set.insert(right_table_expression_column.name);
|
||||
}
|
||||
|
||||
JoinClausesAndActions result;
|
||||
result.join_expression_actions = join_expression_actions;
|
||||
auto join_left_table_expressions = extractTableExpressionsSet(join_node.getLeftTableExpression());
|
||||
auto join_right_table_expressions = extractTableExpressionsSet(join_node.getRightTableExpression());
|
||||
|
||||
const auto & function_name = join_expressions_actions_root_node->function->getName();
|
||||
JoinClausesAndActions result;
|
||||
//result.join_expression_actions = join_expression_actions;
|
||||
|
||||
const auto & function_name = function_node->getFunction()->getName();
|
||||
if (function_name == "or")
|
||||
{
|
||||
for (const auto & child : join_expressions_actions_root_node->children)
|
||||
for (const auto & child : function_node->getArguments())
|
||||
{
|
||||
result.join_clauses.emplace_back();
|
||||
|
||||
buildJoinClause(join_expression_actions,
|
||||
join_expression_dag_input_nodes,
|
||||
buildJoinClause(//join_expression_actions,
|
||||
//join_expression_dag_input_nodes,
|
||||
left_join_actions,
|
||||
right_join_actions,
|
||||
planner_context,
|
||||
child,
|
||||
join_left_actions_names_set,
|
||||
join_right_actions_names_set,
|
||||
join_left_table_expressions,
|
||||
join_right_table_expressions,
|
||||
join_node,
|
||||
result.join_clauses.back());
|
||||
}
|
||||
@ -383,11 +495,15 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
{
|
||||
result.join_clauses.emplace_back();
|
||||
|
||||
buildJoinClause(join_expression_actions,
|
||||
join_expression_dag_input_nodes,
|
||||
join_expressions_actions_root_node,
|
||||
join_left_actions_names_set,
|
||||
join_right_actions_names_set,
|
||||
buildJoinClause(
|
||||
left_join_actions,
|
||||
right_join_actions,
|
||||
planner_context,
|
||||
//join_expression_actions,
|
||||
//join_expression_dag_input_nodes,
|
||||
join_expression, //join_expressions_actions_root_node,
|
||||
join_left_table_expressions,
|
||||
join_right_table_expressions,
|
||||
join_node,
|
||||
result.join_clauses.back());
|
||||
}
|
||||
@ -412,12 +528,12 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
const ActionsDAG::Node * dag_filter_condition_node = nullptr;
|
||||
|
||||
if (left_filter_condition_nodes.size() > 1)
|
||||
dag_filter_condition_node = &join_expression_actions->addFunction(and_function, left_filter_condition_nodes, {});
|
||||
dag_filter_condition_node = &left_join_actions->addFunction(and_function, left_filter_condition_nodes, {});
|
||||
else
|
||||
dag_filter_condition_node = left_filter_condition_nodes[0];
|
||||
|
||||
join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node};
|
||||
join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
|
||||
left_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
|
||||
|
||||
add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name);
|
||||
}
|
||||
@ -428,12 +544,12 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
const ActionsDAG::Node * dag_filter_condition_node = nullptr;
|
||||
|
||||
if (right_filter_condition_nodes.size() > 1)
|
||||
dag_filter_condition_node = &join_expression_actions->addFunction(and_function, right_filter_condition_nodes, {});
|
||||
dag_filter_condition_node = &right_join_actions->addFunction(and_function, right_filter_condition_nodes, {});
|
||||
else
|
||||
dag_filter_condition_node = right_filter_condition_nodes[0];
|
||||
|
||||
join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node};
|
||||
join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
|
||||
right_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
|
||||
|
||||
add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name);
|
||||
}
|
||||
@ -470,10 +586,10 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
}
|
||||
|
||||
if (!left_key_node->result_type->equals(*common_type))
|
||||
left_key_node = &join_expression_actions->addCast(*left_key_node, common_type, {});
|
||||
left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {});
|
||||
|
||||
if (!right_key_node->result_type->equals(*common_type))
|
||||
right_key_node = &join_expression_actions->addCast(*right_key_node, common_type, {});
|
||||
right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {});
|
||||
}
|
||||
|
||||
if (join_clause.isNullsafeCompareKey(i) && left_key_node->result_type->isNullable() && right_key_node->result_type->isNullable())
|
||||
@ -490,22 +606,29 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
|
||||
* SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b)
|
||||
*/
|
||||
auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext());
|
||||
left_key_node = &join_expression_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {});
|
||||
right_key_node = &join_expression_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {});
|
||||
left_key_node = &left_join_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {});
|
||||
right_key_node = &right_join_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {});
|
||||
}
|
||||
|
||||
join_expression_actions->addOrReplaceInOutputs(*left_key_node);
|
||||
join_expression_actions->addOrReplaceInOutputs(*right_key_node);
|
||||
left_join_actions->addOrReplaceInOutputs(*left_key_node);
|
||||
right_join_actions->addOrReplaceInOutputs(*right_key_node);
|
||||
|
||||
add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name);
|
||||
add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name);
|
||||
}
|
||||
}
|
||||
|
||||
result.left_join_expressions_actions = join_expression_actions->clone();
|
||||
result.left_join_expressions_actions = left_join_actions->clone();
|
||||
result.left_join_tmp_expression_actions = std::move(left_join_actions);
|
||||
result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names);
|
||||
|
||||
result.right_join_expressions_actions = join_expression_actions->clone();
|
||||
// for (const auto & name : join_right_actions_names)
|
||||
// std::cerr << ".. " << name << std::endl;
|
||||
|
||||
// std::cerr << right_join_actions->dumpDAG() << std::endl;
|
||||
|
||||
result.right_join_expressions_actions = right_join_actions->clone();
|
||||
result.right_join_tmp_expression_actions = std::move(right_join_actions);
|
||||
result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names);
|
||||
|
||||
return result;
|
||||
@ -525,10 +648,10 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
"JOIN {} join does not have ON section",
|
||||
join_node_typed.formatASTForErrorMessage());
|
||||
|
||||
auto join_expression_input_columns = left_table_expression_columns;
|
||||
join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end());
|
||||
// auto join_expression_input_columns = left_table_expression_columns;
|
||||
// join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end());
|
||||
|
||||
return buildJoinClausesAndActions(join_expression_input_columns, left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context);
|
||||
return buildJoinClausesAndActions(/*join_expression_input_columns,*/ left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context);
|
||||
}
|
||||
|
||||
std::optional<bool> tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join_node)
|
||||
|
@ -165,7 +165,8 @@ struct JoinClausesAndActions
|
||||
/// Join clauses. Actions dag nodes point into join_expression_actions.
|
||||
JoinClauses join_clauses;
|
||||
/// Whole JOIN ON section expressions
|
||||
ActionsDAGPtr join_expression_actions;
|
||||
ActionsDAGPtr left_join_tmp_expression_actions;
|
||||
ActionsDAGPtr right_join_tmp_expression_actions;
|
||||
/// Left join expressions actions
|
||||
ActionsDAGPtr left_join_expressions_actions;
|
||||
/// Right join expressions actions
|
||||
|
@ -357,6 +357,7 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con
|
||||
|
||||
QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node,
|
||||
const ContextPtr & context,
|
||||
//PlannerContext & planner_context,
|
||||
ResultReplacementMap * result_replacement_map)
|
||||
{
|
||||
auto & query_node_typed = query_node->as<QueryNode &>();
|
||||
@ -406,6 +407,13 @@ QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr &
|
||||
if (result_replacement_map)
|
||||
result_replacement_map->emplace(table_expression, dummy_table_node);
|
||||
|
||||
dummy_table_node->setAlias(table_expression->getAlias());
|
||||
|
||||
// auto & src_table_expression_data = planner_context.getOrCreateTableExpressionData(table_expression);
|
||||
// auto & dst_table_expression_data = planner_context.getOrCreateTableExpressionData(dummy_table_node);
|
||||
|
||||
// dst_table_expression_data = src_table_expression_data;
|
||||
|
||||
replacement_map.emplace(table_expression.get(), std::move(dummy_table_node));
|
||||
}
|
||||
|
||||
|
@ -436,7 +436,6 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
|
||||
AggregateProjectionCandidates candidates;
|
||||
|
||||
const auto & parts = reading.getParts();
|
||||
const auto & query_info = reading.getQueryInfo();
|
||||
|
||||
const auto metadata = reading.getStorageMetadata();
|
||||
ContextPtr context = reading.getContext();
|
||||
@ -481,8 +480,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
|
||||
auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock(
|
||||
metadata,
|
||||
candidate.dag->getRequiredColumnsNames(),
|
||||
dag.filter_node != nullptr,
|
||||
query_info,
|
||||
(dag.filter_node ? dag.dag : nullptr),
|
||||
parts,
|
||||
max_added_blocks.get(),
|
||||
context);
|
||||
|
@ -23,6 +23,8 @@
|
||||
#include <Processors/Transforms/ReverseTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexAnnoy.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexUSearch.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreePrefetchedReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolInOrder.h>
|
||||
@ -1337,26 +1339,12 @@ static void buildIndexes(
|
||||
const Names & primary_key_column_names = primary_key.column_names;
|
||||
|
||||
const auto & settings = context->getSettingsRef();
|
||||
if (settings.query_plan_optimize_primary_key)
|
||||
{
|
||||
NameSet array_join_name_set;
|
||||
if (query_info.syntax_analyzer_result)
|
||||
array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet();
|
||||
|
||||
indexes.emplace(ReadFromMergeTree::Indexes{{
|
||||
filter_actions_dag,
|
||||
context,
|
||||
primary_key_column_names,
|
||||
primary_key.expression}, {}, {}, {}, {}, false, {}});
|
||||
}
|
||||
else
|
||||
{
|
||||
indexes.emplace(ReadFromMergeTree::Indexes{{
|
||||
query_info,
|
||||
context,
|
||||
primary_key_column_names,
|
||||
primary_key.expression}, {}, {}, {}, {}, false, {}});
|
||||
}
|
||||
indexes.emplace(ReadFromMergeTree::Indexes{{
|
||||
filter_actions_dag,
|
||||
context,
|
||||
primary_key_column_names,
|
||||
primary_key.expression}, {}, {}, {}, {}, false, {}});
|
||||
|
||||
if (metadata_snapshot->hasPartitionKey())
|
||||
{
|
||||
@ -1369,11 +1357,7 @@ static void buildIndexes(
|
||||
}
|
||||
|
||||
/// TODO Support row_policy_filter and additional_filters
|
||||
if (settings.allow_experimental_analyzer)
|
||||
indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context);
|
||||
else
|
||||
indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context);
|
||||
|
||||
indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context);
|
||||
MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(indexes->part_offset_condition, filter_actions_dag, context);
|
||||
|
||||
indexes->use_skip_indexes = settings.use_skip_indexes;
|
||||
@ -1385,14 +1369,18 @@ static void buildIndexes(
|
||||
if (!indexes->use_skip_indexes)
|
||||
return;
|
||||
|
||||
const SelectQueryInfo * info = &query_info;
|
||||
std::optional<SelectQueryInfo> info_copy;
|
||||
if (settings.allow_experimental_analyzer)
|
||||
auto get_query_info = [&]() -> const SelectQueryInfo &
|
||||
{
|
||||
info_copy.emplace(query_info);
|
||||
info_copy->filter_actions_dag = filter_actions_dag;
|
||||
info = &*info_copy;
|
||||
}
|
||||
if (settings.allow_experimental_analyzer)
|
||||
{
|
||||
info_copy.emplace(query_info);
|
||||
info_copy->filter_actions_dag = filter_actions_dag;
|
||||
return *info_copy;
|
||||
}
|
||||
|
||||
return query_info;
|
||||
};
|
||||
|
||||
std::unordered_set<std::string> ignored_index_names;
|
||||
|
||||
@ -1433,14 +1421,30 @@ static void buildIndexes(
|
||||
if (inserted)
|
||||
{
|
||||
skip_indexes.merged_indices.emplace_back();
|
||||
skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot);
|
||||
skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(get_query_info(), metadata_snapshot);
|
||||
}
|
||||
|
||||
skip_indexes.merged_indices[it->second].addIndex(index_helper);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto condition = index_helper->createIndexCondition(*info, context);
|
||||
MergeTreeIndexConditionPtr condition;
|
||||
if (index_helper->isVectorSearch())
|
||||
{
|
||||
#ifdef ENABLE_ANNOY
|
||||
if (const auto * annoy = typeid_cast<const MergeTreeIndexAnnoy *>(index_helper.get()))
|
||||
condition = annoy->createIndexCondition(get_query_info(), context);
|
||||
#endif
|
||||
#ifdef ENABLE_USEARCH
|
||||
if (const auto * usearch = typeid_cast<const MergeTreeIndexUSearch *>(index_helper.get()))
|
||||
condition = usearch->createIndexCondition(get_query_info(), context);
|
||||
#endif
|
||||
if (!condition)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name);
|
||||
}
|
||||
else
|
||||
condition = index_helper->createIndexCondition(filter_actions_dag, context);
|
||||
|
||||
if (!condition->alwaysUnknownOrTrue())
|
||||
skip_indexes.useful_indices.emplace_back(index_helper, condition);
|
||||
}
|
||||
@ -1473,34 +1477,15 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
Poco::Logger * log,
|
||||
std::optional<Indexes> & indexes)
|
||||
{
|
||||
const auto & settings = context->getSettingsRef();
|
||||
if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key)
|
||||
{
|
||||
auto updated_query_info_with_filter_dag = query_info;
|
||||
updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info);
|
||||
|
||||
return selectRangesToReadImpl(
|
||||
std::move(parts),
|
||||
std::move(alter_conversions),
|
||||
metadata_snapshot_base,
|
||||
metadata_snapshot,
|
||||
updated_query_info_with_filter_dag,
|
||||
context,
|
||||
num_streams,
|
||||
max_block_numbers_to_read,
|
||||
data,
|
||||
real_column_names,
|
||||
sample_factor_column_queried,
|
||||
log,
|
||||
indexes);
|
||||
}
|
||||
auto updated_query_info_with_filter_dag = query_info;
|
||||
updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info);
|
||||
|
||||
return selectRangesToReadImpl(
|
||||
std::move(parts),
|
||||
std::move(alter_conversions),
|
||||
metadata_snapshot_base,
|
||||
metadata_snapshot,
|
||||
query_info,
|
||||
updated_query_info_with_filter_dag,
|
||||
context,
|
||||
num_streams,
|
||||
max_block_numbers_to_read,
|
||||
|
@ -30,19 +30,9 @@ void ReadFromStorageStep::applyFilters()
|
||||
if (!context)
|
||||
return;
|
||||
|
||||
std::shared_ptr<const KeyCondition> key_condition;
|
||||
if (!context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
if (auto * source = dynamic_cast<SourceWithKeyCondition *>(processor.get()))
|
||||
source->setKeyCondition(query_info, context);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
if (auto * source = dynamic_cast<SourceWithKeyCondition *>(processor.get()))
|
||||
source->setKeyCondition(filter_nodes.nodes, context);
|
||||
}
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
if (auto * source = dynamic_cast<SourceWithKeyCondition *>(processor.get()))
|
||||
source->setKeyCondition(filter_nodes.nodes, context);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -16,15 +16,6 @@ protected:
|
||||
/// Represents pushed down filters in source
|
||||
std::shared_ptr<const KeyCondition> key_condition;
|
||||
|
||||
void setKeyConditionImpl(const SelectQueryInfo & query_info, ContextPtr context, const Block & keys)
|
||||
{
|
||||
key_condition = std::make_shared<const KeyCondition>(
|
||||
query_info,
|
||||
context,
|
||||
keys.getNames(),
|
||||
std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(keys.getColumnsWithTypeAndName())));
|
||||
}
|
||||
|
||||
void setKeyConditionImpl(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context, const Block & keys)
|
||||
{
|
||||
std::unordered_map<std::string, DB::ColumnWithTypeAndName> node_name_to_input_column;
|
||||
@ -46,10 +37,7 @@ public:
|
||||
/// Set key_condition directly. It is used for filter push down in source.
|
||||
virtual void setKeyCondition(const std::shared_ptr<const KeyCondition> & key_condition_) { key_condition = key_condition_; }
|
||||
|
||||
/// Set key_condition created by query_info and context. It is used for filter push down when allow_experimental_analyzer is false.
|
||||
virtual void setKeyCondition(const SelectQueryInfo & /*query_info*/, ContextPtr /*context*/) { }
|
||||
|
||||
/// Set key_condition created by nodes and context. It is used for filter push down when allow_experimental_analyzer is true.
|
||||
/// Set key_condition created by nodes and context.
|
||||
virtual void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & /*nodes*/, ContextPtr /*context*/) { }
|
||||
};
|
||||
}
|
||||
|
@ -29,10 +29,14 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <QueryPipeline/QueryPipeline.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Processors/ISource.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/SourceStepWithFilter.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/AsynchronousReadBufferFromHDFS.h>
|
||||
@ -123,7 +127,6 @@ public:
|
||||
String compression_method_,
|
||||
Block sample_block_,
|
||||
ContextPtr context_,
|
||||
const SelectQueryInfo & query_info_,
|
||||
UInt64 max_block_size_,
|
||||
const StorageHive & storage_,
|
||||
const Names & text_input_field_names_ = {})
|
||||
@ -140,7 +143,6 @@ public:
|
||||
, text_input_field_names(text_input_field_names_)
|
||||
, format_settings(getFormatSettings(getContext()))
|
||||
, read_settings(getContext()->getReadSettings())
|
||||
, query_info(query_info_)
|
||||
{
|
||||
to_read_block = sample_block;
|
||||
|
||||
@ -395,7 +397,6 @@ private:
|
||||
const Names & text_input_field_names;
|
||||
FormatSettings format_settings;
|
||||
ReadSettings read_settings;
|
||||
SelectQueryInfo query_info;
|
||||
|
||||
HiveFilePtr current_file;
|
||||
String current_path;
|
||||
@ -574,7 +575,7 @@ static HiveFilePtr createHiveFile(
|
||||
|
||||
HiveFiles StorageHive::collectHiveFilesFromPartition(
|
||||
const Apache::Hadoop::Hive::Partition & partition,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const HDFSFSPtr & fs,
|
||||
const ContextPtr & context_,
|
||||
@ -638,7 +639,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition(
|
||||
for (size_t i = 0; i < partition_names.size(); ++i)
|
||||
ranges.emplace_back(fields[i]);
|
||||
|
||||
const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr);
|
||||
const KeyCondition partition_key_condition(filter_actions_dag, getContext(), partition_names, partition_minmax_idx_expr);
|
||||
if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true)
|
||||
return {};
|
||||
}
|
||||
@ -648,7 +649,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition(
|
||||
hive_files.reserve(file_infos.size());
|
||||
for (const auto & file_info : file_infos)
|
||||
{
|
||||
auto hive_file = getHiveFileIfNeeded(file_info, fields, query_info, hive_table_metadata, context_, prune_level);
|
||||
auto hive_file = getHiveFileIfNeeded(file_info, fields, filter_actions_dag, hive_table_metadata, context_, prune_level);
|
||||
if (hive_file)
|
||||
{
|
||||
LOG_TRACE(
|
||||
@ -672,7 +673,7 @@ StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hiv
|
||||
HiveFilePtr StorageHive::getHiveFileIfNeeded(
|
||||
const FileInfo & file_info,
|
||||
const FieldVector & fields,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const ContextPtr & context_,
|
||||
PruneLevel prune_level) const
|
||||
@ -706,7 +707,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded(
|
||||
|
||||
if (prune_level >= PruneLevel::File)
|
||||
{
|
||||
const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr);
|
||||
const KeyCondition hivefile_key_condition(filter_actions_dag, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr);
|
||||
if (hive_file->useFileMinMaxIndex())
|
||||
{
|
||||
/// Load file level minmax index and apply
|
||||
@ -758,10 +759,77 @@ bool StorageHive::supportsSubsetOfColumns() const
|
||||
return format_name == "Parquet" || format_name == "ORC";
|
||||
}
|
||||
|
||||
Pipe StorageHive::read(
|
||||
class ReadFromHive : public SourceStepWithFilter
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "ReadFromHive"; }
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
void applyFilters() override;
|
||||
|
||||
ReadFromHive(
|
||||
Block header,
|
||||
std::shared_ptr<StorageHive> storage_,
|
||||
std::shared_ptr<StorageHiveSource::SourcesInfo> sources_info_,
|
||||
HDFSBuilderWrapper builder_,
|
||||
HDFSFSPtr fs_,
|
||||
HiveMetastoreClient::HiveTableMetadataPtr hive_table_metadata_,
|
||||
Block sample_block_,
|
||||
Poco::Logger * log_,
|
||||
ContextPtr context_,
|
||||
size_t max_block_size_,
|
||||
size_t num_streams_)
|
||||
: SourceStepWithFilter(DataStream{.header = std::move(header)})
|
||||
, storage(std::move(storage_))
|
||||
, sources_info(std::move(sources_info_))
|
||||
, builder(std::move(builder_))
|
||||
, fs(std::move(fs_))
|
||||
, hive_table_metadata(std::move(hive_table_metadata_))
|
||||
, sample_block(std::move(sample_block_))
|
||||
, log(log_)
|
||||
, context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, num_streams(num_streams_)
|
||||
{
|
||||
}
|
||||
|
||||
private:
|
||||
std::shared_ptr<StorageHive> storage;
|
||||
std::shared_ptr<StorageHiveSource::SourcesInfo> sources_info;
|
||||
HDFSBuilderWrapper builder;
|
||||
HDFSFSPtr fs;
|
||||
HiveMetastoreClient::HiveTableMetadataPtr hive_table_metadata;
|
||||
Block sample_block;
|
||||
Poco::Logger * log;
|
||||
|
||||
ContextPtr context;
|
||||
size_t max_block_size;
|
||||
size_t num_streams;
|
||||
|
||||
std::optional<HiveFiles> hive_files;
|
||||
|
||||
void createFiles(const ActionsDAGPtr & filter_actions_dag);
|
||||
};
|
||||
|
||||
void ReadFromHive::applyFilters()
|
||||
{
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
|
||||
createFiles(filter_actions_dag);
|
||||
}
|
||||
|
||||
void ReadFromHive::createFiles(const ActionsDAGPtr & filter_actions_dag)
|
||||
{
|
||||
if (hive_files)
|
||||
return;
|
||||
|
||||
hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag, hive_table_metadata, fs, context);
|
||||
LOG_INFO(log, "Collect {} hive files to read", hive_files->size());
|
||||
}
|
||||
|
||||
void StorageHive::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
SelectQueryInfo &,
|
||||
ContextPtr context_,
|
||||
QueryProcessingStage::Enum /* processed_stage */,
|
||||
size_t max_block_size,
|
||||
@ -774,15 +842,7 @@ Pipe StorageHive::read(
|
||||
auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url);
|
||||
auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table);
|
||||
|
||||
/// Collect Hive files to read
|
||||
HiveFiles hive_files = collectHiveFiles(num_streams, query_info, hive_table_metadata, fs, context_);
|
||||
LOG_INFO(log, "Collect {} hive files to read", hive_files.size());
|
||||
|
||||
if (hive_files.empty())
|
||||
return {};
|
||||
|
||||
auto sources_info = std::make_shared<StorageHiveSource::SourcesInfo>();
|
||||
sources_info->hive_files = std::move(hive_files);
|
||||
sources_info->database_name = hive_database;
|
||||
sources_info->table_name = hive_table;
|
||||
sources_info->hive_metastore_client = hive_metastore_client;
|
||||
@ -822,6 +882,36 @@ Pipe StorageHive::read(
|
||||
sources_info->need_file_column = true;
|
||||
}
|
||||
|
||||
auto this_ptr = std::static_pointer_cast<StorageHive>(shared_from_this());
|
||||
|
||||
auto reading = std::make_unique<ReadFromHive>(
|
||||
StorageHiveSource::getHeader(sample_block, sources_info),
|
||||
std::move(this_ptr),
|
||||
std::move(sources_info),
|
||||
std::move(builder),
|
||||
std::move(fs),
|
||||
std::move(hive_table_metadata),
|
||||
std::move(sample_block),
|
||||
log,
|
||||
context_,
|
||||
max_block_size,
|
||||
num_streams);
|
||||
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
void ReadFromHive::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
createFiles(nullptr);
|
||||
|
||||
if (hive_files->empty())
|
||||
{
|
||||
pipeline.init(Pipe(std::make_shared<NullSource>(getOutputStream().header)));
|
||||
return;
|
||||
}
|
||||
|
||||
sources_info->hive_files = std::move(*hive_files);
|
||||
|
||||
if (num_streams > sources_info->hive_files.size())
|
||||
num_streams = sources_info->hive_files.size();
|
||||
|
||||
@ -830,22 +920,29 @@ Pipe StorageHive::read(
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<StorageHiveSource>(
|
||||
sources_info,
|
||||
hdfs_namenode_url,
|
||||
format_name,
|
||||
compression_method,
|
||||
storage->hdfs_namenode_url,
|
||||
storage->format_name,
|
||||
storage->compression_method,
|
||||
sample_block,
|
||||
context_,
|
||||
query_info,
|
||||
context,
|
||||
max_block_size,
|
||||
*this,
|
||||
text_input_field_names));
|
||||
*storage,
|
||||
storage->text_input_field_names));
|
||||
}
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
if (pipe.empty())
|
||||
pipe = Pipe(std::make_shared<NullSource>(getOutputStream().header));
|
||||
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
}
|
||||
|
||||
HiveFiles StorageHive::collectHiveFiles(
|
||||
size_t max_threads,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const HDFSFSPtr & fs,
|
||||
const ContextPtr & context_,
|
||||
@ -871,7 +968,7 @@ HiveFiles StorageHive::collectHiveFiles(
|
||||
[&]()
|
||||
{
|
||||
auto hive_files_in_partition
|
||||
= collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_, prune_level);
|
||||
= collectHiveFilesFromPartition(partition, filter_actions_dag, hive_table_metadata, fs, context_, prune_level);
|
||||
if (!hive_files_in_partition.empty())
|
||||
{
|
||||
std::lock_guard lock(hive_files_mutex);
|
||||
@ -897,7 +994,7 @@ HiveFiles StorageHive::collectHiveFiles(
|
||||
pool.scheduleOrThrowOnError(
|
||||
[&]()
|
||||
{
|
||||
auto hive_file = getHiveFileIfNeeded(file_info, {}, query_info, hive_table_metadata, context_, prune_level);
|
||||
auto hive_file = getHiveFileIfNeeded(file_info, {}, filter_actions_dag, hive_table_metadata, context_, prune_level);
|
||||
if (hive_file)
|
||||
{
|
||||
std::lock_guard lock(hive_files_mutex);
|
||||
@ -925,13 +1022,12 @@ NamesAndTypesList StorageHive::getVirtuals() const
|
||||
std::optional<UInt64> StorageHive::totalRows(const Settings & settings) const
|
||||
{
|
||||
/// query_info is not used when prune_level == PruneLevel::None
|
||||
SelectQueryInfo query_info;
|
||||
return totalRowsImpl(settings, query_info, getContext(), PruneLevel::None);
|
||||
return totalRowsImpl(settings, nullptr, getContext(), PruneLevel::None);
|
||||
}
|
||||
|
||||
std::optional<UInt64> StorageHive::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const
|
||||
std::optional<UInt64> StorageHive::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const
|
||||
{
|
||||
return totalRowsImpl(context_->getSettingsRef(), query_info, context_, PruneLevel::Partition);
|
||||
return totalRowsImpl(context_->getSettingsRef(), filter_actions_dag, context_, PruneLevel::Partition);
|
||||
}
|
||||
|
||||
void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const
|
||||
@ -946,7 +1042,7 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt
|
||||
}
|
||||
|
||||
std::optional<UInt64>
|
||||
StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const
|
||||
StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const
|
||||
{
|
||||
/// Row-based format like Text doesn't support totalRowsByPartitionPredicate
|
||||
if (!supportsSubsetOfColumns())
|
||||
@ -958,7 +1054,7 @@ StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & qu
|
||||
HDFSFSPtr fs = createHDFSFS(builder.get());
|
||||
HiveFiles hive_files = collectHiveFiles(
|
||||
settings.max_threads,
|
||||
query_info,
|
||||
filter_actions_dag,
|
||||
hive_table_metadata,
|
||||
fs,
|
||||
context_,
|
||||
|
@ -42,10 +42,11 @@ public:
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
SelectQueryInfo &,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
@ -58,9 +59,12 @@ public:
|
||||
bool supportsSubsetOfColumns() const;
|
||||
|
||||
std::optional<UInt64> totalRows(const Settings & settings) const override;
|
||||
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override;
|
||||
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override;
|
||||
|
||||
protected:
|
||||
friend class ReadFromHive;
|
||||
|
||||
private:
|
||||
using FileFormat = IHiveFile::FileFormat;
|
||||
using FileInfo = HiveMetastoreClient::FileInfo;
|
||||
@ -88,7 +92,7 @@ private:
|
||||
|
||||
HiveFiles collectHiveFiles(
|
||||
size_t max_threads,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const HDFSFSPtr & fs,
|
||||
const ContextPtr & context_,
|
||||
@ -96,7 +100,7 @@ private:
|
||||
|
||||
HiveFiles collectHiveFilesFromPartition(
|
||||
const Apache::Hadoop::Hive::Partition & partition,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const HDFSFSPtr & fs,
|
||||
const ContextPtr & context_,
|
||||
@ -105,7 +109,7 @@ private:
|
||||
HiveFilePtr getHiveFileIfNeeded(
|
||||
const FileInfo & file_info,
|
||||
const FieldVector & fields,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const ContextPtr & context_,
|
||||
PruneLevel prune_level = PruneLevel::Max) const;
|
||||
@ -113,7 +117,7 @@ private:
|
||||
void lazyInitialize();
|
||||
|
||||
std::optional<UInt64>
|
||||
totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const;
|
||||
totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const;
|
||||
|
||||
String hive_metastore_url;
|
||||
|
||||
|
@ -669,7 +669,7 @@ public:
|
||||
virtual std::optional<UInt64> totalRows(const Settings &) const { return {}; }
|
||||
|
||||
/// Same as above but also take partition predicate into account.
|
||||
virtual std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const { return {}; }
|
||||
virtual std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr &, ContextPtr) const { return {}; }
|
||||
|
||||
/// If it is possible to quickly determine exact number of bytes for the table on storage:
|
||||
/// - memory (approximated, resident)
|
||||
|
@ -762,92 +762,6 @@ void KeyCondition::getAllSpaceFillingCurves()
|
||||
}
|
||||
}
|
||||
|
||||
KeyCondition::KeyCondition(
|
||||
const ASTPtr & query,
|
||||
const ASTs & additional_filter_asts,
|
||||
Block block_with_constants,
|
||||
PreparedSetsPtr prepared_sets,
|
||||
ContextPtr context,
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr_,
|
||||
NameSet array_joined_column_names_,
|
||||
bool single_point_,
|
||||
bool strict_)
|
||||
: key_expr(key_expr_)
|
||||
, key_subexpr_names(getAllSubexpressionNames(*key_expr))
|
||||
, array_joined_column_names(std::move(array_joined_column_names_))
|
||||
, single_point(single_point_)
|
||||
, strict(strict_)
|
||||
{
|
||||
size_t key_index = 0;
|
||||
for (const auto & name : key_column_names)
|
||||
{
|
||||
if (!key_columns.contains(name))
|
||||
{
|
||||
key_columns[name] = key_columns.size();
|
||||
key_indices.push_back(key_index);
|
||||
}
|
||||
++key_index;
|
||||
}
|
||||
|
||||
if (context->getSettingsRef().analyze_index_with_space_filling_curves)
|
||||
getAllSpaceFillingCurves();
|
||||
|
||||
ASTPtr filter_node;
|
||||
if (query)
|
||||
filter_node = buildFilterNode(query, additional_filter_asts);
|
||||
|
||||
if (!filter_node)
|
||||
{
|
||||
has_filter = false;
|
||||
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
has_filter = true;
|
||||
|
||||
/** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
|
||||
* the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
|
||||
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
|
||||
* when parsing the AST into internal RPN representation.
|
||||
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
|
||||
* are pushed down and applied (when possible) to leaf nodes.
|
||||
*/
|
||||
auto inverted_filter_node = DB::cloneASTWithInversionPushDown(filter_node);
|
||||
|
||||
RPNBuilder<RPNElement> builder(
|
||||
inverted_filter_node,
|
||||
std::move(context),
|
||||
std::move(block_with_constants),
|
||||
std::move(prepared_sets),
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
|
||||
|
||||
rpn = std::move(builder).extractRPN();
|
||||
|
||||
findHyperrectanglesForArgumentsOfSpaceFillingCurves();
|
||||
}
|
||||
|
||||
KeyCondition::KeyCondition(
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr_,
|
||||
bool single_point_,
|
||||
bool strict_)
|
||||
: KeyCondition(
|
||||
query_info.query,
|
||||
query_info.filter_asts,
|
||||
KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context),
|
||||
query_info.prepared_sets,
|
||||
context,
|
||||
key_column_names,
|
||||
key_expr_,
|
||||
query_info.syntax_analyzer_result ? query_info.syntax_analyzer_result->getArrayJoinSourceNameSet() : NameSet{},
|
||||
single_point_,
|
||||
strict_)
|
||||
{
|
||||
}
|
||||
|
||||
KeyCondition::KeyCondition(
|
||||
ActionsDAGPtr filter_dag,
|
||||
ContextPtr context,
|
||||
@ -883,6 +797,13 @@ KeyCondition::KeyCondition(
|
||||
|
||||
has_filter = true;
|
||||
|
||||
/** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
|
||||
* the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
|
||||
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
|
||||
* when parsing the AST into internal RPN representation.
|
||||
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
|
||||
* are pushed down and applied (when possible) to leaf nodes.
|
||||
*/
|
||||
auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context);
|
||||
assert(inverted_dag->getOutputs().size() == 1);
|
||||
|
||||
|
@ -39,30 +39,6 @@ struct ActionDAGNodes;
|
||||
class KeyCondition
|
||||
{
|
||||
public:
|
||||
/// Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters
|
||||
KeyCondition(
|
||||
const ASTPtr & query,
|
||||
const ASTs & additional_filter_asts,
|
||||
Block block_with_constants,
|
||||
PreparedSetsPtr prepared_sets_,
|
||||
ContextPtr context,
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr,
|
||||
NameSet array_joined_column_names,
|
||||
bool single_point_ = false,
|
||||
bool strict_ = false);
|
||||
|
||||
/** Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters.
|
||||
* Select query, additional filters, prepared sets are initialized using query info.
|
||||
*/
|
||||
KeyCondition(
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr_,
|
||||
bool single_point_ = false,
|
||||
bool strict_ = false);
|
||||
|
||||
/// Construct key condition from ActionsDAG nodes
|
||||
KeyCondition(
|
||||
ActionsDAGPtr filter_dag,
|
||||
|
@ -1075,26 +1075,30 @@ Block MergeTreeData::getBlockWithVirtualPartColumns(const MergeTreeData::DataPar
|
||||
|
||||
|
||||
std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
|
||||
const SelectQueryInfo & query_info, ContextPtr local_context, const DataPartsVector & parts) const
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const
|
||||
{
|
||||
if (parts.empty())
|
||||
return 0u;
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
ASTPtr expression_ast;
|
||||
Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, true /* one_part */);
|
||||
|
||||
// Generate valid expressions for filtering
|
||||
bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, local_context, virtual_columns_block, expression_ast);
|
||||
auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr);
|
||||
|
||||
PartitionPruner partition_pruner(metadata_snapshot, query_info, local_context, true /* strict */);
|
||||
// Generate valid expressions for filtering
|
||||
bool valid = true;
|
||||
for (const auto * input : filter_dag->getInputs())
|
||||
if (!virtual_columns_block.has(input->result_name))
|
||||
valid = false;
|
||||
|
||||
PartitionPruner partition_pruner(metadata_snapshot, filter_dag, local_context, true /* strict */);
|
||||
if (partition_pruner.isUseless() && !valid)
|
||||
return {};
|
||||
|
||||
std::unordered_set<String> part_values;
|
||||
if (valid && expression_ast)
|
||||
if (valid)
|
||||
{
|
||||
virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */);
|
||||
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, local_context, expression_ast);
|
||||
VirtualColumnUtils::filterBlockWithDAG(filter_dag, virtual_columns_block, local_context);
|
||||
part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
|
||||
if (part_values.empty())
|
||||
return 0;
|
||||
@ -3985,8 +3989,15 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW
|
||||
/// FIXME refactor removePartsFromWorkingSet(...), do not remove parts twice
|
||||
removePartsFromWorkingSet(txn, parts_to_remove, clear_without_timeout, lock);
|
||||
|
||||
/// We can only create a covering part for a blocks range that starts with 0 (otherwise we may get "intersecting parts"
|
||||
/// if we remove a range from the middle when dropping a part).
|
||||
/// Maybe we could do it by incrementing mutation version to get a name for the empty covering part,
|
||||
/// but it's okay to simply avoid creating it for DROP PART (for a part in the middle).
|
||||
/// NOTE: Block numbers in ReplicatedMergeTree start from 0. For MergeTree, is_new_syntax is always false.
|
||||
assert(!create_empty_part || supportsReplication());
|
||||
bool range_in_the_middle = drop_range.min_block;
|
||||
bool is_new_syntax = format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
|
||||
if (create_empty_part && !parts_to_remove.empty() && is_new_syntax)
|
||||
if (create_empty_part && !parts_to_remove.empty() && is_new_syntax && !range_in_the_middle)
|
||||
{
|
||||
/// We are going to remove a lot of parts from zookeeper just after returning from this function.
|
||||
/// And we will remove parts from disk later (because some queries may use them).
|
||||
@ -3995,12 +4006,9 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW
|
||||
/// We don't need to commit it to zk, and don't even need to activate it.
|
||||
|
||||
MergeTreePartInfo empty_info = drop_range;
|
||||
empty_info.level = empty_info.mutation = 0;
|
||||
if (!empty_info.min_block)
|
||||
empty_info.min_block = MergeTreePartInfo::MAX_BLOCK_NUMBER;
|
||||
empty_info.min_block = empty_info.level = empty_info.mutation = 0;
|
||||
for (const auto & part : parts_to_remove)
|
||||
{
|
||||
empty_info.min_block = std::min(empty_info.min_block, part->info.min_block);
|
||||
empty_info.level = std::max(empty_info.level, part->info.level);
|
||||
empty_info.mutation = std::max(empty_info.mutation, part->info.mutation);
|
||||
}
|
||||
@ -6617,8 +6625,7 @@ using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
|
||||
Block MergeTreeData::getMinMaxCountProjectionBlock(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const Names & required_columns,
|
||||
bool has_filter,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_dag,
|
||||
const DataPartsVector & parts,
|
||||
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
||||
ContextPtr query_context) const
|
||||
@ -6668,7 +6675,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
|
||||
Block virtual_columns_block;
|
||||
auto virtual_block = getSampleBlockWithVirtualColumns();
|
||||
bool has_virtual_column = std::any_of(required_columns.begin(), required_columns.end(), [&](const auto & name) { return virtual_block.has(name); });
|
||||
if (has_virtual_column || has_filter)
|
||||
if (has_virtual_column || filter_dag)
|
||||
{
|
||||
virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */, true /* ignore_empty */);
|
||||
if (virtual_columns_block.rows() == 0)
|
||||
@ -6680,7 +6687,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
|
||||
std::optional<PartitionPruner> partition_pruner;
|
||||
std::optional<KeyCondition> minmax_idx_condition;
|
||||
DataTypes minmax_columns_types;
|
||||
if (has_filter)
|
||||
if (filter_dag)
|
||||
{
|
||||
if (metadata_snapshot->hasPartitionKey())
|
||||
{
|
||||
@ -6689,16 +6696,15 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
|
||||
minmax_columns_types = getMinMaxColumnsTypes(partition_key);
|
||||
|
||||
minmax_idx_condition.emplace(
|
||||
query_info, query_context, minmax_columns_names,
|
||||
filter_dag, query_context, minmax_columns_names,
|
||||
getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(query_context)));
|
||||
partition_pruner.emplace(metadata_snapshot, query_info, query_context, false /* strict */);
|
||||
partition_pruner.emplace(metadata_snapshot, filter_dag, query_context, false /* strict */);
|
||||
}
|
||||
|
||||
const auto * predicate = filter_dag->getOutputs().at(0);
|
||||
|
||||
// Generate valid expressions for filtering
|
||||
ASTPtr expression_ast;
|
||||
VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, query_context, virtual_columns_block, expression_ast);
|
||||
if (expression_ast)
|
||||
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, query_context, expression_ast);
|
||||
VirtualColumnUtils::filterBlockWithPredicate(predicate, virtual_columns_block, query_context);
|
||||
|
||||
rows = virtual_columns_block.rows();
|
||||
part_name_column = virtual_columns_block.getByName("_part").column;
|
||||
|
@ -404,8 +404,7 @@ public:
|
||||
Block getMinMaxCountProjectionBlock(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const Names & required_columns,
|
||||
bool has_filter,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_dag,
|
||||
const DataPartsVector & parts,
|
||||
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
||||
ContextPtr query_context) const;
|
||||
@ -1222,7 +1221,7 @@ protected:
|
||||
boost::iterator_range<DataPartIteratorByStateAndInfo> range, const ColumnsDescription & storage_columns);
|
||||
|
||||
std::optional<UInt64> totalRowsByPartitionPredicateImpl(
|
||||
const SelectQueryInfo & query_info, ContextPtr context, const DataPartsVector & parts) const;
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const;
|
||||
|
||||
static decltype(auto) getStateModifier(DataPartState state)
|
||||
{
|
||||
|
@ -784,7 +784,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(
|
||||
= {ColumnWithTypeAndName(part_offset_type->createColumn(), part_offset_type, "_part_offset"),
|
||||
ColumnWithTypeAndName(part_type->createColumn(), part_type, "_part")};
|
||||
|
||||
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), sample);
|
||||
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), &sample);
|
||||
if (!dag)
|
||||
return;
|
||||
|
||||
@ -810,7 +810,7 @@ std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPar
|
||||
if (!filter_dag)
|
||||
return {};
|
||||
auto sample = data.getSampleBlockWithVirtualColumns();
|
||||
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), sample);
|
||||
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), &sample);
|
||||
if (!dag)
|
||||
return {};
|
||||
|
||||
@ -819,34 +819,6 @@ std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPar
|
||||
return VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
|
||||
}
|
||||
|
||||
|
||||
std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(
|
||||
const MergeTreeData & data,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const ASTPtr & query,
|
||||
ContextPtr context)
|
||||
{
|
||||
std::unordered_set<String> part_values;
|
||||
ASTPtr expression_ast;
|
||||
auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */);
|
||||
|
||||
if (virtual_columns_block.rows() == 0)
|
||||
return {};
|
||||
|
||||
// Generate valid expressions for filtering
|
||||
VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast);
|
||||
|
||||
// If there is still something left, fill the virtual block and do the filtering.
|
||||
if (expression_ast)
|
||||
{
|
||||
virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */);
|
||||
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast);
|
||||
return VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
const std::optional<PartitionPruner> & partition_pruner,
|
||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||
|
@ -169,12 +169,6 @@ public:
|
||||
/// If possible, filter using expression on virtual columns.
|
||||
/// Example: SELECT count() FROM table WHERE _part = 'part_name'
|
||||
/// If expression found, return a set with allowed part names (std::nullopt otherwise).
|
||||
static std::optional<std::unordered_set<String>> filterPartsByVirtualColumns(
|
||||
const MergeTreeData & data,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const ASTPtr & query,
|
||||
ContextPtr context);
|
||||
|
||||
static std::optional<std::unordered_set<String>> filterPartsByVirtualColumns(
|
||||
const MergeTreeData & data,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
|
@ -23,6 +23,7 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_NUMBER_OF_COLUMNS;
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
template <typename Distance>
|
||||
@ -331,6 +332,11 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec
|
||||
return std::make_shared<MergeTreeIndexConditionAnnoy>(index, query, distance_function, context);
|
||||
};
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG");
|
||||
}
|
||||
|
||||
MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index)
|
||||
{
|
||||
static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2;
|
||||
|
@ -88,7 +88,7 @@ private:
|
||||
};
|
||||
|
||||
|
||||
class MergeTreeIndexAnnoy : public IMergeTreeIndex
|
||||
class MergeTreeIndexAnnoy final : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
|
||||
@ -98,7 +98,9 @@ public:
|
||||
|
||||
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override;
|
||||
bool isVectorSearch() const override { return true; }
|
||||
|
||||
private:
|
||||
const UInt64 trees;
|
||||
|
@ -43,9 +43,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(con
|
||||
return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names);
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexConditionBloomFilter>(query_info, context, index.sample_block, hash_functions);
|
||||
return std::make_shared<MergeTreeIndexConditionBloomFilter>(filter_actions_dag, context, index.sample_block, hash_functions);
|
||||
}
|
||||
|
||||
static void assertIndexColumnsType(const Block & header)
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
|
||||
|
||||
private:
|
||||
size_t bits_per_row;
|
||||
|
@ -97,39 +97,18 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr &
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter(
|
||||
const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_)
|
||||
: WithContext(context_), header(header_), query_info(info_), hash_functions(hash_functions_)
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_)
|
||||
: WithContext(context_), header(header_), hash_functions(hash_functions_)
|
||||
{
|
||||
if (context_->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
if (!query_info.filter_actions_dag)
|
||||
{
|
||||
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
RPNBuilder<RPNElement> builder(
|
||||
query_info.filter_actions_dag->getOutputs().at(0),
|
||||
context_,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
|
||||
rpn = std::move(builder).extractRPN();
|
||||
return;
|
||||
}
|
||||
|
||||
ASTPtr filter_node = buildFilterNode(query_info.query);
|
||||
|
||||
if (!filter_node)
|
||||
if (!filter_actions_dag)
|
||||
{
|
||||
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_);
|
||||
RPNBuilder<RPNElement> builder(
|
||||
filter_node,
|
||||
filter_actions_dag->getOutputs().at(0),
|
||||
context_,
|
||||
std::move(block_with_constants),
|
||||
query_info.prepared_sets,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
|
||||
rpn = std::move(builder).extractRPN();
|
||||
}
|
||||
|
@ -44,7 +44,7 @@ public:
|
||||
std::vector<std::pair<size_t, ColumnPtr>> predicate;
|
||||
};
|
||||
|
||||
MergeTreeIndexConditionBloomFilter(const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_);
|
||||
MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_);
|
||||
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
|
||||
@ -58,7 +58,6 @@ public:
|
||||
|
||||
private:
|
||||
const Block & header;
|
||||
const SelectQueryInfo & query_info;
|
||||
const size_t hash_functions;
|
||||
std::vector<RPNElement> rpn;
|
||||
|
||||
|
@ -138,7 +138,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos,
|
||||
}
|
||||
|
||||
MergeTreeConditionFullText::MergeTreeConditionFullText(
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
ContextPtr context,
|
||||
const Block & index_sample_block,
|
||||
const BloomFilterParameters & params_,
|
||||
@ -147,38 +147,16 @@ MergeTreeConditionFullText::MergeTreeConditionFullText(
|
||||
, index_data_types(index_sample_block.getNamesAndTypesList().getTypes())
|
||||
, params(params_)
|
||||
, token_extractor(token_extactor_)
|
||||
, prepared_sets(query_info.prepared_sets)
|
||||
{
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
if (!query_info.filter_actions_dag)
|
||||
{
|
||||
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
RPNBuilder<RPNElement> builder(
|
||||
query_info.filter_actions_dag->getOutputs().at(0),
|
||||
context,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
|
||||
rpn = std::move(builder).extractRPN();
|
||||
return;
|
||||
}
|
||||
|
||||
ASTPtr filter_node = buildFilterNode(query_info.query);
|
||||
|
||||
if (!filter_node)
|
||||
if (!filter_actions_dag)
|
||||
{
|
||||
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context);
|
||||
RPNBuilder<RPNElement> builder(
|
||||
filter_node,
|
||||
filter_actions_dag->getOutputs().at(0),
|
||||
context,
|
||||
std::move(block_with_constants),
|
||||
query_info.prepared_sets,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
|
||||
rpn = std::move(builder).extractRPN();
|
||||
}
|
||||
@ -747,9 +725,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator(const
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const
|
||||
const ActionsDAGPtr & filter_dag, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeConditionFullText>(query, context, index.sample_block, params, token_extractor.get());
|
||||
return std::make_shared<MergeTreeConditionFullText>(filter_dag, context, index.sample_block, params, token_extractor.get());
|
||||
}
|
||||
|
||||
MergeTreeIndexPtr bloomFilterIndexCreator(
|
||||
|
@ -62,7 +62,7 @@ class MergeTreeConditionFullText final : public IMergeTreeIndexCondition
|
||||
{
|
||||
public:
|
||||
MergeTreeConditionFullText(
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
ContextPtr context,
|
||||
const Block & index_sample_block,
|
||||
const BloomFilterParameters & params_,
|
||||
@ -144,9 +144,6 @@ private:
|
||||
BloomFilterParameters params;
|
||||
TokenExtractorPtr token_extractor;
|
||||
RPN rpn;
|
||||
|
||||
/// Sets from syntax analyzer.
|
||||
PreparedSetsPtr prepared_sets;
|
||||
};
|
||||
|
||||
class MergeTreeIndexFullText final : public IMergeTreeIndex
|
||||
@ -166,7 +163,7 @@ public:
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
|
||||
MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
const ActionsDAGPtr & filter_dag, ContextPtr context) const override;
|
||||
|
||||
BloomFilterParameters params;
|
||||
/// Function for selecting next token.
|
||||
|
@ -79,7 +79,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator(cons
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition(
|
||||
const SelectQueryInfo &, ContextPtr) const
|
||||
const ActionsDAGPtr &, ContextPtr) const
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported");
|
||||
}
|
||||
|
@ -70,7 +70,7 @@ public:
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
|
||||
MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
|
||||
|
||||
MergeTreeIndexMergedConditionPtr createIndexMergedCondition(
|
||||
const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override;
|
||||
|
@ -184,7 +184,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos,
|
||||
}
|
||||
|
||||
MergeTreeConditionInverted::MergeTreeConditionInverted(
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
ContextPtr context_,
|
||||
const Block & index_sample_block,
|
||||
const GinFilterParameters & params_,
|
||||
@ -192,41 +192,20 @@ MergeTreeConditionInverted::MergeTreeConditionInverted(
|
||||
: WithContext(context_), header(index_sample_block)
|
||||
, params(params_)
|
||||
, token_extractor(token_extactor_)
|
||||
, prepared_sets(query_info.prepared_sets)
|
||||
{
|
||||
if (context_->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
if (!query_info.filter_actions_dag)
|
||||
{
|
||||
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
rpn = std::move(
|
||||
RPNBuilder<RPNElement>(
|
||||
query_info.filter_actions_dag->getOutputs().at(0), context_,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out)
|
||||
{
|
||||
return this->traverseAtomAST(node, out);
|
||||
}).extractRPN());
|
||||
return;
|
||||
}
|
||||
|
||||
ASTPtr filter_node = buildFilterNode(query_info.query);
|
||||
if (!filter_node)
|
||||
if (!filter_actions_dag)
|
||||
{
|
||||
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_);
|
||||
RPNBuilder<RPNElement> builder(
|
||||
filter_node,
|
||||
context_,
|
||||
std::move(block_with_constants),
|
||||
query_info.prepared_sets,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return traverseAtomAST(node, out); });
|
||||
rpn = std::move(builder).extractRPN();
|
||||
rpn = std::move(
|
||||
RPNBuilder<RPNElement>(
|
||||
filter_actions_dag->getOutputs().at(0), context_,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out)
|
||||
{
|
||||
return this->traverseAtomAST(node, out);
|
||||
}).extractRPN());
|
||||
}
|
||||
|
||||
/// Keep in-sync with MergeTreeConditionFullText::alwaysUnknownOrTrue
|
||||
@ -721,9 +700,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregatorForPart
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexInverted::createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeConditionInverted>(query, context, index.sample_block, params, token_extractor.get());
|
||||
return std::make_shared<MergeTreeConditionInverted>(filter_actions_dag, context, index.sample_block, params, token_extractor.get());
|
||||
};
|
||||
|
||||
MergeTreeIndexPtr invertedIndexCreator(
|
||||
|
@ -64,7 +64,7 @@ class MergeTreeConditionInverted final : public IMergeTreeIndexCondition, WithCo
|
||||
{
|
||||
public:
|
||||
MergeTreeConditionInverted(
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
ContextPtr context,
|
||||
const Block & index_sample_block,
|
||||
const GinFilterParameters & params_,
|
||||
@ -169,7 +169,7 @@ public:
|
||||
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store, const MergeTreeWriterSettings & /*settings*/) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
|
||||
|
||||
GinFilterParameters params;
|
||||
/// Function for selecting next token.
|
||||
|
@ -156,20 +156,17 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s
|
||||
namespace
|
||||
{
|
||||
|
||||
KeyCondition buildCondition(const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context)
|
||||
KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context)
|
||||
{
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
return KeyCondition{query_info.filter_actions_dag, context, index.column_names, index.expression};
|
||||
|
||||
return KeyCondition{query_info, context, index.column_names, index.expression};
|
||||
return KeyCondition{filter_actions_dag, context, index.column_names, index.expression};
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax(
|
||||
const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context)
|
||||
const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context)
|
||||
: index_data_types(index.data_types)
|
||||
, condition(buildCondition(index, query_info, context))
|
||||
, condition(buildCondition(index, filter_actions_dag, context))
|
||||
{
|
||||
}
|
||||
|
||||
@ -200,9 +197,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator(const Me
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexConditionMinMax>(index, query, context);
|
||||
return std::make_shared<MergeTreeIndexConditionMinMax>(index, filter_actions_dag, context);
|
||||
}
|
||||
|
||||
MergeTreeIndexFormat MergeTreeIndexMinMax::getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & relative_path_prefix) const
|
||||
|
@ -52,7 +52,7 @@ class MergeTreeIndexConditionMinMax final : public IMergeTreeIndexCondition
|
||||
public:
|
||||
MergeTreeIndexConditionMinMax(
|
||||
const IndexDescription & index,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
ContextPtr context);
|
||||
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
@ -79,7 +79,7 @@ public:
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
|
||||
MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
|
||||
|
||||
const char* getSerializedFileExtension() const override { return ".idx2"; }
|
||||
MergeTreeIndexFormat getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & path_prefix) const override; /// NOLINT
|
||||
|
@ -247,7 +247,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
|
||||
const String & index_name_,
|
||||
const Block & index_sample_block,
|
||||
size_t max_rows_,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_dag,
|
||||
ContextPtr context)
|
||||
: index_name(index_name_)
|
||||
, max_rows(max_rows_)
|
||||
@ -256,42 +256,20 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
|
||||
if (!key_columns.contains(name))
|
||||
key_columns.insert(name);
|
||||
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
if (!query_info.filter_actions_dag)
|
||||
return;
|
||||
if (!filter_dag)
|
||||
return;
|
||||
|
||||
if (checkDAGUseless(*query_info.filter_actions_dag->getOutputs().at(0), context))
|
||||
return;
|
||||
if (checkDAGUseless(*filter_dag->getOutputs().at(0), context))
|
||||
return;
|
||||
|
||||
const auto * filter_node = query_info.filter_actions_dag->getOutputs().at(0);
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG({filter_node}, {}, context);
|
||||
const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0);
|
||||
auto filter_actions_dag = filter_dag->clone();
|
||||
const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
|
||||
filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node);
|
||||
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
|
||||
filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node);
|
||||
|
||||
filter_actions_dag->removeUnusedActions();
|
||||
actions = std::make_shared<ExpressionActions>(filter_actions_dag);
|
||||
}
|
||||
else
|
||||
{
|
||||
ASTPtr ast_filter_node = buildFilterNode(query_info.query);
|
||||
if (!ast_filter_node)
|
||||
return;
|
||||
|
||||
if (checkASTUseless(ast_filter_node))
|
||||
return;
|
||||
|
||||
auto expression_ast = ast_filter_node->clone();
|
||||
|
||||
/// Replace logical functions with bit functions.
|
||||
/// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h).
|
||||
traverseAST(expression_ast);
|
||||
|
||||
auto syntax_analyzer_result = TreeRewriter(context).analyze(expression_ast, index_sample_block.getNamesAndTypesList());
|
||||
actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true);
|
||||
}
|
||||
filter_actions_dag->removeUnusedActions();
|
||||
actions = std::make_shared<ExpressionActions>(filter_actions_dag);
|
||||
}
|
||||
|
||||
bool MergeTreeIndexConditionSet::alwaysUnknownOrTrue() const
|
||||
@ -704,9 +682,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const Merge
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexConditionSet>(index.name, index.sample_block, max_rows, query, context);
|
||||
return std::make_shared<MergeTreeIndexConditionSet>(index.name, index.sample_block, max_rows, filter_actions_dag, context);
|
||||
}
|
||||
|
||||
MergeTreeIndexPtr setIndexCreator(const IndexDescription & index)
|
||||
|
@ -87,7 +87,7 @@ public:
|
||||
const String & index_name_,
|
||||
const Block & index_sample_block,
|
||||
size_t max_rows_,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ActionsDAGPtr & filter_dag,
|
||||
ContextPtr context);
|
||||
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
@ -149,7 +149,7 @@ public:
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
|
||||
MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
|
||||
|
||||
size_t max_rows = 0;
|
||||
};
|
||||
|
@ -36,6 +36,7 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_NUMBER_OF_COLUMNS;
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -366,6 +367,11 @@ MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const Sel
|
||||
return std::make_shared<MergeTreeIndexConditionUSearch>(index, query, distance_function, context);
|
||||
};
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG");
|
||||
}
|
||||
|
||||
MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index)
|
||||
{
|
||||
static constexpr auto default_distance_function = DISTANCE_FUNCTION_L2;
|
||||
|
@ -100,7 +100,9 @@ public:
|
||||
|
||||
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override;
|
||||
bool isVectorSearch() const override { return true; }
|
||||
|
||||
private:
|
||||
const String distance_function;
|
||||
|
@ -170,7 +170,9 @@ struct IMergeTreeIndex
|
||||
}
|
||||
|
||||
virtual MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query_info, ContextPtr context) const = 0;
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const = 0;
|
||||
|
||||
virtual bool isVectorSearch() const { return false; }
|
||||
|
||||
virtual MergeTreeIndexMergedConditionPtr createIndexMergedCondition(
|
||||
const SelectQueryInfo & /*query_info*/, StorageMetadataPtr /*storage_metadata*/) const
|
||||
|
@ -9,10 +9,7 @@ namespace
|
||||
|
||||
KeyCondition buildKeyCondition(const KeyDescription & partition_key, const SelectQueryInfo & query_info, ContextPtr context, bool strict)
|
||||
{
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
return {query_info.filter_actions_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict};
|
||||
|
||||
return {query_info, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict};
|
||||
return {query_info.filter_actions_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -202,17 +202,6 @@ public:
|
||||
traverseTree(RPNBuilderTreeNode(filter_actions_dag_node, tree_context));
|
||||
}
|
||||
|
||||
RPNBuilder(const ASTPtr & filter_node,
|
||||
ContextPtr query_context_,
|
||||
Block block_with_constants_,
|
||||
PreparedSetsPtr prepared_sets_,
|
||||
const ExtractAtomFromTreeFunction & extract_atom_from_tree_function_)
|
||||
: tree_context(std::move(query_context_), std::move(block_with_constants_), std::move(prepared_sets_))
|
||||
, extract_atom_from_tree_function(extract_atom_from_tree_function_)
|
||||
{
|
||||
traverseTree(RPNBuilderTreeNode(filter_node.get(), tree_context));
|
||||
}
|
||||
|
||||
RPNElements && extractRPN() && { return std::move(rpn_elements); }
|
||||
|
||||
private:
|
||||
|
@ -1056,11 +1056,6 @@ StorageFileSource::~StorageFileSource()
|
||||
beforeDestroy();
|
||||
}
|
||||
|
||||
void StorageFileSource::setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_)
|
||||
{
|
||||
setKeyConditionImpl(query_info_, context_, block_for_format);
|
||||
}
|
||||
|
||||
void StorageFileSource::setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_)
|
||||
{
|
||||
setKeyConditionImpl(nodes, context_, block_for_format);
|
||||
|
@ -256,8 +256,6 @@ private:
|
||||
return storage->getName();
|
||||
}
|
||||
|
||||
void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override;
|
||||
|
||||
void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override;
|
||||
|
||||
bool tryGetCountFromCache(const struct stat & file_stat);
|
||||
|
@ -262,10 +262,10 @@ std::optional<UInt64> StorageMergeTree::totalRows(const Settings &) const
|
||||
return getTotalActiveSizeInRows();
|
||||
}
|
||||
|
||||
std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const
|
||||
std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const
|
||||
{
|
||||
auto parts = getVisibleDataPartsVector(local_context);
|
||||
return totalRowsByPartitionPredicateImpl(query_info, local_context, parts);
|
||||
return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts);
|
||||
}
|
||||
|
||||
std::optional<UInt64> StorageMergeTree::totalBytes(const Settings &) const
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
size_t num_streams) override;
|
||||
|
||||
std::optional<UInt64> totalRows(const Settings &) const override;
|
||||
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const override;
|
||||
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr) const override;
|
||||
std::optional<UInt64> totalBytes(const Settings &) const override;
|
||||
std::optional<UInt64> totalBytesUncompressed(const Settings &) const override;
|
||||
|
||||
|
@ -5470,11 +5470,11 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalRows(const Settings & set
|
||||
return res;
|
||||
}
|
||||
|
||||
std::optional<UInt64> StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const
|
||||
std::optional<UInt64> StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const
|
||||
{
|
||||
DataPartsVector parts;
|
||||
foreachActiveParts([&](auto & part) { parts.push_back(part); }, local_context->getSettingsRef().select_sequential_consistency);
|
||||
return totalRowsByPartitionPredicateImpl(query_info, local_context, parts);
|
||||
return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts);
|
||||
}
|
||||
|
||||
std::optional<UInt64> StorageReplicatedMergeTree::totalBytes(const Settings & settings) const
|
||||
|
@ -163,7 +163,7 @@ public:
|
||||
size_t num_streams) override;
|
||||
|
||||
std::optional<UInt64> totalRows(const Settings & settings) const override;
|
||||
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context) const override;
|
||||
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
|
||||
std::optional<UInt64> totalBytes(const Settings & settings) const override;
|
||||
std::optional<UInt64> totalBytesUncompressed(const Settings & settings) const override;
|
||||
|
||||
|
@ -151,11 +151,6 @@ public:
|
||||
|
||||
String getName() const override;
|
||||
|
||||
void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override
|
||||
{
|
||||
setKeyConditionImpl(query_info_, context_, sample_block);
|
||||
}
|
||||
|
||||
void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override
|
||||
{
|
||||
setKeyConditionImpl(nodes, context_, sample_block);
|
||||
|
@ -172,11 +172,6 @@ public:
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override
|
||||
{
|
||||
setKeyConditionImpl(query_info_, context_, block_for_format);
|
||||
}
|
||||
|
||||
void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override
|
||||
{
|
||||
setKeyConditionImpl(nodes, context_, block_for_format);
|
||||
|
@ -104,7 +104,7 @@ ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr
|
||||
MutableColumnPtr database_column = ColumnString::create();
|
||||
MutableColumnPtr engine_column;
|
||||
|
||||
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, sample);
|
||||
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample);
|
||||
if (dag)
|
||||
{
|
||||
bool filter_by_engine = false;
|
||||
|
@ -253,19 +253,7 @@ static void makeSets(const ExpressionActionsPtr & actions, const ContextPtr & co
|
||||
if (!future_set->get())
|
||||
{
|
||||
if (auto * set_from_subquery = typeid_cast<FutureSetFromSubquery *>(future_set.get()))
|
||||
{
|
||||
auto plan = set_from_subquery->build(context);
|
||||
|
||||
if (!plan)
|
||||
continue;
|
||||
|
||||
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
|
||||
pipeline.complete(std::make_shared<EmptySink>(Block()));
|
||||
|
||||
CompletedPipelineExecutor executor(pipeline);
|
||||
executor.execute();
|
||||
}
|
||||
set_from_subquery->buildSetInplace(context);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -406,7 +394,7 @@ ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, con
|
||||
}
|
||||
|
||||
block.insert({ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "_idx"});
|
||||
return splitFilterDagForAllowedInputs(predicate, block);
|
||||
return splitFilterDagForAllowedInputs(predicate, &block);
|
||||
}
|
||||
|
||||
ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
@ -480,7 +468,7 @@ static bool canEvaluateSubtree(const ActionsDAG::Node * node, const Block & allo
|
||||
|
||||
static const ActionsDAG::Node * splitFilterNodeForAllowedInputs(
|
||||
const ActionsDAG::Node * node,
|
||||
const Block & allowed_inputs,
|
||||
const Block * allowed_inputs,
|
||||
ActionsDAG::Nodes & additional_nodes)
|
||||
{
|
||||
if (node->type == ActionsDAG::ActionType::FUNCTION)
|
||||
@ -555,13 +543,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs(
|
||||
}
|
||||
}
|
||||
|
||||
if (!canEvaluateSubtree(node, allowed_inputs))
|
||||
if (allowed_inputs && !canEvaluateSubtree(node, *allowed_inputs))
|
||||
return nullptr;
|
||||
|
||||
return node;
|
||||
}
|
||||
|
||||
ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block & allowed_inputs)
|
||||
ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs)
|
||||
{
|
||||
if (!predicate)
|
||||
return nullptr;
|
||||
@ -576,7 +564,7 @@ ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate,
|
||||
|
||||
void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context)
|
||||
{
|
||||
auto dag = splitFilterDagForAllowedInputs(predicate, block);
|
||||
auto dag = splitFilterDagForAllowedInputs(predicate, &block);
|
||||
if (dag)
|
||||
filterBlockWithDAG(dag, block, context);
|
||||
}
|
||||
|
@ -42,7 +42,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block,
|
||||
void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context);
|
||||
|
||||
/// Extract a part of predicate that can be evaluated using only columns from input_names.
|
||||
ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block & allowed_inputs);
|
||||
ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs);
|
||||
|
||||
/// Extract from the input stream a set of `name` column values
|
||||
template <typename T>
|
||||
|
@ -1,6 +1,7 @@
|
||||
|
||||
#include <Storages/buildQueryTreeForShard.h>
|
||||
|
||||
#include <Analyzer/createUniqueTableAliases.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
@ -372,6 +373,10 @@ QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeN
|
||||
|
||||
removeGroupingFunctionSpecializations(query_tree_to_modify);
|
||||
|
||||
// std::cerr << "====================== build 1 \n" << query_tree_to_modify->dumpTree() << std::endl;
|
||||
createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext());
|
||||
// std::cerr << "====================== build 2 \n" << query_tree_to_modify->dumpTree() << std::endl;
|
||||
|
||||
return query_tree_to_modify;
|
||||
}
|
||||
|
||||
|
@ -72,10 +72,20 @@ class BuildConfig:
|
||||
include_paths=[
|
||||
"./src",
|
||||
"./contrib/*-cmake",
|
||||
"./contrib/consistent-hashing",
|
||||
"./contrib/murmurhash",
|
||||
"./contrib/libfarmhash",
|
||||
"./contrib/pdqsort",
|
||||
"./contrib/cityhash102",
|
||||
"./contrib/sparse-checkout",
|
||||
"./contrib/libmetrohash",
|
||||
"./contrib/update-submodules.sh",
|
||||
"./contrib/CMakeLists.txt",
|
||||
"./cmake",
|
||||
"./base",
|
||||
"./programs",
|
||||
"./packages",
|
||||
"./docker/packager/packager",
|
||||
],
|
||||
exclude_files=[".md"],
|
||||
docker=["clickhouse/binary-builder"],
|
||||
|
@ -10,7 +10,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.regression_for_in_operator_view
|
||||
TABLE id: 3, alias: __table1, table_name: default.regression_for_in_operator_view
|
||||
WHERE
|
||||
FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -27,7 +27,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.regression_for_in_operator_view
|
||||
TABLE id: 3, alias: __table1, table_name: default.regression_for_in_operator_view
|
||||
WHERE
|
||||
FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
|
@ -34,7 +34,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3
|
||||
COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.bug
|
||||
TABLE id: 3, alias: __table1, table_name: default.bug
|
||||
WHERE
|
||||
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -77,7 +77,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3
|
||||
COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3
|
||||
JOIN TREE
|
||||
QUERY id: 3, is_subquery: 1
|
||||
QUERY id: 3, alias: __table1, is_subquery: 1
|
||||
PROJECTION COLUMNS
|
||||
k UInt64
|
||||
s UInt64
|
||||
@ -86,7 +86,7 @@ QUERY id: 0
|
||||
COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7
|
||||
COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7
|
||||
JOIN TREE
|
||||
TABLE id: 7, table_name: default.bug
|
||||
TABLE id: 7, alias: __table2, table_name: default.bug
|
||||
WHERE
|
||||
FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -151,7 +151,7 @@ QUERY id: 0
|
||||
COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3
|
||||
CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.bug
|
||||
TABLE id: 3, alias: __table1, table_name: default.bug
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
21 1
|
||||
22 1
|
||||
@ -184,7 +184,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3
|
||||
CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.bug
|
||||
TABLE id: 3, alias: __table1, table_name: default.bug
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
1 21
|
||||
1 22
|
||||
@ -222,7 +222,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3
|
||||
COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.bug
|
||||
TABLE id: 3, alias: __table1, table_name: default.bug
|
||||
WHERE
|
||||
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -265,7 +265,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3
|
||||
COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3
|
||||
JOIN TREE
|
||||
QUERY id: 3, is_subquery: 1
|
||||
QUERY id: 3, alias: __table1, is_subquery: 1
|
||||
PROJECTION COLUMNS
|
||||
k UInt64
|
||||
s UInt64
|
||||
@ -274,7 +274,7 @@ QUERY id: 0
|
||||
COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7
|
||||
COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7
|
||||
JOIN TREE
|
||||
TABLE id: 7, table_name: default.bug
|
||||
TABLE id: 7, alias: __table2, table_name: default.bug
|
||||
WHERE
|
||||
FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -347,7 +347,7 @@ QUERY id: 0
|
||||
COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3
|
||||
CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.bug
|
||||
TABLE id: 3, alias: __table1, table_name: default.bug
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
21 1
|
||||
22 1
|
||||
@ -380,7 +380,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3
|
||||
CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.bug
|
||||
TABLE id: 3, alias: __table1, table_name: default.bug
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
21 1
|
||||
22 1
|
||||
@ -413,5 +413,5 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3
|
||||
CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.bug
|
||||
TABLE id: 3, alias: __table1, table_name: default.bug
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
|
@ -49,7 +49,7 @@ QUERY id: 0
|
||||
LIST id: 9, nodes: 1
|
||||
COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 11, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 12, nodes: 1
|
||||
CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32
|
||||
@ -124,7 +124,7 @@ QUERY id: 0
|
||||
LIST id: 9, nodes: 1
|
||||
COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 11, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 12, nodes: 1
|
||||
CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32
|
||||
@ -194,7 +194,7 @@ QUERY id: 0
|
||||
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
|
||||
CONSTANT id: 11, constant_value: UInt64_5, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 7, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 12, nodes: 1
|
||||
CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32
|
||||
@ -276,7 +276,7 @@ QUERY id: 0
|
||||
LIST id: 9, nodes: 1
|
||||
COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 11, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 12, nodes: 1
|
||||
CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32
|
||||
|
@ -49,14 +49,14 @@ QUERY id: 0
|
||||
LIST id: 3, nodes: 1
|
||||
COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5
|
||||
JOIN TREE
|
||||
QUERY id: 5, is_subquery: 1
|
||||
QUERY id: 5, alias: __table1, is_subquery: 1
|
||||
PROJECTION COLUMNS
|
||||
x UInt64
|
||||
PROJECTION
|
||||
LIST id: 6, nodes: 1
|
||||
COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 8, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 9, nodes: 1
|
||||
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
|
||||
@ -83,14 +83,14 @@ QUERY id: 0
|
||||
LIST id: 3, nodes: 1
|
||||
COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5
|
||||
JOIN TREE
|
||||
QUERY id: 5, is_subquery: 1
|
||||
QUERY id: 5, alias: __table1, is_subquery: 1
|
||||
PROJECTION COLUMNS
|
||||
x UInt64
|
||||
PROJECTION
|
||||
LIST id: 6, nodes: 1
|
||||
COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 8, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 9, nodes: 1
|
||||
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
|
||||
@ -119,14 +119,14 @@ QUERY id: 0
|
||||
LIST id: 3, nodes: 1
|
||||
COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5
|
||||
JOIN TREE
|
||||
QUERY id: 5, is_subquery: 1
|
||||
QUERY id: 5, alias: __table1, is_subquery: 1
|
||||
PROJECTION COLUMNS
|
||||
x UInt64
|
||||
PROJECTION
|
||||
LIST id: 6, nodes: 1
|
||||
COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 8, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 9, nodes: 1
|
||||
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
|
||||
@ -171,7 +171,7 @@ QUERY id: 0
|
||||
JOIN TREE
|
||||
JOIN id: 8, strictness: ALL, kind: FULL
|
||||
LEFT TABLE EXPRESSION
|
||||
QUERY id: 3, alias: s, is_subquery: 1
|
||||
QUERY id: 3, alias: __table1, is_subquery: 1
|
||||
PROJECTION COLUMNS
|
||||
key UInt64
|
||||
PROJECTION
|
||||
@ -182,12 +182,12 @@ QUERY id: 0
|
||||
COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13
|
||||
CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 13, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 13, alias: __table2, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 15, nodes: 1
|
||||
CONSTANT id: 16, constant_value: UInt64_4, constant_value_type: UInt8
|
||||
RIGHT TABLE EXPRESSION
|
||||
TABLE id: 5, alias: t, table_name: default.test
|
||||
TABLE id: 5, alias: __table3, table_name: default.test
|
||||
JOIN EXPRESSION
|
||||
LIST id: 17, nodes: 1
|
||||
COLUMN id: 18, column_name: key, result_type: UInt64, source_id: 8
|
||||
@ -220,7 +220,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3
|
||||
COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.test
|
||||
TABLE id: 3, alias: __table1, table_name: default.test
|
||||
ORDER BY
|
||||
LIST id: 5, nodes: 2
|
||||
SORT id: 6, sort_direction: ASCENDING, with_fill: 0
|
||||
@ -246,7 +246,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3
|
||||
COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.test
|
||||
TABLE id: 3, alias: __table1, table_name: default.test
|
||||
ORDER BY
|
||||
LIST id: 5, nodes: 2
|
||||
SORT id: 6, sort_direction: ASCENDING, with_fill: 0
|
||||
@ -270,7 +270,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.test
|
||||
TABLE id: 3, alias: __table1, table_name: default.test
|
||||
GROUP BY
|
||||
LIST id: 4, nodes: 1
|
||||
COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3
|
||||
@ -297,9 +297,9 @@ QUERY id: 0
|
||||
JOIN TREE
|
||||
JOIN id: 6, strictness: ALL, kind: INNER
|
||||
LEFT TABLE EXPRESSION
|
||||
TABLE id: 3, table_name: default.t1
|
||||
TABLE id: 3, alias: __table1, table_name: default.t1
|
||||
RIGHT TABLE EXPRESSION
|
||||
TABLE id: 5, table_name: default.t2
|
||||
TABLE id: 5, alias: __table2, table_name: default.t2
|
||||
JOIN EXPRESSION
|
||||
FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
|
@ -1,9 +1,9 @@
|
||||
===http===
|
||||
{"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
|
||||
{"query":"DESC TABLE system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
|
||||
{"query":"SELECT 1 AS `1` FROM `system`.`one`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
|
||||
{"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
|
||||
{"query":"DESC TABLE system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
|
||||
{"query":"SELECT 1 AS `1` FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
|
||||
{"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
|
||||
{"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
|
||||
{"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"}
|
||||
{"initial query spans with proper parent":"2"}
|
||||
|
@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1
|
||||
"foo"
|
||||
1
|
||||
execute: --allow_experimental_analyzer=1 --stage fetch_columns
|
||||
"dummy_0"
|
||||
"__table1.dummy"
|
||||
0
|
||||
execute: --allow_experimental_analyzer=1 --stage with_mergeable_state
|
||||
"1_UInt8"
|
||||
|
@ -917,9 +917,9 @@ from
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Window (Window step for window \'\')
|
||||
Window (Window step for window \'PARTITION BY p_0\')
|
||||
Window (Window step for window \'PARTITION BY p_0 ORDER BY o_1 ASC\')
|
||||
Sorting (Sorting for window \'PARTITION BY p_0 ORDER BY o_1 ASC\')
|
||||
Window (Window step for window \'PARTITION BY __table1.p\')
|
||||
Window (Window step for window \'PARTITION BY __table1.p ORDER BY __table1.o ASC\')
|
||||
Sorting (Sorting for window \'PARTITION BY __table1.p ORDER BY __table1.o ASC\')
|
||||
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
|
||||
ReadFromSystemNumbers
|
||||
explain select
|
||||
@ -930,11 +930,11 @@ from
|
||||
from numbers(16)) t
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Window (Window step for window \'ORDER BY o_0 ASC, number_1 ASC\')
|
||||
Sorting (Sorting for window \'ORDER BY o_0 ASC, number_1 ASC\')
|
||||
Window (Window step for window \'ORDER BY number_1 ASC\')
|
||||
Window (Window step for window \'ORDER BY __table1.o ASC, __table1.number ASC\')
|
||||
Sorting (Sorting for window \'ORDER BY __table1.o ASC, __table1.number ASC\')
|
||||
Window (Window step for window \'ORDER BY __table1.number 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 number_1 ASC\')
|
||||
Sorting (Sorting for window \'ORDER BY __table1.number ASC\')
|
||||
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
|
||||
ReadFromSystemNumbers
|
||||
-- A test case for the sort comparator found by fuzzer.
|
||||
|
@ -45,7 +45,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.constraint_test_constants
|
||||
TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants
|
||||
WHERE
|
||||
FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -63,7 +63,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.constraint_test_constants
|
||||
TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants
|
||||
WHERE
|
||||
FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -80,5 +80,5 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.constraint_test_constants
|
||||
TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
|
@ -8,7 +8,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.t_constraints_where
|
||||
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
|
||||
WHERE
|
||||
CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
@ -22,7 +22,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.t_constraints_where
|
||||
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
|
||||
WHERE
|
||||
CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
@ -36,7 +36,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.t_constraints_where
|
||||
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
|
||||
WHERE
|
||||
CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
@ -50,7 +50,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.t_constraints_where
|
||||
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
|
||||
WHERE
|
||||
FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -68,7 +68,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.t_constraints_where
|
||||
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
|
||||
PREWHERE
|
||||
FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -85,5 +85,5 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.t_constraints_where
|
||||
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
|
@ -20,7 +20,7 @@ QUERY id: 0
|
||||
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
|
||||
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE id: 5, table_name: default.column_swap_test_test
|
||||
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -50,7 +50,7 @@ QUERY id: 0
|
||||
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
|
||||
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE id: 5, table_name: default.column_swap_test_test
|
||||
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
|
||||
PREWHERE
|
||||
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -80,7 +80,7 @@ QUERY id: 0
|
||||
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
|
||||
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE id: 5, table_name: default.column_swap_test_test
|
||||
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -110,7 +110,7 @@ QUERY id: 0
|
||||
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
|
||||
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE id: 5, table_name: default.column_swap_test_test
|
||||
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -140,7 +140,7 @@ QUERY id: 0
|
||||
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
|
||||
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE id: 5, table_name: default.column_swap_test_test
|
||||
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -162,7 +162,7 @@ QUERY id: 0
|
||||
COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5
|
||||
CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE id: 5, table_name: default.column_swap_test_test
|
||||
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -191,7 +191,7 @@ QUERY id: 0
|
||||
CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8
|
||||
COLUMN id: 9, column_name: a, result_type: String, source_id: 7
|
||||
JOIN TREE
|
||||
TABLE id: 7, table_name: default.column_swap_test_test
|
||||
TABLE id: 7, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -223,7 +223,7 @@ QUERY id: 0
|
||||
CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8
|
||||
COLUMN id: 9, column_name: a, result_type: String, source_id: 7
|
||||
JOIN TREE
|
||||
TABLE id: 7, table_name: default.column_swap_test_test
|
||||
TABLE id: 7, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -248,7 +248,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
|
||||
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.column_swap_test_test
|
||||
TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -270,7 +270,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
|
||||
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.column_swap_test_test
|
||||
TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -292,7 +292,7 @@ QUERY id: 0
|
||||
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
|
||||
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.column_swap_test_test
|
||||
TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -310,7 +310,7 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.column_swap_test_test
|
||||
TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test
|
||||
WHERE
|
||||
FUNCTION id: 4, function_name: equals, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
@ -327,5 +327,5 @@ QUERY id: 0
|
||||
LIST id: 1, nodes: 1
|
||||
COLUMN id: 2, column_name: a, result_type: UInt32, source_id: 3
|
||||
JOIN TREE
|
||||
TABLE id: 3, table_name: default.t_bad_constraint
|
||||
TABLE id: 3, alias: __table1, table_name: default.t_bad_constraint
|
||||
SETTINGS allow_experimental_analyzer=1
|
||||
|
@ -56,7 +56,7 @@ QUERY id: 0
|
||||
CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8
|
||||
CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 12, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 12, alias: __table1, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 15, nodes: 1
|
||||
CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8
|
||||
@ -82,7 +82,7 @@ QUERY id: 0
|
||||
CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8
|
||||
CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 12, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 12, alias: __table1, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 15, nodes: 1
|
||||
CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8
|
||||
@ -111,7 +111,7 @@ QUERY id: 0
|
||||
CONSTANT id: 15, constant_value: UInt64_2, constant_value_type: UInt8
|
||||
CONSTANT id: 16, constant_value: UInt64_0, constant_value_type: UInt8
|
||||
JOIN TREE
|
||||
TABLE_FUNCTION id: 14, table_function_name: numbers
|
||||
TABLE_FUNCTION id: 14, alias: __table1, table_function_name: numbers
|
||||
ARGUMENTS
|
||||
LIST id: 17, nodes: 1
|
||||
CONSTANT id: 18, constant_value: UInt64_100, constant_value_type: UInt8
|
||||
|
@ -28,7 +28,7 @@ Aggregating
|
||||
Filter
|
||||
Filter
|
||||
> (analyzer) filter should be pushed down after aggregating, column after aggregation is const
|
||||
COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8)
|
||||
COLUMN Const(UInt8) -> notEquals(__table1.y, 0_UInt8)
|
||||
Aggregating
|
||||
Filter
|
||||
Filter
|
||||
@ -49,9 +49,9 @@ Aggregating
|
||||
Filter column: notEquals(y, 0)
|
||||
> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased
|
||||
Filter column
|
||||
ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8))
|
||||
ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))
|
||||
Aggregating
|
||||
Filter column: notEquals(y_1, 0_UInt8)
|
||||
Filter column: notEquals(__table1.y, 0_UInt8)
|
||||
0 1
|
||||
1 2
|
||||
2 3
|
||||
@ -68,9 +68,9 @@ Aggregating
|
||||
Filter column: notEquals(y, 0)
|
||||
> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted
|
||||
Filter column
|
||||
FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2
|
||||
FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2
|
||||
Aggregating
|
||||
Filter column: notEquals(y_1, 0_UInt8)
|
||||
Filter column: notEquals(__table1.y, 0_UInt8)
|
||||
0 1
|
||||
1 2
|
||||
2 3
|
||||
@ -87,9 +87,9 @@ Aggregating
|
||||
Filter column: notEquals(y, 0)
|
||||
> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed
|
||||
Filter column
|
||||
FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8))
|
||||
FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))
|
||||
Aggregating
|
||||
Filter column: notEquals(y_1, 0_UInt8)
|
||||
Filter column: notEquals(__table1.y, 0_UInt8)
|
||||
0 1
|
||||
1 2
|
||||
2 3
|
||||
@ -105,9 +105,9 @@ Aggregating
|
||||
Filter column: and(notEquals(y, 0), minus(y, 4))
|
||||
> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased
|
||||
Filter column
|
||||
ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8))
|
||||
ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))
|
||||
Aggregating
|
||||
Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8))
|
||||
Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))
|
||||
0 1
|
||||
1 2
|
||||
2 3
|
||||
@ -121,9 +121,9 @@ Filter column: and(notEquals(y, 2), notEquals(x, 0))
|
||||
ARRAY JOIN x
|
||||
Filter column: notEquals(y, 2)
|
||||
> (analyzer) filter is split, one part is filtered before ARRAY JOIN
|
||||
Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8))
|
||||
ARRAY JOIN x_0
|
||||
Filter column: notEquals(y_1, 2_UInt8)
|
||||
Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8))
|
||||
ARRAY JOIN __table1.x
|
||||
Filter column: notEquals(__table2.y, 2_UInt8)
|
||||
1 3
|
||||
> filter is pushed down before Distinct
|
||||
Distinct
|
||||
@ -132,7 +132,7 @@ Filter column: notEquals(y, 2)
|
||||
> (analyzer) filter is pushed down before Distinct
|
||||
Distinct
|
||||
Distinct
|
||||
Filter column: notEquals(y_1, 2_UInt8)
|
||||
Filter column: notEquals(__table1.y, 2_UInt8)
|
||||
0 0
|
||||
0 1
|
||||
1 0
|
||||
@ -144,7 +144,7 @@ Filter column: and(notEquals(x, 0), notEquals(y, 0))
|
||||
> (analyzer) filter is pushed down before sorting steps
|
||||
Sorting
|
||||
Sorting
|
||||
Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8))
|
||||
Filter column: and(notEquals(__table1.x, 0_UInt8), notEquals(__table1.y, 0_UInt8))
|
||||
1 2
|
||||
1 1
|
||||
> filter is pushed down before TOTALS HAVING and aggregating
|
||||
@ -154,7 +154,7 @@ Filter column: notEquals(y, 2)
|
||||
> (analyzer) filter is pushed down before TOTALS HAVING and aggregating
|
||||
TotalsHaving
|
||||
Aggregating
|
||||
Filter column: notEquals(y_0, 2_UInt8)
|
||||
Filter column: notEquals(__table1.y, 2_UInt8)
|
||||
0 12
|
||||
1 15
|
||||
3 10
|
||||
@ -174,7 +174,7 @@ Join
|
||||
> (analyzer) one condition of filter is pushed down before LEFT JOIN
|
||||
Join
|
||||
Join
|
||||
Filter column: notEquals(number_0, 1_UInt8)
|
||||
Filter column: notEquals(__table1.number, 1_UInt8)
|
||||
0 0
|
||||
3 3
|
||||
> one condition of filter is pushed down before INNER JOIN
|
||||
@ -185,7 +185,7 @@ Join
|
||||
> (analyzer) one condition of filter is pushed down before INNER JOIN
|
||||
Join
|
||||
Join
|
||||
Filter column: notEquals(number_0, 1_UInt8)
|
||||
Filter column: notEquals(__table1.number, 1_UInt8)
|
||||
3 3
|
||||
> filter is pushed down before UNION
|
||||
Union
|
||||
|
@ -36,7 +36,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
|
||||
explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from (
|
||||
select number as x, number + 1 as y from numbers(10)) group by y
|
||||
) where y != 0
|
||||
settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8)"
|
||||
settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(__table1.y, 0_UInt8)"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select s, y, y != 0 from (select sum(x) as s, y from (
|
||||
select number as x, number + 1 as y from numbers(10)) group by y
|
||||
@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
) where y != 0 and s != 4
|
||||
settings enable_optimize_predicate_expression=0" |
|
||||
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8))"
|
||||
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select s, y from (
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
@ -76,7 +76,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
) where y != 0 and s - 4
|
||||
settings enable_optimize_predicate_expression=0" |
|
||||
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2"
|
||||
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select s, y from (
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
@ -96,7 +96,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q "
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
) where y != 0 and s - 8 and s - 4
|
||||
settings enable_optimize_predicate_expression=0" |
|
||||
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8))"
|
||||
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select s, y from (
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
@ -116,7 +116,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q "
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
) where y != 0 and s != 8 and y - 4
|
||||
settings enable_optimize_predicate_expression=0" |
|
||||
grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8))\|ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8))"
|
||||
grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select s, y from (
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
@ -134,7 +134,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
|
||||
explain actions = 1 select x, y from (
|
||||
select range(number) as x, number + 1 as y from numbers(3)
|
||||
) array join x where y != 2 and x != 0" |
|
||||
grep -o "Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8))\|ARRAY JOIN x_0\|Filter column: notEquals(y_1, 2_UInt8)"
|
||||
grep -o "Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8))\|ARRAY JOIN __table1.x\|Filter column: notEquals(__table2.y, 2_UInt8)"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select x, y from (
|
||||
select range(number) as x, number + 1 as y from numbers(3)
|
||||
@ -166,7 +166,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
|
||||
select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10))
|
||||
) where y != 2
|
||||
settings enable_optimize_predicate_expression=0" |
|
||||
grep -o "Distinct\|Filter column: notEquals(y_1, 2_UInt8)"
|
||||
grep -o "Distinct\|Filter column: notEquals(__table1.y, 2_UInt8)"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select x, y from (
|
||||
select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10))
|
||||
@ -186,7 +186,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q "
|
||||
select number % 2 as x, number % 3 as y from numbers(6) order by y desc
|
||||
) where x != 0 and y != 0
|
||||
settings enable_optimize_predicate_expression = 0" |
|
||||
grep -o "Sorting\|Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8))"
|
||||
grep -o "Sorting\|Filter column: and(notEquals(__table1.x, 0_UInt8), notEquals(__table1.y, 0_UInt8))"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select x, y from (
|
||||
select number % 2 as x, number % 3 as y from numbers(6) order by y desc
|
||||
@ -206,7 +206,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
|
||||
select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals
|
||||
) where y != 2
|
||||
settings enable_optimize_predicate_expression=0" |
|
||||
grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(y_0, 2_UInt8)"
|
||||
grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(__table1.y, 2_UInt8)"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select * from (
|
||||
select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals
|
||||
@ -236,7 +236,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
|
||||
select number as a, r.b from numbers(4) as l any left join (
|
||||
select number + 2 as b from numbers(3)
|
||||
) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" |
|
||||
grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)"
|
||||
grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select number as a, r.b from numbers(4) as l any left join (
|
||||
select number + 2 as b from numbers(3)
|
||||
@ -255,7 +255,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
|
||||
select number as a, r.b from numbers(4) as l any inner join (
|
||||
select number + 2 as b from numbers(3)
|
||||
) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" |
|
||||
grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)"
|
||||
grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
select number as a, r.b from numbers(4) as l any inner join (
|
||||
select number + 2 as b from numbers(3)
|
||||
|
@ -7,19 +7,19 @@ Partial sorting plan
|
||||
Prefix sort description: n ASC
|
||||
Result sort description: n ASC, x ASC
|
||||
optimize_read_in_window_order=1, allow_experimental_analyzer=1
|
||||
Prefix sort description: n_0 ASC
|
||||
Result sort description: n_0 ASC, x_1 ASC
|
||||
Prefix sort description: __table1.n ASC
|
||||
Result sort description: __table1.n ASC, __table1.x ASC
|
||||
No sorting plan
|
||||
optimize_read_in_window_order=0
|
||||
Sort description: n ASC, x ASC
|
||||
optimize_read_in_window_order=0, allow_experimental_analyzer=1
|
||||
Sort description: n_0 ASC, x_1 ASC
|
||||
Sort description: __table1.n ASC, __table1.x ASC
|
||||
optimize_read_in_window_order=1
|
||||
Prefix sort description: n ASC, x ASC
|
||||
Result sort description: n ASC, x ASC
|
||||
optimize_read_in_window_order=1, allow_experimental_analyzer=1
|
||||
Prefix sort description: n_0 ASC, x_1 ASC
|
||||
Result sort description: n_0 ASC, x_1 ASC
|
||||
Prefix sort description: __table1.n ASC, __table1.x ASC
|
||||
Result sort description: __table1.n ASC, __table1.x ASC
|
||||
Complex ORDER BY
|
||||
optimize_read_in_window_order=0
|
||||
3 3 1
|
||||
|
@ -37,59 +37,59 @@
|
||||
"Node Type": "Aggregating",
|
||||
"Header": [
|
||||
{
|
||||
"Name": "number_0",
|
||||
"Name": "__table1.number",
|
||||
"Type": "UInt64"
|
||||
},
|
||||
{
|
||||
"Name": "quantile(0.2_Float64)(number_0)",
|
||||
"Name": "quantile(0.2_Float64)(__table1.number)",
|
||||
"Type": "Float64"
|
||||
},
|
||||
{
|
||||
"Name": "sumIf(number_0, greater(number_0, 0_UInt8))",
|
||||
"Name": "sumIf(__table1.number, greater(__table1.number, 0_UInt8))",
|
||||
"Type": "UInt64"
|
||||
}
|
||||
],
|
||||
"Keys": ["number_0"],
|
||||
"Keys": ["__table1.number"],
|
||||
"Aggregates": [
|
||||
{
|
||||
"Name": "quantile(0.2_Float64)(number_0)",
|
||||
"Name": "quantile(0.2_Float64)(__table1.number)",
|
||||
"Function": {
|
||||
"Name": "quantile",
|
||||
"Parameters": ["0.2"],
|
||||
"Argument Types": ["UInt64"],
|
||||
"Result Type": "Float64"
|
||||
},
|
||||
"Arguments": ["number_0"]
|
||||
"Arguments": ["__table1.number"]
|
||||
},
|
||||
{
|
||||
"Name": "sumIf(number_0, greater(number_0, 0_UInt8))",
|
||||
"Name": "sumIf(__table1.number, greater(__table1.number, 0_UInt8))",
|
||||
"Function": {
|
||||
"Name": "sumIf",
|
||||
"Argument Types": ["UInt64", "UInt8"],
|
||||
"Result Type": "UInt64"
|
||||
},
|
||||
"Arguments": ["number_0", "greater(number_0, 0_UInt8)"]
|
||||
"Arguments": ["__table1.number", "greater(__table1.number, 0_UInt8)"]
|
||||
}
|
||||
],
|
||||
--------
|
||||
"Node Type": "ArrayJoin",
|
||||
"Left": false,
|
||||
"Columns": ["x_0", "y_1"],
|
||||
"Columns": ["__table1.x", "__table1.y"],
|
||||
--------
|
||||
"Node Type": "Distinct",
|
||||
"Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"],
|
||||
"Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"],
|
||||
--
|
||||
"Node Type": "Distinct",
|
||||
"Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"],
|
||||
"Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"],
|
||||
--------
|
||||
"Sort Description": [
|
||||
{
|
||||
"Column": "number_0",
|
||||
"Column": "__table1.number",
|
||||
"Ascending": false,
|
||||
"With Fill": false
|
||||
},
|
||||
{
|
||||
"Column": "plus(number_0, 1_UInt8)",
|
||||
"Column": "plus(__table1.number, 1_UInt8)",
|
||||
"Ascending": true,
|
||||
"With Fill": false
|
||||
}
|
||||
|
@ -30,7 +30,7 @@ SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key ==
|
||||
|
||||
SELECT '--';
|
||||
SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2;
|
||||
SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION }
|
||||
SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_FOUND_COLUMN_IN_BLOCK }
|
||||
|
||||
SELECT '--';
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2;
|
||||
|
@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1
|
||||
"foo"
|
||||
1
|
||||
execute: --allow_experimental_analyzer=1 --stage fetch_columns
|
||||
"dummy_0"
|
||||
"__table1.dummy"
|
||||
0
|
||||
execute: --allow_experimental_analyzer=1 --stage with_mergeable_state
|
||||
"1_UInt8"
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user