ClickHouse/src/Storages/MergeTree/MergeTreeIndexFullText.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

772 lines
28 KiB
C++
Raw Normal View History

2019-05-10 03:42:28 +00:00
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
2019-02-20 11:22:07 +00:00
2022-01-17 04:33:47 +00:00
#include <Columns/ColumnArray.h>
2019-02-20 11:22:07 +00:00
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
2019-02-20 11:22:07 +00:00
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
2019-02-20 12:12:41 +00:00
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/misc.h>
2019-02-20 12:12:41 +00:00
#include <Storages/MergeTree/MergeTreeData.h>
2019-03-07 09:15:58 +00:00
#include <Storages/MergeTree/RPNBuilder.h>
2022-10-26 10:43:29 +00:00
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
2019-02-24 21:17:52 +00:00
#include <Parsers/ASTIdentifier.h>
2019-02-20 12:12:41 +00:00
#include <Parsers/ASTLiteral.h>
2019-02-24 21:17:52 +00:00
#include <Parsers/ASTSubquery.h>
2022-10-26 10:43:29 +00:00
#include <Parsers/ASTSelectQuery.h>
#include <Core/Defines.h>
2019-02-20 12:12:41 +00:00
#include <Poco/Logger.h>
2019-02-20 11:22:07 +00:00
2019-02-20 11:22:07 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int LOGICAL_ERROR;
2019-02-20 11:22:07 +00:00
extern const int INCORRECT_QUERY;
2020-07-10 17:53:58 +00:00
extern const int BAD_ARGUMENTS;
2019-02-20 11:22:07 +00:00
}
2020-05-28 12:37:05 +00:00
MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText(
const String & index_name_,
size_t columns_number,
const BloomFilterParameters & params_)
: index_name(index_name_)
, params(params_)
2019-02-22 19:59:40 +00:00
, bloom_filters(
2020-05-28 12:37:05 +00:00
columns_number, BloomFilter(params))
, has_elems(false)
{
}
2019-02-20 11:22:07 +00:00
2019-05-10 03:42:28 +00:00
void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const
2019-02-20 11:22:07 +00:00
{
if (empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name));
2019-02-20 11:22:07 +00:00
2019-02-23 09:26:32 +00:00
for (const auto & bloom_filter : bloom_filters)
2020-05-28 12:37:05 +00:00
ostr.write(reinterpret_cast<const char *>(bloom_filter.getFilter().data()), params.filter_size);
2019-02-20 11:22:07 +00:00
}
void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version)
2019-02-20 11:22:07 +00:00
{
if (version != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version);
2019-02-23 09:26:32 +00:00
for (auto & bloom_filter : bloom_filters)
2019-02-22 19:59:40 +00:00
{
2022-11-11 09:56:18 +00:00
istr.readStrict(reinterpret_cast<char *>(bloom_filter.getFilter().data()), params.filter_size);
2019-02-22 19:59:40 +00:00
}
2019-02-20 12:48:50 +00:00
has_elems = true;
2019-02-20 11:22:07 +00:00
}
2019-03-11 17:59:36 +00:00
2020-05-28 12:37:05 +00:00
MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText(
const Names & index_columns_,
const String & index_name_,
const BloomFilterParameters & params_,
TokenExtractorPtr token_extractor_)
: index_columns(index_columns_)
, index_name (index_name_)
, params(params_)
, token_extractor(token_extractor_)
, granule(
std::make_shared<MergeTreeIndexGranuleFullText>(
index_name, index_columns.size(), params))
{
}
2019-03-11 17:59:36 +00:00
2019-05-10 03:42:28 +00:00
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorFullText::getGranuleAndReset()
2019-03-11 17:59:36 +00:00
{
2020-05-28 12:37:05 +00:00
auto new_granule = std::make_shared<MergeTreeIndexGranuleFullText>(
index_name, index_columns.size(), params);
2019-03-11 17:59:36 +00:00
new_granule.swap(granule);
return new_granule;
}
2019-05-10 03:42:28 +00:00
void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, size_t limit)
2019-02-20 11:22:07 +00:00
{
if (*pos >= block.rows())
throw Exception(
"The provided position is not less than the number of block rows. Position: "
+ toString(*pos) + ", Block rows: " + toString(block.rows()) + ".", ErrorCodes::LOGICAL_ERROR);
size_t rows_read = std::min(limit, block.rows() - *pos);
2020-05-28 12:37:05 +00:00
for (size_t col = 0; col < index_columns.size(); ++col)
2019-02-20 11:22:07 +00:00
{
const auto & column_with_type = block.getByName(index_columns[col]);
const auto & column = column_with_type.column;
2021-09-21 18:28:52 +00:00
size_t current_position = *pos;
if (isArray(column_with_type.type))
2019-02-22 19:59:40 +00:00
{
const auto & column_array = assert_cast<const ColumnArray &>(*column);
const auto & column_offsets = column_array.getOffsets();
const auto & column_key = column_array.getData();
for (size_t i = 0; i < rows_read; ++i)
{
size_t element_start_row = column_offsets[current_position - 1];
size_t elements_size = column_offsets[current_position] - element_start_row;
2021-12-20 12:55:07 +00:00
for (size_t row_num = 0; row_num < elements_size; ++row_num)
{
auto ref = column_key.getDataAt(element_start_row + row_num);
2021-10-04 15:53:17 +00:00
token_extractor->stringPaddedToBloomFilter(ref.data, ref.size, granule->bloom_filters[col]);
}
2021-09-21 18:28:52 +00:00
current_position += 1;
}
}
else
{
for (size_t i = 0; i < rows_read; ++i)
{
2021-09-21 18:28:52 +00:00
auto ref = column->getDataAt(current_position + i);
2021-10-04 15:53:17 +00:00
token_extractor->stringPaddedToBloomFilter(ref.data, ref.size, granule->bloom_filters[col]);
}
2019-02-22 19:59:40 +00:00
}
2019-02-20 11:22:07 +00:00
}
2021-09-21 18:28:52 +00:00
2019-03-11 17:59:36 +00:00
granule->has_elems = true;
2021-09-21 18:28:52 +00:00
*pos += rows_read;
2019-02-20 11:22:07 +00:00
}
2019-05-10 03:42:28 +00:00
MergeTreeConditionFullText::MergeTreeConditionFullText(
2019-02-20 16:24:46 +00:00
const SelectQueryInfo & query_info,
ContextPtr context,
2020-05-28 12:37:05 +00:00
const Block & index_sample_block,
const BloomFilterParameters & params_,
TokenExtractorPtr token_extactor_)
: index_columns(index_sample_block.getNames())
, index_data_types(index_sample_block.getNamesAndTypesList().getTypes())
, params(params_)
, token_extractor(token_extactor_)
, prepared_sets(query_info.prepared_sets)
2019-02-20 16:24:46 +00:00
{
2022-11-17 18:44:26 +00:00
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;
}
2022-10-26 10:43:29 +00:00
ASTPtr filter_node = buildFilterNode(query_info.query);
if (!filter_node)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context);
RPNBuilder<RPNElement> builder(
filter_node,
context,
std::move(block_with_constants),
query_info.prepared_sets,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
2019-02-20 16:24:46 +00:00
}
2019-05-10 03:42:28 +00:00
bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const
2019-02-20 16:24:46 +00:00
{
/// Check like in KeyCondition.
std::vector<bool> rpn_stack;
for (const auto & element : rpn)
{
if (element.function == RPNElement::FUNCTION_UNKNOWN
|| element.function == RPNElement::ALWAYS_TRUE)
{
rpn_stack.push_back(true);
}
else if (element.function == RPNElement::FUNCTION_EQUALS
2019-02-20 20:17:44 +00:00
|| element.function == RPNElement::FUNCTION_NOT_EQUALS
|| element.function == RPNElement::FUNCTION_HAS
2019-02-24 21:17:52 +00:00
|| element.function == RPNElement::FUNCTION_IN
|| element.function == RPNElement::FUNCTION_NOT_IN
|| element.function == RPNElement::FUNCTION_MULTI_SEARCH
2019-02-20 20:17:44 +00:00
|| element.function == RPNElement::ALWAYS_FALSE)
2019-02-20 16:24:46 +00:00
{
rpn_stack.push_back(false);
}
else if (element.function == RPNElement::FUNCTION_NOT)
{
// do nothing
}
else if (element.function == RPNElement::FUNCTION_AND)
{
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
rpn_stack.back() = arg1 && arg2;
}
else if (element.function == RPNElement::FUNCTION_OR)
{
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
rpn_stack.back() = arg1 || arg2;
}
else
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
}
return rpn_stack[0];
}
2019-05-10 03:42:28 +00:00
bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
2019-02-20 16:24:46 +00:00
{
2019-05-10 03:42:28 +00:00
std::shared_ptr<MergeTreeIndexGranuleFullText> granule
= std::dynamic_pointer_cast<MergeTreeIndexGranuleFullText>(idx_granule);
2019-02-20 16:24:46 +00:00
if (!granule)
throw Exception(
"BloomFilter index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
/// Check like in KeyCondition.
std::vector<BoolMask> rpn_stack;
2019-02-22 07:59:07 +00:00
for (const auto & element : rpn)
2019-02-20 16:24:46 +00:00
{
if (element.function == RPNElement::FUNCTION_UNKNOWN)
{
rpn_stack.emplace_back(true, true);
}
else if (element.function == RPNElement::FUNCTION_EQUALS
|| element.function == RPNElement::FUNCTION_NOT_EQUALS
|| element.function == RPNElement::FUNCTION_HAS)
2019-02-20 16:24:46 +00:00
{
rpn_stack.emplace_back(granule->bloom_filters[element.key_column].contains(*element.bloom_filter), true);
2019-02-20 16:24:46 +00:00
if (element.function == RPNElement::FUNCTION_NOT_EQUALS)
rpn_stack.back() = !rpn_stack.back();
}
2019-02-24 21:17:52 +00:00
else if (element.function == RPNElement::FUNCTION_IN
|| element.function == RPNElement::FUNCTION_NOT_IN)
2019-02-24 21:17:52 +00:00
{
2019-02-25 08:43:19 +00:00
std::vector<bool> result(element.set_bloom_filters.back().size(), true);
2019-02-24 21:17:52 +00:00
2019-02-25 18:38:57 +00:00
for (size_t column = 0; column < element.set_key_position.size(); ++column)
2019-02-24 21:17:52 +00:00
{
2019-02-25 18:38:57 +00:00
const size_t key_idx = element.set_key_position[column];
2019-02-24 21:17:52 +00:00
2019-02-25 08:43:19 +00:00
const auto & bloom_filters = element.set_bloom_filters[column];
for (size_t row = 0; row < bloom_filters.size(); ++row)
result[row] = result[row] && granule->bloom_filters[key_idx].contains(bloom_filters[row]);
2019-02-24 21:17:52 +00:00
}
2019-02-25 08:43:19 +00:00
rpn_stack.emplace_back(
std::find(std::cbegin(result), std::cend(result), true) != std::end(result), true);
2019-02-24 21:17:52 +00:00
if (element.function == RPNElement::FUNCTION_NOT_IN)
rpn_stack.back() = !rpn_stack.back();
}
else if (element.function == RPNElement::FUNCTION_MULTI_SEARCH)
{
std::vector<bool> result(element.set_bloom_filters.back().size(), true);
const auto & bloom_filters = element.set_bloom_filters[0];
for (size_t row = 0; row < bloom_filters.size(); ++row)
result[row] = result[row] && granule->bloom_filters[element.key_column].contains(bloom_filters[row]);
rpn_stack.emplace_back(
std::find(std::cbegin(result), std::cend(result), true) != std::end(result), true);
}
2019-02-20 16:24:46 +00:00
else if (element.function == RPNElement::FUNCTION_NOT)
{
rpn_stack.back() = !rpn_stack.back();
}
else if (element.function == RPNElement::FUNCTION_AND)
{
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
rpn_stack.back() = arg1 & arg2;
}
else if (element.function == RPNElement::FUNCTION_OR)
{
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
rpn_stack.back() = arg1 | arg2;
}
else if (element.function == RPNElement::ALWAYS_FALSE)
{
rpn_stack.emplace_back(false, true);
}
else if (element.function == RPNElement::ALWAYS_TRUE)
{
rpn_stack.emplace_back(true, false);
}
else
throw Exception("Unexpected function type in BloomFilterCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
2019-02-20 16:24:46 +00:00
}
if (rpn_stack.size() != 1)
throw Exception("Unexpected stack size in BloomFilterCondition::mayBeTrueOnGranule", ErrorCodes::LOGICAL_ERROR);
2019-02-20 16:24:46 +00:00
return rpn_stack[0].can_be_true;
}
bool MergeTreeConditionFullText::getKey(const std::string & key_column_name, size_t & key_column_num)
2019-02-20 16:24:46 +00:00
{
auto it = std::find(index_columns.begin(), index_columns.end(), key_column_name);
2020-05-28 12:37:05 +00:00
if (it == index_columns.end())
2019-02-20 16:24:46 +00:00
return false;
2020-05-28 12:37:05 +00:00
key_column_num = static_cast<size_t>(it - index_columns.begin());
2019-02-20 16:24:46 +00:00
return true;
}
2022-10-26 10:43:29 +00:00
bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out)
2019-02-20 16:24:46 +00:00
{
{
Field const_value;
DataTypePtr const_type;
2019-02-20 16:24:46 +00:00
2022-10-26 10:43:29 +00:00
if (node.tryGetConstant(const_value, const_type))
{
/// Check constant like in KeyCondition
if (const_value.getType() == Field::Types::UInt64
|| const_value.getType() == Field::Types::Int64
|| const_value.getType() == Field::Types::Float64)
{
/// Zero in all types is represented in memory the same way as in UInt64.
out.function = const_value.get<UInt64>()
? RPNElement::ALWAYS_TRUE
: RPNElement::ALWAYS_FALSE;
return true;
}
}
}
2022-10-26 10:43:29 +00:00
if (node.isFunction())
{
2022-10-26 10:43:29 +00:00
auto function_node = node.toFunctionNode();
auto function_name = function_node.getFunctionName();
2022-10-26 10:43:29 +00:00
size_t arguments_size = function_node.getArgumentsSize();
if (arguments_size != 2)
return false;
2022-10-26 10:43:29 +00:00
auto left_argument = function_node.getArgumentAt(0);
auto right_argument = function_node.getArgumentAt(1);
if (functionIsInOrGlobalInOperator(function_name))
{
2022-10-26 10:43:29 +00:00
if (tryPrepareSetBloomFilter(left_argument, right_argument, out))
{
2022-10-26 10:43:29 +00:00
if (function_name == "notIn")
{
out.function = RPNElement::FUNCTION_NOT_IN;
return true;
}
2022-10-26 10:43:29 +00:00
else if (function_name == "in")
{
out.function = RPNElement::FUNCTION_IN;
return true;
}
}
2019-02-20 16:24:46 +00:00
}
2022-10-26 10:43:29 +00:00
else if (function_name == "equals" ||
function_name == "notEquals" ||
function_name == "has" ||
function_name == "mapContains" ||
function_name == "like" ||
function_name == "notLike" ||
function_name == "hasToken" ||
function_name == "startsWith" ||
function_name == "endsWith" ||
function_name == "multiSearchAny")
{
Field const_value;
DataTypePtr const_type;
2022-10-26 10:43:29 +00:00
if (right_argument.tryGetConstant(const_value, const_type))
{
2022-10-26 10:43:29 +00:00
if (traverseTreeEquals(function_name, left_argument, const_type, const_value, out))
return true;
}
2022-10-26 10:43:29 +00:00
else if (left_argument.tryGetConstant(const_value, const_type) && (function_name == "equals" || function_name == "notEquals"))
{
2022-10-26 10:43:29 +00:00
if (traverseTreeEquals(function_name, right_argument, const_type, const_value, out))
return true;
}
}
}
return false;
}
2022-10-26 10:43:29 +00:00
bool MergeTreeConditionFullText::traverseTreeEquals(
const String & function_name,
2022-10-26 10:43:29 +00:00
const RPNBuilderTreeNode & key_node,
const DataTypePtr & value_type,
const Field & value_field,
RPNElement & out)
{
auto value_data_type = WhichDataType(value_type);
if (!value_data_type.isStringOrFixedString() && !value_data_type.isArray())
return false;
Field const_value = value_field;
2022-10-26 10:43:29 +00:00
auto column_name = key_node.getColumnName();
size_t key_column_num = 0;
2022-10-26 10:43:29 +00:00
bool key_exists = getKey(column_name, key_column_num);
bool map_key_exists = getKey(fmt::format("mapKeys({})", column_name), key_column_num);
2022-10-26 10:43:29 +00:00
if (key_node.isFunction())
{
2022-10-26 10:43:29 +00:00
auto key_function_node = key_node.toFunctionNode();
auto key_function_node_function_name = key_function_node.getFunctionName();
if (key_function_node_function_name == "arrayElement")
{
/** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map
* we return default value for arrayElement.
*
* We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists.
*/
if (value_field == value_type->getDefault())
return false;
2022-10-26 10:43:29 +00:00
auto first_argument = key_function_node.getArgumentAt(0);
2022-10-31 11:30:00 +00:00
const auto map_column_name = first_argument.getColumnName();
size_t map_keys_key_column_num = 0;
auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name);
bool map_keys_exists = getKey(map_keys_index_column_name, map_keys_key_column_num);
size_t map_values_key_column_num = 0;
auto map_values_index_column_name = fmt::format("mapValues({})", map_column_name);
bool map_values_exists = getKey(map_values_index_column_name, map_values_key_column_num);
if (map_keys_exists)
{
2022-10-26 10:43:29 +00:00
auto second_argument = key_function_node.getArgumentAt(1);
DataTypePtr const_type;
2022-10-26 10:43:29 +00:00
if (second_argument.tryGetConstant(const_value, const_type))
{
key_column_num = map_keys_key_column_num;
key_exists = true;
}
else
{
return false;
}
}
else if (map_values_exists)
{
key_column_num = map_values_key_column_num;
key_exists = true;
}
else
{
return false;
}
}
}
if (!key_exists && !map_key_exists)
return false;
if (map_key_exists && (function_name == "has" || function_name == "mapContains"))
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_HAS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
2021-10-04 15:53:17 +00:00
auto & value = const_value.get<String>();
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
}
else if (function_name == "has")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_HAS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
2021-10-04 15:53:17 +00:00
auto & value = const_value.get<String>();
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
}
if (function_name == "notEquals")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_NOT_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
2021-10-04 15:53:17 +00:00
const auto & value = const_value.get<String>();
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
}
else if (function_name == "equals")
{
out.key_column = key_column_num;
2021-10-04 15:53:17 +00:00
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
const auto & value = const_value.get<String>();
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
}
else if (function_name == "like")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
2021-10-04 15:53:17 +00:00
const auto & value = const_value.get<String>();
2021-10-05 21:37:45 +00:00
token_extractor->stringLikeToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
}
else if (function_name == "notLike")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_NOT_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
2021-10-04 15:53:17 +00:00
const auto & value = const_value.get<String>();
2021-10-05 21:37:45 +00:00
token_extractor->stringLikeToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
}
else if (function_name == "hasToken")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
2021-10-04 15:53:17 +00:00
const auto & value = const_value.get<String>();
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
2019-02-20 16:24:46 +00:00
}
else if (function_name == "startsWith")
2019-02-20 16:24:46 +00:00
{
out.key_column = key_column_num;
2021-10-04 15:53:17 +00:00
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
const auto & value = const_value.get<String>();
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
}
else if (function_name == "endsWith")
{
out.key_column = key_column_num;
2021-10-04 15:53:17 +00:00
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
const auto & value = const_value.get<String>();
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
return true;
}
else if (function_name == "multiSearchAny")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_MULTI_SEARCH;
/// 2d vector is not needed here but is used because already exists for FUNCTION_IN
std::vector<std::vector<BloomFilter>> bloom_filters;
bloom_filters.emplace_back();
for (const auto & element : const_value.get<Array>())
2019-02-20 19:27:23 +00:00
{
if (element.getType() != Field::Types::String)
return false;
2019-02-20 16:24:46 +00:00
bloom_filters.back().emplace_back(params);
2021-10-04 15:53:17 +00:00
const auto & value = element.get<String>();
2021-10-05 21:37:45 +00:00
token_extractor->stringToBloomFilter(value.data(), value.size(), bloom_filters.back().back());
2019-02-20 19:27:23 +00:00
}
out.set_bloom_filters = std::move(bloom_filters);
return true;
2019-02-20 16:24:46 +00:00
}
return false;
}
2019-05-10 03:42:28 +00:00
bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
2022-10-26 10:43:29 +00:00
const RPNBuilderTreeNode & left_argument,
const RPNBuilderTreeNode & right_argument,
2019-02-24 21:17:52 +00:00
RPNElement & out)
2019-02-22 10:51:19 +00:00
{
2019-02-24 21:17:52 +00:00
std::vector<KeyTuplePositionMapping> key_tuple_mapping;
DataTypes data_types;
2022-10-31 11:30:00 +00:00
auto left_argument_function_node_optional = left_argument.toFunctionNodeOrNull();
if (left_argument_function_node_optional && left_argument_function_node_optional->getFunctionName() == "tuple")
2019-02-24 21:17:52 +00:00
{
2022-10-31 11:30:00 +00:00
const auto & left_argument_function_node = *left_argument_function_node_optional;
size_t left_argument_function_node_arguments_size = left_argument_function_node.getArgumentsSize();
2022-10-26 10:43:29 +00:00
2022-10-31 11:30:00 +00:00
for (size_t i = 0; i < left_argument_function_node_arguments_size; ++i)
2019-02-24 21:17:52 +00:00
{
2022-10-31 11:30:00 +00:00
size_t key = 0;
if (getKey(left_argument_function_node.getArgumentAt(i).getColumnName(), key))
2019-02-24 21:17:52 +00:00
{
2022-10-31 11:30:00 +00:00
key_tuple_mapping.emplace_back(i, key);
data_types.push_back(index_data_types[key]);
2019-02-24 21:17:52 +00:00
}
}
}
else
{
size_t key = 0;
2022-10-26 10:43:29 +00:00
if (getKey(left_argument.getColumnName(), key))
2019-02-24 21:17:52 +00:00
{
2019-02-25 08:43:19 +00:00
key_tuple_mapping.emplace_back(0, key);
2020-05-28 12:37:05 +00:00
data_types.push_back(index_data_types[key]);
2019-02-24 21:17:52 +00:00
}
}
if (key_tuple_mapping.empty())
return false;
2022-10-26 10:43:29 +00:00
auto prepared_set = right_argument.tryGetPreparedSet(data_types);
2022-07-18 15:53:30 +00:00
if (!prepared_set)
2019-02-24 21:17:52 +00:00
return false;
2019-02-25 08:43:19 +00:00
for (const auto & data_type : prepared_set->getDataTypes())
if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString)
return false;
2019-05-10 03:42:28 +00:00
std::vector<std::vector<BloomFilter>> bloom_filters;
2019-02-25 18:38:57 +00:00
std::vector<size_t> key_position;
2019-02-24 21:17:52 +00:00
Columns columns = prepared_set->getSetElements();
2020-03-09 02:05:04 +00:00
for (const auto & elem : key_tuple_mapping)
2019-02-24 21:17:52 +00:00
{
2019-02-25 08:43:19 +00:00
bloom_filters.emplace_back();
2020-03-09 02:05:04 +00:00
key_position.push_back(elem.key_index);
2019-02-25 18:38:57 +00:00
2020-03-09 02:05:04 +00:00
size_t tuple_idx = elem.tuple_index;
2019-02-24 21:17:52 +00:00
const auto & column = columns[tuple_idx];
for (size_t row = 0; row < prepared_set->getTotalRowCount(); ++row)
{
2020-05-28 12:37:05 +00:00
bloom_filters.back().emplace_back(params);
2019-02-24 21:17:52 +00:00
auto ref = column->getDataAt(row);
2021-10-04 15:53:17 +00:00
token_extractor->stringPaddedToBloomFilter(ref.data, ref.size, bloom_filters.back().back());
2019-02-24 21:17:52 +00:00
}
}
2019-02-25 18:38:57 +00:00
out.set_key_position = std::move(key_position);
2019-02-24 21:17:52 +00:00
out.set_bloom_filters = std::move(bloom_filters);
return true;
}
2019-02-22 10:51:19 +00:00
2019-05-10 03:42:28 +00:00
MergeTreeIndexGranulePtr MergeTreeIndexFullText::createIndexGranule() const
2019-02-20 12:12:41 +00:00
{
2020-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeIndexGranuleFullText>(index.name, index.column_names.size(), params);
2019-02-20 12:12:41 +00:00
}
2019-05-10 03:42:28 +00:00
MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator() const
2019-03-11 17:59:36 +00:00
{
2020-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeIndexAggregatorFullText>(index.column_names, index.name, params, token_extractor.get());
2019-03-11 17:59:36 +00:00
}
2019-06-19 15:30:48 +00:00
MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const
2019-02-20 12:12:41 +00:00
{
2020-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeConditionFullText>(query, context, index.sample_block, params, token_extractor.get());
}
2019-02-20 12:12:41 +00:00
2019-05-10 03:42:28 +00:00
bool MergeTreeIndexFullText::mayBenefitFromIndexForIn(const ASTPtr & node) const
2019-02-25 08:43:19 +00:00
{
2020-05-28 12:37:05 +00:00
return std::find(std::cbegin(index.column_names), std::cend(index.column_names), node->getColumnName()) != std::cend(index.column_names);
2019-02-25 08:43:19 +00:00
}
2020-05-28 13:45:08 +00:00
MergeTreeIndexPtr bloomFilterIndexCreator(
2020-05-28 13:09:03 +00:00
const IndexDescription & index)
2019-02-20 12:12:41 +00:00
{
2020-05-28 12:37:05 +00:00
if (index.type == NgramTokenExtractor::getName())
{
size_t n = index.arguments[0].get<size_t>();
2020-07-10 08:13:21 +00:00
BloomFilterParameters params(
index.arguments[1].get<size_t>(),
index.arguments[2].get<size_t>(),
index.arguments[3].get<size_t>());
2019-02-20 12:12:41 +00:00
2020-05-28 12:37:05 +00:00
auto tokenizer = std::make_unique<NgramTokenExtractor>(n);
2019-02-20 12:12:41 +00:00
2020-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeIndexFullText>(index, params, std::move(tokenizer));
}
else if (index.type == SplitTokenExtractor::getName())
{
2020-07-10 08:13:21 +00:00
BloomFilterParameters params(
index.arguments[0].get<size_t>(),
index.arguments[1].get<size_t>(),
index.arguments[2].get<size_t>());
2019-02-20 12:12:41 +00:00
2020-05-28 12:37:05 +00:00
auto tokenizer = std::make_unique<SplitTokenExtractor>();
2019-02-20 12:12:41 +00:00
2020-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeIndexFullText>(index, params, std::move(tokenizer));
}
else
2019-02-20 12:12:41 +00:00
{
2020-05-28 12:37:05 +00:00
throw Exception("Unknown index type: " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
}
}
2019-02-20 12:12:41 +00:00
2020-05-28 13:09:03 +00:00
void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/)
2020-05-28 12:37:05 +00:00
{
for (const auto & index_data_type : index.data_types)
2021-09-06 10:22:06 +00:00
{
WhichDataType data_type(index_data_type);
if (data_type.isArray())
{
const auto & array_type = assert_cast<const DataTypeArray &>(*index_data_type);
data_type = WhichDataType(array_type.getNestedType());
2022-03-02 10:08:35 +00:00
}
else if (data_type.isLowCarnality())
{
const auto & low_cardinality = assert_cast<const DataTypeLowCardinality &>(*index_data_type);
data_type = WhichDataType(low_cardinality.getDictionaryType());
}
if (!data_type.isString() && !data_type.isFixedString())
throw Exception("Bloom filter index can be used only with `String`, `FixedString`, `LowCardinality(String)`, `LowCardinality(FixedString)` column or Array with `String` or `FixedString` values column.", ErrorCodes::INCORRECT_QUERY);
2019-02-20 12:12:41 +00:00
}
2020-05-28 12:37:05 +00:00
if (index.type == NgramTokenExtractor::getName())
2019-02-25 18:46:54 +00:00
{
2020-05-28 12:37:05 +00:00
if (index.arguments.size() != 4)
2019-02-24 18:55:56 +00:00
throw Exception("`ngrambf` index must have exactly 4 arguments.", ErrorCodes::INCORRECT_QUERY);
2019-02-25 18:46:54 +00:00
}
2020-05-28 12:37:05 +00:00
else if (index.type == SplitTokenExtractor::getName())
2019-02-25 18:46:54 +00:00
{
2020-05-28 12:37:05 +00:00
if (index.arguments.size() != 3)
2019-02-25 14:23:19 +00:00
throw Exception("`tokenbf` index must have exactly 3 arguments.", ErrorCodes::INCORRECT_QUERY);
2019-02-25 18:46:54 +00:00
}
else
{
2020-05-28 12:37:05 +00:00
throw Exception("Unknown index type: " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
2019-02-20 12:12:41 +00:00
}
2020-07-10 08:13:21 +00:00
assert(index.arguments.size() >= 3);
2020-07-10 08:21:40 +00:00
for (const auto & arg : index.arguments)
if (arg.getType() != Field::Types::UInt64)
throw Exception("All parameters to *bf_v1 index must be unsigned integers", ErrorCodes::BAD_ARGUMENTS);
2020-07-10 08:13:21 +00:00
/// Just validate
BloomFilterParameters params(
index.arguments[0].get<size_t>(),
index.arguments[1].get<size_t>(),
index.arguments[2].get<size_t>());
2019-02-20 12:12:41 +00:00
}
2019-02-20 11:22:07 +00:00
}