mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Rename MergeTreeIndexFullText --> MergeTreeIndexBloomFilterText
This commit is contained in:
parent
f0faac2e8b
commit
0ae422d33c
@ -7,7 +7,7 @@
|
|||||||
#include <Disks/DiskLocal.h>
|
#include <Disks/DiskLocal.h>
|
||||||
#include <Interpreters/GinFilter.h>
|
#include <Interpreters/GinFilter.h>
|
||||||
#include <Storages/MergeTree/GinIndexStore.h>
|
#include <Storages/MergeTree/GinIndexStore.h>
|
||||||
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
|
#include <Storages/MergeTree/MergeTreeIndexBloomFilterText.h>
|
||||||
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
|
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
|
||||||
#include <string>
|
#include <string>
|
||||||
#include <algorithm>
|
#include <algorithm>
|
||||||
|
@ -921,7 +921,7 @@ static void assertIndexColumnsType(const Block & header)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexPtr bloomFilterIndexCreatorNew(
|
MergeTreeIndexPtr bloomFilterIndexCreator(
|
||||||
const IndexDescription & index)
|
const IndexDescription & index)
|
||||||
{
|
{
|
||||||
double max_conflict_probability = 0.025;
|
double max_conflict_probability = 0.025;
|
||||||
@ -938,7 +938,7 @@ MergeTreeIndexPtr bloomFilterIndexCreatorNew(
|
|||||||
index, bits_per_row_and_size_of_hash_functions.first, bits_per_row_and_size_of_hash_functions.second);
|
index, bits_per_row_and_size_of_hash_functions.first, bits_per_row_and_size_of_hash_functions.second);
|
||||||
}
|
}
|
||||||
|
|
||||||
void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach)
|
void bloomFilterIndexValidator(const IndexDescription & index, bool attach)
|
||||||
{
|
{
|
||||||
assertIndexColumnsType(index.sample_block);
|
assertIndexColumnsType(index.sample_block);
|
||||||
|
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
|
#include <Storages/MergeTree/MergeTreeIndexBloomFilterText.h>
|
||||||
|
|
||||||
#include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
#include <Common/OptimizedRegularExpression.h>
|
#include <Common/OptimizedRegularExpression.h>
|
||||||
@ -32,7 +32,7 @@ namespace ErrorCodes
|
|||||||
extern const int BAD_ARGUMENTS;
|
extern const int BAD_ARGUMENTS;
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText(
|
MergeTreeIndexGranuleBloomFilterText::MergeTreeIndexGranuleBloomFilterText(
|
||||||
const String & index_name_,
|
const String & index_name_,
|
||||||
size_t columns_number,
|
size_t columns_number,
|
||||||
const BloomFilterParameters & params_)
|
const BloomFilterParameters & params_)
|
||||||
@ -44,7 +44,7 @@ MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText(
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const
|
void MergeTreeIndexGranuleBloomFilterText::serializeBinary(WriteBuffer & ostr) const
|
||||||
{
|
{
|
||||||
if (empty())
|
if (empty())
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name));
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name));
|
||||||
@ -53,7 +53,7 @@ void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const
|
|||||||
ostr.write(reinterpret_cast<const char *>(bloom_filter.getFilter().data()), params.filter_size);
|
ostr.write(reinterpret_cast<const char *>(bloom_filter.getFilter().data()), params.filter_size);
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version)
|
void MergeTreeIndexGranuleBloomFilterText::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version)
|
||||||
{
|
{
|
||||||
if (version != 1)
|
if (version != 1)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version);
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version);
|
||||||
@ -66,7 +66,7 @@ void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTr
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText(
|
MergeTreeIndexAggregatorBloomFilterText::MergeTreeIndexAggregatorBloomFilterText(
|
||||||
const Names & index_columns_,
|
const Names & index_columns_,
|
||||||
const String & index_name_,
|
const String & index_name_,
|
||||||
const BloomFilterParameters & params_,
|
const BloomFilterParameters & params_,
|
||||||
@ -76,20 +76,20 @@ MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText(
|
|||||||
, params(params_)
|
, params(params_)
|
||||||
, token_extractor(token_extractor_)
|
, token_extractor(token_extractor_)
|
||||||
, granule(
|
, granule(
|
||||||
std::make_shared<MergeTreeIndexGranuleFullText>(
|
std::make_shared<MergeTreeIndexGranuleBloomFilterText>(
|
||||||
index_name, index_columns.size(), params))
|
index_name, index_columns.size(), params))
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorFullText::getGranuleAndReset()
|
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorBloomFilterText::getGranuleAndReset()
|
||||||
{
|
{
|
||||||
auto new_granule = std::make_shared<MergeTreeIndexGranuleFullText>(
|
auto new_granule = std::make_shared<MergeTreeIndexGranuleBloomFilterText>(
|
||||||
index_name, index_columns.size(), params);
|
index_name, index_columns.size(), params);
|
||||||
new_granule.swap(granule);
|
new_granule.swap(granule);
|
||||||
return new_granule;
|
return new_granule;
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, size_t limit)
|
void MergeTreeIndexAggregatorBloomFilterText::update(const Block & block, size_t * pos, size_t limit)
|
||||||
{
|
{
|
||||||
if (*pos >= block.rows())
|
if (*pos >= block.rows())
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. "
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. "
|
||||||
@ -137,7 +137,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos,
|
|||||||
*pos += rows_read;
|
*pos += rows_read;
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeConditionFullText::MergeTreeConditionFullText(
|
MergeTreeConditionBloomFilterText::MergeTreeConditionBloomFilterText(
|
||||||
const ActionsDAGPtr & filter_actions_dag,
|
const ActionsDAGPtr & filter_actions_dag,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
const Block & index_sample_block,
|
const Block & index_sample_block,
|
||||||
@ -162,7 +162,7 @@ MergeTreeConditionFullText::MergeTreeConditionFullText(
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Keep in-sync with MergeTreeConditionGinFilter::alwaysUnknownOrTrue
|
/// Keep in-sync with MergeTreeConditionGinFilter::alwaysUnknownOrTrue
|
||||||
bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const
|
bool MergeTreeConditionBloomFilterText::alwaysUnknownOrTrue() const
|
||||||
{
|
{
|
||||||
/// Check like in KeyCondition.
|
/// Check like in KeyCondition.
|
||||||
std::vector<bool> rpn_stack;
|
std::vector<bool> rpn_stack;
|
||||||
@ -212,10 +212,10 @@ bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Keep in-sync with MergeTreeIndexConditionGin::mayBeTrueOnTranuleInPart
|
/// Keep in-sync with MergeTreeIndexConditionGin::mayBeTrueOnTranuleInPart
|
||||||
bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
|
bool MergeTreeConditionBloomFilterText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
|
||||||
{
|
{
|
||||||
std::shared_ptr<MergeTreeIndexGranuleFullText> granule
|
std::shared_ptr<MergeTreeIndexGranuleBloomFilterText> granule
|
||||||
= std::dynamic_pointer_cast<MergeTreeIndexGranuleFullText>(idx_granule);
|
= std::dynamic_pointer_cast<MergeTreeIndexGranuleBloomFilterText>(idx_granule);
|
||||||
if (!granule)
|
if (!granule)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "BloomFilter index condition got a granule with the wrong type.");
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "BloomFilter index condition got a granule with the wrong type.");
|
||||||
|
|
||||||
@ -323,13 +323,13 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx
|
|||||||
return rpn_stack[0].can_be_true;
|
return rpn_stack[0].can_be_true;
|
||||||
}
|
}
|
||||||
|
|
||||||
std::optional<size_t> MergeTreeConditionFullText::getKeyIndex(const std::string & key_column_name)
|
std::optional<size_t> MergeTreeConditionBloomFilterText::getKeyIndex(const std::string & key_column_name)
|
||||||
{
|
{
|
||||||
const auto it = std::ranges::find(index_columns, key_column_name);
|
const auto it = std::ranges::find(index_columns, key_column_name);
|
||||||
return it == index_columns.end() ? std::nullopt : std::make_optional<size_t>(std::ranges::distance(index_columns.cbegin(), it));
|
return it == index_columns.end() ? std::nullopt : std::make_optional<size_t>(std::ranges::distance(index_columns.cbegin(), it));
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out)
|
bool MergeTreeConditionBloomFilterText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out)
|
||||||
{
|
{
|
||||||
{
|
{
|
||||||
Field const_value;
|
Field const_value;
|
||||||
@ -419,7 +419,7 @@ bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode &
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeConditionFullText::traverseTreeEquals(
|
bool MergeTreeConditionBloomFilterText::traverseTreeEquals(
|
||||||
const String & function_name,
|
const String & function_name,
|
||||||
const RPNBuilderTreeNode & key_node,
|
const RPNBuilderTreeNode & key_node,
|
||||||
const DataTypePtr & value_type,
|
const DataTypePtr & value_type,
|
||||||
@ -638,7 +638,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals(
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
|
bool MergeTreeConditionBloomFilterText::tryPrepareSetBloomFilter(
|
||||||
const RPNBuilderTreeNode & left_argument,
|
const RPNBuilderTreeNode & left_argument,
|
||||||
const RPNBuilderTreeNode & right_argument,
|
const RPNBuilderTreeNode & right_argument,
|
||||||
RPNElement & out)
|
RPNElement & out)
|
||||||
@ -714,23 +714,23 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexGranulePtr MergeTreeIndexFullText::createIndexGranule() const
|
MergeTreeIndexGranulePtr MergeTreeIndexBloomFilterText::createIndexGranule() const
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeIndexGranuleFullText>(index.name, index.column_names.size(), params);
|
return std::make_shared<MergeTreeIndexGranuleBloomFilterText>(index.name, index.column_names.size(), params);
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
|
MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilterText::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeIndexAggregatorFullText>(index.column_names, index.name, params, token_extractor.get());
|
return std::make_shared<MergeTreeIndexAggregatorBloomFilterText>(index.column_names, index.name, params, token_extractor.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition(
|
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilterText::createIndexCondition(
|
||||||
const ActionsDAGPtr & filter_dag, ContextPtr context) const
|
const ActionsDAGPtr & filter_dag, ContextPtr context) const
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeConditionFullText>(filter_dag, context, index.sample_block, params, token_extractor.get());
|
return std::make_shared<MergeTreeConditionBloomFilterText>(filter_dag, context, index.sample_block, params, token_extractor.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexPtr bloomFilterIndexCreator(
|
MergeTreeIndexPtr bloomFilterIndexTextCreator(
|
||||||
const IndexDescription & index)
|
const IndexDescription & index)
|
||||||
{
|
{
|
||||||
if (index.type == NgramTokenExtractor::getName())
|
if (index.type == NgramTokenExtractor::getName())
|
||||||
@ -743,7 +743,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator(
|
|||||||
|
|
||||||
auto tokenizer = std::make_unique<NgramTokenExtractor>(n);
|
auto tokenizer = std::make_unique<NgramTokenExtractor>(n);
|
||||||
|
|
||||||
return std::make_shared<MergeTreeIndexFullText>(index, params, std::move(tokenizer));
|
return std::make_shared<MergeTreeIndexBloomFilterText>(index, params, std::move(tokenizer));
|
||||||
}
|
}
|
||||||
else if (index.type == SplitTokenExtractor::getName())
|
else if (index.type == SplitTokenExtractor::getName())
|
||||||
{
|
{
|
||||||
@ -754,7 +754,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator(
|
|||||||
|
|
||||||
auto tokenizer = std::make_unique<SplitTokenExtractor>();
|
auto tokenizer = std::make_unique<SplitTokenExtractor>();
|
||||||
|
|
||||||
return std::make_shared<MergeTreeIndexFullText>(index, params, std::move(tokenizer));
|
return std::make_shared<MergeTreeIndexBloomFilterText>(index, params, std::move(tokenizer));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -762,7 +762,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/)
|
void bloomFilterIndexTextValidator(const IndexDescription & index, bool /*attach*/)
|
||||||
{
|
{
|
||||||
for (const auto & index_data_type : index.data_types)
|
for (const auto & index_data_type : index.data_types)
|
||||||
{
|
{
|
@ -11,14 +11,14 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct MergeTreeIndexGranuleFullText final : public IMergeTreeIndexGranule
|
struct MergeTreeIndexGranuleBloomFilterText final : public IMergeTreeIndexGranule
|
||||||
{
|
{
|
||||||
explicit MergeTreeIndexGranuleFullText(
|
explicit MergeTreeIndexGranuleBloomFilterText(
|
||||||
const String & index_name_,
|
const String & index_name_,
|
||||||
size_t columns_number,
|
size_t columns_number,
|
||||||
const BloomFilterParameters & params_);
|
const BloomFilterParameters & params_);
|
||||||
|
|
||||||
~MergeTreeIndexGranuleFullText() override = default;
|
~MergeTreeIndexGranuleBloomFilterText() override = default;
|
||||||
|
|
||||||
void serializeBinary(WriteBuffer & ostr) const override;
|
void serializeBinary(WriteBuffer & ostr) const override;
|
||||||
void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override;
|
void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override;
|
||||||
@ -32,17 +32,17 @@ struct MergeTreeIndexGranuleFullText final : public IMergeTreeIndexGranule
|
|||||||
bool has_elems;
|
bool has_elems;
|
||||||
};
|
};
|
||||||
|
|
||||||
using MergeTreeIndexGranuleFullTextPtr = std::shared_ptr<MergeTreeIndexGranuleFullText>;
|
using MergeTreeIndexGranuleBloomFilterTextPtr = std::shared_ptr<MergeTreeIndexGranuleBloomFilterText>;
|
||||||
|
|
||||||
struct MergeTreeIndexAggregatorFullText final : IMergeTreeIndexAggregator
|
struct MergeTreeIndexAggregatorBloomFilterText final : IMergeTreeIndexAggregator
|
||||||
{
|
{
|
||||||
explicit MergeTreeIndexAggregatorFullText(
|
explicit MergeTreeIndexAggregatorBloomFilterText(
|
||||||
const Names & index_columns_,
|
const Names & index_columns_,
|
||||||
const String & index_name_,
|
const String & index_name_,
|
||||||
const BloomFilterParameters & params_,
|
const BloomFilterParameters & params_,
|
||||||
TokenExtractorPtr token_extractor_);
|
TokenExtractorPtr token_extractor_);
|
||||||
|
|
||||||
~MergeTreeIndexAggregatorFullText() override = default;
|
~MergeTreeIndexAggregatorBloomFilterText() override = default;
|
||||||
|
|
||||||
bool empty() const override { return !granule || granule->empty(); }
|
bool empty() const override { return !granule || granule->empty(); }
|
||||||
MergeTreeIndexGranulePtr getGranuleAndReset() override;
|
MergeTreeIndexGranulePtr getGranuleAndReset() override;
|
||||||
@ -54,21 +54,21 @@ struct MergeTreeIndexAggregatorFullText final : IMergeTreeIndexAggregator
|
|||||||
BloomFilterParameters params;
|
BloomFilterParameters params;
|
||||||
TokenExtractorPtr token_extractor;
|
TokenExtractorPtr token_extractor;
|
||||||
|
|
||||||
MergeTreeIndexGranuleFullTextPtr granule;
|
MergeTreeIndexGranuleBloomFilterTextPtr granule;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
class MergeTreeConditionFullText final : public IMergeTreeIndexCondition
|
class MergeTreeConditionBloomFilterText final : public IMergeTreeIndexCondition
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
MergeTreeConditionFullText(
|
MergeTreeConditionBloomFilterText(
|
||||||
const ActionsDAGPtr & filter_actions_dag,
|
const ActionsDAGPtr & filter_actions_dag,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
const Block & index_sample_block,
|
const Block & index_sample_block,
|
||||||
const BloomFilterParameters & params_,
|
const BloomFilterParameters & params_,
|
||||||
TokenExtractorPtr token_extactor_);
|
TokenExtractorPtr token_extactor_);
|
||||||
|
|
||||||
~MergeTreeConditionFullText() override = default;
|
~MergeTreeConditionBloomFilterText() override = default;
|
||||||
|
|
||||||
bool alwaysUnknownOrTrue() const override;
|
bool alwaysUnknownOrTrue() const override;
|
||||||
bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override;
|
bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override;
|
||||||
@ -146,10 +146,10 @@ private:
|
|||||||
RPN rpn;
|
RPN rpn;
|
||||||
};
|
};
|
||||||
|
|
||||||
class MergeTreeIndexFullText final : public IMergeTreeIndex
|
class MergeTreeIndexBloomFilterText final : public IMergeTreeIndex
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
MergeTreeIndexFullText(
|
MergeTreeIndexBloomFilterText(
|
||||||
const IndexDescription & index_,
|
const IndexDescription & index_,
|
||||||
const BloomFilterParameters & params_,
|
const BloomFilterParameters & params_,
|
||||||
std::unique_ptr<ITokenExtractor> && token_extractor_)
|
std::unique_ptr<ITokenExtractor> && token_extractor_)
|
||||||
@ -157,7 +157,7 @@ public:
|
|||||||
, params(params_)
|
, params(params_)
|
||||||
, token_extractor(std::move(token_extractor_)) {}
|
, token_extractor(std::move(token_extractor_)) {}
|
||||||
|
|
||||||
~MergeTreeIndexFullText() override = default;
|
~MergeTreeIndexBloomFilterText() override = default;
|
||||||
|
|
||||||
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
||||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
@ -5,7 +5,6 @@
|
|||||||
#include <Storages/MergeTree/KeyCondition.h>
|
#include <Storages/MergeTree/KeyCondition.h>
|
||||||
#include <Storages/MergeTree/MergeTreeData.h>
|
#include <Storages/MergeTree/MergeTreeData.h>
|
||||||
#include <base/types.h>
|
#include <base/types.h>
|
||||||
#include <atomic>
|
|
||||||
#include <memory>
|
#include <memory>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -115,14 +115,14 @@ MergeTreeIndexFactory::MergeTreeIndexFactory()
|
|||||||
registerCreator("set", setIndexCreator);
|
registerCreator("set", setIndexCreator);
|
||||||
registerValidator("set", setIndexValidator);
|
registerValidator("set", setIndexValidator);
|
||||||
|
|
||||||
registerCreator("ngrambf_v1", bloomFilterIndexCreator);
|
registerCreator("ngrambf_v1", bloomFilterIndexTextCreator);
|
||||||
registerValidator("ngrambf_v1", bloomFilterIndexValidator);
|
registerValidator("ngrambf_v1", bloomFilterIndexTextValidator);
|
||||||
|
|
||||||
registerCreator("tokenbf_v1", bloomFilterIndexCreator);
|
registerCreator("tokenbf_v1", bloomFilterIndexTextCreator);
|
||||||
registerValidator("tokenbf_v1", bloomFilterIndexValidator);
|
registerValidator("tokenbf_v1", bloomFilterIndexTextValidator);
|
||||||
|
|
||||||
registerCreator("bloom_filter", bloomFilterIndexCreatorNew);
|
registerCreator("bloom_filter", bloomFilterIndexCreator);
|
||||||
registerValidator("bloom_filter", bloomFilterIndexValidatorNew);
|
registerValidator("bloom_filter", bloomFilterIndexValidator);
|
||||||
|
|
||||||
registerCreator("hypothesis", hypothesisIndexCreator);
|
registerCreator("hypothesis", hypothesisIndexCreator);
|
||||||
registerValidator("hypothesis", hypothesisIndexValidator);
|
registerValidator("hypothesis", hypothesisIndexValidator);
|
||||||
|
@ -221,12 +221,12 @@ void minmaxIndexValidator(const IndexDescription & index, bool attach);
|
|||||||
MergeTreeIndexPtr setIndexCreator(const IndexDescription & index);
|
MergeTreeIndexPtr setIndexCreator(const IndexDescription & index);
|
||||||
void setIndexValidator(const IndexDescription & index, bool attach);
|
void setIndexValidator(const IndexDescription & index, bool attach);
|
||||||
|
|
||||||
|
MergeTreeIndexPtr bloomFilterIndexTextCreator(const IndexDescription & index);
|
||||||
|
void bloomFilterIndexTextValidator(const IndexDescription & index, bool attach);
|
||||||
|
|
||||||
MergeTreeIndexPtr bloomFilterIndexCreator(const IndexDescription & index);
|
MergeTreeIndexPtr bloomFilterIndexCreator(const IndexDescription & index);
|
||||||
void bloomFilterIndexValidator(const IndexDescription & index, bool attach);
|
void bloomFilterIndexValidator(const IndexDescription & index, bool attach);
|
||||||
|
|
||||||
MergeTreeIndexPtr bloomFilterIndexCreatorNew(const IndexDescription & index);
|
|
||||||
void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach);
|
|
||||||
|
|
||||||
MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index);
|
MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index);
|
||||||
void hypothesisIndexValidator(const IndexDescription & index, bool attach);
|
void hypothesisIndexValidator(const IndexDescription & index, bool attach);
|
||||||
|
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
|
#include <Storages/MergeTree/MergeTreeIndexBloomFilterText.h>
|
||||||
|
|
||||||
#include <Common/PODArray_fwd.h>
|
#include <Common/PODArray_fwd.h>
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user