Rename "inverted" to "fulltext"

This commit is contained in:
Robert Schulze 2024-05-02 07:21:35 +00:00
parent 08bcbfc251
commit 0f87653fef
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
17 changed files with 52 additions and 56 deletions

View File

@ -2,7 +2,7 @@
slug: /en/engines/table-engines/mergetree-family/invertedindexes
sidebar_label: Full-text Indexes
description: Quickly find search terms in text.
keywords: [full-text search, text search, inverted, index, indices]
keywords: [full-text search, text search, index, indices]
---
# Full-text Search using Full-text Indexes [experimental]

View File

@ -494,7 +494,7 @@ Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, ran
#### Special-purpose
- Experimental indexes to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details.
- An experimental inverted index to support full-text search. See [here](invertedindexes.md) for details.
- An experimental full-text index to support full-text search. See [here](invertedindexes.md) for details.
### Functions Support {#functions-support}
@ -502,31 +502,31 @@ Conditions in the `WHERE` clause contains calls of the functions that operate wi
Indexes of type `set` can be utilized by all functions. The other index types are supported as follows:
| Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | inverted |
|------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------|----------|
| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [like](/docs/en/sql-reference/functions/string-search-functions.md/#like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [match](/docs/en/sql-reference/functions/string-search-functions.md/#match) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ |
| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ |
| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ |
| [in](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notIn](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [empty](/docs/en/sql-reference/functions/array-functions/#empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [notEmpty](/docs/en/sql-reference/functions/array-functions/#notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [has](/docs/en/sql-reference/functions/array-functions/#has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ |
| [hasAny](/docs/en/sql-reference/functions/array-functions/#hasany) | ✗ | ✗ | ✔ | ✔ | ✔ | ✗ |
| [hasAll](/docs/en/sql-reference/functions/array-functions/#hasall) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ |
| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ |
| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ |
| hasTokenCaseInsensitive (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ |
| hasTokenCaseInsensitiveOrNull (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ |
| Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | full_text |
|------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------|-----------|
| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notEquals(!=, &lt;&gt;)](/docs/en/sql-reference/functions/comparison-functions.md/#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [like](/docs/en/sql-reference/functions/string-search-functions.md/#like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [match](/docs/en/sql-reference/functions/string-search-functions.md/#match) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ |
| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ |
| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ |
| [in](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notIn](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [empty](/docs/en/sql-reference/functions/array-functions/#empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [notEmpty](/docs/en/sql-reference/functions/array-functions/#notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [has](/docs/en/sql-reference/functions/array-functions/#has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ |
| [hasAny](/docs/en/sql-reference/functions/array-functions/#hasany) | ✗ | ✗ | ✔ | ✔ | ✔ | ✗ |
| [hasAll](/docs/en/sql-reference/functions/array-functions/#hasall) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ |
| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ |
| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ |
| hasTokenCaseInsensitive (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ |
| hasTokenCaseInsensitiveOrNull (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ |
Functions with a constant argument that is less than ngram size cant be used by `ngrambf_v1` for query optimization.

View File

@ -306,7 +306,7 @@ void FstBuilder::add(std::string_view current_word, Output current_output)
size_t current_word_len = current_word.size();
if (current_word_len > MAX_TERM_LENGTH)
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot build inverted index: The maximum term length is {}, this is exceeded by term {}", MAX_TERM_LENGTH, current_word_len);
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot build full-text index: The maximum term length is {}, this is exceeded by term {}", MAX_TERM_LENGTH, current_word_len);
size_t prefix_length_plus1 = getCommonPrefixLength(current_word, previous_word) + 1;

View File

@ -235,7 +235,7 @@ class IColumn;
M(Bool, do_not_merge_across_partitions_select_final, false, "Merge parts only in one partition in select final", 0) \
M(Bool, split_parts_ranges_into_intersecting_and_non_intersecting_final, true, "Split parts ranges into intersecting and non intersecting during FINAL optimization", 0) \
M(Bool, split_intersecting_parts_ranges_into_layers_final, true, "Split intersecting parts ranges into layers during FINAL optimization", 0) \
M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental fulltext (inverted) index.", 0) \
M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \
\
M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \
M(Bool, mysql_map_string_to_text_in_show_columns, true, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Has an effect only when the connection is made through the MySQL wire protocol.", 0) \

View File

@ -29,7 +29,7 @@ GinFilterParameters::GinFilterParameters(size_t ngrams_, UInt64 max_rows_per_pos
max_rows_per_postings_list = std::numeric_limits<UInt64>::max();
if (ngrams > 8)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of inverted index filter cannot be greater than 8");
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of full-text index filter cannot be greater than 8");
}
GinFilter::GinFilter(const GinFilterParameters & params_)

View File

@ -2,12 +2,11 @@
#include <Storages/MergeTree/GinIndexStore.h>
#include <vector>
#include <memory>
namespace DB
{
static inline constexpr auto INVERTED_INDEX_NAME = "inverted";
static inline constexpr auto FULL_TEXT_INDEX_NAME = "full_text";
static inline constexpr UInt64 UNLIMITED_ROWS_PER_POSTINGS_LIST = 0;
static inline constexpr UInt64 MIN_ROWS_PER_POSTINGS_LIST = 8 * 1024;
static inline constexpr UInt64 DEFAULT_MAX_ROWS_PER_POSTINGS_LIST = 64 * 1024;
@ -34,7 +33,7 @@ struct GinSegmentWithRowIdRange
using GinSegmentWithRowIdRangeVector = std::vector<GinSegmentWithRowIdRange>;
/// GinFilter provides underlying functionalities for building inverted index and also
/// GinFilter provides underlying functionalities for building full-text index and also
/// it does filtering the unmatched rows according to its query string.
/// It also builds and uses skipping index which stores (segmentID, RowIDStart, RowIDEnd) triples.
class GinFilter
@ -44,7 +43,7 @@ public:
explicit GinFilter(const GinFilterParameters & params_);
/// Add term (located at 'data' with length 'len') and its row ID to the postings list builder
/// for building inverted index for the given store.
/// for building full-text index for the given store.
void add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store) const;
/// Accumulate (segmentID, RowIDStart, RowIDEnd) for building skipping index

View File

@ -748,15 +748,12 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext());
if (properties.indices.has(index_desc.name))
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {} is not allowed. Please use different index names.", backQuoteIfNeed(index_desc.name));
const auto & settings = getContext()->getSettingsRef();
if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index)
{
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index')");
}
if (index_desc.type == FULL_TEXT_INDEX_NAME && !settings.allow_experimental_inverted_index)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is not enabled (the setting 'allow_experimental_inverted_index')");
if (index_desc.type == "annoy" && !settings.allow_experimental_annoy_index)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index is disabled. Turn on allow_experimental_annoy_index");
if (index_desc.type == "usearch" && !settings.allow_experimental_usearch_index)
throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index is disabled. Turn on allow_experimental_usearch_index");

View File

@ -1100,11 +1100,11 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
return result;
}
bool AlterCommands::hasInvertedIndex(const StorageInMemoryMetadata & metadata)
bool AlterCommands::hasFullTextIndex(const StorageInMemoryMetadata & metadata)
{
for (const auto & index : metadata.secondary_indices)
{
if (index.type == INVERTED_INDEX_NAME)
if (index.type == FULL_TEXT_INDEX_NAME)
return true;
}
return false;

View File

@ -234,8 +234,8 @@ public:
/// additional mutation command (MATERIALIZE_TTL) will be returned.
MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const;
/// Check if commands have any inverted index
static bool hasInvertedIndex(const StorageInMemoryMetadata & metadata);
/// Check if commands have any full-text index
static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata);
};
}

View File

@ -240,7 +240,7 @@ UInt32 GinIndexStore::getNumOfSegments()
readBinary(version, *istr);
if (version > static_cast<std::underlying_type_t<Format>>(CURRENT_GIN_FILE_FORMAT_VERSION))
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported inverted index version {}", version);
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported full-text index version {}", version);
readVarUInt(result, *istr);
}

View File

@ -13,8 +13,8 @@
#include <vector>
#include <absl/container/flat_hash_map.h>
/// GinIndexStore manages the generalized inverted index ("gin") for a data part, and it is made up of one or more immutable
/// index segments.
/// GinIndexStore manages the generalized inverted index ("gin") (full-text index )for a data part, and it is made up of one or more
/// immutable index segments.
///
/// There are 4 types of index files in a store:
/// 1. Segment ID file(.gin_sid): it contains one byte for version followed by the next available segment ID.

View File

@ -3008,9 +3008,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
commands.apply(new_metadata, local_context);
if (AlterCommands::hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index)
if (AlterCommands::hasFullTextIndex(new_metadata) && !settings.allow_experimental_inverted_index)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Experimental Inverted Index feature is not enabled (turn on setting 'allow_experimental_inverted_index')");
"Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_inverted_index')");
for (const auto & disk : getDisks())
if (!disk->supportsHardLinks())

View File

@ -61,7 +61,7 @@ void MergeTreeDataPartChecksum::checkEqual(const MergeTreeDataPartChecksum & rhs
void MergeTreeDataPartChecksum::checkSize(const IDataPartStorage & storage, const String & name) const
{
/// Skip inverted index files, these have a default MergeTreeDataPartChecksum with file_size == 0
/// Skip full-text index files, these have a default MergeTreeDataPartChecksum with file_size == 0
if (isGinFile(name))
return;
@ -88,7 +88,7 @@ void MergeTreeDataPartChecksums::checkEqual(const MergeTreeDataPartChecksums & r
for (const auto & [name, checksum] : files)
{
/// Exclude files written by inverted index from check. No correct checksums are available for them currently.
/// Exclude files written by full-text index from check. No correct checksums are available for them currently.
if (name.ends_with(".gin_dict") || name.ends_with(".gin_post") || name.ends_with(".gin_seg") || name.ends_with(".gin_sid"))
continue;

View File

@ -468,7 +468,7 @@ void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::Data
if (!skip_indices_aggregators[i]->empty())
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed_hashing);
/// Register additional files written only by the inverted index. Required because otherwise DROP TABLE complains about unknown
/// Register additional files written only by the full-text index. Required because otherwise DROP TABLE complains about unknown
/// files. Note that the provided actual checksums are bogus. The problem is that at this point the file writes happened already and
/// we'd need to re-open + hash the files (fixing this is TODO). For now, CHECK TABLE skips these four files.
if (typeid_cast<const MergeTreeIndexFullText *>(&*skip_indices[i]) != nullptr)

View File

@ -652,7 +652,7 @@ static NameSet collectFilesToSkip(
files_to_skip.insert(index->getFileName() + index->getSerializedFileExtension());
files_to_skip.insert(index->getFileName() + mrk_extension);
// Skip all inverted index files, for they will be rebuilt
// Skip all full-text index files, for they will be rebuilt
if (dynamic_cast<const MergeTreeIndexFullText *>(index.get()))
{
auto index_filename = index->getFileName();
@ -731,7 +731,7 @@ static NameToNameVector collectFilesForRenames(
if (command.type == MutationCommand::Type::DROP_INDEX)
{
static const std::array<String, 2> suffixes = {".idx2", ".idx"};
static const std::array<String, 4> gin_suffixes = {".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; /// .gin_* is inverted index
static const std::array<String, 4> gin_suffixes = {".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; /// .gin_* means generalized inverted index (aka. full-text-index)
for (const auto & suffix : suffixes)
{

View File

@ -250,7 +250,7 @@ static IMergeTreeDataPart::Checksums checkDataPart(
continue;
}
/// Exclude files written by inverted index from check. No correct checksums are available for them currently.
/// Exclude files written by full-text index from check. No correct checksums are available for them currently.
if (isGinFile(file_name))
continue;

View File

@ -11,6 +11,6 @@ CREATE TABLE tab
ENGINE = MergeTree
ORDER BY key;
ALTER TABLE tab ADD INDEX inv_idx(str) TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED }
ALTER TABLE tab ADD INDEX inv_idx(str) TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED }
DROP TABLE tab;