Skip to content

Commit e08e135

Browse files
authored
Merge pull request ClickHouse#79024 from rschu1ze/gin-renaming
Rename `full_text` index to `gin` index
2 parents bd60e45 + 6ae10b7 commit e08e135

39 files changed

+370
-291
lines changed

docs/en/engines/table-engines/mergetree-family/invertedindexes.md

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,7 @@ CREATE TABLE tab
5454
(
5555
`key` UInt64,
5656
`str` String,
57-
INDEX inv_idx(str) TYPE full_text(0) GRANULARITY 1
57+
INDEX inv_idx(str) TYPE gin(0) GRANULARITY 1
5858
)
5959
ENGINE = MergeTree
6060
ORDER BY key
@@ -66,20 +66,20 @@ In earlier versions of ClickHouse, the corresponding index type name was `invert
6666

6767
where `N` specifies the tokenizer:
6868

69-
- `full_text(0)` (or shorter: `full_text()`) set the tokenizer to "tokens", i.e. split strings along spaces,
70-
- `full_text(N)` with `N` between 2 and 8 sets the tokenizer to "ngrams(N)"
69+
- `gin(0)` (or shorter: `gin()`) set the tokenizer to "tokens", i.e. split strings along spaces,
70+
- `gin(N)` with `N` between 2 and 8 sets the tokenizer to "ngrams(N)"
7171

7272
The maximum rows per postings list can be specified as the second parameter. This parameter can be used to control postings list sizes to avoid generating huge postings list files. The following variants exist:
7373

74-
- `full_text(ngrams, max_rows_per_postings_list)`: Use given max_rows_per_postings_list (assuming it is not 0)
75-
- `full_text(ngrams, 0)`: No limitation of maximum rows per postings list
76-
- `full_text(ngrams)`: Use a default maximum rows which is 64K.
74+
- `gin(ngrams, max_rows_per_postings_list)`: Use given max_rows_per_postings_list (assuming it is not 0)
75+
- `gin(ngrams, 0)`: No limitation of maximum rows per postings list
76+
- `gin(ngrams)`: Use a default maximum rows which is 64K.
7777

7878
Being a type of skipping index, full-text indexes can be dropped or added to a column after table creation:
7979

8080
```sql
8181
ALTER TABLE tab DROP INDEX inv_idx;
82-
ALTER TABLE tab ADD INDEX inv_idx(s) TYPE full_text(2);
82+
ALTER TABLE tab ADD INDEX inv_idx(s) TYPE gin(2);
8383
```
8484

8585
To use the index, no special functions or syntax are required. Typical string search predicates automatically leverage the index. As
@@ -177,7 +177,7 @@ We will use `ALTER TABLE` and add an full-text index on the lowercase of the `co
177177

178178
```sql
179179
ALTER TABLE hackernews
180-
ADD INDEX comment_lowercase(lower(comment)) TYPE full_text;
180+
ADD INDEX comment_lowercase(lower(comment)) TYPE gin;
181181

182182
ALTER TABLE hackernews MATERIALIZE INDEX comment_lowercase;
183183
```

src/Interpreters/GinFilter.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,7 @@
1010
#include <Interpreters/GinFilter.h>
1111
#include <Storages/MergeTree/GinIndexStore.h>
1212
#include <Storages/MergeTree/MergeTreeIndexBloomFilterText.h>
13-
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
13+
#include <Storages/MergeTree/MergeTreeIndexGin.h>
1414
#include <string>
1515
#include <algorithm>
1616
#include <city.h>

src/Interpreters/GinFilter.h

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@ namespace DB
88

99
static inline constexpr auto FULL_TEXT_INDEX_NAME = "full_text";
1010
static inline constexpr auto INVERTED_INDEX_NAME = "inverted";
11+
static inline constexpr auto GIN_INDEX_NAME = "gin";
1112
static inline constexpr UInt64 UNLIMITED_ROWS_PER_POSTINGS_LIST = 0;
1213
static inline constexpr UInt64 MIN_ROWS_PER_POSTINGS_LIST = 8 * 1024;
1314
static inline constexpr UInt64 DEFAULT_MAX_ROWS_PER_POSTINGS_LIST = 64 * 1024;

src/Interpreters/InterpreterCreateQuery.cpp

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -793,13 +793,15 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
793793
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {} is not allowed. Please use a different index name", backQuoteIfNeed(index_desc.name));
794794

