Merge pull request #70616 from rschu1ze/query-time-ef-search

Vector search: allow to specify HNSW parameter `ef_search` at query time
This commit is contained in:
Robert Schulze 2024-10-21 06:47:46 +00:00 committed by GitHub
commit d02a31da44
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
24 changed files with 281 additions and 165 deletions

2
contrib/usearch vendored

@ -1 +1 @@
Subproject commit d1d33eac94acd3b628e0b446c927ec3295ef63c7
Subproject commit 1706420acafbd83d852c512dcf343af0a4059e48

View File

@ -43,7 +43,7 @@ CREATE TABLE table
(
id Int64,
vectors Array(Float32),
INDEX index_name vectors TYPE vector_similarity(method, distance_function[, quantization, connectivity, expansion_add, expansion_search]) [GRANULARITY N]
INDEX index_name vectors TYPE vector_similarity(method, distance_function[, quantization, hnsw_max_connections_per_layer, hnsw_candidate_list_size_for_construction]) [GRANULARITY N]
)
ENGINE = MergeTree
ORDER BY id;
@ -55,11 +55,13 @@ Parameters:
line between two points in Euclidean space), or `cosineDistance` (the [cosine
distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors).
- `quantization`: either `f64`, `f32`, `f16`, `bf16`, or `i8` for storing the vector with reduced precision (optional, default: `bf16`)
- `m`: the number of neighbors per graph node (optional, default: 16)
- `ef_construction`: (optional, default: 128)
- `ef_search`: (optional, default: 64)
- `hnsw_max_connections_per_layer`: the number of neighbors per HNSW graph node, also known as `M` in the [HNSW
paper](https://doi.org/10.1109/TPAMI.2018.2889473) (optional, default: 16)
- `hnsw_candidate_list_size_for_construction`: the size of the dynamic candidate list when constructing the HNSW graph, also known as
`ef_construction` in the original [HNSW paper](https://doi.org/10.1109/TPAMI.2018.2889473) (optional, default: 128)
Value 0 for parameters `m`, `ef_construction`, and `ef_search` refers to the default value.
Values 0 for parameters `hnsw_max_connections_per_layer` and `hnsw_candidate_list_size_for_construction` means using the default values of
these parameters.
Example:
@ -115,6 +117,11 @@ ANN indexes are built during column insertion and merge. As a result, `INSERT` a
tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively when are far more read requests than write
requests.
:::tip
To reduce the cost of building vector similarity indexes, consider setting `materialize_skip_indexes_on_insert` which disables the
construction of skipping indexes on newly inserted parts. Search would fall back to exact search but as inserted parts are typically small
compared to the total table size, the performance impact of that would be negligible.
ANN indexes support this type of query:
``` sql
@ -124,6 +131,7 @@ FROM table
WHERE ... -- WHERE clause is optional
ORDER BY Distance(vectors, reference_vector)
LIMIT N
SETTINGS enable_analyzer = 0; -- Temporary limitation, will be lifted
```
:::tip
@ -135,6 +143,10 @@ clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Dista
```
:::
To search using a different value of HNSW parameter `hnsw_candidate_list_size_for_search` (default: 64), also known as `ef_search` in the
original [HNSW paper](https://doi.org/10.1109/TPAMI.2018.2889473), run the `SELECT` query with `SETTINGS hnsw_candidate_list_size_for_search
= <value>`.
**Restrictions**: Approximate algorithms used to determine the nearest neighbors require a limit, hence queries without `LIMIT` clause
cannot utilize ANN indexes. Also, ANN indexes are only used if the query has a `LIMIT` value smaller than setting
`max_limit_for_ann_queries` (default: 1 million rows). This is a safeguard to prevent large memory allocations by external libraries for

View File

@ -49,7 +49,7 @@ Default value: 8192.
Maximum size of data granules in bytes.
Default value: 10Mb.
Default value: 10485760 (ca. 10 MiB).
To restrict the granule size only by number of rows, set to 0 (not recommended).

View File

@ -24,7 +24,7 @@ Returns a random UInt32 number with uniform distribution.
Uses a linear congruential generator with an initial state obtained from the system, which means that while it appears random, it's not truly random and can be predictable if the initial state is known. For scenarios where true randomness is crucial, consider using alternative methods like system-level calls or integrating with external libraries.
### Syntax
**Syntax**
```sql
rand()
@ -32,15 +32,15 @@ rand()
Alias: `rand32`
### Arguments
**Arguments**
None.
### Returned value
**Returned value**
Returns a number of type UInt32.
### Example
**Example**
```sql
SELECT rand();
@ -54,23 +54,23 @@ SELECT rand();
Returns a random UInt64 integer (UInt64) number
### Syntax
**Syntax**
```sql
rand64()
```
### Arguments
**Arguments**
None.
### Returned value
**Arguments**
Returns a number UInt64 number with uniform distribution.
Uses a linear congruential generator with an initial state obtained from the system, which means that while it appears random, it's not truly random and can be predictable if the initial state is known. For scenarios where true randomness is crucial, consider using alternative methods like system-level calls or integrating with external libraries.
### Example
**Example**
```sql
SELECT rand64();
@ -84,21 +84,21 @@ SELECT rand64();
Returns a random Float64 number.
### Syntax
**Syntax**
```sql
randCanonical()
```
### Arguments
**Arguments**
None.
### Returned value
**Arguments**
Returns a Float64 value between 0 (inclusive) and 1 (exclusive).
### Example
**Example**
```sql
SELECT randCanonical();
@ -112,25 +112,25 @@ SELECT randCanonical();
Generates a single constant column filled with a random value. Unlike `rand`, this function ensures the same random value appears in every row of the generated column, making it useful for scenarios requiring a consistent random seed across rows in a single query.
### Syntax
**Syntax**
```sql
randConstant([x]);
```
### Arguments
**Arguments**
- **[x] (Optional):** An optional expression that influences the generated random value. Even if provided, the resulting value will still be constant within the same query execution. Different queries using the same expression will likely generate different constant values.
### Returned value
**Arguments**
Returns a column of type UInt32 containing the same random value in each row.
### Implementation details
**Implementation details**
The actual output will be different for each query execution, even with the same optional expression. The optional parameter may not significantly change the generated value compared to using `randConstant` alone.
### Examples
**Example**
```sql
SELECT randConstant() AS random_value;
@ -156,22 +156,22 @@ SELECT randConstant(10) AS random_value;
Returns a random Float64 drawn uniformly from interval [`min`, `max`].
### Syntax
**Syntax**
```sql
randUniform(min, max)
```
### Arguments
**Arguments**
- `min` - `Float64` - left boundary of the range,
- `max` - `Float64` - right boundary of the range.
### Returned value
**Arguments**
A random number of type [Float64](../data-types/float.md).
### Example
**Example**
```sql
SELECT randUniform(5.5, 10) FROM numbers(5)

View File

@ -18,9 +18,9 @@ generateRandom(['name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_str
- `name` — Name of corresponding column.
- `TypeName` — Type of corresponding column.
- `max_array_length` — Maximum elements for all generated arrays or maps. Defaults to `10`.
- `max_string_length` — Maximum string length for all generated strings. Defaults to `10`.
- `random_seed` — Specify random seed manually to produce stable results. If NULL — seed is randomly generated.
- `max_string_length` — Maximum string length for all generated strings. Defaults to `10`.
- `max_array_length` — Maximum elements for all generated arrays or maps. Defaults to `10`.
**Returned Value**

View File

@ -18,9 +18,9 @@ generateRandom('name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_stri
- `name` — название соответствующего столбца.
- `TypeName` — тип соответствующего столбца.
- `max_array_length` — максимальная длина массива для всех сгенерированных массивов. По умолчанию `10`.
- `max_string_length` — максимальная длина строки для всех генерируемых строк. По умолчанию `10`.
- `random_seed` — укажите состояние генератора случайных чисел вручную, чтобы получить стабильные результаты. Если значение равно `NULL` - генератор инициализируется случайным состоянием.
- `max_string_length` — максимальная длина строки для всех генерируемых строк. По умолчанию `10`.
- `max_array_length` — максимальная длина массива для всех сгенерированных массивов. По умолчанию `10`.
**Возвращаемое значение**

View File

@ -18,9 +18,9 @@ generateRandom('name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_stri
- `name` — 对应列的名称。
- `TypeName` — 对应列的类型。
- `max_array_length` — 生成数组的最大长度。 默认为10。
- `max_string_length` — 生成字符串的最大长度。 默认为10。
- `random_seed` — 手动指定随机种子以产生稳定的结果。 如果为NULL-种子是随机生成的。
- `max_string_length` — 生成字符串的最大长度。 默认为10。
- `max_array_length` — 生成数组的最大长度。 默认为10。
**返回值**

View File

@ -5556,7 +5556,10 @@ If it is set to true, allow to specify experimental compression codecs (but we d
Only in ClickHouse Cloud. Allow to create ShareSet and SharedJoin
)", 0) \
M(UInt64, max_limit_for_ann_queries, 1'000'000, R"(
SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.
SELECT queries with LIMIT bigger than this setting cannot use vector similarity indexes. Helps to prevent memory overflows in vector similarity indexes.
)", 0) \
M(UInt64, hnsw_candidate_list_size_for_search, 0, R"(
The size of the dynamic candidate list when searching the vector similarity index, also known as 'ef_search'. 0 means USearch's default value (64).
)", 0) \
M(Bool, throw_on_unsupported_query_inside_transaction, true, R"(
Throw exception if unsupported query is used inside transaction

View File

@ -103,6 +103,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"input_format_orc_dictionary_as_low_cardinality", false, true, "Treat ORC dictionary encoded columns as LowCardinality columns while reading ORC files"},
{"allow_experimental_refreshable_materialized_view", false, true, "Not experimental anymore"},
{"max_parts_to_move", 1000, 1000, "New setting"},
{"hnsw_candidate_list_size_for_search", 0, 0, "New setting"},
{"allow_reorder_prewhere_conditions", false, true, "New setting"},
{"input_format_parquet_bloom_filter_push_down", false, true, "When reading Parquet files, skip whole row groups based on the WHERE/PREWHERE expressions and bloom filter in the Parquet metadata."},
{"date_time_64_output_format_cut_trailing_zeros_align_to_groups_of_thousands", false, false, "Dynamically trim the trailing zeros of datetime64 values to adjust the output scale to (0, 3, 6), corresponding to 'seconds', 'milliseconds', and 'microseconds'."}

View File

@ -159,6 +159,14 @@ bool IndicesDescription::has(const String & name) const
return false;
}
bool IndicesDescription::hasType(const String & type) const
{
for (const auto & index : *this)
if (index.type == type)
return true;
return false;
}
String IndicesDescription::toString() const
{
if (empty())

View File

@ -65,6 +65,8 @@ struct IndicesDescription : public std::vector<IndexDescription>, IHints<>
{
/// Index with name exists
bool has(const String & name) const;
/// Index with type exists
bool hasType(const String & type) const;
/// Convert description to string
String toString() const;
/// Parse description from string

View File

@ -261,6 +261,7 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED;
extern const int TOO_MANY_SIMULTANEOUS_QUERIES;
extern const int INCORRECT_QUERY;
extern const int INVALID_SETTING_VALUE;
extern const int CANNOT_RESTORE_TABLE;
extern const int ZERO_COPY_REPLICATION_ERROR;
extern const int NOT_INITIALIZED;
@ -759,6 +760,16 @@ void MergeTreeData::checkProperties(
}
}
/// If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs.
/// SET allow_experimental_vector_similarity_index = 1;
/// CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0;
/// INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000);
/// WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100;
/// As a workaround, force enabled adaptive index granularity for now (it is the default anyways).
if (new_metadata.secondary_indices.hasType("vector_similarity") && (*getSettings())[MergeTreeSetting::index_granularity_bytes] == 0)
throw Exception(ErrorCodes::INVALID_SETTING_VALUE,
"Experimental vector similarity index can only be used with MergeTree setting 'index_granularity_bytes' != 0");
if (!new_metadata.projections.empty())
{
std::unordered_set<String> projections_names;
@ -3310,6 +3321,16 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Experimental vector similarity index is disabled (turn on setting 'allow_experimental_vector_similarity_index')");
/// If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs.
/// SET allow_experimental_vector_similarity_index = 1;
/// CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0;
/// INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000);
/// WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100;
/// As a workaround, force enabled adaptive index granularity for now (it is the default anyways).
if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && (*getSettings())[MergeTreeSetting::index_granularity_bytes] == 0)
throw Exception(ErrorCodes::INVALID_SETTING_VALUE,
"Experimental vector similarity index can only be used with MergeTree setting 'index_granularity_bytes' != 0");
for (const auto & disk : getDisks())
if (!disk->supportsHardLinks() && !commands.isSettingsAlter() && !commands.isCommentAlter())
throw Exception(

View File

@ -10,6 +10,7 @@
#include <Common/typeid_cast.h>
#include <Core/Field.h>
#include <Core/ServerSettings.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeArray.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
@ -45,6 +46,11 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
namespace Setting
{
extern const SettingsUInt64 hnsw_candidate_list_size_for_search;
}
namespace
{
@ -104,7 +110,7 @@ USearchIndexWithSerialization::USearchIndexWithSerialization(
{
USearchIndex::metric_t metric(dimensions, metric_kind, scalar_kind);
unum::usearch::index_dense_config_t config(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search);
unum::usearch::index_dense_config_t config(usearch_hnsw_params.connectivity, usearch_hnsw_params.expansion_add, unum::usearch::default_expansion_search());
config.enable_key_lookups = false; /// we don't do row-to-vector lookups
auto result = USearchIndex::make(metric, config);
@ -399,6 +405,7 @@ MergeTreeIndexConditionVectorSimilarity::MergeTreeIndexConditionVectorSimilarity
ContextPtr context)
: vector_similarity_condition(query, context)
, metric_kind(metric_kind_)
, expansion_search(context->getSettingsRef()[Setting::hnsw_candidate_list_size_for_search])
{
}
@ -430,13 +437,17 @@ std::vector<UInt64> MergeTreeIndexConditionVectorSimilarity::calculateApproximat
const USearchIndexWithSerializationPtr index = granule->index;
if (vector_similarity_condition.getDimensions() != index->dimensions())
throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) "
"does not match the dimension in the index ({})",
throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) does not match the dimension in the index ({})",
vector_similarity_condition.getDimensions(), index->dimensions());
const std::vector<Float64> reference_vector = vector_similarity_condition.getReferenceVector();
auto search_result = index->search(reference_vector.data(), limit);
/// We want to run the search with the user-provided value for setting hnsw_candidate_list_size_for_search (aka. expansion_search).
/// The way to do this in USearch is to call index_dense_gt::change_expansion_search. Unfortunately, this introduces a need to
/// synchronize index access, see https://github.com/unum-cloud/usearch/issues/500. As a workaround, we extended USearch' search method
/// to accept a custom expansion_add setting. The config value is only used on the fly, i.e. not persisted in the index.
auto search_result = index->search(reference_vector.data(), limit, USearchIndex::any_thread(), false, (expansion_search == 0) ? unum::usearch::default_expansion_search() : expansion_search);
if (!search_result)
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index. Error: {}", String(search_result.error.release()));
@ -501,13 +512,12 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index)
UsearchHnswParams usearch_hnsw_params;
/// Optional parameters:
const bool has_six_args = (index.arguments.size() == 6);
if (has_six_args)
const bool has_five_args = (index.arguments.size() == 5);
if (has_five_args)
{
scalar_kind = quantizationToScalarKind.at(index.arguments[2].safeGet<String>());
usearch_hnsw_params = {.m = index.arguments[3].safeGet<UInt64>(),
.ef_construction = index.arguments[4].safeGet<UInt64>(),
.ef_search = index.arguments[5].safeGet<UInt64>()};
usearch_hnsw_params = {.connectivity = index.arguments[3].safeGet<UInt64>(),
.expansion_add = index.arguments[4].safeGet<UInt64>()};
}
return std::make_shared<MergeTreeIndexVectorSimilarity>(index, metric_kind, scalar_kind, usearch_hnsw_params);
@ -516,25 +526,23 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index)
void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* attach */)
{
const bool has_two_args = (index.arguments.size() == 2);
const bool has_six_args = (index.arguments.size() == 6);
const bool has_five_args = (index.arguments.size() == 5);
/// Check number and type of arguments
if (!has_two_args && !has_six_args)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must have two or six arguments");
if (!has_two_args && !has_five_args)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must have two or five arguments");
if (index.arguments[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (method) must be of type String");
if (index.arguments[1].getType() != Field::Types::String)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (metric) must be of type String");
if (has_six_args)
if (has_five_args)
{
if (index.arguments[2].getType() != Field::Types::String)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Third argument of vector similarity index (quantization) must be of type String");
if (index.arguments[3].getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Fourth argument of vector similarity index (M) must be of type UInt64");
throw Exception(ErrorCodes::INCORRECT_QUERY, "Fourth argument of vector similarity index (hnsw_max_connections_per_layer) must be of type UInt64");
if (index.arguments[4].getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Fifth argument of vector similarity index (ef_construction) must be of type UInt64");
if (index.arguments[5].getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Sixth argument of vector similarity index (ef_search) must be of type UInt64");
throw Exception(ErrorCodes::INCORRECT_QUERY, "Fifth argument of vector similarity index (hnsw_candidate_list_size_for_construction) must be of type UInt64");
}
/// Check that passed arguments are supported
@ -542,18 +550,17 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta
throw Exception(ErrorCodes::INCORRECT_DATA, "First argument (method) of vector similarity index is not supported. Supported methods are: {}", joinByComma(methods));
if (!distanceFunctionToMetricKind.contains(index.arguments[1].safeGet<String>()))
throw Exception(ErrorCodes::INCORRECT_DATA, "Second argument (distance function) of vector similarity index is not supported. Supported distance function are: {}", joinByComma(distanceFunctionToMetricKind));
if (has_six_args)
if (has_five_args)
{
if (!quantizationToScalarKind.contains(index.arguments[2].safeGet<String>()))
throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind));
/// Call Usearch's own parameter validation method for HNSW-specific parameters
UInt64 m = index.arguments[3].safeGet<UInt64>();
UInt64 ef_construction = index.arguments[4].safeGet<UInt64>();
UInt64 ef_search = index.arguments[5].safeGet<UInt64>();
unum::usearch::index_dense_config_t config(m, ef_construction, ef_search);
UInt64 connectivity = index.arguments[3].safeGet<UInt64>();
UInt64 expansion_add = index.arguments[4].safeGet<UInt64>();
UInt64 expansion_search = unum::usearch::default_expansion_search();
unum::usearch::index_dense_config_t config(connectivity, expansion_add, expansion_search);
if (auto error = config.validate(); error)
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release()));
}

View File

@ -13,9 +13,8 @@ namespace DB
struct UsearchHnswParams
{
size_t m = unum::usearch::default_connectivity();
size_t ef_construction = unum::usearch::default_expansion_add();
size_t ef_search = unum::usearch::default_expansion_search();
size_t connectivity = unum::usearch::default_connectivity();
size_t expansion_add = unum::usearch::default_expansion_add();
};
using USearchIndex = unum::usearch::index_dense_t;
@ -142,6 +141,7 @@ public:
private:
const VectorSimilarityCondition vector_similarity_condition;
const unum::usearch::metric_kind_t metric_kind;
const size_t expansion_search;
};

View File

@ -40,3 +40,4 @@ Expression (Projection)
Condition: true
Parts: 1/1
Granules: 4/4
index_granularity_bytes = 0 is disallowed

View File

@ -37,7 +37,7 @@ DROP TABLE tab;
SELECT 'Correctness of index with > 1 mark';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192;
INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000);
WITH [1.0, 0.0] AS reference_vec
@ -56,7 +56,7 @@ DROP TABLE tab;
SELECT 'Issue #69085: Reference vector computed by a subquery';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
-- works
@ -100,3 +100,20 @@ FROM tab
ORDER BY distance
LIMIT 1
SETTINGS enable_analyzer = 0;
DROP TABLE tab;
SELECT 'index_granularity_bytes = 0 is disallowed';
-- If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs.
-- SET allow_experimental_vector_similarity_index = 1;
-- CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0;
-- INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000);
-- WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100;
-- As a workaround, force enabled adaptive index granularity for now (it is the default anyways).
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; -- { serverError INVALID_SETTING_VALUE }
CREATE TABLE tab(id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0;
ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError INVALID_SETTING_VALUE }
DROP TABLE tab;

View File

@ -0,0 +1,41 @@
-- Tags: no-fasttest, long, no-asan, no-asan, no-ubsan, no-debug
-- ^^ Disable test for slow builds: generating data takes time but a sufficiently large data set
-- is necessary for different hnsw_candidate_list_size_for_search settings to make a difference
-- Tests vector search with setting 'hnsw_candidate_list_size_for_search'
SET allow_experimental_vector_similarity_index = 1;
SET enable_analyzer = 0;
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192;
-- Generate random values but with a fixed seed (conceptually), so that the data is deterministic.
-- Unfortunately, no random functions in ClickHouse accepts a seed. Instead, abuse the numbers table + hash functions to provide
-- deterministic randomness.
INSERT INTO tab SELECT number, [sipHash64(number)/18446744073709551615, wyHash64(number)/18446744073709551615] FROM numbers(370000); -- 18446744073709551615 is the biggest UInt64
DROP TABLE IF EXISTS results;
CREATE TABLE results(id Int32) ENGINE = Memory;
-- Standard vector search with default hnsw_candidate_list_size_for_search = 64
INSERT INTO results
SELECT id
FROM tab
ORDER BY L2Distance(vec, [0.5, 0.5])
LIMIT 1;
-- Vector search with custom hnsw_candidate_list_size_for_search
INSERT INTO results
SELECT id
FROM tab
ORDER BY L2Distance(vec, [0.5, 0.5])
LIMIT 1
SETTINGS hnsw_candidate_list_size_for_search = 1;
-- Expect that matches are different
SELECT count(distinct *) FROM results;
DROP TABLE results;
DROP TABLE tab;

View File

@ -1,4 +1,4 @@
Two or six index arguments
Two or five index arguments
1st argument (method) must be String and hnsw
2nd argument (distance function) must be String and L2Distance or cosineDistance
3nd argument (quantization), if given, must be String and f32, f16, ...

View File

@ -6,12 +6,12 @@ SET allow_experimental_vector_similarity_index = 1;
DROP TABLE IF EXISTS tab;
SELECT 'Two or six index arguments';
SELECT 'Two or five index arguments';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant_have_one_arg')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant', 'have', 'three_args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant', 'have', 'more', 'than', 'six', 'args', '!')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant', 'have', 'more', 'than', 'five', 'args', '!')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
SELECT '1st argument (method) must be String and hnsw';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3, 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
@ -22,11 +22,11 @@ CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similar
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'invalid_distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT '3nd argument (quantization), if given, must be String and f32, f16, ...';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 1, 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'invalid', 2, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'invalid', 2, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT '4nd argument (M), if given, must be UInt64 and > 1';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 'invalid', 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT 'Must be created on single column';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS }

View File

@ -37,7 +37,7 @@ Expression (Projection)
Parts: 1/1
Granules: 2/4
Special cases
-- Non-default metric, M, ef_construction, ef_search
-- Non-default metric, hnsw_max_connections_per_layer, hnsw_candidate_list_size_for_construction
6 [1,9.3] 0.005731362878640178
4 [2.4,5.2] 0.09204062768384846
1 [2,3.2] 0.15200169244542905

View File

@ -53,8 +53,8 @@ DROP TABLE tab;
SELECT 'Special cases'; -- Not a systematic test, just to check that no bad things happen.
SELECT '-- Non-default metric, M, ef_construction, ef_search';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 42, 99, 66) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
SELECT '-- Non-default metric, hnsw_max_connections_per_layer, hnsw_candidate_list_size_for_construction';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 42, 99) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
WITH [0.0, 2.0] AS reference_vec
@ -82,11 +82,11 @@ SETTINGS max_limit_for_ann_queries = 2; -- LIMIT 3 > 2 --> don't use the ann ind
DROP TABLE tab;
SELECT '-- Non-default quantization';
CREATE TABLE tab_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);

View File

@ -31,6 +31,7 @@ AnyEvent
AppleClang
Approximative
ArrayJoin
ArrowCompression
ArrowStream
AsyncInsertCacheSize
AsynchronousHeavyMetricsCalculationTimeSpent
@ -123,6 +124,7 @@ CMPLNT
CMake
CMakeLists
CODECS
CORS
COVID
CPUFrequencyMHz
CPUs
@ -138,11 +140,13 @@ CacheDictionaryThreadsActive
CacheDictionaryUpdateQueueBatches
CacheDictionaryUpdateQueueKeys
CacheFileSegments
CacheWarmer
CamelCase
Cap'n
CapContains
CapUnion
CapnProto
CapnProtoEnumComparingMode
CatBoost
CellAreaM
CellAreaRads
@ -209,6 +213,7 @@ DDLWORKER
DDLWorker
DDLWorkerThreads
DDLWorkerThreadsActive
DDLs
DECRYPT
DELETEs
DESC
@ -217,6 +222,7 @@ DOGEFI
Damerau
DataGrip
DataLens
DataPacket
DataTime
DataTypes
DatabaseCatalog
@ -228,11 +234,15 @@ DatabaseOnDiskThreadsActive
DatabaseOrdinaryThreads
DatabaseOrdinaryThreadsActive
DateTime
DateTimeInputFormat
DateTimeOutputFormat
DateTimeOverflowBehavior
DateTimes
DbCL
Decrypted
Deduplicate
Deduplication
DefaultTableEngine
DelayedInserts
DeliveryTag
DeltaLake
@ -248,7 +258,11 @@ DiskSpaceReservedForMerge
DiskTotal
DiskUnreserved
DiskUsed
DistributedCacheLogMode
DistributedCachePoolBehaviourOnLimit
DistributedDDLOutputMode
DistributedFilesToInsert
DistributedProductMode
DistributedSend
DockerHub
DoubleDelta
@ -257,6 +271,7 @@ Dresseler
Durre
ECMA
ETag
EachRow
Ecto
EdgeAngle
EdgeLengthKm
@ -269,6 +284,7 @@ Enum
Enums
Eoan
EphemeralNode
EscapingRule
Ethereum
ExactEdgeLengthKm
ExactEdgeLengthM
@ -373,6 +389,7 @@ IMDS
INFILE
INSERTed
INSERTs
INVOKER
IOPrefetchThreads
IOPrefetchThreadsActive
IOThreads
@ -384,7 +401,10 @@ IOWriterThreadsActive
IPTrie
IProcessor
IPv
ITION
Identifiant
IdentifierQuotingRule
IdentifierQuotingStyle
Incrementing
IndexesAreNeighbors
InfluxDB
@ -403,6 +423,7 @@ IntervalMilliseconds
IntervalMinute
IntervalMonth
IntervalNanosecond
IntervalOutputFormat
IntervalQuarter
IntervalSecond
IntervalWeek
@ -464,6 +485,8 @@ Jepsen
JetBrains
Jitter
Joda
JoinAlgorithm
JoinStrictness
JumpConsistentHash
Jupyter
KDevelop
@ -512,10 +535,16 @@ LinfNorm
LinfNormalize
LinksDeployment
Linq
ListObject
ListObjects
LoadAverage
LoadBalancing
LocalFSReadMethod
LocalThread
LocalThreadActive
LogQL
LogQueriesType
LogsLevel
Logstash
LookML
LoongArch
@ -552,6 +581,7 @@ MaxDDLEntryID
MaxMind
MaxPartCountForPartition
MaxPushedDDLEntryID
MaxThreads
Mbps
McNeal
Memcheck
@ -559,6 +589,7 @@ MemoryCode
MemoryDataAndStack
MemoryResident
MemoryResidentMax
MemorySample
MemorySanitizer
MemoryShared
MemoryTracking
@ -594,6 +625,7 @@ Mongo
Mongodb
Monotonicity
MsgPack
MsgPackUUIDRepresentation
MultiLineString
MultiPolygon
Multiline
@ -602,6 +634,7 @@ Multithreading
Multiword
MurmurHash
MySQLConnection
MySQLDataTypesSupport
MySQLDump
MySQLThreads
NATS
@ -637,6 +670,7 @@ NetworkSendPackets
Noaa
NodeJs
NonMonotonic
NonZeroUInt
NuRaft
NumHexagons
NumPy
@ -649,6 +683,7 @@ NumberOfTables
OFNS
OLAP
OLTP
ORCCompression
OSContextSwitches
OSGuestNiceTime
OSGuestNiceTimeCPU
@ -715,6 +750,8 @@ OrDefault
OrNull
OrZero
OvercommitTracker
OverflowMode
OverflowModeGroupBy
PAAMAYIM
PCRE
PRCP
@ -728,8 +765,11 @@ ParallelFormattingOutputFormatThreadsActive
ParallelParsingInputFormat
ParallelParsingInputFormatThreads
ParallelParsingInputFormatThreadsActive
ParallelReplicasMode
Parametrized
ParquetCompression
ParquetMetadata
ParquetVersion
Parsers
PartMutation
Partitioner
@ -746,6 +786,7 @@ PartsWide
PeerDB
PendingAsyncInsert
Percona
PerfEventInfo
PhpStorm
PlantUML
Poess
@ -797,6 +838,8 @@ QueryCacheBytes
QueryCacheEntries
QueryCacheHits
QueryCacheMisses
QueryCacheNondeterministicFunctionHandling
QueryCacheSystemTableHandling
QueryPreempted
QueryThread
QuickAssist
@ -805,6 +848,7 @@ QuoteMeta
RBAC
RClickHouse
RHEL
RIPEMD
ROLLUP
RWLock
RWLockActiveReaders
@ -857,7 +901,7 @@ RestartReplicaThreads
RestartReplicaThreadsActive
RestoreThreads
RestoreThreadsActive
RIPEMD
RetryStrategy
RoaringBitmap
RocksDB
Rollup
@ -881,6 +925,7 @@ SQLAlchemy
SQLConsoleDetail
SQLInsert
SQLSTATE
SQLSecurityType
SSDCache
SSDComplexKeyCache
SSDs
@ -893,6 +938,7 @@ Sankey
Scalable
Scatterplot
Schaefer
SchemaInferenceMode
Schemas
Schwartzian
SeasClick
@ -901,8 +947,12 @@ SelfManaged
Sematext
SendExternalTables
SendScalars
SetOperationMode
ShareAlike
ShareSet
SharedJoin
SharedMergeTree
ShortCircuitFunctionEvaluation
Shortkeys
Signup
SimHash
@ -953,6 +1003,7 @@ SystemReplicasThreadsActive
TABLUM
TAVG
TCPConnection
TCPHandler
TCPThreads
TDigest
TINYINT
@ -1020,8 +1071,10 @@ TotalPrimaryKeyBytesInMemory
TotalPrimaryKeyBytesInMemoryAllocated
TotalRowsOfMergeTreeTables
TotalTemporaryFiles
TotalsMode
Tradeoff
Transactional
TransactionsWaitCSNMode
Tsai
Tukey
TwoColumnList
@ -1043,6 +1096,7 @@ URLHash
URLHierarchy
URLPathHierarchy
USearch
USearch
UTCTimestamp
UUIDNumToString
UUIDStringToNum
@ -1146,6 +1200,7 @@ aggregatio
aggretate
aggthrow
aiochclient
alloc
allocator
alphaTokens
amplab
@ -1428,6 +1483,7 @@ config
configs
conformant
congruential
conjuctive
connectionId
const
contrib
@ -1437,9 +1493,11 @@ corrMatrix
corrStable
corrmatrix
corrstable
cors
cosineDistance
countDigits
countEqual
countIf
countMatches
countMatchesCaseInsensitive
countSubstrings
@ -1561,7 +1619,9 @@ denormalizing
denormals
dequeued
dequeues
dereference
deserialization
deserialize
deserialized
deserializing
dest
@ -1604,6 +1664,7 @@ domainWithoutWWW
domainWithoutWWWRFC
dont
dotProduct
dotall
downsampling
dplyr
dragonbox
@ -1707,6 +1768,7 @@ formatReadableSize
formatReadableTimeDelta
formatRow
formatRowNoNewline
formatdatetime
formatschema
formatter
formatters
@ -1850,6 +1912,7 @@ heredocs
hilbertDecode
hilbertEncode
hiveHash
hnsw
holistics
homebrew
hopEnd
@ -1880,6 +1943,7 @@ ilike
incrementing
indexHint
indexOf
inequal
infi
inflight
infty
@ -1956,6 +2020,7 @@ kRing
kafka
kafkaMurmurHash
kafkacat
keepalive
keepermap
kerberized
kerberos
@ -2148,15 +2213,19 @@ multiSearchFirstPosition
multiSearchFirstPositionCaseInsensitive
multiSearchFirstPositionCaseInsensitiveUTF
multiSearchFirstPositionUTF
multibuffer
multibyte
multidirectory
multiif
multiline
multilinestring
multiplyDecimal
multipolygon
multiread
multisearchany
multisets
multithread
multithreading
multiword
munmap
murmurHash
@ -2208,6 +2277,7 @@ ngrambf
ngrams
noaa
nonNegativeDerivative
nonconst
noop
normalizeQuery
normalizeQueryKeepNames
@ -2229,6 +2299,7 @@ nullIf
nullability
nullable
nullables
nullptr
num
numerics
nypd
@ -2258,6 +2329,7 @@ pageviews
parallelization
parallelize
parallelized
param
params
parseDateTime
parseDateTimeBestEffort
@ -2276,13 +2348,16 @@ parseReadableSizeOrNull
parseReadableSizeOrZero
parseTimeDelta
parseable
parsedatetime
parsers
partitionID
partitionId
pathFull
pclmulqdq
pcre
perf
performant
perkey
perl
persistency
phpclickhouse
@ -2317,6 +2392,7 @@ positionUTF
positiveModulo
postfix
postfixes
postgres
postgresql
pre
pread
@ -2326,7 +2402,11 @@ prebuilt
preemptable
preferServerCiphers
prefetch
prefetched
prefetches
prefetching
prefetchsize
preimage
preloaded
prem
prepend
@ -2480,6 +2560,7 @@ reinterpretAsInt
reinterpretAsString
reinterpretAsUInt
reinterpretAsUUID
remerge
remoteSecure
repivot
replaceAll
@ -2487,6 +2568,7 @@ replaceOne
replaceRegexpAll
replaceRegexpOne
replacingmergetree
replcase
replicatable
replicatedmergetree
replxx
@ -2494,6 +2576,7 @@ repo
representable
requestor
requireTLSv
rerange
resharding
reshards
resolvers
@ -2525,6 +2608,7 @@ rowbinary
rowbinarywithdefaults
rowbinarywithnames
rowbinarywithnamesandtypes
rowlist
rsync
rsyslog
runnable
@ -2713,6 +2797,7 @@ subtrees
subtype
sudo
sumCount
sumIf
sumKahan
sumMap
sumMapFiltered
@ -2758,6 +2843,7 @@ theilsu
themself
threadpool
throwIf
throwif
timeDiff
timeSeriesData
timeSeriesMetrics
@ -2923,9 +3009,11 @@ typename
ubuntu
uint
ulid
unacked
unary
unbin
uncomment
undelete
undrop
unencoded
unencrypted
@ -2954,6 +3042,7 @@ uniqthetasketch
unix
unixODBC
unixodbc
unmerged
unoptimized
unparsed
unpooled
@ -3060,90 +3149,3 @@ znode
znodes
zookeeperSessionUptime
zstd
postgres
ArrowCompression
CapnProtoEnumComparingMode
DateTimeInputFormat
DateTimeOutputFormat
DateTimeOverflowBehavior
deserialize
dotall
EachRow
EscapingRule
IdentifierQuotingRule
IdentifierQuotingStyle
IntervalOutputFormat
MsgPackUUIDRepresentation
ORCCompression
ParquetCompression
ParquetVersion
SchemaInferenceMode
alloc
CacheWarmer
conjuctive
cors
CORS
countIf
DefaultTableEngine
dereference
DistributedDDLOutputMode
DistributedProductMode
formatdatetime
inequal
INVOKER
ITION
JoinAlgorithm
JoinStrictness
keepalive
ListObject
ListObjects
LoadBalancing
LocalFSReadMethod
LogQueriesType
LogsLevel
MaxThreads
MemorySample
multibuffer
multiif
multiread
multithreading
MySQLDataTypesSupport
nonconst
NonZeroUInt
nullptr
OverflowMode
OverflowModeGroupBy
ParallelReplicasMode
param
parsedatetime
perf
PerfEventInfo
perkey
prefetched
prefetches
prefetching
preimage
QueryCacheNondeterministicFunctionHandling
QueryCacheSystemTableHandling
remerge
replcase
rerange
RetryStrategy
rowlist
SetOperationMode
ShortCircuitFunctionEvaluation
SQLSecurityType
sumIf
TCPHandler
throwif
TotalsMode
TransactionsWaitCSNMode
undelete
unmerged
DataPacket
DDLs
DistributedCacheLogMode
DistributedCachePoolBehaviourOnLimit
SharedJoin
ShareSet
unacked