diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 93ce7660f457..8e8fc8bddbe0 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -199,6 +199,9 @@ class ExecutableFunctionCapture : public IExecutableFunction } } + const ExpressionActionsPtr & getActions() const { return expression_actions; } + const LambdaCapturePtr & getCapture() const { return capture; } + private: ExpressionActionsPtr expression_actions; LambdaCapturePtr capture; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 21abfd8c7869..4bcd1f70bd53 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -18,6 +19,7 @@ #include #include #include +#include #include @@ -41,7 +43,7 @@ namespace ErrorCodes namespace { -void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool allow_experimental_analyzer, bool legacy = false) +void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, const ContextPtr & context, bool use_analyzer, bool legacy = false) { switch (node.type) { @@ -53,29 +55,64 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o /// If it was created from ASTLiteral, then result_name can be an alias. /// We need to convert value back to string here. const auto * column_const = typeid_cast(node.column.get()); - if (column_const && !allow_experimental_analyzer) + if (column_const && !use_analyzer) writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out); else writeString(node.result_name, out); break; } case ActionsDAG::ActionType::ALIAS: - appendColumnNameWithoutAlias(*node.children.front(), out, allow_experimental_analyzer, legacy); + appendColumnNameWithoutAlias(*node.children.front(), out, context, use_analyzer, legacy); break; case ActionsDAG::ActionType::ARRAY_JOIN: writeCString("arrayJoin(", out); - appendColumnNameWithoutAlias(*node.children.front(), out, allow_experimental_analyzer, legacy); + appendColumnNameWithoutAlias(*node.children.front(), out, context, use_analyzer, legacy); writeChar(')', out); break; case ActionsDAG::ActionType::FUNCTION: { - auto name = node.function_base->getName(); - if (legacy && name == "modulo") - writeCString("moduloLegacy", out); + if (const auto * func_capture = typeid_cast(node.function.get())) + { + const auto & capture = func_capture->getCapture(); + auto capture_dag = func_capture->getActions()->getActionsDAG().clone(); + if (!node.children.empty()) + { + auto captured_columns_dag = ActionsDAG::cloneSubDAG(node.children, false); + auto & outputs = captured_columns_dag.getOutputs(); + for (size_t i = 0; i < capture->captured_names.size(); ++i) + outputs[i] = &captured_columns_dag.addAlias(*outputs[i], capture->captured_names[i]); + + capture_dag = ActionsDAG::merge(std::move(captured_columns_dag), std::move(capture_dag)); + } + + writeString("lambda(tuple(", out); + bool first = true; + for (const auto & arg : capture->lambda_arguments) + { + if (!first) + writeCString(", ", out); + first = false; + + writeString(arg.name, out); + } + writeString("), ", out); + + ActionsDAGWithInversionPushDown inverted_capture_dag(capture_dag.getOutputs().at(0), context); + appendColumnNameWithoutAlias(*inverted_capture_dag.predicate, out, context, use_analyzer, legacy); + writeChar(')', out); + break; + } else - writeString(name, out); + { + auto name = node.function_base->getName(); + if (legacy && name == "modulo") + writeCString("moduloLegacy", out); + else + writeString(name, out); + + writeChar('(', out); + } - writeChar('(', out); bool first = true; for (const auto * arg : node.children) { @@ -83,17 +120,17 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o writeCString(", ", out); first = false; - appendColumnNameWithoutAlias(*arg, out, allow_experimental_analyzer, legacy); + appendColumnNameWithoutAlias(*arg, out, context, use_analyzer, legacy); } writeChar(')', out); } } } -String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool allow_experimental_analyzer, bool legacy = false) +String getColumnNameWithoutAlias(const ActionsDAG::Node & node, const ContextPtr & context, bool use_analyzer, bool legacy = false) { WriteBufferFromOwnString out; - appendColumnNameWithoutAlias(node, out, allow_experimental_analyzer, legacy); + appendColumnNameWithoutAlias(node, out, context, use_analyzer, legacy); return std::move(out.str()); } @@ -144,7 +181,7 @@ std::string RPNBuilderTreeNode::getColumnName() const if (ast_node) return ast_node->getColumnNameWithoutAlias(); - return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings()[Setting::allow_experimental_analyzer]); + return getColumnNameWithoutAlias(*dag_node, getTreeContext().getQueryContext(), getTreeContext().getSettings()[Setting::allow_experimental_analyzer]); } std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const @@ -156,7 +193,7 @@ std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const return adjusted_ast->getColumnNameWithoutAlias(); } - return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings()[Setting::allow_experimental_analyzer], true /*legacy*/); + return getColumnNameWithoutAlias(*dag_node, getTreeContext().getQueryContext(), getTreeContext().getSettings()[Setting::allow_experimental_analyzer], true /*legacy*/); } bool RPNBuilderTreeNode::isFunction() const diff --git a/tests/queries/0_stateless/03448_analyzer_skip_index_and_lambdas.reference b/tests/queries/0_stateless/03448_analyzer_skip_index_and_lambdas.reference new file mode 100644 index 000000000000..52ef44ca2ec5 --- /dev/null +++ b/tests/queries/0_stateless/03448_analyzer_skip_index_and_lambdas.reference @@ -0,0 +1,63 @@ +-- { echo On } + +EXPLAIN indexes = 1, description=0 +SELECT arr +FROM index_test +WHERE has(arrayMap(x -> lower(x), arr), lower('a_12')) +SETTINGS enable_analyzer = 1; +Expression + Filter + ReadFromMergeTree + Indexes: + Skip + Name: array_index + Description: bloom_filter GRANULARITY 1 + Parts: 1/1 + Granules: 1/4 +SELECT arr +FROM index_test +WHERE has(arrayMap(x -> lower(x), arr), lower('a_12')) +SETTINGS enable_analyzer = 1; +['A_0','A_1','A_2','A_3','A_4','A_5','A_6','A_7','A_8','A_9','A_10','A_11','A_12'] +['A_0','A_1','A_2','A_3','A_4','A_5','A_6','A_7','A_8','A_9','A_10','A_11','A_12','A_13'] +['A_0','A_1','A_2','A_3','A_4','A_5','A_6','A_7','A_8','A_9','A_10','A_11','A_12','A_13','A_14'] +EXPLAIN indexes = 1, description=0 +SELECT arr +FROM index_test +WHERE has(arrayMap((x, y) -> concat(lower(x), y), arr, arr), 'a_12A_12') +SETTINGS enable_analyzer = 1; +Expression + Filter + ReadFromMergeTree + Indexes: + Skip + Name: array_index_2 + Description: bloom_filter GRANULARITY 1 + Parts: 1/1 + Granules: 1/4 +SELECT arr +FROM index_test +WHERE has(arrayMap((x, y) -> concat(lower(x), y), arr, arr), 'a_12A_12') +SETTINGS enable_analyzer = 1; +['A_0','A_1','A_2','A_3','A_4','A_5','A_6','A_7','A_8','A_9','A_10','A_11','A_12'] +['A_0','A_1','A_2','A_3','A_4','A_5','A_6','A_7','A_8','A_9','A_10','A_11','A_12','A_13'] +['A_0','A_1','A_2','A_3','A_4','A_5','A_6','A_7','A_8','A_9','A_10','A_11','A_12','A_13','A_14'] +EXPLAIN indexes = 1, description=0 +SELECT arr +FROM index_test +WHERE has(arrayMap((x, y) -> concat(lower(x), y, '_', toString(id)), arr, arr), 'a_12A_12_13') +SETTINGS enable_analyzer = 1; +Expression + Filter + ReadFromMergeTree + Indexes: + Skip + Name: array_index_3 + Description: bloom_filter GRANULARITY 1 + Parts: 1/1 + Granules: 1/4 +SELECT arr +FROM index_test +WHERE has(arrayMap((x, y) -> concat(lower(x), y, '_', toString(id)), arr, arr), 'a_12A_12_13') +SETTINGS enable_analyzer = 1; +['A_0','A_1','A_2','A_3','A_4','A_5','A_6','A_7','A_8','A_9','A_10','A_11','A_12'] diff --git a/tests/queries/0_stateless/03448_analyzer_skip_index_and_lambdas.sql b/tests/queries/0_stateless/03448_analyzer_skip_index_and_lambdas.sql new file mode 100644 index 000000000000..bccbb7fa3e74 --- /dev/null +++ b/tests/queries/0_stateless/03448_analyzer_skip_index_and_lambdas.sql @@ -0,0 +1,52 @@ +-- Tags: no-random-merge-tree-settings, no-random-settings, no-parallel-replicas + +DROP TABLE IF EXISTS index_test; +CREATE TABLE index_test +( + id UInt32, + arr Array(String), + INDEX array_index arrayMap(x -> lower(x), arr) TYPE bloom_filter(0.01) GRANULARITY 1, + INDEX array_index_2 arrayMap((x, y) -> concat(lower(x), y), arr, arr) TYPE bloom_filter(0.01) GRANULARITY 1, + INDEX array_index_3 arrayMap((x, y) -> concat(lower(x), y, '_', toString(id)), arr, arr) TYPE bloom_filter(0.01) GRANULARITY 1 +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS allow_suspicious_indices = 1, index_granularity = 4; + +insert into index_test select number, arrayMap(x -> 'A_' || toString(x) , range(number)) from numbers(16); + +-- { echo On } + +EXPLAIN indexes = 1, description=0 +SELECT arr +FROM index_test +WHERE has(arrayMap(x -> lower(x), arr), lower('a_12')) +SETTINGS enable_analyzer = 1; + +SELECT arr +FROM index_test +WHERE has(arrayMap(x -> lower(x), arr), lower('a_12')) +SETTINGS enable_analyzer = 1; + + +EXPLAIN indexes = 1, description=0 +SELECT arr +FROM index_test +WHERE has(arrayMap((x, y) -> concat(lower(x), y), arr, arr), 'a_12A_12') +SETTINGS enable_analyzer = 1; + +SELECT arr +FROM index_test +WHERE has(arrayMap((x, y) -> concat(lower(x), y), arr, arr), 'a_12A_12') +SETTINGS enable_analyzer = 1; + +EXPLAIN indexes = 1, description=0 +SELECT arr +FROM index_test +WHERE has(arrayMap((x, y) -> concat(lower(x), y, '_', toString(id)), arr, arr), 'a_12A_12_13') +SETTINGS enable_analyzer = 1; + +SELECT arr +FROM index_test +WHERE has(arrayMap((x, y) -> concat(lower(x), y, '_', toString(id)), arr, arr), 'a_12A_12_13') +SETTINGS enable_analyzer = 1;