mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Analyzer added indexes support
This commit is contained in:
parent
b068119d27
commit
99de697ea2
@ -2050,4 +2050,123 @@ bool ActionsDAG::isSortingPreserved(
|
||||
return true;
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
const NodeRawConstPtrs & filter_nodes,
|
||||
const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column,
|
||||
const ContextPtr & context)
|
||||
{
|
||||
if (filter_nodes.empty()) {
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
struct Frame
|
||||
{
|
||||
const ActionsDAG::Node * node = nullptr;
|
||||
bool visited_children = false;
|
||||
};
|
||||
|
||||
auto result_dag = std::make_shared<ActionsDAG>();
|
||||
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
|
||||
|
||||
size_t filter_nodes_size = filter_nodes.size();
|
||||
|
||||
std::vector<Frame> nodes_to_process;
|
||||
nodes_to_process.reserve(filter_nodes_size);
|
||||
|
||||
for (const auto & node : filter_nodes)
|
||||
nodes_to_process.push_back({node, false /*visited_children*/});
|
||||
|
||||
while (!nodes_to_process.empty())
|
||||
{
|
||||
auto & node_to_process = nodes_to_process.back();
|
||||
const auto * node = node_to_process.node;
|
||||
|
||||
/// Already visited node
|
||||
if (node_to_result_node.contains(node))
|
||||
{
|
||||
nodes_to_process.pop_back();
|
||||
continue;
|
||||
}
|
||||
|
||||
const ActionsDAG::Node * result_node = nullptr;
|
||||
|
||||
auto input_node_it = node_name_to_input_node_column.find(node->result_name);
|
||||
if (input_node_it != node_name_to_input_node_column.end())
|
||||
{
|
||||
result_node = &result_dag->addInput(input_node_it->second);
|
||||
node_to_result_node.emplace(node, result_node);
|
||||
nodes_to_process.pop_back();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!node_to_process.visited_children)
|
||||
{
|
||||
node_to_process.visited_children = true;
|
||||
|
||||
for (const auto & child : node->children)
|
||||
nodes_to_process.push_back({child, false /*visited_children*/});
|
||||
|
||||
/// If node has children process them first
|
||||
if (!node->children.empty())
|
||||
continue;
|
||||
}
|
||||
|
||||
auto node_type = node->type;
|
||||
|
||||
switch (node_type)
|
||||
{
|
||||
case ActionsDAG::ActionType::INPUT:
|
||||
{
|
||||
result_node = &result_dag->addInput({node->column, node->result_type, node->result_name});
|
||||
break;
|
||||
}
|
||||
case ActionsDAG::ActionType::COLUMN:
|
||||
{
|
||||
result_node = &result_dag->addColumn({node->column, node->result_type, node->result_name});
|
||||
break;
|
||||
}
|
||||
case ActionsDAG::ActionType::ALIAS:
|
||||
{
|
||||
const auto * child = node->children.front();
|
||||
result_node = &result_dag->addAlias(*(node_to_result_node.find(child)->second), node->result_name);
|
||||
break;
|
||||
}
|
||||
case ActionsDAG::ActionType::ARRAY_JOIN:
|
||||
{
|
||||
const auto * child = node->children.front();
|
||||
result_node = &result_dag->addArrayJoin(*(node_to_result_node.find(child)->second), {});
|
||||
break;
|
||||
}
|
||||
case ActionsDAG::ActionType::FUNCTION:
|
||||
{
|
||||
NodeRawConstPtrs function_children;
|
||||
function_children.reserve(node->children.size());
|
||||
|
||||
for (const auto & child : node->children)
|
||||
function_children.push_back(node_to_result_node.find(child)->second);
|
||||
|
||||
result_node = &result_dag->addFunction(node->function_builder, std::move(function_children), {});
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
node_to_result_node.emplace(node, result_node);
|
||||
nodes_to_process.pop_back();
|
||||
}
|
||||
|
||||
auto & result_dag_outputs = result_dag->getOutputs();
|
||||
result_dag_outputs.reserve(filter_nodes_size);
|
||||
|
||||
for (const auto & node : filter_nodes)
|
||||
result_dag_outputs.push_back(node_to_result_node.find(node)->second);
|
||||
|
||||
if (result_dag_outputs.size() > 1)
|
||||
{
|
||||
auto function_builder = FunctionFactory::instance().get("and", context);
|
||||
result_dag_outputs = {&result_dag->addFunction(function_builder, result_dag_outputs, {})};
|
||||
}
|
||||
|
||||
return result_dag;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -323,6 +323,25 @@ public:
|
||||
bool
|
||||
isSortingPreserved(const Block & input_header, const SortDescription & sort_description, const String & ignore_output_column = "") const;
|
||||
|
||||
/** Build filter dag from multiple filter dags.
|
||||
*
|
||||
* If filter nodes are empty, result is nullptr.
|
||||
*
|
||||
* If filter nodes are not empty, nodes and their children are merged into single dag.
|
||||
*
|
||||
* Additionally during dag construction if node has name that exists in node_name_to_input_column map argument
|
||||
* in final dag this node is represented as INPUT node with specified column.
|
||||
*
|
||||
* Result dag has only single output node:
|
||||
* 1. If there is single filter node, result dag output will contain this node.
|
||||
* 2. If there are multiple filter nodes, result dag output will contain single `and` function node
|
||||
* and children of this node will be filter nodes.
|
||||
*/
|
||||
static ActionsDAGPtr buildFilterActionsDAG(
|
||||
const NodeRawConstPtrs & filter_nodes,
|
||||
const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column,
|
||||
const ContextPtr & context);
|
||||
|
||||
private:
|
||||
Node & addNode(Node node);
|
||||
|
||||
|
@ -111,8 +111,8 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression,
|
||||
auto & query_context = planner_context->getQueryContext();
|
||||
|
||||
auto from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info);
|
||||
const auto & columns_names_set = table_expression_data.getColumnsNames();
|
||||
Names columns_names(columns_names_set.begin(), columns_names_set.end());
|
||||
|
||||
Names columns_names = table_expression_data.getColumnNames();
|
||||
|
||||
/** The current user must have the SELECT privilege.
|
||||
* We do not check access rights for table functions because they have been already checked in ITableFunction::execute().
|
||||
|
@ -39,6 +39,8 @@ using ColumnIdentifier = std::string;
|
||||
class TableExpressionData
|
||||
{
|
||||
public:
|
||||
using ColumnNameToColumn = std::unordered_map<std::string, NameAndTypePair>;
|
||||
|
||||
using ColumnNameToColumnIdentifier = std::unordered_map<std::string, ColumnIdentifier>;
|
||||
|
||||
using ColumnIdentifierToColumnName = std::unordered_map<ColumnIdentifier, std::string>;
|
||||
@ -46,7 +48,7 @@ public:
|
||||
/// Return true if column with name exists, false otherwise
|
||||
bool hasColumn(const std::string & column_name) const
|
||||
{
|
||||
return alias_columns_names.contains(column_name) || columns_names.contains(column_name);
|
||||
return alias_columns_names.contains(column_name) || column_name_to_column.contains(column_name);
|
||||
}
|
||||
|
||||
/** Add column in table expression data.
|
||||
@ -59,8 +61,7 @@ public:
|
||||
if (hasColumn(column.name))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists");
|
||||
|
||||
columns_names.insert(column.name);
|
||||
columns.push_back(column);
|
||||
column_name_to_column.emplace(column.name, column);
|
||||
column_name_to_column_identifier.emplace(column.name, column_identifier);
|
||||
column_identifier_to_column_name.emplace(column_identifier, column.name);
|
||||
}
|
||||
@ -73,8 +74,7 @@ public:
|
||||
if (hasColumn(column.name))
|
||||
return;
|
||||
|
||||
columns_names.insert(column.name);
|
||||
columns.push_back(column);
|
||||
column_name_to_column.emplace(column.name, column);
|
||||
column_name_to_column_identifier.emplace(column.name, column_identifier);
|
||||
column_identifier_to_column_name.emplace(column_identifier, column.name);
|
||||
}
|
||||
@ -91,16 +91,22 @@ public:
|
||||
return alias_columns_names;
|
||||
}
|
||||
|
||||
/// Get columns names
|
||||
const NameSet & getColumnsNames() const
|
||||
/// Get column name to column map
|
||||
const ColumnNameToColumn & getColumnNameToColumn() const
|
||||
{
|
||||
return columns_names;
|
||||
return column_name_to_column;
|
||||
}
|
||||
|
||||
/// Get columns
|
||||
const NamesAndTypesList & getColumns() const
|
||||
/// Get column names
|
||||
Names getColumnNames() const
|
||||
{
|
||||
return columns;
|
||||
Names result;
|
||||
result.reserve(column_name_to_column.size());
|
||||
|
||||
for (const auto & [column_name, column] : column_name_to_column)
|
||||
result.push_back(column_name);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Get column name to column identifier map
|
||||
@ -115,6 +121,32 @@ public:
|
||||
return column_identifier_to_column_name;
|
||||
}
|
||||
|
||||
/** Get column for column name.
|
||||
* Exception is thrown if there are no column for column name.
|
||||
*/
|
||||
const NameAndTypePair & getColumnOrThrow(const std::string & column_name) const
|
||||
{
|
||||
auto it = column_name_to_column.find(column_name);
|
||||
if (it == column_name_to_column.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Column for name {} does not exists",
|
||||
column_name);
|
||||
|
||||
return it->second;
|
||||
}
|
||||
|
||||
/** Get column for column name.
|
||||
* Null is returned if there are no column for column name.
|
||||
*/
|
||||
const NameAndTypePair * getColumnOrNull(const std::string & column_name) const
|
||||
{
|
||||
auto it = column_name_to_column.find(column_name);
|
||||
if (it == column_name_to_column.end())
|
||||
return nullptr;
|
||||
|
||||
return &it->second;
|
||||
}
|
||||
|
||||
/** Get column identifier for column name.
|
||||
* Exception is thrown if there are no column identifier for column name.
|
||||
*/
|
||||
@ -184,10 +216,7 @@ public:
|
||||
|
||||
private:
|
||||
/// Valid for table, table function, array join, query, union nodes
|
||||
NamesAndTypesList columns;
|
||||
|
||||
/// Valid for table, table function, array join, query, union nodes
|
||||
NameSet columns_names;
|
||||
ColumnNameToColumn column_name_to_column;
|
||||
|
||||
/// Valid only for table node
|
||||
NameSet alias_columns_names;
|
||||
|
@ -928,6 +928,84 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
const Names & real_column_names,
|
||||
bool sample_factor_column_queried,
|
||||
Poco::Logger * log)
|
||||
{
|
||||
const auto & settings = context->getSettingsRef();
|
||||
if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key)
|
||||
{
|
||||
ActionsDAG::NodeRawConstPtrs nodes;
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
{
|
||||
const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name);
|
||||
nodes.push_back(&node);
|
||||
}
|
||||
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name);
|
||||
nodes.push_back(&node);
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & node : added_filter_nodes.nodes)
|
||||
nodes.push_back(node);
|
||||
|
||||
std::unordered_map<std::string, ColumnWithTypeAndName> node_name_to_input_node_column;
|
||||
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression);
|
||||
for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName())
|
||||
{
|
||||
const auto & column = table_expression_data.getColumnOrThrow(column_name);
|
||||
node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name));
|
||||
}
|
||||
}
|
||||
|
||||
auto updated_query_info_with_filter_dag = query_info;
|
||||
updated_query_info_with_filter_dag.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context);
|
||||
|
||||
return selectRangesToReadImpl(
|
||||
parts,
|
||||
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);
|
||||
}
|
||||
|
||||
return selectRangesToReadImpl(
|
||||
parts,
|
||||
metadata_snapshot_base,
|
||||
metadata_snapshot,
|
||||
query_info,
|
||||
context,
|
||||
num_streams,
|
||||
max_block_numbers_to_read,
|
||||
data,
|
||||
real_column_names,
|
||||
sample_factor_column_queried,
|
||||
log);
|
||||
}
|
||||
|
||||
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
const StorageMetadataPtr & metadata_snapshot_base,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
size_t num_streams,
|
||||
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read,
|
||||
const MergeTreeData & data,
|
||||
const Names & real_column_names,
|
||||
bool sample_factor_column_queried,
|
||||
Poco::Logger * log)
|
||||
{
|
||||
AnalysisResult result;
|
||||
const auto & settings = context->getSettingsRef();
|
||||
@ -952,42 +1030,24 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
|
||||
// Build and check if primary key is used when necessary
|
||||
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
||||
Names primary_key_columns = primary_key.column_names;
|
||||
const Names & primary_key_column_names = primary_key.column_names;
|
||||
std::optional<KeyCondition> key_condition;
|
||||
|
||||
if (settings.query_plan_optimize_primary_key)
|
||||
{
|
||||
ActionDAGNodes nodes;
|
||||
if (prewhere_info)
|
||||
{
|
||||
{
|
||||
const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name);
|
||||
nodes.nodes.push_back(&node);
|
||||
}
|
||||
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name);
|
||||
nodes.nodes.push_back(&node);
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & node : added_filter_nodes.nodes)
|
||||
nodes.nodes.push_back(node);
|
||||
|
||||
NameSet array_join_name_set;
|
||||
if (query_info.syntax_analyzer_result)
|
||||
array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet();
|
||||
|
||||
key_condition.emplace(std::move(nodes),
|
||||
key_condition.emplace(query_info.filter_actions_dag,
|
||||
context,
|
||||
primary_key_columns,
|
||||
primary_key_column_names,
|
||||
primary_key.expression,
|
||||
array_join_name_set);
|
||||
}
|
||||
else
|
||||
{
|
||||
key_condition.emplace(query_info, context, primary_key_columns, primary_key.expression);
|
||||
key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression);
|
||||
}
|
||||
|
||||
if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue())
|
||||
@ -996,7 +1056,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
.result = std::make_exception_ptr(Exception(
|
||||
ErrorCodes::INDEX_NOT_USED,
|
||||
"Primary key ({}) is not used and setting 'force_primary_key' is set",
|
||||
fmt::join(primary_key_columns, ", ")))});
|
||||
fmt::join(primary_key_column_names, ", ")))});
|
||||
}
|
||||
LOG_DEBUG(log, "Key condition: {}", key_condition->toString());
|
||||
|
||||
@ -1380,7 +1440,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const
|
||||
void ReadFromMergeTree::describeIndexes(FormatSettings & format_settings) const
|
||||
{
|
||||
auto result = getAnalysisResult();
|
||||
auto index_stats = std::move(result.index_stats);
|
||||
auto index_stats = result.index_stats;
|
||||
|
||||
std::string prefix(format_settings.offset, format_settings.indent_char);
|
||||
if (!index_stats.empty())
|
||||
|
@ -155,6 +155,19 @@ public:
|
||||
void requestReadingInOrder(size_t prefix_size, int direction, size_t limit);
|
||||
|
||||
private:
|
||||
static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
const StorageMetadataPtr & metadata_snapshot_base,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
size_t num_streams,
|
||||
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read,
|
||||
const MergeTreeData & data,
|
||||
const Names & real_column_names,
|
||||
bool sample_factor_column_queried,
|
||||
Poco::Logger * log);
|
||||
|
||||
int getSortDirection() const
|
||||
{
|
||||
const InputOrderInfoPtr & order_info = query_info.getInputOrderInfo();
|
||||
|
@ -791,7 +791,7 @@ KeyCondition::KeyCondition(
|
||||
}
|
||||
|
||||
KeyCondition::KeyCondition(
|
||||
ActionDAGNodes dag_nodes,
|
||||
ActionsDAGPtr filter_dag,
|
||||
ContextPtr context,
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr_,
|
||||
@ -811,13 +811,13 @@ KeyCondition::KeyCondition(
|
||||
key_columns[name] = i;
|
||||
}
|
||||
|
||||
if (dag_nodes.nodes.empty())
|
||||
if (!filter_dag)
|
||||
{
|
||||
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context);
|
||||
auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context);
|
||||
assert(inverted_dag->getOutputs().size() == 1);
|
||||
|
||||
const auto * inverted_dag_filter_node = inverted_dag->getOutputs()[0];
|
||||
|
@ -243,7 +243,7 @@ public:
|
||||
|
||||
/// Construct key condition from ActionsDAG nodes
|
||||
KeyCondition(
|
||||
ActionDAGNodes dag_nodes,
|
||||
ActionsDAGPtr filter_dag,
|
||||
ContextPtr context,
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr,
|
||||
|
@ -789,14 +789,11 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
{
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
/// TODO: Analyzer syntax analyzer result
|
||||
if (settings.allow_experimental_analyzer)
|
||||
return;
|
||||
|
||||
std::optional<PartitionPruner> partition_pruner;
|
||||
std::optional<KeyCondition> minmax_idx_condition;
|
||||
DataTypes minmax_columns_types;
|
||||
if (metadata_snapshot->hasPartitionKey())
|
||||
|
||||
if (metadata_snapshot->hasPartitionKey() && !settings.allow_experimental_analyzer)
|
||||
{
|
||||
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||
auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key);
|
||||
@ -808,19 +805,9 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
|
||||
if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless()))
|
||||
{
|
||||
String msg = "Neither MinMax index by columns (";
|
||||
bool first = true;
|
||||
for (const String & col : minmax_columns_names)
|
||||
{
|
||||
if (first)
|
||||
first = false;
|
||||
else
|
||||
msg += ", ";
|
||||
msg += col;
|
||||
}
|
||||
msg += ") nor partition expr is used and setting 'force_index_by_date' is set";
|
||||
|
||||
throw Exception(msg, ErrorCodes::INDEX_NOT_USED);
|
||||
throw Exception(ErrorCodes::INDEX_NOT_USED,
|
||||
"Neither MinMax index by columns ({}) nor partition expr is used and setting 'force_index_by_date' is set",
|
||||
fmt::join(minmax_columns_names, ", "));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -18,7 +18,6 @@
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -100,6 +99,22 @@ 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_)
|
||||
{
|
||||
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)
|
||||
|
@ -149,6 +149,22 @@ MergeTreeConditionFullText::MergeTreeConditionFullText(
|
||||
, 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)
|
||||
|
@ -9,8 +9,9 @@ namespace DB
|
||||
|
||||
class MergeTreeIndexHyposesis;
|
||||
|
||||
struct MergeTreeIndexGranuleHypothesis : public IMergeTreeIndexGranule
|
||||
class MergeTreeIndexGranuleHypothesis : public IMergeTreeIndexGranule
|
||||
{
|
||||
public:
|
||||
explicit MergeTreeIndexGranuleHypothesis(
|
||||
const String & index_name_);
|
||||
|
||||
@ -31,8 +32,9 @@ struct MergeTreeIndexGranuleHypothesis : public IMergeTreeIndexGranule
|
||||
};
|
||||
|
||||
|
||||
struct MergeTreeIndexAggregatorHypothesis : IMergeTreeIndexAggregator
|
||||
class MergeTreeIndexAggregatorHypothesis : public IMergeTreeIndexAggregator
|
||||
{
|
||||
public:
|
||||
explicit MergeTreeIndexAggregatorHypothesis(
|
||||
const String & index_name_, const String & column_name_);
|
||||
|
||||
|
@ -155,11 +155,29 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s
|
||||
*pos += rows_read;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
KeyCondition buildCondition(const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context)
|
||||
{
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
NameSet array_join_name_set;
|
||||
if (query_info.syntax_analyzer_result)
|
||||
array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet();
|
||||
|
||||
return KeyCondition{query_info.filter_actions_dag, context, index.column_names, index.expression, array_join_name_set};
|
||||
}
|
||||
|
||||
return KeyCondition{query_info, context, index.column_names, index.expression};
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax(
|
||||
const IndexDescription & index, const SelectQueryInfo & query, ContextPtr context)
|
||||
const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context)
|
||||
: index_data_types(index.data_types)
|
||||
, condition(query, context, index.column_names, index.expression)
|
||||
, condition(buildCondition(index, query_info, context))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -51,7 +51,7 @@ class MergeTreeIndexConditionMinMax final : public IMergeTreeIndexCondition
|
||||
public:
|
||||
MergeTreeIndexConditionMinMax(
|
||||
const IndexDescription & index,
|
||||
const SelectQueryInfo & query,
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context);
|
||||
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
|
@ -207,6 +207,9 @@ struct SelectQueryInfo
|
||||
/// It is needed for PK analysis based on row_level_policy and additional_filters.
|
||||
ASTs filter_asts;
|
||||
|
||||
/// Filter actions dag for current storage
|
||||
ActionsDAGPtr filter_actions_dag;
|
||||
|
||||
ReadInOrderOptimizerPtr order_optimizer;
|
||||
/// Can be modified while reading from storage
|
||||
InputOrderInfoPtr input_order_info;
|
||||
|
@ -360,13 +360,13 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu
|
||||
|
||||
size_t tables_count = selected_tables.size();
|
||||
Float64 num_streams_multiplier
|
||||
= std::min(static_cast<unsigned>(tables_count), std::max(1U, static_cast<unsigned>(context->getSettingsRef().max_streams_multiplier_for_merge_tables)));
|
||||
= std::min(static_cast<size_t>(tables_count), std::max(1UL, static_cast<size_t>(context->getSettingsRef().max_streams_multiplier_for_merge_tables)));
|
||||
size_t num_streams = static_cast<size_t>(requested_num_streams * num_streams_multiplier);
|
||||
size_t remaining_streams = num_streams;
|
||||
|
||||
if (order_info)
|
||||
{
|
||||
query_info.input_order_info = order_info;
|
||||
query_info.input_order_info = order_info;
|
||||
}
|
||||
else if (query_info.order_optimizer)
|
||||
{
|
||||
|
17
tests/queries/0_stateless/02489_analyzer_indexes.reference
Normal file
17
tests/queries/0_stateless/02489_analyzer_indexes.reference
Normal file
@ -0,0 +1,17 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
62
tests/queries/0_stateless/02489_analyzer_indexes.sql
Normal file
62
tests/queries/0_stateless/02489_analyzer_indexes.sql
Normal file
@ -0,0 +1,62 @@
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
DROP TABLE IF EXISTS test_table;
|
||||
CREATE TABLE test_table
|
||||
(
|
||||
id UInt64,
|
||||
value_1 String,
|
||||
value_2 String,
|
||||
value_3 String,
|
||||
INDEX value_1_idx (value_1) TYPE bloom_filter GRANULARITY 1,
|
||||
INDEX value_2_idx (value_2) TYPE ngrambf_v1(3, 512, 2, 0) GRANULARITY 1,
|
||||
INDEX value_3_idx (value_3) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1
|
||||
) ENGINE=MergeTree ORDER BY id;
|
||||
|
||||
INSERT INTO test_table SELECT number, toString(number), toString(number), toString(number) FROM numbers(10);
|
||||
|
||||
SELECT count() FROM test_table WHERE id = 1 SETTINGS force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM test_table WHERE value_1 = '1' SETTINGS force_data_skipping_indices = 'value_1_idx';
|
||||
|
||||
SELECT count() FROM test_table WHERE id = 1 AND value_1 = '1' SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx';
|
||||
|
||||
SELECT count() FROM test_table WHERE value_2 = '1' SETTINGS force_data_skipping_indices = 'value_2_idx';
|
||||
|
||||
SELECT count() FROM test_table WHERE value_1 = '1' AND value_2 = '1' SETTINGS force_data_skipping_indices = 'value_1_idx, value_2_idx';
|
||||
|
||||
SELECT count() FROM test_table WHERE id = 1 AND value_1 = '1' AND value_2 = '1' SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx';
|
||||
|
||||
SELECT count() FROM test_table WHERE value_3 = '1' SETTINGS force_data_skipping_indices = 'value_3_idx';
|
||||
|
||||
SELECT count() FROM test_table WHERE id = 1 AND value_3 = '1' SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_3_idx';
|
||||
|
||||
SELECT count() FROM test_table WHERE id = 1 AND value_1 = '1' AND value_2 = '1' AND value_3 = '1'
|
||||
SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx, value_3_idx';
|
||||
|
||||
SELECT count() FROM test_table AS t1 INNER JOIN (SELECT number AS id FROM numbers(10)) AS t2 ON t1.id = t2.id
|
||||
WHERE t1.id = 1 SETTINGS force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM test_table AS t1 INNER JOIN (SELECT number AS id FROM numbers(10)) AS t2 ON t1.id = t2.id
|
||||
WHERE t1.value_1 = '1' SETTINGS force_data_skipping_indices = 'value_1_idx';
|
||||
|
||||
SELECT count() FROM test_table AS t1 INNER JOIN (SELECT number AS id FROM numbers(10)) AS t2 ON t1.id = t2.id
|
||||
WHERE t1.id = 1 AND t1.value_1 = '1' SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx';
|
||||
|
||||
SELECT count() FROM test_table AS t1 INNER JOIN (SELECT number AS id FROM numbers(10)) AS t2 ON t1.id = t2.id
|
||||
WHERE t1.value_2 = '1' SETTINGS force_data_skipping_indices = 'value_2_idx';
|
||||
|
||||
SELECT count() FROM test_table AS t1 INNER JOIN (SELECT number AS id FROM numbers(10)) AS t2 ON t1.id = t2.id
|
||||
WHERE t1.value_1 = '1' AND t1.value_2 = '1' SETTINGS force_data_skipping_indices = 'value_1_idx, value_2_idx';
|
||||
|
||||
SELECT count() FROM test_table AS t1 INNER JOIN (SELECT number AS id FROM numbers(10)) AS t2 ON t1.id = t2.id
|
||||
WHERE t1.id = 1 AND t1.value_1 = '1' AND t1.value_2 = '1' SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx';
|
||||
|
||||
SELECT count() FROM test_table AS t1 INNER JOIN (SELECT number AS id FROM numbers(10)) AS t2 ON t1.id = t2.id
|
||||
WHERE t1.value_3 = '1' SETTINGS force_data_skipping_indices = 'value_3_idx';
|
||||
|
||||
SELECT count() FROM test_table AS t1 INNER JOIN (SELECT number AS id FROM numbers(10)) AS t2 ON t1.id = t2.id
|
||||
WHERE t1.id = 1 AND t1.value_1 = '1' AND t1.value_2 = '1' AND t1.value_3 = '1'
|
||||
SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx, value_3_idx';
|
||||
|
||||
DROP TABLE test_table;
|
||||
|
Loading…
Reference in New Issue
Block a user