mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Rename "inverted" to "fulltext"
This commit is contained in:
parent
08bcbfc251
commit
0f87653fef
@ -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]
|
||||
|
@ -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(!=, <>)](/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 can’t be used by `ngrambf_v1` for query optimization.
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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) \
|
||||
|
@ -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_)
|
||||
|
@ -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
|
||||
|
@ -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");
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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())
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user