Misc Annoy fixes

This commit is contained in:
Robert Schulze 2023-06-08 08:10:40 +00:00
parent 5344ff2516
commit b8178088d0
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
11 changed files with 548 additions and 345 deletions

View File

@ -1,104 +1,142 @@
# Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex}
Nearest neighborhood search refers to the problem of finding the point(s) with the smallest distance to a given point in an n-dimensional
space. Since exact search is in practice usually typically too slow, the task is often solved with approximate algorithms. A popular use
case of of neighbor search is finding similar pictures (texts) for a given picture (text). Pictures (texts) can be decomposed into
[embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning), and instead of
comparing pictures (texts) pixel-by-pixel (character-by-character), only the embeddings are compared.
Nearest neighborhood search is the problem of finding the M closest points for a given point in an N-dimensional vector space. The most
straightforward approach to solve this problem is a brute force search where the distance between all points in the vector space and the
reference point is computed. This method guarantees perfect accuracy but it is usually too slow for practical applications. Thus, nearest
neighborhood search problems are often solved with [approximative algorithms](https://github.com/erikbern/ann-benchmarks). Approximative
nearest neighborhood search techniques, in conjunction with [embedding
methods](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning) allow to search huge
amounts of media (pictures, songs, articles, etc.) in milliseconds.
In terms of SQL, the problem can be expressed as follows:
Blogs:
- [Vector Search with ClickHouse - Part 1](https://clickhouse.com/blog/vector-search-clickhouse-p1)
- [Vector Search with ClickHouse - Part 2](https://clickhouse.com/blog/vector-search-clickhouse-p2)
In terms of SQL, the nearest neighborhood problem can be expressed as follows:
``` sql
SELECT *
FROM table
WHERE L2Distance(column, Point) < MaxDistance
ORDER BY Distance(vectors, Point)
LIMIT N
```
`vectors` contains N-dimensional values of type [Array](../../../sql-reference/data-types/array.md) or
[Tuple](../../../sql-reference/data-types/tuple.md), for example embeddings. Function `Distance` computes the distance between two vectors.
Often, the the Euclidean (L2) distance is chosen as distance function but [other
distance functions](/docs/en/sql-reference/functions/distance-functions.md) are also possible. `Point` is the reference point, e.g. `(0.17,
0.33, ...)`, and `N` limits the number of search results.
An alternative formulation of the nearest neighborhood search problem looks as follows:
``` sql
SELECT *
FROM table
ORDER BY L2Distance(column, Point)
WHERE Distance(vectors, Point) < MaxDistance
LIMIT N
```
The queries are expensive because the L2 (Euclidean) distance between `Point` and all points in `column` and must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly.
While the first query returns the top-`N` closest points to the reference point, the second query returns all points closer to the reference
point than a maximally allowed radius `MaxDistance`. Parameter `N` limits the number of returned values which is useful for situations where
`MaxDistance` is difficult to determine in advance.
# Creating ANN Indexes
With brute force search, both queries are expensive (linear in the number of points) because the distance between all points in `vectors` and
`Point` must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation
of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer much quicker (in sub-linear time).
As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`.
# Creating and Using ANN Indexes
Syntax to create an ANN index over an `Array` column:
Syntax to create an ANN index over an [Array](../../../sql-reference/data-types/array.md) column:
```sql
CREATE TABLE table
(
`id` Int64,
`embedding` Array(Float32),
INDEX <ann_index_name> embedding TYPE <ann_index_type>(<ann_index_parameters>) GRANULARITY <N>
`vectors` Array(Float32),
INDEX <ann_index_name> vectors TYPE <ann_index_type>(<ann_index_parameters>) [GRANULARITY <N>]
)
ENGINE = MergeTree
ORDER BY id;
```
Syntax to create an ANN index over a `Tuple` column:
Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column:
```sql
CREATE TABLE table
(
`id` Int64,
`embedding` Tuple(Float32[, Float32[, ...]]),
INDEX <ann_index_name> embedding TYPE <ann_index_type>(<ann_index_parameters>) GRANULARITY <N>
`vectors` Tuple(Float32[, Float32[, ...]]),
INDEX <ann_index_name> vectors TYPE <ann_index_type>(<ann_index_parameters>) [GRANULARITY <N>]
)
ENGINE = MergeTree
ORDER BY id;
```
ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will be slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively there are much more read requests than write requests.
Similar to regular skip indexes, ANN indexes are constructed over granules and each indexed block consists of `GRANULARITY = <N>`-many
granules. For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`,
then each indexed block will consist of 16384 rows. However, unlike skip indexes, ANN indexes are not only able to skip the entire indexed
block, they are able to skip individual granules in indexed blocks. As a result, the `GRANULARITY` parameter has a different meaning in ANN
indexes than in normal skip indexes. Basically, the bigger `GRANULARITY` is chosen, the more data is provided to a single ANN index, and the
higher the chance that with the right hyper parameters, the index will remember the data structure better.
# Using ANN Indexes
ANN indexes are built during column insertion and merge. As a result, `INSERT` and `OPTIMIZE` statements will be slower than for ordinary
tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively when are far more read requests than write
requests.
ANN indexes support two types of queries:
- WHERE queries:
``` sql
SELECT *
FROM table
WHERE DistanceFunction(column, Point) < MaxDistance
LIMIT N
```
- ORDER BY queries:
``` sql
SELECT *
FROM table
[WHERE ...]
ORDER BY DistanceFunction(column, Point)
ORDER BY Distance(vectors, Point)
LIMIT N
```
`DistanceFunction` is a [distance function](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a reference vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a floating point value which restricts the size of the neighbourhood.
- WHERE queries:
``` sql
SELECT *
FROM table
WHERE Distance(vectors, Point) < MaxDistance
LIMIT N
```
:::tip
To avoid writing out large vectors, you can use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g.
To avoid writing out large vectors, you can use [query
parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g.
```bash
clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(embedding, {vec: Array(Float32)}) < 1.0"
clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(vectors, {vec: Array(Float32)}) < 1.0"
```
:::
ANN indexes cannot speed up queries that contain both a `WHERE DistanceFunction(column, Point) < MaxDistance` and an `ORDER BY DistanceFunction(column, Point)` clause. Also, the approximate algorithms used to determine the nearest neighbors require a limit, hence queries that use an ANN index must have a `LIMIT` clause.
**Restrictions**: Queries that contain both a `WHERE Distance(vectors, Point) < MaxDistance` and an `ORDER BY Distance(vectors, Point)`
clause cannot use ANN indexes. Also, the 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
approximate neighbor search.
**Differences to Skip Indexes** Similar to regular [skip indexes](https://clickhouse.com/docs/en/optimize/skipping-indexes), ANN indexes are
constructed over granules and each indexed block consists of `GRANULARITY = <N>`-many granules (`<N>` = 1 by default for normal skip
indexes). For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`,
then each indexed block will contain 16384 rows. However, data structures and algorithms for approximate neighborhood search (usually
provided by external libraries) are inherently row-oriented. They store a compact representation of a set of rows and also return rows for
ANN queries. This causes some rather unintuitive differences in the way ANN indexes behave compared to normal skip indexes.
When a user defines a ANN index on a column, ClickHouse internally creates a ANN "sub-index" for each index block. The sub-index is "local"
in the sense that it only knows about the rows of its containing index block. In the previous example and assuming that a column has 65536
rows, we obtain four index blocks (spanning eight granules) and a ANN sub-index for each index block. A sub-index is theoretically able to
return the rows with the N closest points within its index block directly. However, since ClickHouse loads data from disk to memory at the
granularity of granules, sub-indexes extrapolate matching rows to granule granularity. This is different from regular skip indexes which
skip data at the granularity of index blocks.
The `GRANULARITY` parameter determines how many ANN sub-indexes are created. Bigger `GRANULARITY` values mean fewer but larger ANN
sub-indexes, up to the point where a column (or a column part) has only a single sub-index. In that case, the sub-index has a "global" view of
all column rows and can directly return all granules of the column (part) with relevant rows (there are at at most `LIMIT <N>`-many
such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a brute-force distance
calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes returns up to `LIMIT N`-many granules.
As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with both cases equally good, only the
processing performance differs. It is generally recommended to use a large `GRANULARITY` for ANN indexes and fall back to a smaller
`GRANULARITY` values only in case of problems like excessive memory consumption of the ANN structures. If no `GRANULARITY` was specified for
ANN indexes, the default value is 100 million.
An ANN index is only used if the query has a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This is a safety measure which helps to avoid large memory consumption by external libraries for approximate neighbor search.
# Available ANN Indexes
@ -106,51 +144,68 @@ An ANN index is only used if the query has a `LIMIT` value smaller than setting
## Annoy {#annoy}
(currently disabled on ARM due to memory safety problems with the algorithm)
Annoy indexes are currently experimental, to use them you first need to `SET allow_experimental_annoy_index = 1`. They are also currently
disabled on ARM due to memory safety problems with the algorithm.
This type of ANN index implements [the Annoy algorithm](https://github.com/spotify/annoy) which uses a recursive division of the space in random linear surfaces (lines in 2D, planes in 3D etc.).
This type of ANN index implements [the Annoy algorithm](https://github.com/spotify/annoy) which is based on a recursive division of the
space in random linear surfaces (lines in 2D, planes in 3D etc.).
Syntax to create a Annoy index over a `Array` column:
<div class='vimeo-container'>
<iframe src="//www.youtube.com/watch?v=QkCCyLW0ehU"
width="640"
height="360"
frameborder="0"
allow="autoplay;
fullscreen;
picture-in-picture"
allowfullscreen>
</iframe>
</div>
Syntax to create an Annoy index over an [Array](../../../sql-reference/data-types/array.md) column:
```sql
CREATE TABLE table
(
id Int64,
embedding Array(Float32),
INDEX <ann_index_name> embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N
vectors Array(Float32),
INDEX <ann_index_name> vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N]
)
ENGINE = MergeTree
ORDER BY id;
```
Syntax to create a Annoy index over a `Tuple` column:
Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column:
```sql
CREATE TABLE table
(
id Int64,
embedding Tuple(Float32[, Float32[, ...]]),
INDEX <ann_index_name> embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N
vectors Tuple(Float32[, Float32[, ...]]),
INDEX <ann_index_name> vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N]
)
ENGINE = MergeTree
ORDER BY id;
```
Parameter `DistanceName` is name of a distance function (default `L2Distance`). Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results.
Annoy currently supports `L2Distance` and `cosineDistance` as distance function `Distance`. If no distance function was specified during
index creation, `L2Distance` is used as default. Parameter `NumTrees` is the number of trees which the algorithm creates (default if not
specified: 100). Higher values of `NumTree` mean more accurate search results but slower index creation / query times (approximately
linearly) as well as larger index sizes.
:::note
Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(embedding) = 256`.
Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use
[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1
CHECK length(vectors) = 256`.
:::
Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. It can be used to
balance runtime and accuracy at runtime.
Example:
Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger
values mean more accurate results at the cost of longer query runtime:
``` sql
SELECT *
FROM table_name [WHERE ...]
ORDER BY L2Distance(column, Point)
ORDER BY L2Distance(vectors, Point)
LIMIT N
SETTINGS annoy_index_search_k_nodes=100
```

View File

@ -491,7 +491,7 @@ Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, ran
#### Special-purpose
- An experimental index to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details.
- 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.
### Functions Support {#functions-support}

View File

@ -46,7 +46,16 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected
index->part_of_create_index_query = true;
index->set(index->expr, expr);
index->set(index->type, type);
index->granularity = granularity ? granularity->as<ASTLiteral &>().value.safeGet<UInt64>() : 1;
if (granularity)
index->granularity = granularity->as<ASTLiteral &>().value.safeGet<UInt64>();
else
{
if (index->type->name == "annoy")
index->granularity = 100'000'000;
else
index->granularity = 1;
}
node = index;
return true;

View File

@ -141,7 +141,17 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
index->name = name->as<ASTIdentifier &>().name();
index->set(index->expr, expr);
index->set(index->type, type);
index->granularity = granularity ? granularity->as<ASTLiteral &>().value.safeGet<UInt64>() : 1;
if (granularity)
index->granularity = granularity->as<ASTLiteral &>().value.safeGet<UInt64>();
else
{
if (index->type->name == "annoy")
index->granularity = 100'000'000;
else
index->granularity = 1;
}
node = index;
return true;

View File

@ -88,7 +88,7 @@ std::vector<float> ApproximateNearestNeighborCondition::getReferenceVector() con
throw Exception(ErrorCodes::LOGICAL_ERROR, "Reference vector was requested for useless or uninitialized index.");
}
size_t ApproximateNearestNeighborCondition::getNumOfDimensions() const
size_t ApproximateNearestNeighborCondition::getDimensions() const
{
if (index_is_useful && query_information.has_value())
return query_information->reference_vector.size();

View File

@ -90,8 +90,8 @@ public:
/// Distance should be calculated regarding to referenceVector
std::vector<float> getReferenceVector() const;
/// Reference vector's dimension size
size_t getNumOfDimensions() const;
/// Reference vector's dimension count
size_t getDimensions() const;
String getColumnName() const;

View File

@ -27,13 +27,13 @@ namespace ErrorCodes
template <typename Distance>
AnnoyIndexWithSerialization<Distance>::AnnoyIndexWithSerialization(uint64_t dim)
: Base::AnnoyIndex(dim)
AnnoyIndexWithSerialization<Distance>::AnnoyIndexWithSerialization(size_t dimensions)
: Base::AnnoyIndex(dimensions)
{
}
template<typename Distance>
void AnnoyIndexWithSerialization<Distance>::serialize(WriteBuffer& ostr) const
void AnnoyIndexWithSerialization<Distance>::serialize(WriteBuffer & ostr) const
{
chassert(Base::_built);
writeIntBinary(Base::_s, ostr);
@ -43,11 +43,11 @@ void AnnoyIndexWithSerialization<Distance>::serialize(WriteBuffer& ostr) const
writeIntBinary(Base::_K, ostr);
writeIntBinary(Base::_seed, ostr);
writeVectorBinary(Base::_roots, ostr);
ostr.write(reinterpret_cast<const char*>(Base::_nodes), Base::_s * Base::_n_nodes);
ostr.write(reinterpret_cast<const char *>(Base::_nodes), Base::_s * Base::_n_nodes);
}
template<typename Distance>
void AnnoyIndexWithSerialization<Distance>::deserialize(ReadBuffer& istr)
void AnnoyIndexWithSerialization<Distance>::deserialize(ReadBuffer & istr)
{
chassert(!Base::_built);
readIntBinary(Base::_s, istr);
@ -69,7 +69,7 @@ void AnnoyIndexWithSerialization<Distance>::deserialize(ReadBuffer& istr)
}
template<typename Distance>
uint64_t AnnoyIndexWithSerialization<Distance>::getNumOfDimensions() const
size_t AnnoyIndexWithSerialization<Distance>::getDimensions() const
{
return Base::get_f();
}
@ -97,14 +97,14 @@ void MergeTreeIndexGranuleAnnoy<Distance>::serializeBinary(WriteBuffer & ostr) c
{
/// Number of dimensions is required in the index constructor,
/// so it must be written and read separately from the other part
writeIntBinary(index->getNumOfDimensions(), ostr); // write dimension
writeIntBinary(static_cast<UInt64>(index->getDimensions()), ostr); // write dimension
index->serialize(ostr);
}
template <typename Distance>
void MergeTreeIndexGranuleAnnoy<Distance>::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/)
{
uint64_t dimension;
UInt64 dimension;
readIntBinary(dimension, istr);
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(dimension);
index->deserialize(istr);
@ -114,7 +114,7 @@ template <typename Distance>
MergeTreeIndexAggregatorAnnoy<Distance>::MergeTreeIndexAggregatorAnnoy(
const String & index_name_,
const Block & index_sample_block_,
uint64_t trees_)
UInt64 trees_)
: index_name(index_name_)
, index_sample_block(index_sample_block_)
, trees(trees_)
@ -251,10 +251,10 @@ std::vector<size_t> MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI
const AnnoyIndexWithSerializationPtr<Distance> annoy = granule->index;
if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions())
if (ann_condition.getDimensions() != annoy->getDimensions())
throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) "
"does not match the dimension in the index ({})",
ann_condition.getNumOfDimensions(), annoy->getNumOfDimensions());
ann_condition.getDimensions(), annoy->getDimensions());
std::vector<UInt64> neighbors; /// indexes of dots which were closest to the reference vector
std::vector<Float32> distances;
@ -281,7 +281,7 @@ std::vector<size_t> MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI
return granule_numbers;
}
MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_)
MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 trees_, const String & distance_function_)
: IMergeTreeIndex(index_)
, trees(trees_)
, distance_function(distance_function_)
@ -320,9 +320,9 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index)
if (!index.arguments.empty())
distance_function = index.arguments[0].get<String>();
uint64_t trees = default_trees;
UInt64 trees = default_trees;
if (index.arguments.size() > 1)
trees = index.arguments[1].get<uint64_t>();
trees = index.arguments[1].get<UInt64>();
return std::make_shared<MergeTreeIndexAnnoy>(index, trees, distance_function);
}
@ -338,7 +338,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance function argument of Annoy index must be of type String");
if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Number of trees argument of Annoy index must be UInt64");
throw Exception(ErrorCodes::INCORRECT_QUERY, "Number of trees argument of Annoy index must be of type UInt64");
/// Check that the index is created on a single column
@ -351,17 +351,16 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */)
{
String distance_name = index.arguments[0].get<String>();
if (distance_name != "L2Distance" && distance_name != "cosineDistance")
throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index supports only distance functions 'L2Distance' and 'cosineDistance'. Given distance function: {}", distance_name);
throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions 'L2Distance' and 'cosineDistance'");
}
/// Check data type of indexed column:
auto throw_unsupported_underlying_column_exception = [](DataTypePtr data_type)
auto throw_unsupported_underlying_column_exception = []()
{
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32). Given type: {}",
data_type->getName());
"Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32)");
};
DataTypePtr data_type = index.sample_block.getDataTypes()[0];
@ -370,7 +369,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */)
{
TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId();
if (!WhichDataType(nested_type_index).isFloat32())
throw_unsupported_underlying_column_exception(data_type);
throw_unsupported_underlying_column_exception();
}
else if (const auto * data_type_tuple = typeid_cast<const DataTypeTuple *>(data_type.get()))
{
@ -379,11 +378,11 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */)
{
TypeIndex nested_type_index = inner_type->getTypeId();
if (!WhichDataType(nested_type_index).isFloat32())
throw_unsupported_underlying_column_exception(data_type);
throw_unsupported_underlying_column_exception();
}
}
else
throw_unsupported_underlying_column_exception(data_type);
throw_unsupported_underlying_column_exception();
}
}

View File

@ -16,10 +16,10 @@ class AnnoyIndexWithSerialization : public Annoy::AnnoyIndex<UInt64, Float32, Di
using Base = Annoy::AnnoyIndex<UInt64, Float32, Distance, Annoy::Kiss64Random, Annoy::AnnoyIndexMultiThreadedBuildPolicy>;
public:
explicit AnnoyIndexWithSerialization(uint64_t dim);
void serialize(WriteBuffer& ostr) const;
void deserialize(ReadBuffer& istr);
uint64_t getNumOfDimensions() const;
explicit AnnoyIndexWithSerialization(size_t dimensions);
void serialize(WriteBuffer & ostr) const;
void deserialize(ReadBuffer & istr);
size_t getDimensions() const;
};
template <typename Distance>
@ -46,7 +46,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule
template <typename Distance>
struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator
{
MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t trees);
MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, UInt64 trees);
~MergeTreeIndexAggregatorAnnoy() override = default;
bool empty() const override { return !index || index->get_n_items() == 0; }
@ -55,7 +55,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator
const String index_name;
const Block index_sample_block;
const uint64_t trees;
const UInt64 trees;
AnnoyIndexWithSerializationPtr<Distance> index;
};
@ -89,7 +89,7 @@ class MergeTreeIndexAnnoy : public IMergeTreeIndex
{
public:
MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_);
MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 trees_, const String & distance_function_);
~MergeTreeIndexAnnoy() override = default;
@ -100,7 +100,7 @@ public:
bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; }
private:
const uint64_t trees;
const UInt64 trees;
const String distance_function;
};

View File

@ -1,118 +1,144 @@
--- Test with Array ---
WHERE type, L2Distance
1 [0,0,10]
2 [0,0,10.5]
3 [0,0,9.5]
4 [0,0,9.7]
5 [0,0,10.2]
ORDER BY type, L2Distance
1 [0,0,10]
5 [0,0,10.2]
4 [0,0,9.7]
WHERE type, L2Distance, check that index is used
Expression ((Projection + Before ORDER BY))
Limit (preliminary LIMIT (without OFFSET))
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
Skip
Name: annoy_index
Description: annoy GRANULARITY 1
Parts: 1/1
Granules: 1/3
ORDER BY type, L2Distance, check that index is used
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
Skip
Name: annoy_index
Description: annoy GRANULARITY 1
Parts: 1/1
Granules: 3/3
parameter annoy_index_search_k_nodes
parameter max_limit_for_ann_queries
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
--- Test with Tuple ---
WHERE type, L2Distance
1 (0,0,10)
2 (0,0,10.5)
3 (0,0,9.5)
4 (0,0,9.7)
5 (0,0,10.2)
ORDER BY type, L2Distance
1 (0,0,10)
5 (0,0,10.2)
4 (0,0,9.7)
WHERE type, L2Distance, check that index is used
Expression ((Projection + Before ORDER BY))
Limit (preliminary LIMIT (without OFFSET))
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
Skip
Name: annoy_index
Description: annoy GRANULARITY 1
Parts: 1/1
Granules: 1/3
ORDER BY type, L2Distance, check that index is used
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
Skip
Name: annoy_index
Description: annoy GRANULARITY 1
Parts: 1/1
Granules: 3/3
parameter annoy_index_search_k_nodes
parameter max_limit_for_ann_queries
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
--- Test alternative metric (cosine distance) and non-default NumTrees ---
WHERE type, L2Distance
1 [0,0,10]
2 [0,0,10.5]
3 [0,0,9.5]
4 [0,0,9.7]
5 [0,0,10.2]
ORDER BY type, L2Distance
1 [0,0,10]
5 [0,0,10.2]
4 [0,0,9.7]
--- Negative tests ---
--- Test default GRANULARITY (should be 100 mio. for annoy)---
CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX annoy_index vector TYPE annoy GRANULARITY 100000000\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192
CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX annoy_index vector TYPE annoy GRANULARITY 100000000\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192
--- Test with Array, GRANULARITY = 1, index_granularity = 5 ---
WHERE type, L2Distance, check that index is used
Expression ((Projection + Before ORDER BY))
Limit (preliminary LIMIT (without OFFSET))
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
Skip
Name: annoy_index
Description: annoy GRANULARITY 1
Parts: 1/1
Granules: 1/3
ORDER BY type, L2Distance, check that index is used
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
Skip
Name: annoy_index
Description: annoy GRANULARITY 1
Parts: 1/1
Granules: 3/3
Reference ARRAYs with non-matching dimension are rejected
Special case: MaximumDistance is negative
WHERE type, L2Distance
Special case: setting annoy_index_search_k_nodes
Special case: setting max_limit_for_ann_queries
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
--- Test with Tuple, GRANULARITY = 1, index_granularity = 5 ---
WHERE type, L2Distance, check that index is used
Expression ((Projection + Before ORDER BY))
Limit (preliminary LIMIT (without OFFSET))
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
Skip
Name: annoy_index
Description: annoy GRANULARITY 1
Parts: 1/1
Granules: 1/3
ORDER BY type, L2Distance, check that index is used
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 3/3
Skip
Name: annoy_index
Description: annoy GRANULARITY 1
Parts: 1/1
Granules: 3/3
--- Test non-default metric (cosine distance) + non-default NumTrees (200) ---
--- Test with Array, GRANULARITY = 2, index_granularity = 4 ---
WHERE type, L2Distance, check that index is used
Expression ((Projection + Before ORDER BY))
Limit (preliminary LIMIT (without OFFSET))
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: annoy_index
Description: annoy GRANULARITY 2
Parts: 0/1
Granules: 2/4
ORDER BY type, L2Distance, check that index is used
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: annoy_index
Description: annoy GRANULARITY 2
Parts: 1/1
Granules: 4/4
--- Test with Array, GRANULARITY = 4, index_granularity = 4 ---
WHERE type, L2Distance, check that index is used
Expression ((Projection + Before ORDER BY))
Limit (preliminary LIMIT (without OFFSET))
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: annoy_index
Description: annoy GRANULARITY 4
Parts: 0/1
Granules: 3/4
ORDER BY type, L2Distance, check that index is used
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: annoy_index
Description: annoy GRANULARITY 4
Parts: 1/1
Granules: 4/4

View File

@ -1,150 +1,251 @@
-- Tags: disabled, no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check
-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check
SET allow_experimental_annoy_index = 1;
SELECT '--- Test with Array ---';
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5;
INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]);
SELECT 'WHERE type, L2Distance';
SELECT *
FROM tab
WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0
LIMIT 5;
SELECT 'ORDER BY type, L2Distance';
SELECT *
FROM tab
ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0])
LIMIT 3;
-- Produces different error code with analyzer, TODO: check
-- SELECT 'Reference ARRAYs with non-matching dimension are rejected';
-- SELECT *
-- FROM tab
-- ORDER BY L2Distance(embedding, [0.0, 0.0])
-- LIMIT 3; -- { serverError INCORRECT_QUERY }
SELECT 'WHERE type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0
LIMIT 5;
SELECT 'ORDER BY type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0])
LIMIT 3;
SELECT 'parameter annoy_index_search_k_nodes';
SELECT *
FROM tab
ORDER BY L2Distance(embedding, [5.3, 7.3, 2.1])
LIMIT 5
SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results
SELECT 'parameter max_limit_for_ann_queries';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(embedding, [5.3, 7.3, 2.1])
LIMIT 5
SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index
DROP TABLE tab;
SELECT '--- Test with Tuple ---';
CREATE TABLE tab(id Int32, embedding Tuple(Float32, Float32, Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5;
INSERT INTO tab VALUES (1, (0.0, 0.0, 10.0)), (2, (0.0, 0.0, 10.5)), (3, (0.0, 0.0, 9.5)), (4, (0.0, 0.0, 9.7)), (5, (0.0, 0.0, 10.2)), (6, (10.0, 0.0, 0.0)), (7, (9.5, 0.0, 0.0)), (8, (9.7, 0.0, 0.0)), (9, (10.2, 0.0, 0.0)), (10, (10.5, 0.0, 0.0)), (11, (0.0, 10.0, 0.0)), (12, (0.0, 9.5, 0.0)), (13, (0.0, 9.7, 0.0)), (14, (0.0, 10.2, 0.0)), (15, (0.0, 10.5, 0.0));
SELECT 'WHERE type, L2Distance';
SELECT *
FROM tab
WHERE L2Distance(embedding, (0.0, 0.0, 10.0)) < 1.0
LIMIT 5;
SELECT 'ORDER BY type, L2Distance';
SELECT *
FROM tab
ORDER BY L2Distance(embedding, (0.0, 0.0, 10.0))
LIMIT 3;
SELECT 'WHERE type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
WHERE L2Distance(embedding, (0.0, 0.0, 10.0)) < 1.0
LIMIT 5;
SELECT 'ORDER BY type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(embedding, (0.0, 0.0, 10.0))
LIMIT 3;
SELECT 'parameter annoy_index_search_k_nodes';
SELECT *
FROM tab
ORDER BY L2Distance(embedding, (5.3, 7.3, 2.1))
LIMIT 5
SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results
SELECT 'parameter max_limit_for_ann_queries';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(embedding, (5.3, 7.3, 2.1))
LIMIT 5
SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index
DROP TABLE tab;
SELECT '--- Test alternative metric (cosine distance) and non-default NumTrees ---';
CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('cosineDistance', 200)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5;
INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]);
SELECT 'WHERE type, L2Distance';
SELECT *
FROM tab
WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0
LIMIT 5;
SELECT 'ORDER BY type, L2Distance';
SELECT *
FROM tab
ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0])
LIMIT 3;
DROP TABLE tab;
SET allow_experimental_analyzer = 0;
SELECT '--- Negative tests ---';
DROP TABLE IF EXISTS tab;
-- must have at most 2 arguments
CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
-- first argument (distance_function) must be String
CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
-- 2nd argument (number of trees) must be UInt64
CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
-- reject unsupported distance functions
CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
-- must be created on single column
CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index (embedding, id) TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS }
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index (vector, id) TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS }
-- reject unsupported distance functions
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
-- must be created on Array/Tuple(Float32) columns
SET allow_suspicious_low_cardinality_types = 1;
CREATE TABLE tab(id Int32, embedding Float32, INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
CREATE TABLE tab(id Int32, embedding Array(Float64), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
CREATE TABLE tab(id Int32, embedding LowCardinality(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
CREATE TABLE tab(id Int32, embedding Nullable(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
CREATE TABLE tab(id Int32, vector Float32, INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
CREATE TABLE tab(id Int32, vector Array(Float64), INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
CREATE TABLE tab(id Int32, vector Tuple(Float64), INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
CREATE TABLE tab(id Int32, vector LowCardinality(Float32), INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
CREATE TABLE tab(id Int32, vector Nullable(Float32), INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
SELECT '--- Test default GRANULARITY (should be 100 mio. for annoy)---';
CREATE TABLE tab (id Int32, vector Array(Float32), INDEX annoy_index(vector) TYPE annoy) ENGINE=MergeTree ORDER BY id;
SHOW CREATE TABLE tab;
DROP TABLE tab;
CREATE TABLE tab (id Int32, vector Array(Float32)) ENGINE=MergeTree ORDER BY id;
ALTER TABLE tab ADD INDEX annoy_index(vector) TYPE annoy;
SHOW CREATE TABLE tab;
DROP TABLE tab;
SELECT '--- Test with Array, GRANULARITY = 1, index_granularity = 5 ---';
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy() GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5;
INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]);
-- rows = 15, index_granularity = 5, GRANULARITY = 1 gives 3 annoy-indexed blocks (each comprising a single granule)
-- condition 'L2Distance(vector, reference_vector) < 1.0' ensures that only one annoy-indexed block produces results --> "Granules: 1/3"
-- See (*) why commented out
-- SELECT 'WHERE type, L2Distance';
-- SELECT *
-- FROM tab
-- WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0
-- LIMIT 3;
SELECT 'WHERE type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0
LIMIT 3;
-- See (*) why commented out
-- SELECT 'ORDER BY type, L2Distance';
-- SELECT *
-- FROM tab
-- ORDER BY L2Distance(vector, [0.0, 0.0, 10.0])
-- LIMIT 3;
SELECT 'ORDER BY type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(vector, [0.0, 0.0, 10.0])
LIMIT 3;
-- Test special cases. Corresponding special case tests are omitted from later tests.
SELECT 'Reference ARRAYs with non-matching dimension are rejected';
SELECT *
FROM tab
ORDER BY L2Distance(vector, [0.0, 0.0])
LIMIT 3; -- { serverError INCORRECT_QUERY }
SELECT 'Special case: MaximumDistance is negative';
SELECT 'WHERE type, L2Distance';
SELECT *
FROM tab
WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < -1.0
LIMIT 3; -- { serverError INCORRECT_QUERY }
SELECT 'Special case: setting annoy_index_search_k_nodes';
SELECT *
FROM tab
ORDER BY L2Distance(vector, [5.3, 7.3, 2.1])
LIMIT 3
SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results
SELECT 'Special case: setting max_limit_for_ann_queries';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(vector, [5.3, 7.3, 2.1])
LIMIT 3
SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index
DROP TABLE tab;
-- Test Tuple embeddings. Triggers different logic than Array inside MergeTreeIndexAnnoy but the same logic as Array above MergeTreeIndexAnnoy.
-- Therefore test Tuple case just once.
SELECT '--- Test with Tuple, GRANULARITY = 1, index_granularity = 5 ---';
CREATE TABLE tab(id Int32, vector Tuple(Float32, Float32, Float32), INDEX annoy_index vector TYPE annoy() GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5;
INSERT INTO tab VALUES (1, (0.0, 0.0, 10.0)), (2, (0.0, 0.0, 10.5)), (3, (0.0, 0.0, 9.5)), (4, (0.0, 0.0, 9.7)), (5, (0.0, 0.0, 10.2)), (6, (10.0, 0.0, 0.0)), (7, (9.5, 0.0, 0.0)), (8, (9.7, 0.0, 0.0)), (9, (10.2, 0.0, 0.0)), (10, (10.5, 0.0, 0.0)), (11, (0.0, 10.0, 0.0)), (12, (0.0, 9.5, 0.0)), (13, (0.0, 9.7, 0.0)), (14, (0.0, 10.2, 0.0)), (15, (0.0, 10.5, 0.0));
-- See (*) why commented out
-- SELECT 'WHERE type, L2Distance';
-- SELECT *
-- FROM tab
-- WHERE L2Distance(vector, (0.0, 0.0, 10.0)) < 1.0
-- LIMIT 3;
SELECT 'WHERE type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
WHERE L2Distance(vector, (0.0, 0.0, 10.0)) < 1.0
LIMIT 3;
-- See (*) why commented out
-- SELECT 'ORDER BY type, L2Distance';
-- SELECT *
-- FROM tab
-- ORDER BY L2Distance(vector, (0.0, 0.0, 10.0))
-- LIMIT 3;
SELECT 'ORDER BY type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(vector, (0.0, 0.0, 10.0))
LIMIT 3;
DROP TABLE tab;
-- Not a systematic test, just to make sure no bad things happen
SELECT '--- Test non-default metric (cosine distance) + non-default NumTrees (200) ---';
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy('cosineDistance', 200) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5;
INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]);
-- See (*) why commented out
-- SELECT 'WHERE type, L2Distance';
-- SELECT *
-- FROM tab
-- WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0
-- LIMIT 3;
-- See (*) why commented out
-- SELECT 'ORDER BY type, L2Distance';
-- SELECT *
-- FROM tab
-- ORDER BY L2Distance(vector, [0.0, 0.0, 10.0])
-- LIMIT 3;
DROP TABLE tab;
SELECT '--- Test with Array, GRANULARITY = 2, index_granularity = 4 ---';
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 4;
INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0, 0.0]), (2, [0.0, 0.0, 10.5, 0.0]), (3, [0.0, 0.0, 9.5, 0.0]), (4, [0.0, 0.0, 9.7, 0.0]), (5, [10.0, 0.0, 0.0, 0.0]), (6, [9.5, 0.0, 0.0, 0.0]), (7, [9.7, 0.0, 0.0, 0.0]), (8, [10.2, 0.0, 0.0, 0.0]), (9, [0.0, 10.0, 0.0, 0.0]), (10, [0.0, 9.5, 0.0, 0.0]), (11, [0.0, 9.7, 0.0, 0.0]), (12, [0.0, 9.7, 0.0, 0.0]), (13, [0.0, 0.0, 0.0, 10.3]), (14, [0.0, 0.0, 0.0, 9.5]), (15, [0.0, 0.0, 0.0, 10.0]), (16, [0.0, 0.0, 0.0, 10.5]);
-- rows = 16, index_granularity = 4, GRANULARITY = 2 gives 2 annoy-indexed blocks (each comprising two granules)
-- condition 'L2Distance(vector, reference_vector) < 1.0' ensures that only one annoy-indexed block produces results --> "Granules: 2/4"
-- See (*) why commented out
-- SELECT 'WHERE type, L2Distance';
-- SELECT *
-- FROM tab
-- WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0
-- LIMIT 3;
SELECT 'WHERE type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0
LIMIT 3;
-- See (*) why commented out
-- SELECT 'ORDER BY type, L2Distance';
-- SELECT *
-- FROM tab
-- ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0])
-- LIMIT 3;
SELECT 'ORDER BY type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0])
LIMIT 3;
DROP TABLE tab;
SELECT '--- Test with Array, GRANULARITY = 4, index_granularity = 4 ---';
CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy() GRANULARITY 4) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 4;
INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0, 0.0]), (2, [0.0, 0.0, 10.5, 0.0]), (3, [0.0, 0.0, 9.5, 0.0]), (4, [0.0, 0.0, 9.7, 0.0]), (5, [10.0, 0.0, 0.0, 0.0]), (6, [9.5, 0.0, 0.0, 0.0]), (7, [9.7, 0.0, 0.0, 0.0]), (8, [10.2, 0.0, 0.0, 0.0]), (9, [0.0, 10.0, 0.0, 0.0]), (10, [0.0, 9.5, 0.0, 0.0]), (11, [0.0, 9.7, 0.0, 0.0]), (12, [0.0, 9.7, 0.0, 0.0]), (13, [0.0, 0.0, 0.0, 10.3]), (14, [0.0, 0.0, 0.0, 9.5]), (15, [0.0, 0.0, 0.0, 10.0]), (16, [0.0, 0.0, 0.0, 10.5]);
-- rows = 16, index_granularity = 4, GRANULARITY = 4 gives a single annoy-indexed block (comprising all granules)
-- no two matches happen to be located in the same granule, so with LIMIT = 3, we'll get "Granules: 2/4"
-- See (*) why commented out
-- SELECT 'WHERE type, L2Distance';
-- SELECT *
-- FROM tab
-- WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0
-- LIMIT 3;
SELECT 'WHERE type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0
LIMIT 3;
-- See (*) why commented out
-- SELECT 'ORDER BY type, L2Distance';
-- SELECT *
-- FROM tab
-- ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0])
-- LIMIT 3;
SELECT 'ORDER BY type, L2Distance, check that index is used';
EXPLAIN indexes=1
SELECT *
FROM tab
ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0])
LIMIT 3;
DROP TABLE tab;
-- (*) Storage and search in Annoy indexes is inherently random. Tests which check for exact row matches would be unstable. Therefore,
-- comment them out.

View File

@ -26,6 +26,7 @@ AlertManager
Alexey
AnyEvent
AppleClang
Approximative
ArrayJoin
ArrowStream
AsyncInsertCacheSize
@ -1005,6 +1006,7 @@ anyLast
anyheavy
anylast
appendTrailingCharIfAbsent
approximative
argMax
argMin
argmax
@ -2419,6 +2421,7 @@ unescaping
unhex
unicode
unidimensional
unintuitive
uniq
uniqCombined
uniqExact