From 8997464867be3b732d0eb719fc14c892950dec1b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 31 Aug 2023 05:53:23 +0000 Subject: [PATCH 1/2] Small usearch index improvements: metrics and f16 --- contrib/usearch | 2 +- .../mergetree-family/annindexes.md | 8 +- src/Common/ProfileEvents.cpp | 7 ++ .../MergeTree/MergeTreeIndexAnnoy.cpp | 3 + .../MergeTree/MergeTreeIndexUSearch.cpp | 114 ++++++++++++++---- .../MergeTree/MergeTreeIndexUSearch.h | 19 +-- .../0_stateless/02354_usearch_index.reference | 4 + .../0_stateless/02354_usearch_index.sql | 18 ++- 8 files changed, 139 insertions(+), 36 deletions(-) diff --git a/contrib/usearch b/contrib/usearch index f942b6f334b3..955c6f9c11ad 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit f942b6f334b31716f9bdb02eb6a25fa6b222f5ba +Subproject commit 955c6f9c11adfd89c912e0d1643d160b4e9e543f diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index b75a49d1cd11..f928fa47fee3 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -252,7 +252,7 @@ CREATE TABLE table_with_usearch_index ( id Int64, vectors Array(Float32), - INDEX [ann_index_name] vectors TYPE usearch([Distance]) [GRANULARITY N] + INDEX [ann_index_name] vectors TYPE usearch([Distance[, ScalarKind]]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; @@ -265,7 +265,7 @@ CREATE TABLE table_with_usearch_index ( id Int64, vectors Tuple(Float32[, Float32[, ...]]), - INDEX [ann_index_name] vectors TYPE usearch([Distance]) [GRANULARITY N] + INDEX [ann_index_name] vectors TYPE usearch([Distance[, ScalarKind]]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; @@ -277,5 +277,7 @@ USearch currently supports two distance functions: - `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). +USearch allows storing the vectors in reduced precision formats. Supported scarar kinds are `f64`, `f32`, `f16` or `i8` (default: `f16`). + 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. \ No newline at end of file +distance function was specified during index creation, `L2Distance` is used as default. diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 670c7990ac2c..fd7b47b4f876 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -253,6 +253,13 @@ The server successfully detected this situation and will download merged part fr M(PolygonsAddedToPool, "A polygon has been added to the cache (pool) for the 'pointInPolygon' function.") \ M(PolygonsInPoolAllocatedBytes, "The number of bytes for polygons added to the cache (pool) for the 'pointInPolygon' function.") \ \ + M(USearchAddCount, "Number of vectors added to usearch indexes.") \ + M(USearchAddVisitedMembers, "Number of nodes visited when adding vectors to usearch indexes.") \ + M(USearchAddComputedDistances, "Number of times distance was computed when adding vectors to usearch indexes.") \ + M(USearchSearchCount, "Number of search operations performed in usearch indexes.") \ + M(USearchSearchVisitedMembers, "Number of nodes visited when searching in usearch indexes.") \ + M(USearchSearchComputedDistances, "Number of times distance was computed when searching usearch indexes.") \ + \ M(RWLockAcquiredReadLocks, "Number of times a read lock was acquired (in a heavy RWLock).") \ M(RWLockAcquiredWriteLocks, "Number of times a write lock was acquired (in a heavy RWLock).") \ M(RWLockReadersWaitMilliseconds, "Total time spent waiting for a read lock to be acquired (in a heavy RWLock).") \ diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index f6e3f310ad9a..3ad4f81716e8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -143,6 +143,9 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (rows_read == 0) return; + if (rows_read > std::numeric_limits::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"); diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index 70e2b8f76dfd..1ab85e6bbaf9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -15,6 +15,16 @@ #include #include +namespace ProfileEvents +{ + extern const Event USearchAddCount; + extern const Event USearchAddVisitedMembers; + extern const Event USearchAddComputedDistances; + extern const Event USearchSearchCount; + extern const Event USearchSearchVisitedMembers; + extern const Event USearchSearchComputedDistances; +} + namespace DB { @@ -28,9 +38,20 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + +std::unordered_map nameToScalarKind = { + {"f64", unum::usearch::scalar_kind_t::f64_k}, + {"f32", unum::usearch::scalar_kind_t::f32_k}, + {"f16", unum::usearch::scalar_kind_t::f16_k}, + {"i8", unum::usearch::scalar_kind_t::i8_k}}; + +} + template -USearchIndexWithSerialization::USearchIndexWithSerialization(size_t dimensions) - : Base(Base::make(unum::usearch::metric_punned_t(dimensions, Metric))) +USearchIndexWithSerialization::USearchIndexWithSerialization(size_t dimensions, unum::usearch::scalar_kind_t scalar_kind) + : Base(Base::make(unum::usearch::metric_punned_t(dimensions, Metric, scalar_kind))) { } @@ -67,9 +88,11 @@ size_t USearchIndexWithSerialization::getDimensions() const template MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( const String & index_name_, - const Block & index_sample_block_) + const Block & index_sample_block_, + unum::usearch::scalar_kind_t scalar_kind_) : index_name(index_name_) , index_sample_block(index_sample_block_) + , scalar_kind(scalar_kind_) , index(nullptr) { } @@ -78,9 +101,11 @@ template MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( const String & index_name_, const Block & index_sample_block_, + unum::usearch::scalar_kind_t scalar_kind_, USearchIndexWithSerializationPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) + , scalar_kind(scalar_kind_) , index(std::move(index_)) { } @@ -99,23 +124,25 @@ void MergeTreeIndexGranuleUSearch::deserializeBinary(ReadBuffer & istr, { UInt64 dimension; readIntBinary(dimension, istr); - index = std::make_shared>(dimension); + index = std::make_shared>(dimension, scalar_kind); index->deserialize(istr); } template MergeTreeIndexAggregatorUSearch::MergeTreeIndexAggregatorUSearch( const String & index_name_, - const Block & index_sample_block_) + const Block & index_sample_block_, + unum::usearch::scalar_kind_t scalar_kind_) : index_name(index_name_) , index_sample_block(index_sample_block_) + , scalar_kind(scalar_kind_) { } template MergeTreeIndexGranulePtr MergeTreeIndexAggregatorUSearch::getGranuleAndReset() { - auto granule = std::make_shared>(index_name, index_sample_block, index); + auto granule = std::make_shared>(index_name, index_sample_block, scalar_kind, index); index = nullptr; return granule; } @@ -131,9 +158,13 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); + if (rows_read == 0) return; + if (rows_read > std::numeric_limits::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"); @@ -151,7 +182,6 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t const auto & offsets = column_array->getOffsets(); const size_t num_rows = offsets.size(); - /// Check all sizes are the same size_t size = offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) @@ -159,18 +189,22 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); if (!index) - index = std::make_shared>(size); + index = std::make_shared>(size, scalar_kind); /// Add all rows of block if (!index->reserve(unum::usearch::ceil2(index->size() + num_rows))) throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index"); - if (auto rc = index->add(index->size(), array.data()); !rc) - throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); - for (size_t current_row = 1; current_row < num_rows; ++current_row) - if (auto rc = index->add(index->size(), &array[offsets[current_row - 1]]); !rc) + for (size_t current_row = 0; current_row < num_rows; ++current_row) + { + auto rc = index->add(static_cast(index->size()), &array[offsets[current_row - 1]]); + if (!rc) throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); + ProfileEvents::increment(ProfileEvents::USearchAddCount); + ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members); + ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, rc.computed_distances); + } } else if (const auto & column_tuple = typeid_cast(column_cut.get())) { @@ -187,14 +221,21 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read); if (!index) - index = std::make_shared>(data[0].size()); + index = std::make_shared>(data[0].size(), scalar_kind); if (!index->reserve(unum::usearch::ceil2(index->size() + data.size()))) throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index"); for (const auto & item : data) - if (auto rc = index->add(index->size(), item.data()); !rc) + { + auto rc = index->add(static_cast(index->size()), item.data()); + if (!rc) throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); + + ProfileEvents::increment(ProfileEvents::USearchAddCount); + ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members); + ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, rc.computed_distances); + } } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array or Tuple column"); @@ -257,7 +298,12 @@ std::vector MergeTreeIndexConditionUSearch::getUsefulRangesImpl(MergeTre ann_condition.getDimensions(), index->dimensions()); auto result = index->search(reference_vector.data(), limit); - std::vector neighbors(result.size()); /// indexes of dots which were closest to the reference vector + + ProfileEvents::increment(ProfileEvents::USearchSearchCount); + ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, result.visited_members); + ProfileEvents::increment(ProfileEvents::USearchSearchComputedDistances, result.computed_distances); + + std::vector neighbors(result.size()); /// indexes of dots which were closest to the reference vector std::vector distances(result.size()); result.dump_to(neighbors.data(), distances.data()); @@ -277,27 +323,28 @@ std::vector MergeTreeIndexConditionUSearch::getUsefulRangesImpl(MergeTre return granule_numbers; } -MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_) +MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_, unum::usearch::scalar_kind_t scalar_kind_) : IMergeTreeIndex(index_) , distance_function(distance_function_) + , scalar_kind(scalar_kind_) { } MergeTreeIndexGranulePtr MergeTreeIndexUSearch::createIndexGranule() const { if (distance_function == DISTANCE_FUNCTION_L2) - return std::make_shared>(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block, scalar_kind); else if (distance_function == DISTANCE_FUNCTION_COSINE) - return std::make_shared>(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block, scalar_kind); std::unreachable(); } MergeTreeIndexAggregatorPtr MergeTreeIndexUSearch::createIndexAggregator() const { if (distance_function == DISTANCE_FUNCTION_L2) - return std::make_shared>(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block, scalar_kind); else if (distance_function == DISTANCE_FUNCTION_COSINE) - return std::make_shared>(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block, scalar_kind); std::unreachable(); } @@ -313,18 +360,25 @@ MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) if (!index.arguments.empty()) distance_function = index.arguments[0].get(); - return std::make_shared(index, distance_function); + static constexpr auto default_scalar_kind = unum::usearch::scalar_kind_t::f16_k; + auto scalar_kind = default_scalar_kind; + if (index.arguments.size() > 1) + scalar_kind = nameToScalarKind.at(index.arguments[1].get()); + + return std::make_shared(index, distance_function, scalar_kind); } void usearchIndexValidator(const IndexDescription & index, bool /* attach */) { /// Check number and type of USearch index arguments: - if (index.arguments.size() > 1) + if (index.arguments.size() > 2) throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index must not have more than one parameters"); if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance function argument of USearch index must be of type String"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of USearch index (distance function) must be of type String"); + if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of USearch index (scalar type) must be of type String"); /// Check that the index is created on a single column @@ -340,6 +394,20 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) throw Exception(ErrorCodes::INCORRECT_DATA, "USearch index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE); } + /// Check that a supported kind was passed as a second argument + + if (index.arguments.size() > 1 && !nameToScalarKind.contains(index.arguments[1].get())) + { + String supported_kinds; + for (const auto & [name, kind] : nameToScalarKind) + { + if (!supported_kinds.empty()) + supported_kinds += ", "; + supported_kinds += name; + } + throw Exception(ErrorCodes::INCORRECT_DATA, "Unrecognized scalar kind (second argument) for USearch index. Supported kinds are: {}", supported_kinds); + } + /// Check data type of indexed column: auto throw_unsupported_underlying_column_exception = []() diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h index 98fb05b6f1a3..4f719c6375a2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -12,13 +12,15 @@ namespace DB { +using USearchImplType = unum::usearch::index_dense_gt; + template -class USearchIndexWithSerialization : public unum::usearch::index_dense_t +class USearchIndexWithSerialization : public USearchImplType { - using Base = unum::usearch::index_dense_t; + using Base = USearchImplType; public: - explicit USearchIndexWithSerialization(size_t dimensions); + USearchIndexWithSerialization(size_t dimensions, unum::usearch::scalar_kind_t scalar_kind); void serialize(WriteBuffer & ostr) const; void deserialize(ReadBuffer & istr); size_t getDimensions() const; @@ -31,8 +33,8 @@ using USearchIndexWithSerializationPtr = std::shared_ptr struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule { - MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_); - MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_, USearchIndexWithSerializationPtr index_); + MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_, unum::usearch::scalar_kind_t scalar_kind_); + MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_, unum::usearch::scalar_kind_t scalar_kind_, USearchIndexWithSerializationPtr index_); ~MergeTreeIndexGranuleUSearch() override = default; @@ -43,6 +45,7 @@ struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule const String index_name; const Block index_sample_block; + const unum::usearch::scalar_kind_t scalar_kind; USearchIndexWithSerializationPtr index; }; @@ -50,7 +53,7 @@ struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule template struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator { - MergeTreeIndexAggregatorUSearch(const String & index_name_, const Block & index_sample_block); + MergeTreeIndexAggregatorUSearch(const String & index_name_, const Block & index_sample_block, unum::usearch::scalar_kind_t scalar_kind_); ~MergeTreeIndexAggregatorUSearch() override = default; bool empty() const override { return !index || index->size() == 0; } @@ -59,6 +62,7 @@ struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator const String index_name; const Block index_sample_block; + const unum::usearch::scalar_kind_t scalar_kind; USearchIndexWithSerializationPtr index; }; @@ -90,7 +94,7 @@ class MergeTreeIndexConditionUSearch final : public IMergeTreeIndexConditionAppr class MergeTreeIndexUSearch : public IMergeTreeIndex { public: - MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_); + MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_, unum::usearch::scalar_kind_t scalar_kind_); ~MergeTreeIndexUSearch() override = default; @@ -102,6 +106,7 @@ class MergeTreeIndexUSearch : public IMergeTreeIndex private: const String distance_function; + const unum::usearch::scalar_kind_t scalar_kind; }; } diff --git a/tests/queries/0_stateless/02354_usearch_index.reference b/tests/queries/0_stateless/02354_usearch_index.reference index f966b5ce33fb..893a092a386f 100644 --- a/tests/queries/0_stateless/02354_usearch_index.reference +++ b/tests/queries/0_stateless/02354_usearch_index.reference @@ -146,3 +146,7 @@ Expression (Projection) 9000 [9000,0,0,0] 1 (1,0,0,0) 9000 (9000,0,0,0) +--- Test quantization --- +1 [0,0,10] +2 [0,0,10.5] +3 [0,0,9.5] diff --git a/tests/queries/0_stateless/02354_usearch_index.sql b/tests/queries/0_stateless/02354_usearch_index.sql index f771e2835fae..e534c91b615f 100644 --- a/tests/queries/0_stateless/02354_usearch_index.sql +++ b/tests/queries/0_stateless/02354_usearch_index.sql @@ -8,8 +8,8 @@ SELECT '--- Negative tests ---'; DROP TABLE IF EXISTS tab; --- must have at most 1 arguments -CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('too', 'many')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +-- must have at most 2 arguments +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('too', 'many', 'args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -- first argument (distance_function) must be String CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } @@ -20,6 +20,9 @@ CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index (vector, i -- reject unsupported distance functions CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +-- reject unsupported distance functions +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('L2Distance', 'invalid')) 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, vector Float32, INDEX usearch_index vector TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } @@ -260,3 +263,14 @@ ORDER BY L2Distance(vector, (9000.0, 0.0, 0.0, 0.0)) LIMIT 1; DROP TABLE tab; + +SELECT '--- Test quantization ---'; + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('L2Distance', 'f16')) ENGINE = MergeTree ORDER BY id; +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 * +FROM tab +WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 3; From e018f1d913ec457e4094a630144a52d1899e408b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 14 Sep 2023 12:41:45 +0000 Subject: [PATCH 2/2] Fix spelling --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index f928fa47fee3..8996133f6672 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -277,7 +277,8 @@ USearch currently supports two distance functions: - `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). -USearch allows storing the vectors in reduced precision formats. Supported scarar kinds are `f64`, `f32`, `f16` or `i8` (default: `f16`). +USearch allows storing the vectors in reduced precision formats. Supported scalar kinds are `f64`, `f32`, `f16` or `i8`. If no scalar kind +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.