795795
const auto & settings = getContext()->getSettingsRef();
796-
if (index_desc.type == FULL_TEXT_INDEX_NAME && !settings[Setting::allow_experimental_full_text_index])
796+
if (index_desc.type == GIN_INDEX_NAME && !settings[Setting::allow_experimental_full_text_index])
797797
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "The experimental full-text index feature is disabled. Enable the setting 'allow_experimental_full_text_index' to use it");
798-
/// ----
799-
/// Temporary check during a transition period. Please remove at the end of 2024.
798+
/// ---
799+
/// Temporary checks during a transition period. Remove this block one year after GIN indexes became GA.
800+
if (index_desc.type == FULL_TEXT_INDEX_NAME && !settings[Setting::allow_experimental_full_text_index])
801+
throw Exception(ErrorCodes::ILLEGAL_INDEX, "The 'full_text' index type is deprecated. Please use the 'gin' index type instead");
800802
if (index_desc.type == INVERTED_INDEX_NAME && !settings[Setting::allow_experimental_inverted_index])
801-
throw Exception(ErrorCodes::ILLEGAL_INDEX, "The 'inverted' index type is deprecated. Please use the 'full_text' index type instead");
802-
/// ----
803+
throw Exception(ErrorCodes::ILLEGAL_INDEX, "The 'inverted' index type is deprecated. Please use the 'gin' index type instead");
804+
/// ---
803805
if (index_desc.type == "vector_similarity" && !settings[Setting::allow_experimental_vector_similarity_index])
804806
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "The experimental vector similarity index feature is disabled. Enable the setting 'allow_experimental_vector_similarity_index' to use it");
805807

src/Storages/AlterCommands.cpp

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1228,7 +1228,17 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
12281228
return result;
12291229
}
12301230

1231-
bool AlterCommands::hasFullTextIndex(const StorageInMemoryMetadata & metadata)
1231+
bool AlterCommands::hasGinIndex(const StorageInMemoryMetadata & metadata)
1232+
{
1233+
for (const auto & index : metadata.secondary_indices)
1234+
{
1235+
if (index.type == GIN_INDEX_NAME)
1236+
return true;
1237+
}
1238+
return false;
1239+
}
1240+
1241+
bool AlterCommands::hasLegacyFullTextIndex(const StorageInMemoryMetadata & metadata)
12321242
{
12331243
for (const auto & index : metadata.secondary_indices)
12341244
{

src/Storages/AlterCommands.h

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -236,8 +236,9 @@ class AlterCommands : public std::vector<AlterCommand>
236236
/// additional mutation command (MATERIALIZE_TTL) will be returned.
237237
MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const;
238238

239-
/// Check if commands have any full-text index or a (legacy) inverted index
240-
static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata);
239+
/// Check if commands have any GIN index or a (legacy) full_text or inverted index
240+
static bool hasGinIndex(const StorageInMemoryMetadata & metadata);
241+
static bool hasLegacyFullTextIndex(const StorageInMemoryMetadata & metadata);
241242
static bool hasLegacyInvertedIndex(const StorageInMemoryMetadata & metadata);
242243

243244
/// Check if commands have any vector similarity index

src/Storages/MergeTree/MergeTreeData.cpp

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -286,6 +286,7 @@ namespace ErrorCodes
286286
extern const int NOT_ENOUGH_SPACE;
287287
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
288288
extern const int SUPPORT_IS_DISABLED;
289+
extern const int ILLEGAL_INDEX;
289290
extern const int TOO_MANY_SIMULTANEOUS_QUERIES;
290291
extern const int INCORRECT_QUERY;
291292
extern const int INVALID_SETTING_VALUE;
@@ -3704,17 +3705,22 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
37043705

37053706
commands.apply(new_metadata, local_context);
37063707

3707-
if (AlterCommands::hasFullTextIndex(new_metadata) && !settings[Setting::allow_experimental_full_text_index])
3708+
if (AlterCommands::hasGinIndex(new_metadata) && !settings[Setting::allow_experimental_full_text_index])
37083709
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
37093710
"Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_full_text_index')");
37103711

