mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Remove from indexes as well.
This commit is contained in:
parent
bcd34b25b2
commit
e493789bf3
@ -100,8 +100,6 @@ public:
|
|||||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) 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 override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; }
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const UInt64 trees;
|
const UInt64 trees;
|
||||||
const String distance_function;
|
const String distance_function;
|
||||||
|
@ -38,33 +38,6 @@ MergeTreeIndexGranulePtr MergeTreeIndexBloomFilter::createIndexGranule() const
|
|||||||
return std::make_shared<MergeTreeIndexGranuleBloomFilter>(bits_per_row, hash_functions, index.column_names.size());
|
return std::make_shared<MergeTreeIndexGranuleBloomFilter>(bits_per_row, hash_functions, index.column_names.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeIndexBloomFilter::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
|
||||||
{
|
|
||||||
Names required_columns = index.expression->getRequiredColumns();
|
|
||||||
NameSet required_columns_set(required_columns.begin(), required_columns.end());
|
|
||||||
|
|
||||||
std::vector<ASTPtr> nodes_to_check;
|
|
||||||
nodes_to_check.emplace_back(node);
|
|
||||||
|
|
||||||
while (!nodes_to_check.empty())
|
|
||||||
{
|
|
||||||
auto node_to_check = nodes_to_check.back();
|
|
||||||
nodes_to_check.pop_back();
|
|
||||||
|
|
||||||
const auto & column_name = node_to_check->getColumnName();
|
|
||||||
if (required_columns_set.find(column_name) != required_columns_set.end())
|
|
||||||
return true;
|
|
||||||
|
|
||||||
if (const auto * function = typeid_cast<const ASTFunction *>(node_to_check.get()))
|
|
||||||
{
|
|
||||||
auto & function_arguments_children = function->arguments->children;
|
|
||||||
nodes_to_check.insert(nodes_to_check.end(), function_arguments_children.begin(), function_arguments_children.end());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
|
MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names);
|
return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names);
|
||||||
|
@ -22,8 +22,6 @@ public:
|
|||||||
|
|
||||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const override;
|
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
size_t bits_per_row;
|
size_t bits_per_row;
|
||||||
size_t hash_functions;
|
size_t hash_functions;
|
||||||
|
@ -696,11 +696,6 @@ MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition(
|
|||||||
return std::make_shared<MergeTreeConditionFullText>(query, context, index.sample_block, params, token_extractor.get());
|
return std::make_shared<MergeTreeConditionFullText>(query, context, index.sample_block, params, token_extractor.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeIndexFullText::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
|
||||||
{
|
|
||||||
return std::find(std::cbegin(index.column_names), std::cend(index.column_names), node->getColumnName()) != std::cend(index.column_names);
|
|
||||||
}
|
|
||||||
|
|
||||||
MergeTreeIndexPtr bloomFilterIndexCreator(
|
MergeTreeIndexPtr bloomFilterIndexCreator(
|
||||||
const IndexDescription & index)
|
const IndexDescription & index)
|
||||||
{
|
{
|
||||||
|
@ -167,8 +167,6 @@ public:
|
|||||||
MergeTreeIndexConditionPtr createIndexCondition(
|
MergeTreeIndexConditionPtr createIndexCondition(
|
||||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
|
||||||
|
|
||||||
BloomFilterParameters params;
|
BloomFilterParameters params;
|
||||||
/// Function for selecting next token.
|
/// Function for selecting next token.
|
||||||
std::unique_ptr<ITokenExtractor> token_extractor;
|
std::unique_ptr<ITokenExtractor> token_extractor;
|
||||||
|
@ -91,11 +91,6 @@ MergeTreeIndexMergedConditionPtr MergeTreeIndexHypothesis::createIndexMergedCond
|
|||||||
query_info, storage_metadata->getConstraints(), index.granularity);
|
query_info, storage_metadata->getConstraints(), index.granularity);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeIndexHypothesis::mayBenefitFromIndexForIn(const ASTPtr &) const
|
|
||||||
{
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index)
|
MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index)
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeIndexHypothesis>(index);
|
return std::make_shared<MergeTreeIndexHypothesis>(index);
|
||||||
|
@ -75,8 +75,6 @@ public:
|
|||||||
MergeTreeIndexMergedConditionPtr createIndexMergedCondition(
|
MergeTreeIndexMergedConditionPtr createIndexMergedCondition(
|
||||||
const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override;
|
const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
|
||||||
|
|
||||||
size_t max_rows = 0;
|
size_t max_rows = 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -726,11 +726,6 @@ MergeTreeIndexConditionPtr MergeTreeIndexInverted::createIndexCondition(
|
|||||||
return std::make_shared<MergeTreeConditionInverted>(query, context, index.sample_block, params, token_extractor.get());
|
return std::make_shared<MergeTreeConditionInverted>(query, context, index.sample_block, params, token_extractor.get());
|
||||||
};
|
};
|
||||||
|
|
||||||
bool MergeTreeIndexInverted::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
|
||||||
{
|
|
||||||
return std::find(std::cbegin(index.column_names), std::cend(index.column_names), node->getColumnName()) != std::cend(index.column_names);
|
|
||||||
}
|
|
||||||
|
|
||||||
MergeTreeIndexPtr invertedIndexCreator(
|
MergeTreeIndexPtr invertedIndexCreator(
|
||||||
const IndexDescription & index)
|
const IndexDescription & index)
|
||||||
{
|
{
|
||||||
|
@ -171,8 +171,6 @@ public:
|
|||||||
MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store, 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 SelectQueryInfo & query, ContextPtr context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
|
||||||
|
|
||||||
GinFilterParameters params;
|
GinFilterParameters params;
|
||||||
/// Function for selecting next token.
|
/// Function for selecting next token.
|
||||||
std::unique_ptr<ITokenExtractor> token_extractor;
|
std::unique_ptr<ITokenExtractor> token_extractor;
|
||||||
|
@ -211,21 +211,6 @@ MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition(
|
|||||||
return std::make_shared<MergeTreeIndexConditionMinMax>(index, query, context);
|
return std::make_shared<MergeTreeIndexConditionMinMax>(index, query, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeIndexMinMax::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
|
||||||
{
|
|
||||||
const String column_name = node->getColumnName();
|
|
||||||
|
|
||||||
for (const auto & cname : index.column_names)
|
|
||||||
if (column_name == cname)
|
|
||||||
return true;
|
|
||||||
|
|
||||||
if (const auto * func = typeid_cast<const ASTFunction *>(node.get()))
|
|
||||||
if (func->arguments->children.size() == 1)
|
|
||||||
return mayBenefitFromIndexForIn(func->arguments->children.front());
|
|
||||||
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
MergeTreeIndexFormat MergeTreeIndexMinMax::getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & relative_path_prefix) const
|
MergeTreeIndexFormat MergeTreeIndexMinMax::getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & relative_path_prefix) const
|
||||||
{
|
{
|
||||||
if (data_part_storage.exists(relative_path_prefix + ".idx2"))
|
if (data_part_storage.exists(relative_path_prefix + ".idx2"))
|
||||||
|
@ -81,8 +81,6 @@ public:
|
|||||||
MergeTreeIndexConditionPtr createIndexCondition(
|
MergeTreeIndexConditionPtr createIndexCondition(
|
||||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
|
||||||
|
|
||||||
const char* getSerializedFileExtension() const override { return ".idx2"; }
|
const char* getSerializedFileExtension() const override { return ".idx2"; }
|
||||||
MergeTreeIndexFormat getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & path_prefix) const override; /// NOLINT
|
MergeTreeIndexFormat getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & path_prefix) const override; /// NOLINT
|
||||||
};
|
};
|
||||||
|
@ -709,11 +709,6 @@ MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition(
|
|||||||
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, query, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeIndexSet::mayBenefitFromIndexForIn(const ASTPtr &) const
|
|
||||||
{
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
MergeTreeIndexPtr setIndexCreator(const IndexDescription & index)
|
MergeTreeIndexPtr setIndexCreator(const IndexDescription & index)
|
||||||
{
|
{
|
||||||
size_t max_rows = index.arguments[0].get<size_t>();
|
size_t max_rows = index.arguments[0].get<size_t>();
|
||||||
|
@ -151,8 +151,6 @@ public:
|
|||||||
MergeTreeIndexConditionPtr createIndexCondition(
|
MergeTreeIndexConditionPtr createIndexCondition(
|
||||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
|
||||||
|
|
||||||
size_t max_rows = 0;
|
size_t max_rows = 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -102,8 +102,6 @@ public:
|
|||||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) 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 override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; }
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const String distance_function;
|
const String distance_function;
|
||||||
const unum::usearch::scalar_kind_t scalar_kind;
|
const unum::usearch::scalar_kind_t scalar_kind;
|
||||||
|
@ -160,9 +160,6 @@ struct IMergeTreeIndex
|
|||||||
return {0 /*unknown*/, ""};
|
return {0 /*unknown*/, ""};
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Checks whether the column is in data skipping index.
|
|
||||||
virtual bool mayBenefitFromIndexForIn(const ASTPtr & node) const = 0;
|
|
||||||
|
|
||||||
virtual MergeTreeIndexGranulePtr createIndexGranule() const = 0;
|
virtual MergeTreeIndexGranulePtr createIndexGranule() const = 0;
|
||||||
|
|
||||||
virtual MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const = 0;
|
virtual MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const = 0;
|
||||||
|
Loading…
Reference in New Issue
Block a user