mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Remove Annoy indexes
Annoy indexes fell out of favor in the community, at least when it comes to vector databases. Such indexes work okay-ish low dimensions but they suffers badly from a curse of dimensionality which makes them inapt for a high number of dimensions. Now that Annoy is gone, issue (*) also disappears and we can drop 'no-ubsan', 'no-cpu-aarch64', and 'no-asan' from tests. (*) spotify/annoy#456
This commit is contained in:
parent
7c41939921
commit
218421c255
3
.gitmodules
vendored
3
.gitmodules
vendored
@ -230,9 +230,6 @@
|
||||
[submodule "contrib/minizip-ng"]
|
||||
path = contrib/minizip-ng
|
||||
url = https://github.com/zlib-ng/minizip-ng
|
||||
[submodule "contrib/annoy"]
|
||||
path = contrib/annoy
|
||||
url = https://github.com/ClickHouse/annoy
|
||||
[submodule "contrib/qpl"]
|
||||
path = contrib/qpl
|
||||
url = https://github.com/intel/qpl
|
||||
|
1
contrib/CMakeLists.txt
vendored
1
contrib/CMakeLists.txt
vendored
@ -205,7 +205,6 @@ add_contrib (morton-nd-cmake morton-nd)
|
||||
if (ARCH_S390X)
|
||||
add_contrib(crc32-s390x-cmake crc32-s390x)
|
||||
endif()
|
||||
add_contrib (annoy-cmake annoy)
|
||||
|
||||
option(ENABLE_USEARCH "Enable USearch" ${ENABLE_LIBRARIES})
|
||||
if (ENABLE_USEARCH)
|
||||
|
1
contrib/annoy
vendored
1
contrib/annoy
vendored
@ -1 +0,0 @@
|
||||
Subproject commit f2ac8e7b48f9a9cf676d3b58286e5455aba8e956
|
@ -1,23 +0,0 @@
|
||||
option(ENABLE_ANNOY "Enable Annoy index support" ${ENABLE_LIBRARIES})
|
||||
|
||||
# Annoy index should be disabled with undefined sanitizer. Because of memory storage optimizations
|
||||
# (https://github.com/ClickHouse/annoy/blob/9d8a603a4cd252448589e84c9846f94368d5a289/src/annoylib.h#L442-L463)
|
||||
# UBSan fails and leads to crash. Simmilar issue is already opened in Annoy repo
|
||||
# https://github.com/spotify/annoy/issues/456
|
||||
# Problem with aligment can lead to errors like
|
||||
# (https://stackoverflow.com/questions/46790550/c-undefined-behavior-strict-aliasing-rule-or-incorrect-alignment)
|
||||
# or will lead to crash on arm https://developer.arm.com/documentation/ka003038/latest
|
||||
# This issues should be resolved before annoy became non-experimental (--> setting "allow_experimental_annoy_index")
|
||||
if ((NOT ENABLE_ANNOY) OR (SANITIZE STREQUAL "undefined") OR (ARCH_AARCH64))
|
||||
message (STATUS "Not using annoy")
|
||||
return()
|
||||
endif()
|
||||
|
||||
set(ANNOY_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/annoy")
|
||||
set(ANNOY_SOURCE_DIR "${ANNOY_PROJECT_DIR}/src")
|
||||
|
||||
add_library(_annoy INTERFACE)
|
||||
target_include_directories(_annoy SYSTEM INTERFACE ${ANNOY_SOURCE_DIR})
|
||||
|
||||
add_library(ch_contrib::annoy ALIAS _annoy)
|
||||
target_compile_definitions(_annoy INTERFACE ANNOYLIB_MULTITHREADED_BUILD)
|
@ -126,81 +126,8 @@ was specified for ANN indexes, the default value is 100 million.
|
||||
|
||||
# Available ANN Indexes {#available_ann_indexes}
|
||||
|
||||
- [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy)
|
||||
|
||||
- [USearch](/docs/en/engines/table-engines/mergetree-family/annindexes.md#usearch-usearch)
|
||||
|
||||
## Annoy {#annoy}
|
||||
|
||||
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 is based on the [Annoy library](https://github.com/spotify/annoy) which recursively divides the space into random
|
||||
linear surfaces (lines in 2D, planes in 3D etc.).
|
||||
|
||||
<div class='vimeo-container'>
|
||||
<iframe src="//www.youtube.com/embed/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(Float32)](../../../sql-reference/data-types/array.md) column:
|
||||
|
||||
```sql
|
||||
CREATE TABLE table_with_annoy_index
|
||||
(
|
||||
id Int64,
|
||||
vectors Array(Float32),
|
||||
INDEX [ann_index_name] vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N]
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id;
|
||||
```
|
||||
|
||||
Annoy currently supports two distance functions:
|
||||
- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space
|
||||
([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)).
|
||||
- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors
|
||||
([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)).
|
||||
|
||||
For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. 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
|
||||
All arrays must have same length. To avoid errors, you can use a
|
||||
[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT constraint_name_1 CHECK
|
||||
length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported.
|
||||
:::
|
||||
|
||||
The creation of Annoy indexes (whenever a new part is build, e.g. at the end of a merge) is a relatively slow process. You can increase
|
||||
setting `max_threads_for_annoy_index_creation` (default: 4) which controls how many threads are used to create an Annoy index. Please be
|
||||
careful with this setting, it is possible that multiple indexes are created in parallel in which case there can be overparallelization.
|
||||
|
||||
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
|
||||
ORDER BY L2Distance(vectors, Point)
|
||||
LIMIT N
|
||||
SETTINGS annoy_index_search_k_nodes=100;
|
||||
```
|
||||
|
||||
:::note
|
||||
The Annoy index currently does not work with per-table, non-default `index_granularity` settings (see
|
||||
[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml.
|
||||
:::
|
||||
|
||||
## USearch {#usearch}
|
||||
|
||||
This type of ANN index is based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW
|
||||
@ -211,6 +138,8 @@ that are expensive to load and compare. The library also has several hardware-sp
|
||||
distance computations on modern Arm (NEON and SVE) and x86 (AVX2 and AVX-512) CPUs and OS-specific optimizations to allow efficient
|
||||
navigation around immutable persistent files, without loading them into RAM.
|
||||
|
||||
USearch indexes are currently experimental, to use them you first need to `SET allow_experimental_usearch_index = 1`.
|
||||
|
||||
<div class='vimeo-container'>
|
||||
<iframe src="//www.youtube.com/embed/UMrhB3icP9w"
|
||||
width="640"
|
||||
@ -247,3 +176,15 @@ was specified during index creation, `f16` is used as default.
|
||||
|
||||
For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no
|
||||
distance function was specified during index creation, `L2Distance` is used as default.
|
||||
|
||||
:::note
|
||||
All arrays must have same length. To avoid errors, you can use a
|
||||
[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT constraint_name_1 CHECK
|
||||
length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported.
|
||||
:::
|
||||
|
||||
:::note
|
||||
The USearch index currently does not work with per-table, non-default `index_granularity` settings (see
|
||||
[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml.
|
||||
:::
|
||||
|
||||
|
@ -601,10 +601,6 @@ endif()
|
||||
|
||||
dbms_target_link_libraries(PUBLIC ch_contrib::consistent_hashing)
|
||||
|
||||
if (TARGET ch_contrib::annoy)
|
||||
dbms_target_link_libraries(PUBLIC ch_contrib::annoy)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_contrib::usearch)
|
||||
dbms_target_link_libraries(PUBLIC ch_contrib::usearch)
|
||||
endif()
|
||||
|
@ -58,7 +58,6 @@
|
||||
#cmakedefine01 USE_FILELOG
|
||||
#cmakedefine01 USE_ODBC
|
||||
#cmakedefine01 USE_BLAKE3
|
||||
#cmakedefine01 USE_ANNOY
|
||||
#cmakedefine01 USE_USEARCH
|
||||
#cmakedefine01 USE_SKIM
|
||||
#cmakedefine01 USE_PRQL
|
||||
|
@ -909,12 +909,9 @@ class IColumn;
|
||||
M(Bool, allow_experimental_time_series_table, false, "Allows experimental TimeSeries table engine", 0) \
|
||||
M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \
|
||||
M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \
|
||||
M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \
|
||||
M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \
|
||||
M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \
|
||||
M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \
|
||||
M(UInt64, max_threads_for_annoy_index_creation, 4, "Number of threads used to build Annoy indexes (0 means all cores, not recommended)", 0) \
|
||||
M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \
|
||||
M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \
|
||||
M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \
|
||||
M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \
|
||||
@ -1036,6 +1033,9 @@ class IColumn;
|
||||
MAKE_OBSOLETE(M, UInt64, parallel_replicas_min_number_of_granules_to_enable, 0) \
|
||||
MAKE_OBSOLETE(M, Bool, query_plan_optimize_projection, true) \
|
||||
MAKE_OBSOLETE(M, Bool, query_cache_store_results_of_queries_with_nondeterministic_functions, false) \
|
||||
MAKE_OBSOLETE(M, Bool, allow_experimental_annoy_index, false) \
|
||||
MAKE_OBSOLETE(M, UInt64, max_threads_for_annoy_index_creation, 4) \
|
||||
MAKE_OBSOLETE(M, Int64, annoy_index_search_k_nodes, -1) \
|
||||
MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \
|
||||
MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \
|
||||
MAKE_OBSOLETE(M, Bool, allow_experimental_s3queue, true) \
|
||||
|
@ -1153,7 +1153,6 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
query_context->setSetting("allow_experimental_object_type", 1);
|
||||
query_context->setSetting("allow_experimental_variant_type", 1);
|
||||
query_context->setSetting("allow_experimental_dynamic_type", 1);
|
||||
query_context->setSetting("allow_experimental_annoy_index", 1);
|
||||
query_context->setSetting("allow_experimental_usearch_index", 1);
|
||||
query_context->setSetting("allow_experimental_bigint_types", 1);
|
||||
query_context->setSetting("allow_experimental_window_functions", 1);
|
||||
|
@ -787,8 +787,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index)
|
||||
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Please use index type 'full_text' instead of 'inverted'");
|
||||
/// ----
|
||||
if (index_desc.type == "annoy" && !settings.allow_experimental_annoy_index)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index is disabled. Turn on allow_experimental_annoy_index");
|
||||
if (index_desc.type == "usearch" && !settings.allow_experimental_usearch_index)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index is disabled. Turn on allow_experimental_usearch_index");
|
||||
|
||||
|
@ -13,7 +13,6 @@ class ASTIndexDeclaration : public IAST
|
||||
{
|
||||
public:
|
||||
static const auto DEFAULT_INDEX_GRANULARITY = 1uz;
|
||||
static const auto DEFAULT_ANNOY_INDEX_GRANULARITY = 100'000'000uz;
|
||||
static const auto DEFAULT_USEARCH_INDEX_GRANULARITY = 100'000'000uz;
|
||||
|
||||
ASTIndexDeclaration(ASTPtr expression, ASTPtr type, const String & name_);
|
||||
|
@ -89,9 +89,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected
|
||||
else
|
||||
{
|
||||
auto index_type = index->getType();
|
||||
if (index_type && index_type->name == "annoy")
|
||||
index->granularity = ASTIndexDeclaration::DEFAULT_ANNOY_INDEX_GRANULARITY;
|
||||
else if (index_type && index_type->name == "usearch")
|
||||
if (index_type && index_type->name == "usearch")
|
||||
index->granularity = ASTIndexDeclaration::DEFAULT_USEARCH_INDEX_GRANULARITY;
|
||||
else
|
||||
index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY;
|
||||
|
@ -214,9 +214,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
else
|
||||
{
|
||||
auto index_type = index->getType();
|
||||
if (index_type->name == "annoy")
|
||||
index->granularity = ASTIndexDeclaration::DEFAULT_ANNOY_INDEX_GRANULARITY;
|
||||
else if (index_type->name == "usearch")
|
||||
if (index_type->name == "usearch")
|
||||
index->granularity = ASTIndexDeclaration::DEFAULT_USEARCH_INDEX_GRANULARITY;
|
||||
else
|
||||
index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY;
|
||||
|
@ -24,7 +24,6 @@
|
||||
#include <Processors/Transforms/SelectByIndicesTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexAnnoy.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexUSearch.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreePrefetchedReadPool.h>
|
||||
@ -1478,10 +1477,6 @@ static void buildIndexes(
|
||||
MergeTreeIndexConditionPtr condition;
|
||||
if (index_helper->isVectorSearch())
|
||||
{
|
||||
#if USE_ANNOY
|
||||
if (const auto * annoy = typeid_cast<const MergeTreeIndexAnnoy *>(index_helper.get()))
|
||||
condition = annoy->createIndexCondition(query_info, context);
|
||||
#endif
|
||||
#if USE_USEARCH
|
||||
if (const auto * usearch = typeid_cast<const MergeTreeIndexUSearch *>(index_helper.get()))
|
||||
condition = usearch->createIndexCondition(query_info, context);
|
||||
|
@ -27,7 +27,6 @@ MergeTreeWriterSettings::MergeTreeWriterSettings(
|
||||
, rewrite_primary_key(rewrite_primary_key_)
|
||||
, blocks_are_granules_size(blocks_are_granules_size_)
|
||||
, query_write_settings(query_write_settings_)
|
||||
, max_threads_for_annoy_index_creation(global_settings.max_threads_for_annoy_index_creation)
|
||||
, low_cardinality_max_dictionary_size(global_settings.low_cardinality_max_dictionary_size)
|
||||
, low_cardinality_use_single_dictionary_for_part(global_settings.low_cardinality_use_single_dictionary_for_part != 0)
|
||||
, use_compact_variant_discriminators_serialization(storage_settings->use_compact_variant_discriminators_serialization)
|
||||
|
@ -77,8 +77,6 @@ struct MergeTreeWriterSettings
|
||||
bool blocks_are_granules_size;
|
||||
WriteSettings query_write_settings;
|
||||
|
||||
size_t max_threads_for_annoy_index_creation;
|
||||
|
||||
size_t low_cardinality_max_dictionary_size;
|
||||
bool low_cardinality_use_single_dictionary_for_part;
|
||||
bool use_compact_variant_discriminators_serialization;
|
||||
|
@ -1,416 +0,0 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexAnnoy.h>
|
||||
|
||||
#if USE_ANNOY
|
||||
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int INCORRECT_NUMBER_OF_COLUMNS;
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
template <typename Distance>
|
||||
AnnoyIndexWithSerialization<Distance>::AnnoyIndexWithSerialization(size_t dimensions)
|
||||
: Base::AnnoyIndex(static_cast<int>(dimensions))
|
||||
{
|
||||
}
|
||||
|
||||
template<typename Distance>
|
||||
void AnnoyIndexWithSerialization<Distance>::serialize(WriteBuffer & ostr) const
|
||||
{
|
||||
chassert(Base::_built);
|
||||
writeIntBinary(Base::_s, ostr);
|
||||
writeIntBinary(Base::_n_items, ostr);
|
||||
writeIntBinary(Base::_n_nodes, ostr);
|
||||
writeIntBinary(Base::_nodes_size, ostr);
|
||||
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);
|
||||
}
|
||||
|
||||
template<typename Distance>
|
||||
void AnnoyIndexWithSerialization<Distance>::deserialize(ReadBuffer & istr)
|
||||
{
|
||||
chassert(!Base::_built);
|
||||
readIntBinary(Base::_s, istr);
|
||||
readIntBinary(Base::_n_items, istr);
|
||||
readIntBinary(Base::_n_nodes, istr);
|
||||
readIntBinary(Base::_nodes_size, istr);
|
||||
readIntBinary(Base::_K, istr);
|
||||
readIntBinary(Base::_seed, istr);
|
||||
readVectorBinary(Base::_roots, istr);
|
||||
Base::_nodes = realloc(Base::_nodes, Base::_s * Base::_n_nodes);
|
||||
istr.readStrict(reinterpret_cast<char *>(Base::_nodes), Base::_s * Base::_n_nodes);
|
||||
|
||||
Base::_fd = 0;
|
||||
// set flags
|
||||
Base::_loaded = false;
|
||||
Base::_verbose = false;
|
||||
Base::_on_disk = false;
|
||||
Base::_built = true;
|
||||
}
|
||||
|
||||
template<typename Distance>
|
||||
size_t AnnoyIndexWithSerialization<Distance>::getDimensions() const
|
||||
{
|
||||
return Base::get_f();
|
||||
}
|
||||
|
||||
|
||||
template <typename Distance>
|
||||
MergeTreeIndexGranuleAnnoy<Distance>::MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_)
|
||||
: index_name(index_name_)
|
||||
, index_sample_block(index_sample_block_)
|
||||
, index(nullptr)
|
||||
{}
|
||||
|
||||
template <typename Distance>
|
||||
MergeTreeIndexGranuleAnnoy<Distance>::MergeTreeIndexGranuleAnnoy(
|
||||
const String & index_name_,
|
||||
const Block & index_sample_block_,
|
||||
AnnoyIndexWithSerializationPtr<Distance> index_)
|
||||
: index_name(index_name_)
|
||||
, index_sample_block(index_sample_block_)
|
||||
, index(std::move(index_))
|
||||
{}
|
||||
|
||||
template <typename Distance>
|
||||
void MergeTreeIndexGranuleAnnoy<Distance>::serializeBinary(WriteBuffer & ostr) const
|
||||
{
|
||||
/// Number of dimensions is required in the index constructor,
|
||||
/// so it must be written and read separately from the other part
|
||||
writeIntBinary(static_cast<UInt64>(index->getDimensions()), ostr); // write dimension
|
||||
index->serialize(ostr);
|
||||
}
|
||||
|
||||
template <typename Distance>
|
||||
void MergeTreeIndexGranuleAnnoy<Distance>::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/)
|
||||
{
|
||||
UInt64 dimension;
|
||||
readIntBinary(dimension, istr);
|
||||
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(dimension);
|
||||
index->deserialize(istr);
|
||||
}
|
||||
|
||||
template <typename Distance>
|
||||
MergeTreeIndexAggregatorAnnoy<Distance>::MergeTreeIndexAggregatorAnnoy(
|
||||
const String & index_name_,
|
||||
const Block & index_sample_block_,
|
||||
UInt64 trees_,
|
||||
size_t max_threads_for_creation_)
|
||||
: index_name(index_name_)
|
||||
, index_sample_block(index_sample_block_)
|
||||
, trees(trees_)
|
||||
, max_threads_for_creation(max_threads_for_creation_)
|
||||
{}
|
||||
|
||||
template <typename Distance>
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy<Distance>::getGranuleAndReset()
|
||||
{
|
||||
int threads = (max_threads_for_creation == 0) ? -1 : static_cast<int>(max_threads_for_creation);
|
||||
/// clang-tidy reports a false positive: it considers %p with an outdated pointer in fprintf() (used by logging which we don't do) dereferencing
|
||||
index->build(static_cast<int>(trees), threads);
|
||||
auto granule = std::make_shared<MergeTreeIndexGranuleAnnoy<Distance>>(index_name, index_sample_block, index);
|
||||
index = nullptr;
|
||||
return granule;
|
||||
}
|
||||
|
||||
template <typename Distance>
|
||||
void MergeTreeIndexAggregatorAnnoy<Distance>::update(const Block & block, size_t * pos, size_t limit)
|
||||
{
|
||||
if (*pos >= block.rows())
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"The provided position is not less than the number of block rows. Position: {}, Block rows: {}.",
|
||||
*pos, block.rows());
|
||||
|
||||
size_t rows_read = std::min(limit, block.rows() - *pos);
|
||||
|
||||
if (rows_read == 0)
|
||||
return;
|
||||
|
||||
if (rows_read > std::numeric_limits<uint32_t>::max())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Index granularity is too big: more than 4B rows per index granule.");
|
||||
|
||||
if (index_sample_block.columns() > 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column");
|
||||
|
||||
const String & index_column_name = index_sample_block.getByPosition(0).name;
|
||||
ColumnPtr column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read);
|
||||
|
||||
if (const auto & column_array = typeid_cast<const ColumnArray *>(column_cut.get()))
|
||||
{
|
||||
const auto & column_array_data = column_array->getData();
|
||||
const auto & column_array_data_float = typeid_cast<const ColumnFloat32 &>(column_array_data);
|
||||
const auto & column_array_data_float_data = column_array_data_float.getData();
|
||||
|
||||
const auto & column_array_offsets = column_array->getOffsets();
|
||||
const size_t num_rows = column_array_offsets.size();
|
||||
|
||||
if (column_array->empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty");
|
||||
|
||||
/// The Annoy algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays
|
||||
/// are INSERTed into an Annoy-indexed column or if no value was specified at all in which case the arrays take on their default
|
||||
/// value which is also empty.
|
||||
if (column_array->isDefaultAt(0))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name);
|
||||
|
||||
/// Check all sizes are the same
|
||||
size_t dimension = column_array_offsets[0];
|
||||
for (size_t i = 0; i < num_rows - 1; ++i)
|
||||
if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name);
|
||||
|
||||
/// Also check that previously inserted blocks have the same size as this block.
|
||||
/// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across
|
||||
/// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42.
|
||||
if (index && index->getDimensions() != dimension)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name);
|
||||
|
||||
if (!index)
|
||||
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(dimension);
|
||||
|
||||
/// Add all rows of block
|
||||
index->add_item(index->get_n_items(), column_array_data_float_data.data());
|
||||
for (size_t current_row = 1; current_row < num_rows; ++current_row)
|
||||
index->add_item(index->get_n_items(), &column_array_data_float_data[column_array_offsets[current_row - 1]]);
|
||||
}
|
||||
else if (const auto & column_tuple = typeid_cast<const ColumnTuple *>(column_cut.get()))
|
||||
{
|
||||
const auto & column_tuple_columns = column_tuple->getColumns();
|
||||
|
||||
/// TODO check if calling index->add_item() directly on the block's tuples is faster than materializing everything
|
||||
std::vector<std::vector<Float32>> data(column_tuple->size(), std::vector<Float32>());
|
||||
for (const auto & column : column_tuple_columns)
|
||||
{
|
||||
const auto & pod_array = typeid_cast<const ColumnFloat32 *>(column.get())->getData();
|
||||
for (size_t i = 0; i < pod_array.size(); ++i)
|
||||
data[i].push_back(pod_array[i]);
|
||||
}
|
||||
|
||||
if (data.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read);
|
||||
|
||||
if (!index)
|
||||
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(data[0].size());
|
||||
|
||||
for (const auto & item : data)
|
||||
index->add_item(index->get_n_items(), item.data());
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array or Tuple column");
|
||||
|
||||
*pos += rows_read;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy(
|
||||
const IndexDescription & /*index_description*/,
|
||||
const SelectQueryInfo & query,
|
||||
const String & distance_function_,
|
||||
ContextPtr context)
|
||||
: ann_condition(query, context)
|
||||
, distance_function(distance_function_)
|
||||
, search_k(context->getSettingsRef().annoy_index_search_k_nodes)
|
||||
{}
|
||||
|
||||
bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /*idx_granule*/) const
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "mayBeTrueOnGranule is not supported for ANN skip indexes");
|
||||
}
|
||||
|
||||
bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const
|
||||
{
|
||||
return ann_condition.alwaysUnknownOrTrue(distance_function);
|
||||
}
|
||||
|
||||
std::vector<size_t> MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const
|
||||
{
|
||||
if (distance_function == DISTANCE_FUNCTION_L2)
|
||||
return getUsefulRangesImpl<Annoy::Euclidean>(idx_granule);
|
||||
else if (distance_function == DISTANCE_FUNCTION_COSINE)
|
||||
return getUsefulRangesImpl<Annoy::Angular>(idx_granule);
|
||||
std::unreachable();
|
||||
}
|
||||
|
||||
template <typename Distance>
|
||||
std::vector<size_t> MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const
|
||||
{
|
||||
const UInt64 limit = ann_condition.getLimit();
|
||||
const UInt64 index_granularity = ann_condition.getIndexGranularity();
|
||||
const std::optional<float> comparison_distance = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where
|
||||
? std::optional<float>(ann_condition.getComparisonDistanceForWhereQuery())
|
||||
: std::nullopt;
|
||||
|
||||
if (comparison_distance && comparison_distance.value() < 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance");
|
||||
|
||||
const std::vector<float> reference_vector = ann_condition.getReferenceVector();
|
||||
|
||||
const auto granule = std::dynamic_pointer_cast<MergeTreeIndexGranuleAnnoy<Distance>>(idx_granule);
|
||||
if (granule == nullptr)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type");
|
||||
|
||||
const AnnoyIndexWithSerializationPtr<Distance> annoy = granule->index;
|
||||
|
||||
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.getDimensions(), annoy->getDimensions());
|
||||
|
||||
std::vector<UInt64> neighbors; /// indexes of dots which were closest to the reference vector
|
||||
std::vector<Float32> distances;
|
||||
neighbors.reserve(limit);
|
||||
distances.reserve(limit);
|
||||
|
||||
annoy->get_nns_by_vector(reference_vector.data(), limit, static_cast<int>(search_k), &neighbors, &distances);
|
||||
|
||||
chassert(neighbors.size() == distances.size());
|
||||
|
||||
std::vector<size_t> granules;
|
||||
granules.reserve(neighbors.size());
|
||||
for (size_t i = 0; i < neighbors.size(); ++i)
|
||||
{
|
||||
if (comparison_distance && distances[i] > comparison_distance)
|
||||
continue;
|
||||
granules.push_back(neighbors[i] / index_granularity);
|
||||
}
|
||||
|
||||
/// make unique
|
||||
std::sort(granules.begin(), granules.end());
|
||||
granules.erase(std::unique(granules.begin(), granules.end()), granules.end());
|
||||
|
||||
return granules;
|
||||
}
|
||||
|
||||
MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 trees_, const String & distance_function_)
|
||||
: IMergeTreeIndex(index_)
|
||||
, trees(trees_)
|
||||
, distance_function(distance_function_)
|
||||
{}
|
||||
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const
|
||||
{
|
||||
if (distance_function == DISTANCE_FUNCTION_L2)
|
||||
return std::make_shared<MergeTreeIndexGranuleAnnoy<Annoy::Euclidean>>(index.name, index.sample_block);
|
||||
else if (distance_function == DISTANCE_FUNCTION_COSINE)
|
||||
return std::make_shared<MergeTreeIndexGranuleAnnoy<Annoy::Angular>>(index.name, index.sample_block);
|
||||
std::unreachable();
|
||||
}
|
||||
|
||||
MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator(const MergeTreeWriterSettings & settings) const
|
||||
{
|
||||
/// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171
|
||||
if (distance_function == DISTANCE_FUNCTION_L2)
|
||||
return std::make_shared<MergeTreeIndexAggregatorAnnoy<Annoy::Euclidean>>(index.name, index.sample_block, trees, settings.max_threads_for_annoy_index_creation);
|
||||
else if (distance_function == DISTANCE_FUNCTION_COSINE)
|
||||
return std::make_shared<MergeTreeIndexAggregatorAnnoy<Annoy::Angular>>(index.name, index.sample_block, trees, settings.max_threads_for_annoy_index_creation);
|
||||
std::unreachable();
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexConditionAnnoy>(index, query, distance_function, context);
|
||||
};
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAG *, ContextPtr) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG");
|
||||
}
|
||||
|
||||
MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index)
|
||||
{
|
||||
static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2;
|
||||
String distance_function = DEFAULT_DISTANCE_FUNCTION;
|
||||
if (!index.arguments.empty())
|
||||
distance_function = index.arguments[0].safeGet<String>();
|
||||
|
||||
static constexpr auto DEFAULT_TREES = 100uz;
|
||||
UInt64 trees = DEFAULT_TREES;
|
||||
if (index.arguments.size() > 1)
|
||||
trees = index.arguments[1].safeGet<UInt64>();
|
||||
|
||||
return std::make_shared<MergeTreeIndexAnnoy>(index, trees, distance_function);
|
||||
}
|
||||
|
||||
void annoyIndexValidator(const IndexDescription & index, bool /* attach */)
|
||||
{
|
||||
/// Check number and type of Annoy index arguments:
|
||||
|
||||
if (index.arguments.size() > 2)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters");
|
||||
|
||||
if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String)
|
||||
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 of type UInt64");
|
||||
|
||||
/// Check that the index is created on a single column
|
||||
|
||||
if (index.column_names.size() != 1 || index.data_types.size() != 1)
|
||||
throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column");
|
||||
|
||||
/// Check that a supported metric was passed as first argument
|
||||
|
||||
if (!index.arguments.empty())
|
||||
{
|
||||
String distance_name = index.arguments[0].safeGet<String>();
|
||||
if (distance_name != DISTANCE_FUNCTION_L2 && distance_name != DISTANCE_FUNCTION_COSINE)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE);
|
||||
}
|
||||
|
||||
/// Check data type of indexed column:
|
||||
|
||||
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[, Float32[, ...]])");
|
||||
};
|
||||
|
||||
DataTypePtr data_type = index.sample_block.getDataTypes()[0];
|
||||
|
||||
if (const auto * data_type_array = typeid_cast<const DataTypeArray *>(data_type.get()))
|
||||
{
|
||||
TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId();
|
||||
if (!WhichDataType(nested_type_index).isFloat32())
|
||||
throw_unsupported_underlying_column_exception();
|
||||
}
|
||||
else if (const auto * data_type_tuple = typeid_cast<const DataTypeTuple *>(data_type.get()))
|
||||
{
|
||||
const DataTypes & inner_types = data_type_tuple->getElements();
|
||||
for (const auto & inner_type : inner_types)
|
||||
{
|
||||
TypeIndex nested_type_index = inner_type->getTypeId();
|
||||
if (!WhichDataType(nested_type_index).isFloat32())
|
||||
throw_unsupported_underlying_column_exception();
|
||||
}
|
||||
}
|
||||
else
|
||||
throw_unsupported_underlying_column_exception();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -1,114 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if USE_ANNOY
|
||||
|
||||
#include <Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h>
|
||||
|
||||
#include <annoylib.h>
|
||||
#include <kissrandom.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename Distance>
|
||||
class AnnoyIndexWithSerialization : public Annoy::AnnoyIndex<UInt64, Float32, Distance, Annoy::Kiss64Random, Annoy::AnnoyIndexMultiThreadedBuildPolicy>
|
||||
{
|
||||
using Base = Annoy::AnnoyIndex<UInt64, Float32, Distance, Annoy::Kiss64Random, Annoy::AnnoyIndexMultiThreadedBuildPolicy>;
|
||||
|
||||
public:
|
||||
explicit AnnoyIndexWithSerialization(size_t dimensions);
|
||||
void serialize(WriteBuffer & ostr) const;
|
||||
void deserialize(ReadBuffer & istr);
|
||||
size_t getDimensions() const;
|
||||
};
|
||||
|
||||
template <typename Distance>
|
||||
using AnnoyIndexWithSerializationPtr = std::shared_ptr<AnnoyIndexWithSerialization<Distance>>;
|
||||
|
||||
|
||||
template <typename Distance>
|
||||
struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule
|
||||
{
|
||||
MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_);
|
||||
MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexWithSerializationPtr<Distance> index_);
|
||||
|
||||
~MergeTreeIndexGranuleAnnoy() override = default;
|
||||
|
||||
void serializeBinary(WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override;
|
||||
|
||||
bool empty() const override { return !index.get(); }
|
||||
|
||||
const String index_name;
|
||||
const Block index_sample_block;
|
||||
AnnoyIndexWithSerializationPtr<Distance> index;
|
||||
};
|
||||
|
||||
|
||||
template <typename Distance>
|
||||
struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator
|
||||
{
|
||||
MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, UInt64 trees, size_t max_threads_for_creation);
|
||||
~MergeTreeIndexAggregatorAnnoy() override = default;
|
||||
|
||||
bool empty() const override { return !index || index->get_n_items() == 0; }
|
||||
MergeTreeIndexGranulePtr getGranuleAndReset() override;
|
||||
void update(const Block & block, size_t * pos, size_t limit) override;
|
||||
|
||||
const String index_name;
|
||||
const Block index_sample_block;
|
||||
const UInt64 trees;
|
||||
const size_t max_threads_for_creation;
|
||||
AnnoyIndexWithSerializationPtr<Distance> index;
|
||||
};
|
||||
|
||||
|
||||
class MergeTreeIndexConditionAnnoy final : public IMergeTreeIndexConditionApproximateNearestNeighbor
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexConditionAnnoy(
|
||||
const IndexDescription & index_description,
|
||||
const SelectQueryInfo & query,
|
||||
const String & distance_function,
|
||||
ContextPtr context);
|
||||
|
||||
~MergeTreeIndexConditionAnnoy() override = default;
|
||||
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override;
|
||||
std::vector<size_t> getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const override;
|
||||
|
||||
private:
|
||||
template <typename Distance>
|
||||
std::vector<size_t> getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const;
|
||||
|
||||
const ApproximateNearestNeighborCondition ann_condition;
|
||||
const String distance_function;
|
||||
const Int64 search_k;
|
||||
};
|
||||
|
||||
|
||||
class MergeTreeIndexAnnoy final : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
|
||||
MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 trees_, const String & distance_function_);
|
||||
|
||||
~MergeTreeIndexAnnoy() override = default;
|
||||
|
||||
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG *, ContextPtr) const override;
|
||||
bool isVectorSearch() const override { return true; }
|
||||
|
||||
private:
|
||||
const UInt64 trees;
|
||||
const String distance_function;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -127,10 +127,6 @@ MergeTreeIndexFactory::MergeTreeIndexFactory()
|
||||
registerCreator("hypothesis", hypothesisIndexCreator);
|
||||
|
||||
registerValidator("hypothesis", hypothesisIndexValidator);
|
||||
#if USE_ANNOY
|
||||
registerCreator("annoy", annoyIndexCreator);
|
||||
registerValidator("annoy", annoyIndexValidator);
|
||||
#endif
|
||||
|
||||
#if USE_USEARCH
|
||||
registerCreator("usearch", usearchIndexCreator);
|
||||
|
@ -231,11 +231,6 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool attach);
|
||||
MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index);
|
||||
void hypothesisIndexValidator(const IndexDescription & index, bool attach);
|
||||
|
||||
#if USE_ANNOY
|
||||
MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index);
|
||||
void annoyIndexValidator(const IndexDescription & index, bool attach);
|
||||
#endif
|
||||
|
||||
#if USE_USEARCH
|
||||
MergeTreeIndexPtr usearchIndexCreator(const IndexDescription& index);
|
||||
void usearchIndexValidator(const IndexDescription& index, bool attach);
|
||||
|
@ -164,9 +164,6 @@ endif()
|
||||
if (TARGET ch_contrib::bcrypt)
|
||||
set(USE_BCRYPT 1)
|
||||
endif()
|
||||
if (TARGET ch_contrib::annoy)
|
||||
set(USE_ANNOY 1)
|
||||
endif()
|
||||
if (TARGET ch_contrib::usearch)
|
||||
set(USE_USEARCH 1)
|
||||
endif()
|
||||
|
@ -1,16 +1,6 @@
|
||||
Issue #52258: Empty Arrays or Arrays with default values are rejected
|
||||
- Annoy
|
||||
- Usearch
|
||||
It is possible to create parts with different Array vector sizes but there will be an error at query time
|
||||
- Annoy
|
||||
- Usearch
|
||||
Correctness of index with > 1 mark
|
||||
- Annoy
|
||||
1 [1,0] 0
|
||||
9000 [9000,0] 0
|
||||
1 (1,0) 0
|
||||
9000 (9000,0) 0
|
||||
- Usearch
|
||||
1 [1,0] 0
|
||||
9000 [9000,0] 0
|
||||
1 (1,0) 0
|
||||
|
@ -1,11 +1,9 @@
|
||||
-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-ordinary-database, no-asan
|
||||
-- Tags: no-fasttest, no-ordinary-database
|
||||
|
||||
-- Tests vector search in ClickHouse, i.e. Annoy and Usearch indexes. Both index types share similarities in implementation and usage,
|
||||
-- therefore they are tested in a single file.
|
||||
-- Tests vector search in ClickHouse, i.e. Usearch indexes.
|
||||
|
||||
-- This file contains tests for various bugs and special cases
|
||||
|
||||
SET allow_experimental_annoy_index = 1;
|
||||
SET allow_experimental_usearch_index = 1;
|
||||
|
||||
SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof
|
||||
@ -14,19 +12,6 @@ DROP TABLE IF EXISTS tab;
|
||||
|
||||
SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected';
|
||||
|
||||
SELECT '- Annoy';
|
||||
|
||||
CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE annoy()) ENGINE = MergeTree() ORDER BY (id);
|
||||
INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA }
|
||||
INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA }
|
||||
DROP TABLE tab;
|
||||
|
||||
CREATE TABLE tab (id UInt64, vec Tuple(Float32, Float32), INDEX idx vec TYPE annoy()) ENGINE = MergeTree() ORDER BY (id);
|
||||
INSERT INTO tab (id) VALUES (1); -- works fine, takes on default tuple (0.0, 0.0)
|
||||
DROP TABLE tab;
|
||||
|
||||
SELECT '- Usearch';
|
||||
|
||||
CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree() ORDER BY (id);
|
||||
INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA }
|
||||
INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA }
|
||||
@ -38,23 +23,6 @@ DROP TABLE tab;
|
||||
|
||||
SELECT 'It is possible to create parts with different Array vector sizes but there will be an error at query time';
|
||||
|
||||
SELECT '- Annoy';
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id;
|
||||
SYSTEM STOP MERGES tab;
|
||||
INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]);
|
||||
INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]);
|
||||
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3; -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
|
||||
|
||||
DROP TABLE tab;
|
||||
|
||||
SELECT '- Usearch';
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id;
|
||||
SYSTEM STOP MERGES tab;
|
||||
INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]);
|
||||
@ -70,45 +38,6 @@ DROP TABLE tab;
|
||||
|
||||
SELECT 'Correctness of index with > 1 mark';
|
||||
|
||||
SELECT '- Annoy';
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy()) 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
|
||||
INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000);
|
||||
|
||||
WITH [1.0, 0.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 1;
|
||||
|
||||
WITH [9000.0, 0.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 1;
|
||||
|
||||
DROP TABLE tab;
|
||||
|
||||
-- same, but with Tuples
|
||||
CREATE TABLE tab(id Int32, vec Tuple(Float32, Float32), INDEX idx vec TYPE annoy()) 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
|
||||
INSERT INTO tab SELECT number, (toFloat32(number), 0.0) from numbers(10000);
|
||||
|
||||
WITH (1.0, 0.0) AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 1;
|
||||
|
||||
WITH (9000.0, 0.0) AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 1;
|
||||
|
||||
DROP TABLE tab;
|
||||
|
||||
SELECT '- Usearch';
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) 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
|
||||
INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000);
|
||||
|
||||
|
@ -1,7 +1,3 @@
|
||||
Test the default index granularity for vector search indexes (CREATE TABLE AND ALTER TABLE), should be 100 million for Annoy and USearch
|
||||
- Annoy
|
||||
100000000
|
||||
100000000
|
||||
- Usearch
|
||||
Test the default index granularity for vector search indexes (CREATE TABLE AND ALTER TABLE), should be 100 million for USearch
|
||||
100000000
|
||||
100000000
|
||||
|
@ -1,29 +1,14 @@
|
||||
-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-ordinary-database, no-asan
|
||||
-- Tags: no-fasttest, no-ordinary-database
|
||||
|
||||
-- Tests vector search in ClickHouse, i.e. Annoy and Usearch indexes. Both index types share similarities in implementation and usage,
|
||||
-- therefore they are tested in a single file.
|
||||
-- Tests vector search in ClickHouse, i.e. Usearch indexes.
|
||||
|
||||
-- This file contains tests for the non-standard default granularity of vector search indexes.
|
||||
|
||||
SET allow_experimental_annoy_index = 1;
|
||||
SET allow_experimental_usearch_index = 1;
|
||||
|
||||
SELECT 'Test the default index granularity for vector search indexes (CREATE TABLE AND ALTER TABLE), should be 100 million for Annoy and USearch';
|
||||
|
||||
SELECT '- Annoy';
|
||||
SELECT 'Test the default index granularity for vector search indexes (CREATE TABLE AND ALTER TABLE), should be 100 million for USearch';
|
||||
|
||||
DROP TABLE IF EXISTS tab;
|
||||
CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE annoy) ENGINE=MergeTree ORDER BY id;
|
||||
SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx';
|
||||
|
||||
DROP TABLE tab;
|
||||
CREATE TABLE tab (id Int32, vec Array(Float32)) ENGINE=MergeTree ORDER BY id;
|
||||
ALTER TABLE tab ADD INDEX idx(vec) TYPE annoy;
|
||||
SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx';
|
||||
|
||||
SELECT '- Usearch';
|
||||
|
||||
DROP TABLE tab;
|
||||
CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE usearch) ENGINE=MergeTree ORDER BY id;
|
||||
SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx';
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
At most two index arguments
|
||||
1st argument (distance function) must be String
|
||||
Rejects unsupported distance functions
|
||||
2nd argument (Annoy: number of trees, USearch: scalar kind) must be UInt64 (Annoy) / String (Usearch)
|
||||
2nd argument (scalar kind) must be String
|
||||
Rejects unsupported scalar kinds (only Usearch)
|
||||
Must be created on single column
|
||||
Must be created on Array(Float32) or Tuple(Float32, Float, ...) columns
|
||||
|
@ -1,63 +1,47 @@
|
||||
-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-ordinary-database, no-asan
|
||||
-- Tags: no-fasttest, no-ordinary-database
|
||||
|
||||
-- Tests vector search in ClickHouse, i.e. Annoy and Usearch indexes. Both index types share similarities in implementation and usage,
|
||||
-- therefore they are tested in a single file.
|
||||
-- Tests vector search in ClickHouse, i.e. Usearch indexes.
|
||||
|
||||
-- This file tests that various conditions are checked during creation of vector search indexes.
|
||||
|
||||
SET allow_experimental_annoy_index = 1;
|
||||
SET allow_experimental_usearch_index = 1;
|
||||
|
||||
DROP TABLE IF EXISTS tab;
|
||||
|
||||
SELECT 'At most two index arguments';
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('too', 'many', 'args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
|
||||
|
||||
SELECT '1st argument (distance function) must be String';
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
|
||||
|
||||
SELECT 'Rejects unsupported distance functions';
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy('invalidDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('invalidDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
|
||||
|
||||
SELECT '2nd argument (Annoy: number of trees, USearch: scalar kind) must be UInt64 (Annoy) / String (Usearch)';
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
|
||||
SELECT '2nd argument (scalar kind) must be String';
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
|
||||
|
||||
SELECT 'Rejects unsupported scalar kinds (only Usearch)';
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('L2Distance', 'invalidKind')) 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 annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS }
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS }
|
||||
|
||||
SELECT 'Must be created on Array(Float32) or Tuple(Float32, Float, ...) columns';
|
||||
|
||||
SET allow_suspicious_low_cardinality_types = 1;
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Tuple(Float64), INDEX idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
CREATE TABLE tab(id Int32, vec Tuple(Float64), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
|
||||
CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
|
||||
SELECT 'Rejects INSERTs of Arrays with different sizes';
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id;
|
||||
INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA }
|
||||
DROP TABLE tab;
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id;
|
||||
INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA }
|
||||
DROP TABLE tab;
|
||||
|
@ -1,12 +1,4 @@
|
||||
ARRAY, 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block
|
||||
- Annoy: WHERE-type
|
||||
5 [0,2] 0
|
||||
6 [0,2.1] 0.09999990463256836
|
||||
7 [0,2.2] 0.20000004768371582
|
||||
- Annoy: ORDER-BY-type
|
||||
5 [0,2] 0
|
||||
6 [0,2.1] 0.09999990463256836
|
||||
7 [0,2.2] 0.20000004768371582
|
||||
- Usearch: WHERE-type
|
||||
5 [0,2] 0
|
||||
6 [0,2.1] 0.09999990463256836
|
||||
@ -15,37 +7,6 @@ ARRAY, 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule,
|
||||
5 [0,2] 0
|
||||
6 [0,2.1] 0.09999990463256836
|
||||
7 [0,2.2] 0.20000004768371582
|
||||
- Annoy: WHERE-type, EXPLAIN
|
||||
Expression ((Projection + Before ORDER BY))
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Expression
|
||||
ReadFromMergeTree (default.tab_annoy)
|
||||
Indexes:
|
||||
PrimaryKey
|
||||
Condition: true
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
Skip
|
||||
Name: idx
|
||||
Description: annoy GRANULARITY 100000000
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
- Annoy: ORDER-BY-type, EXPLAIN
|
||||
Expression (Projection)
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
ReadFromMergeTree (default.tab_annoy)
|
||||
Indexes:
|
||||
PrimaryKey
|
||||
Condition: true
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
Skip
|
||||
Name: idx
|
||||
Description: annoy GRANULARITY 100000000
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
- Usearch: WHERE-type, EXPLAIN
|
||||
Expression ((Projection + Before ORDER BY))
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
@ -78,14 +39,6 @@ Expression (Projection)
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
ARRAY vectors, 12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block
|
||||
- Annoy: WHERE-type
|
||||
6 [0,2] 0
|
||||
7 [0,2.1] 0.09999990463256836
|
||||
8 [0,2.2] 0.20000004768371582
|
||||
- Annoy: ORDER-BY-type
|
||||
6 [0,2] 0
|
||||
7 [0,2.1] 0.09999990463256836
|
||||
8 [0,2.2] 0.20000004768371582
|
||||
- Usearch: WHERE-type
|
||||
6 [0,2] 0
|
||||
7 [0,2.1] 0.09999990463256836
|
||||
@ -94,37 +47,6 @@ ARRAY vectors, 12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2
|
||||
6 [0,2] 0
|
||||
7 [0,2.1] 0.09999990463256836
|
||||
8 [0,2.2] 0.20000004768371582
|
||||
- Annoy: WHERE-type, EXPLAIN
|
||||
Expression ((Projection + Before ORDER BY))
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Expression
|
||||
ReadFromMergeTree (default.tab_annoy)
|
||||
Indexes:
|
||||
PrimaryKey
|
||||
Condition: true
|
||||
Parts: 1/1
|
||||
Granules: 4/4
|
||||
Skip
|
||||
Name: idx
|
||||
Description: annoy GRANULARITY 2
|
||||
Parts: 1/1
|
||||
Granules: 1/4
|
||||
- Annoy: ORDER-BY-type, EXPLAIN
|
||||
Expression (Projection)
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
ReadFromMergeTree (default.tab_annoy)
|
||||
Indexes:
|
||||
PrimaryKey
|
||||
Condition: true
|
||||
Parts: 1/1
|
||||
Granules: 4/4
|
||||
Skip
|
||||
Name: idx
|
||||
Description: annoy GRANULARITY 2
|
||||
Parts: 1/1
|
||||
Granules: 2/4
|
||||
- Usearch: WHERE-type, EXPLAIN
|
||||
Expression ((Projection + Before ORDER BY))
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
@ -157,14 +79,6 @@ Expression (Projection)
|
||||
Parts: 1/1
|
||||
Granules: 2/4
|
||||
TUPLE vectors and special cases
|
||||
- Annoy: WHERE-type
|
||||
0 (4.6,2.3) 0.5527864045000421
|
||||
1 (2,3.2) 0.15200169244542905
|
||||
2 (4.2,3.4) 0.37080174340866845
|
||||
- Annoy: ORDER-BY-type
|
||||
6 (1,9.3) 0.005731362878640178
|
||||
1 (2,3.2) 0.15200169244542905
|
||||
7 (5.5,4.7) 0.3503476876550442
|
||||
- Usearch: WHERE-type
|
||||
0 (4.6,2.3) 0.5527864045000421
|
||||
1 (2,3.2) 0.15200169244542905
|
||||
@ -174,19 +88,6 @@ TUPLE vectors and special cases
|
||||
1 (2,3.2) 0.15200169244542905
|
||||
7 (5.5,4.7) 0.3503476876550442
|
||||
- Special case: MaximumDistance is negative
|
||||
- Special case: MaximumDistance is negative
|
||||
- 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_annoy)
|
||||
Indexes:
|
||||
PrimaryKey
|
||||
Condition: true
|
||||
Parts: 1/1
|
||||
Granules: 4/4
|
||||
- Special case: setting "max_limit_for_ann_queries"
|
||||
Expression (Projection)
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
|
@ -1,41 +1,21 @@
|
||||
-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-ordinary-database, no-asan
|
||||
-- Tags: no-fasttest, no-ordinary-database
|
||||
|
||||
-- Tests vector search in ClickHouse, i.e. Annoy and Usearch indexes. Both index types share similarities in implementation and usage,
|
||||
-- therefore they are tested in a single file.
|
||||
-- Tests vector search in ClickHouse, i.e. Usearch indexes.
|
||||
|
||||
-- This file tests various simple approximate nearest neighborhood (ANN) queries that utilize vector search indexes.
|
||||
|
||||
SET allow_experimental_annoy_index = 1;
|
||||
SET allow_experimental_usearch_index = 1;
|
||||
|
||||
SET enable_analyzer = 0;
|
||||
|
||||
SELECT 'ARRAY, 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block';
|
||||
|
||||
DROP TABLE IF EXISTS tab_annoy;
|
||||
DROP TABLE IF EXISTS tab_usearch;
|
||||
|
||||
CREATE TABLE tab_annoy(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192;
|
||||
INSERT INTO tab_annoy VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]);
|
||||
|
||||
CREATE TABLE tab_usearch(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192;
|
||||
INSERT INTO tab_usearch VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]);
|
||||
|
||||
|
||||
SELECT '- Annoy: WHERE-type';
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
WHERE L2Distance(vec, reference_vec) < 1.0
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Annoy: ORDER-BY-type';
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Usearch: WHERE-type';
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
@ -50,22 +30,6 @@ FROM tab_usearch
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Annoy: WHERE-type, EXPLAIN';
|
||||
EXPLAIN indexes=1
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
WHERE L2Distance(vec, reference_vec) < 1.0
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Annoy: ORDER-BY-type, EXPLAIN';
|
||||
EXPLAIN indexes=1
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Usearch: WHERE-type, EXPLAIN';
|
||||
EXPLAIN indexes=1
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
@ -82,32 +46,14 @@ FROM tab_usearch
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
DROP TABLE tab_annoy;
|
||||
DROP TABLE tab_usearch;
|
||||
|
||||
|
||||
SELECT 'ARRAY vectors, 12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block';
|
||||
|
||||
CREATE TABLE tab_annoy(id Int32, vec Array(Float32), INDEX idx vec TYPE annoy() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
|
||||
INSERT INTO tab_annoy VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [1.5, 0.0]), (6, [0.0, 2.0]), (7, [0.0, 2.1]), (8, [0.0, 2.2]), (9, [0.0, 2.3]), (10, [0.0, 2.4]), (11, [0.0, 2.5]);
|
||||
|
||||
CREATE TABLE tab_usearch(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
|
||||
INSERT INTO tab_usearch VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [1.5, 0.0]), (6, [0.0, 2.0]), (7, [0.0, 2.1]), (8, [0.0, 2.2]), (9, [0.0, 2.3]), (10, [0.0, 2.4]), (11, [0.0, 2.5]);
|
||||
|
||||
SELECT '- Annoy: WHERE-type';
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
WHERE L2Distance(vec, reference_vec) < 1.0
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Annoy: ORDER-BY-type';
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Usearch: WHERE-type';
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
@ -122,22 +68,6 @@ FROM tab_usearch
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Annoy: WHERE-type, EXPLAIN';
|
||||
EXPLAIN indexes=1
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
WHERE L2Distance(vec, reference_vec) < 1.0
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Annoy: ORDER-BY-type, EXPLAIN';
|
||||
EXPLAIN indexes=1
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Usearch: WHERE-type, EXPLAIN';
|
||||
EXPLAIN indexes=1
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
@ -154,34 +84,16 @@ FROM tab_usearch
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
DROP TABLE tab_annoy;
|
||||
DROP TABLE tab_usearch;
|
||||
|
||||
|
||||
SELECT 'TUPLE vectors and special cases';
|
||||
-- Not a systematic test, just to check that no bad things happen.
|
||||
-- Just for jun, use metric = 'cosineDistance' (Annoy/Usearch), tree_count = 200 (Annoy), scalarKind = 'f64' (Usearch)
|
||||
|
||||
CREATE TABLE tab_annoy(id Int32, vec Tuple(Float32, Float32), INDEX idx vec TYPE annoy('cosineDistance', 200) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
|
||||
INSERT INTO tab_annoy 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));
|
||||
-- Just for jun, use metric = 'cosineDistance', scalarKind = 'f64'
|
||||
|
||||
CREATE TABLE tab_usearch(id Int32, vec Tuple(Float32, Float32), INDEX idx vec TYPE usearch('cosineDistance', 'f64') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
|
||||
INSERT INTO tab_usearch 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));
|
||||
|
||||
SELECT '- Annoy: WHERE-type';
|
||||
WITH (0.0, 2.0) AS reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
WHERE cosineDistance(vec, reference_vec) < 1.0
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Annoy: ORDER-BY-type';
|
||||
WITH (0.0, 2.0) AS reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
ORDER BY cosineDistance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Usearch: WHERE-type';
|
||||
WITH (0.0, 2.0) AS reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
@ -196,13 +108,6 @@ FROM tab_usearch
|
||||
ORDER BY cosineDistance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Special case: MaximumDistance is negative';
|
||||
WITH (0.0, 2.0) as reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
WHERE cosineDistance(vec, reference_vec) < -1.0
|
||||
LIMIT 3; -- { serverError INCORRECT_QUERY }
|
||||
|
||||
SELECT '- Special case: MaximumDistance is negative';
|
||||
WITH (0.0, 2.0) as reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
@ -210,23 +115,6 @@ FROM tab_usearch
|
||||
WHERE cosineDistance(vec, reference_vec) < -1.0
|
||||
LIMIT 3; -- { serverError INCORRECT_QUERY }
|
||||
|
||||
SELECT '- Special case: setting "annoy_index_search_k_nodes"';
|
||||
WITH (0.0, 2.0) as reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
ORDER BY cosineDistance(vec, reference_vec)
|
||||
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
|
||||
WITH (0.0, 2.0) as reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
FROM tab_annoy
|
||||
ORDER BY cosineDistance(vec, reference_vec)
|
||||
LIMIT 3
|
||||
SETTINGS max_limit_for_ann_queries=2; -- LIMIT 3 > 2 --> don't use the ann index
|
||||
|
||||
SELECT '- Special case: setting "max_limit_for_ann_queries"';
|
||||
EXPLAIN indexes=1
|
||||
WITH (0.0, 2.0) as reference_vec
|
||||
@ -236,5 +124,4 @@ ORDER BY cosineDistance(vec, reference_vec)
|
||||
LIMIT 3
|
||||
SETTINGS max_limit_for_ann_queries=2; -- LIMIT 3 > 2 --> don't use the ann index
|
||||
|
||||
DROP TABLE tab_annoy;
|
||||
DROP TABLE tab_usearch;
|
||||
|
Loading…
Reference in New Issue
Block a user