3712+
if (AlterCommands::hasLegacyFullTextIndex(new_metadata) && !settings[Setting::allow_experimental_full_text_index])
3713+
throw Exception(ErrorCodes::ILLEGAL_INDEX, "The 'full_text' index type is deprecated. Please use the 'gin' index type instead");
3714+
3715+
/// ---
3716+
/// Temporary checks during a transition period. Remove this block one year after GIN indexes became GA.
37113717
if (AlterCommands::hasLegacyInvertedIndex(new_metadata) && !settings[Setting::allow_experimental_inverted_index])
3712-
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
3713-
"Experimental inverted index feature is not enabled (turn on setting 'allow_experimental_inverted_index')");
3718+
throw Exception(ErrorCodes::ILLEGAL_INDEX, "The 'inverted' index type is deprecated. Please use the 'gin' index type instead");
37143719

37153720
if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && !settings[Setting::allow_experimental_vector_similarity_index])
37163721
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
37173722
"Experimental vector similarity index is disabled (turn on setting 'allow_experimental_vector_similarity_index')");
3723+
/// ---
37183724

37193725
/// If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs.
37203726
/// SET allow_experimental_vector_similarity_index = 1;

src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,5 @@
11
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
2-
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
2+
#include <Storages/MergeTree/MergeTreeIndexGin.h>
33
#include <Storages/MergeTree/MergeTreeSettings.h>
44
#include <Common/ElapsedTimeProfileEventIncrement.h>
55
#include <Common/MemoryTrackerBlockerInThread.h>
@@ -299,7 +299,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices()
299299
settings.query_write_settings));
300300

301301
GinIndexStorePtr store = nullptr;
302-
if (typeid_cast<const MergeTreeIndexFullText *>(&*skip_index) != nullptr)
302+
if (typeid_cast<const MergeTreeIndexGin *>(&*skip_index) != nullptr)
303303
{
304304
store = std::make_shared<GinIndexStore>(stream_name, data_part_storage, data_part_storage, (*storage_settings)[MergeTreeSetting::max_digestion_size_per_segment]);
305305
gin_index_stores[stream_name] = store;
@@ -378,7 +378,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
378378
WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing;
379379

380380
GinIndexStorePtr store;
381-
if (typeid_cast<const MergeTreeIndexFullText *>(&*index_helper) != nullptr)
381+
if (typeid_cast<const MergeTreeIndexGin *>(&*index_helper) != nullptr)
382382
{
383383
String stream_name = index_helper->getFileName();
384384
auto it = gin_index_stores.find(stream_name);
@@ -491,7 +491,7 @@ void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::Data
491491
/// Register additional files written only by the full-text index. Required because otherwise DROP TABLE complains about unknown
492492
/// files. Note that the provided actual checksums are bogus. The problem is that at this point the file writes happened already and
493493
/// we'd need to re-open + hash the files (fixing this is TODO). For now, CHECK TABLE skips these four files.
494-
if (typeid_cast<const MergeTreeIndexFullText *>(&*skip_indices[i]) != nullptr)
494+
if (typeid_cast<const MergeTreeIndexGin *>(&*skip_indices[i]) != nullptr)
495495
{
496496
String filename_without_extension = skip_indices[i]->getFileName();
497497
checksums.files[filename_without_extension + ".gin_dict"] = MergeTreeDataPartChecksums::Checksum();

src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,7 @@
1010
#include <Storages/MergeTree/KeyCondition.h>
1111
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
1212
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
13-
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
13+
#include <Storages/MergeTree/MergeTreeIndexGin.h>
1414
#include <Storages/ReadInOrderOptimizer.h>
1515
#include <Storages/VirtualColumnUtils.h>
1616
#include <Parsers/ASTIdentifier.h>
@@ -1590,7 +1590,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
15901590
size_t last_index_mark = 0;
15911591

15921592
PostingsCacheForStore cache_in_store;
1593-
if (dynamic_cast<const MergeTreeIndexFullText *>(index_helper.get()))
1593+
if (dynamic_cast<const MergeTreeIndexGin *>(index_helper.get()))
15941594
cache_in_store.store = GinIndexStoreFactory::instance().get(index_helper->getFileName(), part->getDataPartStoragePtr());
15951595

15961596
for (size_t i = 0; i < ranges_size; ++i)
@@ -1627,7 +1627,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
16271627
else
16281628
{
16291629
bool result = false;
1630-
const auto * gin_filter_condition = dynamic_cast<const MergeTreeConditionFullText *>(&*condition);
1630+
const auto * gin_filter_condition = dynamic_cast<const MergeTreeIndexConditionGin *>(&*condition);
16311631
if (!gin_filter_condition)
16321632
result = condition->mayBeTrueOnGranule(granule);
16331633
else

0 commit comments

Comments
 (0)