Skip to content

Commit ad11e3e

Browse files
committed
Cosmetics
1 parent 78c5eb8 commit ad11e3e

11 files changed

+155
-148
lines changed

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

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -60,10 +60,6 @@ ENGINE = MergeTree
6060
ORDER BY key
6161
```
6262

63-
:::note
64-
In earlier versions of ClickHouse, the corresponding index type name was `inverted`.
65-
:::
66-
6763
where `N` specifies the tokenizer:
6864

6965
- `gin(0)` (or shorter: `gin()`) set the tokenizer to "tokens", i.e. split strings along spaces,

src/Interpreters/GinFilter.cpp

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -11,8 +11,6 @@
1111
#include <Storages/MergeTree/GinIndexStore.h>
1212
#include <Storages/MergeTree/MergeTreeIndexBloomFilterText.h>
1313
#include <Storages/MergeTree/MergeTreeIndexGin.h>
14-
#include <string>
15-
#include <algorithm>
1614
#include <city.h>
1715

1816
namespace DB

src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,6 @@
11
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
2+
3+
#include <Storages/MergeTree/MergeTreeData.h>
24
#include <Storages/MergeTree/MergeTreeIndexGin.h>
35
#include <Storages/MergeTree/MergeTreeSettings.h>
46
#include <Common/ElapsedTimeProfileEventIncrement.h>

src/Storages/MergeTree/MergeTreeIndexGin.cpp

Lines changed: 93 additions & 72 deletions
Large diffs are not rendered by default.

src/Storages/MergeTree/MergeTreeIndexGin.h

Lines changed: 20 additions & 38 deletions
Original file line numberDiff line numberDiff line change
@@ -3,30 +3,28 @@
33
#include <Interpreters/GinFilter.h>
44
#include <Interpreters/ITokenExtractor.h>
55
#include <Storages/MergeTree/KeyCondition.h>
6-
#include <Storages/MergeTree/MergeTreeData.h>
7-
#include <base/types.h>
8-
#include <memory>
6+
#include <Storages/MergeTree/MergeTreeIndices.h>
97

108
namespace DB
119
{
10+
1211
struct MergeTreeIndexGranuleGin final : public IMergeTreeIndexGranule
1312
{
14-
explicit MergeTreeIndexGranuleGin(
13+
MergeTreeIndexGranuleGin(
1514
const String & index_name_,
1615
size_t columns_number,
17-
const GinFilterParameters & params_);
16+
const GinFilterParameters & gin_filter_params_);
1817

1918
~MergeTreeIndexGranuleGin() override = default;
2019

2120
void serializeBinary(WriteBuffer & ostr) const override;
2221
void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override;
2322

2423
bool empty() const override { return !has_elems; }
25-
2624
size_t memoryUsageBytes() const override;
2725

2826
const String index_name;
29-
const GinFilterParameters params;
27+
const GinFilterParameters gin_filter_params;
3028
GinFilters gin_filters;
3129
bool has_elems;
3230
};
@@ -35,28 +33,25 @@ using MergeTreeIndexGranuleGinPtr = std::shared_ptr<MergeTreeIndexGranuleGin>;
3533

3634
struct MergeTreeIndexAggregatorGin final : IMergeTreeIndexAggregator
3735
{
38-
explicit MergeTreeIndexAggregatorGin(
36+
MergeTreeIndexAggregatorGin(
3937
GinIndexStorePtr store_,
4038
const Names & index_columns_,
4139
const String & index_name_,
42-
const GinFilterParameters & params_,
40+
const GinFilterParameters & gin_filter_params_,
4341
TokenExtractorPtr token_extractor_);
4442

4543
~MergeTreeIndexAggregatorGin() override = default;
4644

4745
bool empty() const override { return !granule || granule->empty(); }
4846
MergeTreeIndexGranulePtr getGranuleAndReset() override;
49-
5047
void update(const Block & block, size_t * pos, size_t limit) override;
51-
5248
void addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter);
5349

5450
GinIndexStorePtr store;
5551
Names index_columns;
5652
const String index_name;
57-
const GinFilterParameters params;
53+
const GinFilterParameters gin_filter_params;
5854
TokenExtractorPtr token_extractor;
59-
6055
MergeTreeIndexGranuleGinPtr granule;
6156
};
6257

@@ -65,22 +60,17 @@ class MergeTreeIndexConditionGin final : public IMergeTreeIndexCondition, WithCo
6560
{
6661
public:
6762
MergeTreeIndexConditionGin(
68-
const ActionsDAG::Node * predicate,
69-
ContextPtr context,
70-
const Block & index_sample_block,
71-
const GinFilterParameters & params_,
72-
TokenExtractorPtr token_extactor_);
63+
const ActionsDAG::Node * predicate,
64+
ContextPtr context,
65+
const Block & index_sample_block,
66+
const GinFilterParameters & gin_filter_params_,
67+
TokenExtractorPtr token_extactor_);
7368

7469
~MergeTreeIndexConditionGin() override = default;
7570

7671
bool alwaysUnknownOrTrue() const override;
77-
bool mayBeTrueOnGranule([[maybe_unused]]MergeTreeIndexGranulePtr idx_granule) const override
78-
{
79-
/// should call mayBeTrueOnGranuleInPart instead
80-
assert(false);
81-
return false;
82-
}
83-
bool mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule, [[maybe_unused]] PostingsCacheForStore & cache_store) const;
72+
bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override;
73+
bool mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule, PostingsCacheForStore & cache_store) const;
8474

8575
private:
8676
struct KeyTuplePositionMapping
@@ -90,6 +80,7 @@ class MergeTreeIndexConditionGin final : public IMergeTreeIndexCondition, WithCo
9080
size_t tuple_index;
9181
size_t key_index;
9282
};
83+
9384
/// Uses RPN like KeyCondition
9485
struct RPNElement
9586
{
@@ -135,7 +126,6 @@ class MergeTreeIndexConditionGin final : public IMergeTreeIndexCondition, WithCo
135126
using RPN = std::vector<RPNElement>;
136127

137128
bool traverseAtomAST(const RPNBuilderTreeNode & node, RPNElement & out);
138-
139129
bool traverseASTEquals(
140130
const String & function_name,
141131
const RPNBuilderTreeNode & key_ast,
@@ -145,14 +135,10 @@ class MergeTreeIndexConditionGin final : public IMergeTreeIndexCondition, WithCo
145135

146136
bool tryPrepareSetGinFilter(const RPNBuilderTreeNode & lhs, const RPNBuilderTreeNode & rhs, RPNElement & out);
147137

148-
static bool createFunctionEqualsCondition(
149-
RPNElement & out, const Field & value, const GinFilterParameters & params, TokenExtractorPtr token_extractor);
150-
151138
const Block & header;
152-
GinFilterParameters params;
139+
GinFilterParameters gin_filter_params;
153140
TokenExtractorPtr token_extractor;
154141
RPN rpn;
155-
/// Sets from syntax analyzer.
156142
PreparedSetsPtr prepared_sets;
157143
};
158144

@@ -161,11 +147,8 @@ class MergeTreeIndexGin final : public IMergeTreeIndex
161147
public:
162148
MergeTreeIndexGin(
163149
const IndexDescription & index_,
164-
const GinFilterParameters & params_,
165-
std::unique_ptr<ITokenExtractor> && token_extractor_)
166-
: IMergeTreeIndex(index_)
167-
, params(params_)
168-
, token_extractor(std::move(token_extractor_)) {}
150+
const GinFilterParameters & gin_filter_params_,
151+
std::unique_ptr<ITokenExtractor> && token_extractor_);
169152

170153
~MergeTreeIndexGin() override = default;
171154

@@ -174,8 +157,7 @@ class MergeTreeIndexGin final : public IMergeTreeIndex
174157
MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store, const MergeTreeWriterSettings & /*settings*/) const override;
175158
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG::Node * predicate, ContextPtr context) const override;
176159

177-
GinFilterParameters params;
178-
/// Function for selecting next token.
160+
GinFilterParameters gin_filter_params;
179161
std::unique_ptr<ITokenExtractor> token_extractor;
180162
};
181163

src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -592,17 +592,17 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta
592592

593593
/// Check that the index is created on a single column
594594
if (index.column_names.size() != 1 || index.data_types.size() != 1)
595-
throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity indexes must be created on a single column");
595+
throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity index must be created on a single column");
596596

597597
/// Check that the data type is Array(Float32|Float64|BFloat16)
598598
DataTypePtr data_type = index.sample_block.getDataTypes()[0];
599599
const auto * data_type_array = typeid_cast<const DataTypeArray *>(data_type.get());
600600
if (!data_type_array)
601-
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32|Float64|BFloat16)");
601+
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32|Float64|BFloat16)");
602602
TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId();
603603
WhichDataType which(nested_type_index);
604604
if (!which.isNativeFloat() && !which.isBFloat16())
605-
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32|Float64|BFloat16)");
605+
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32|Float64|BFloat16)");
606606
}
607607

608608
}
Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,6 @@
1+
Two or less arguments
2+
1st argument (tokenizer) must be UInt64
3+
2st argument (max_rows_per_postings_list) must be UInt64
4+
2st argument (max_rows_per_postings_list) must be bigger than MIN_ROWS_PER_POSTINGS_LIST
5+
Must be created on single column
6+
Must be created on String or FixedString or Array(String) or Array(FixedString) or LowCardinality(String) or LowCardinality(FixedString) columns
Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,27 @@
1+
-- Tests that various conditions are checked during creation of GIN indexes.
2+
3+
-- Using SETTINGS min_bytes_for_full_part_storage = 0 because GIN indexes currently don't work with packed parts
4+
5+
SET allow_experimental_full_text_index = 1;
6+
7+
DROP TABLE IF EXISTS tab;
8+
9+
SELECT 'Two or less arguments';
10+
CREATE TABLE tab (key UInt64, str String, INDEX inv_idx str TYPE gin('cant_have', 'three', 'args')) ENGINE = MergeTree ORDER BY key SETTINGS min_bytes_for_full_part_storage = 0; -- { serverError INCORRECT_QUERY }
11+
12+
SELECT '1st argument (tokenizer) must be UInt64';
13+
CREATE TABLE tab (key UInt64, str String, INDEX inv_idx str TYPE gin('string_arg')) ENGINE = MergeTree ORDER BY key SETTINGS min_bytes_for_full_part_storage = 0; -- { serverError INCORRECT_QUERY }
14+
15+
SELECT '2st argument (max_rows_per_postings_list) must be UInt64';
16+
CREATE TABLE tab (key UInt64, str String, INDEX inv_idx str TYPE gin(1, 'string_arg')) ENGINE = MergeTree ORDER BY key SETTINGS min_bytes_for_full_part_storage = 0; -- { serverError INCORRECT_QUERY }
17+
18+
SELECT '2st argument (max_rows_per_postings_list) must be bigger than MIN_ROWS_PER_POSTINGS_LIST';
19+
CREATE TABLE tab (key UInt64, str String, INDEX inv_idx str TYPE gin(1, 1)) ENGINE = MergeTree ORDER BY key SETTINGS min_bytes_for_full_part_storage = 0; -- { serverError INCORRECT_QUERY }
20+
21+
SELECT 'Must be created on single column';
22+
CREATE TABLE tab (key UInt64, str1 String, str2 String, INDEX inv_idx (str1, str2) TYPE gin(1, 9999)) ENGINE = MergeTree ORDER BY key SETTINGS min_bytes_for_full_part_storage = 0; -- { serverError INCORRECT_NUMBER_OF_COLUMNS }
23+
24+
SELECT 'Must be created on String or FixedString or Array(String) or Array(FixedString) or LowCardinality(String) or LowCardinality(FixedString) columns';
25+
CREATE TABLE tab (key UInt64, str UInt64, INDEX inv_idx str TYPE gin(1, 9999)) ENGINE = MergeTree ORDER BY key SETTINGS min_bytes_for_full_part_storage = 0; -- { serverError INCORRECT_QUERY }
26+
CREATE TABLE tab (key UInt64, str Float32, INDEX inv_idx str TYPE gin(1, 999)) ENGINE = MergeTree ORDER BY key SETTINGS min_bytes_for_full_part_storage = 0; -- { serverError INCORRECT_QUERY }
27+
CREATE TABLE tab (key UInt64, str Nullable(String), INDEX inv_idx str TYPE gin(1, 999)) ENGINE = MergeTree ORDER BY key SETTINGS min_bytes_for_full_part_storage = 0; -- { serverError INCORRECT_QUERY }

tests/queries/0_stateless/02346_gin_index_detach_attach.sql

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,7 @@
22

33
SET allow_experimental_full_text_index = 1;
44

5-
CREATE TABLE t
5+
CREATE TABLE tab
66
(
77
key UInt64,
88
str String,
@@ -12,7 +12,7 @@ ENGINE = MergeTree
1212
ORDER BY key
1313
SETTINGS min_bytes_for_full_part_storage = 0; -- GIN indexes currently don't work with packed parts
1414

15-
INSERT INTO t VALUES (1, 'Hello World');
15+
INSERT INTO tab VALUES (1, 'Hello World');
1616

17-
ALTER TABLE t DETACH PART 'all_1_1_0';
18-
ALTER TABLE t ATTACH PART 'all_1_1_0';
17+
ALTER TABLE tab DETACH PART 'all_1_1_0';
18+
ALTER TABLE tab ATTACH PART 'all_1_1_0';

tests/queries/0_stateless/02346_gin_index_on_multiple_columns.reference

Lines changed: 0 additions & 2 deletions
This file was deleted.

0 commit comments

Comments
 (0)