Analyzer added indexes support

This commit is contained in:
Maksim Kita 2022-11-17 19:44:26 +01:00
parent b068119d27
commit 99de697ea2
18 changed files with 431 additions and 71 deletions

View File

@ -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;
}
}

View File

@ -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);

View File

@ -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().

View File

@ -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;

View File

@ -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())

View File

@ -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();

View File

@ -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];

View File

@ -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,

View File

@ -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, ", "));
}
}

View File

@ -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)

View File

@ -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)

View File

@ -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_);

View File

@ -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))
{
}

View File

@ -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;

View File

@ -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;

View File

@ -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)
{

View File

@ -0,0 +1,17 @@
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1

View 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;