Bump usearch to 2.13.2

This commit is contained in:
Robert Schulze 2024-08-13 12:31:13 +00:00
parent 58d76fabf6
commit 6170a8663f
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
7 changed files with 90 additions and 71 deletions

2
contrib/SimSIMD vendored

@ -1 +1 @@
Subproject commit 18d17686124ddebd9fe55eee56b2e0273a613d4b Subproject commit 91a76d1ac519b3b9dc8957734a3dabd985f00c26

2
contrib/usearch vendored

@ -1 +1 @@
Subproject commit e6c81f78c64c0d8119f854691a06e60660638a25 Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4

View File

@ -59,6 +59,8 @@ Parameters:
- `ef_construction`: (optional, default: 128) - `ef_construction`: (optional, default: 128)
- `ef_search`: (optional, default: 64) - `ef_search`: (optional, default: 64)
Value 0 for parameters `m`, `ef_construction`, and `ef_search` refers to the default value.
Example: Example:
```sql ```sql

View File

@ -2,9 +2,6 @@
#if USE_USEARCH #if USE_USEARCH
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wpass-failed"
#include <Columns/ColumnArray.h> #include <Columns/ColumnArray.h>
#include <Common/BitHelpers.h> #include <Common/BitHelpers.h>
#include <Common/formatReadable.h> #include <Common/formatReadable.h>
@ -46,15 +43,15 @@ namespace
{ {
/// The only indexing method currently supported by USearch /// The only indexing method currently supported by USearch
std::set<String> methods = {"hnsw"}; const std::set<String> methods = {"hnsw"};
/// Maps from user-facing name to internal name /// Maps from user-facing name to internal name
std::unordered_map<String, unum::usearch::metric_kind_t> distanceFunctionToMetricKind = { const std::unordered_map<String, unum::usearch::metric_kind_t> distanceFunctionToMetricKind = {
{"L2Distance", unum::usearch::metric_kind_t::l2sq_k}, {"L2Distance", unum::usearch::metric_kind_t::l2sq_k},
{"cosineDistance", unum::usearch::metric_kind_t::cos_k}}; {"cosineDistance", unum::usearch::metric_kind_t::cos_k}};
/// Maps from user-facing name to internal name /// Maps from user-facing name to internal name
std::unordered_map<String, unum::usearch::scalar_kind_t> quantizationToScalarKind = { const std::unordered_map<String, unum::usearch::scalar_kind_t> quantizationToScalarKind = {
{"f32", unum::usearch::scalar_kind_t::f32_k}, {"f32", unum::usearch::scalar_kind_t::f32_k},
{"f16", unum::usearch::scalar_kind_t::f16_k}, {"f16", unum::usearch::scalar_kind_t::f16_k},
{"i8", unum::usearch::scalar_kind_t::i8_k}}; {"i8", unum::usearch::scalar_kind_t::i8_k}};
@ -96,13 +93,18 @@ USearchIndexWithSerialization::USearchIndexWithSerialization(
unum::usearch::scalar_kind_t scalar_kind, unum::usearch::scalar_kind_t scalar_kind,
UsearchHnswParams usearch_hnsw_params) UsearchHnswParams usearch_hnsw_params)
{ {
unum::usearch::metric_punned_t metric(dimensions, metric_kind, scalar_kind); USearchIndex::metric_t metric(dimensions, metric_kind, scalar_kind);
unum::usearch::index_dense_config_t config(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search); unum::usearch::index_dense_config_t config(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search);
config.enable_key_lookups = false; /// we don't do row-to-vector lookups config.enable_key_lookups = false; /// we don't do row-to-vector lookups
USearchIndex usearch_index = USearchIndex::make(metric, config); if (auto error = config.validate(); error) /// already called in vectorSimilarityIndexValidator, call again because usearch may change the config in-place
swap(usearch_index); throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release()));
if (auto result = USearchIndex::make(metric, config); !result)
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not create vector similarity index. Error: {}", String(result.error.release()));
else
swap(result.index);
} }
void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const
@ -113,9 +115,8 @@ void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const
return true; return true;
}; };
auto result = Base::save_to_stream(callback); if (auto result = Base::save_to_stream(callback); !result)
if (result.error) throw Exception(ErrorCodes::INCORRECT_DATA, "Could not save vector similarity index. Error: {}", String(result.error.release()));
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not save vector similarity index, error: " + String(result.error.release()));
} }
void USearchIndexWithSerialization::deserialize(ReadBuffer & istr) void USearchIndexWithSerialization::deserialize(ReadBuffer & istr)
@ -126,26 +127,43 @@ void USearchIndexWithSerialization::deserialize(ReadBuffer & istr)
return true; return true;
}; };
auto result = Base::load_from_stream(callback); if (auto result = Base::load_from_stream(callback); !result)
if (result.error)
/// See the comment in MergeTreeIndexGranuleVectorSimilarity::deserializeBinary why we throw here /// See the comment in MergeTreeIndexGranuleVectorSimilarity::deserializeBinary why we throw here
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not load vector similarity index, error: " + String(result.error.release()) + " Please drop the index and create it again."); throw Exception(ErrorCodes::INCORRECT_DATA, "Could not load vector similarity index. Please drop the index and create it again. Error: {}", String(result.error.release()));
if (!try_reserve(limits()))
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index");
} }
USearchIndexWithSerialization::Statistics USearchIndexWithSerialization::getStatistics() const USearchIndexWithSerialization::Statistics USearchIndexWithSerialization::getStatistics() const
{ {
USearchIndex::stats_t global_stats = Base::stats();
Statistics statistics = { Statistics statistics = {
.max_level = max_level(), .max_level = max_level(),
.connectivity = connectivity(), .connectivity = connectivity(),
.size = size(), /// number of vectors .size = size(),
.capacity = capacity(), /// number of vectors reserved .capacity = capacity(),
.memory_usage = memory_usage(), /// in bytes, the value is not exact .memory_usage = memory_usage(),
.bytes_per_vector = bytes_per_vector(), .bytes_per_vector = bytes_per_vector(),
.scalar_words = scalar_words(), .scalar_words = scalar_words(),
.statistics = stats()}; .nodes = global_stats.nodes,
.edges = global_stats.edges,
.max_edges = global_stats.max_edges,
.level_stats = {}};
for (size_t i = 0; i < statistics.max_level; ++i)
statistics.level_stats.push_back(Base::stats(i));
return statistics; return statistics;
} }
String USearchIndexWithSerialization::Statistics::toString() const
{
return fmt::format("max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}, bytes_per_vector = {}, scalar_words = {}, nodes = {}, edges = {}, max_edges = {}",
max_level, connectivity, size, capacity, ReadableSize(memory_usage), bytes_per_vector, scalar_words, nodes, edges, max_edges);
}
MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity(
const String & index_name_, const String & index_name_,
const Block & index_sample_block_, const Block & index_sample_block_,
@ -186,8 +204,7 @@ void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr)
index->serialize(ostr); index->serialize(ostr);
auto statistics = index->getStatistics(); auto statistics = index->getStatistics();
LOG_TRACE(logger, "Wrote vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", LOG_TRACE(logger, "Wrote vector similarity index: {}", statistics.toString());
statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage));
} }
void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/)
@ -209,8 +226,7 @@ void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr,
index->deserialize(istr); index->deserialize(istr);
auto statistics = index->getStatistics(); auto statistics = index->getStatistics();
LOG_TRACE(logger, "Loaded vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", LOG_TRACE(logger, "Loaded vector similarity index: {}", statistics.toString());
statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage));
} }
MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilarity( MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilarity(
@ -290,19 +306,24 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_
if (!index) if (!index)
index = std::make_shared<USearchIndexWithSerialization>(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); index = std::make_shared<USearchIndexWithSerialization>(dimensions, metric_kind, scalar_kind, usearch_hnsw_params);
/// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp
if (index->size() + num_rows > std::numeric_limits<UInt32>::max())
throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index in column {} would exceed 4 billion entries", index_column_name);
/// Reserving space is mandatory /// Reserving space is mandatory
if (!index->reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows))) if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows)))
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index");
for (size_t row = 0; row < num_rows; ++row) for (size_t row = 0; row < num_rows; ++row)
{ {
auto rc = index->add(static_cast<UInt32>(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); if (auto result = index->add(static_cast<UInt32>(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result)
if (!rc) throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release()));
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index, error: " + String(rc.error.release())); else
{
ProfileEvents::increment(ProfileEvents::USearchAddCount); ProfileEvents::increment(ProfileEvents::USearchAddCount);
ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members); ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members);
ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, rc.computed_distances); ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances);
}
} }
} }
else else
@ -356,17 +377,16 @@ std::vector<size_t> MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer
const std::vector<float> reference_vector = vector_similarity_condition.getReferenceVector(); const std::vector<float> reference_vector = vector_similarity_condition.getReferenceVector();
auto result = index->search(reference_vector.data(), limit); auto search_result = index->search(reference_vector.data(), limit);
if (result.error) if (!search_result)
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index, error: " + String(result.error.release())); throw Exception(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index. Error: {}", String(search_result.error.release()));
ProfileEvents::increment(ProfileEvents::USearchSearchCount); ProfileEvents::increment(ProfileEvents::USearchSearchCount);
ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, result.visited_members); ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, search_result.visited_members);
ProfileEvents::increment(ProfileEvents::USearchSearchComputedDistances, result.computed_distances); ProfileEvents::increment(ProfileEvents::USearchSearchComputedDistances, search_result.computed_distances);
std::vector<USearchIndex::vector_key_t> neighbors(result.size()); /// indexes of dots which were closest to the reference vector std::vector<USearchIndex::vector_key_t> neighbors(search_result.size()); /// indexes of vectors which were closest to the reference vector
std::vector<USearchIndex::distance_t> distances(result.size()); search_result.dump_to(neighbors.data());
result.dump_to(neighbors.data(), distances.data());
std::vector<size_t> granules; std::vector<size_t> granules;
granules.reserve(neighbors.size()); granules.reserve(neighbors.size());
@ -414,14 +434,13 @@ MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition(
MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index)
{ {
const bool has_six_args = (index.arguments.size() == 6); /// Default parameters:
unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet<String>()); unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet<String>());
/// use defaults for the other parameters
unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k; unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k;
UsearchHnswParams usearch_hnsw_params; UsearchHnswParams usearch_hnsw_params;
/// Optional parameters:
const bool has_six_args = (index.arguments.size() == 6);
if (has_six_args) if (has_six_args)
{ {
scalar_kind = quantizationToScalarKind.at(index.arguments[2].safeGet<String>()); scalar_kind = quantizationToScalarKind.at(index.arguments[2].safeGet<String>());
@ -466,12 +485,16 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta
{ {
if (!quantizationToScalarKind.contains(index.arguments[2].safeGet<String>())) if (!quantizationToScalarKind.contains(index.arguments[2].safeGet<String>()))
throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind)); throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind));
if (index.arguments[3].safeGet<UInt64>() < 2)
throw Exception(ErrorCodes::INCORRECT_DATA, "Fourth argument (M) of vector similarity index must be > 1"); /// Call Usearche's own parameter validation method for HNSW-specific parameters
if (index.arguments[4].safeGet<UInt64>() < 1) UInt64 m = index.arguments[3].safeGet<UInt64>();
throw Exception(ErrorCodes::INCORRECT_DATA, "Fifth argument (ef_construction) of vector similarity index must be > 0"); UInt64 ef_construction = index.arguments[4].safeGet<UInt64>();
if (index.arguments[5].safeGet<UInt64>() < 1) UInt64 ef_search = index.arguments[5].safeGet<UInt64>();
throw Exception(ErrorCodes::INCORRECT_DATA, "Sixth argument (ef_search) of vector similarity index must be > 0");
unum::usearch::index_dense_config_t config(m, ef_construction, ef_search);
if (auto error = config.validate(); error)
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release()));
} }
/// Check that the index is created on a single column /// Check that the index is created on a single column

View File

@ -4,12 +4,9 @@
#if USE_USEARCH #if USE_USEARCH
#pragma clang diagnostic push #include <Storages/MergeTree/VectorSimilarityCondition.h>
#pragma clang diagnostic ignored "-Wpass-failed" #include <Common/Logger.h>
# include <Storages/MergeTree/VectorSimilarityCondition.h> #include <usearch/index_dense.hpp>
# include <Common/Logger.h>
# include <usearch/index_dense.hpp>
#pragma clang diagnostic pop
namespace DB namespace DB
{ {
@ -21,7 +18,7 @@ struct UsearchHnswParams
size_t ef_search = unum::usearch::default_expansion_search(); size_t ef_search = unum::usearch::default_expansion_search();
}; };
using USearchIndex = unum::usearch::index_dense_gt</*key_at*/ uint32_t, /*compressed_slot_at*/ uint32_t>; using USearchIndex = unum::usearch::index_dense_t;
class USearchIndexWithSerialization : public USearchIndex class USearchIndexWithSerialization : public USearchIndex
{ {
@ -41,13 +38,18 @@ public:
{ {
size_t max_level; size_t max_level;
size_t connectivity; size_t connectivity;
size_t size; size_t size; /// number of indexed vectors
size_t capacity; size_t capacity; /// reserved number of indexed vectors
size_t memory_usage; size_t memory_usage; /// byte size (not exact)
/// advanced stats:
size_t bytes_per_vector; size_t bytes_per_vector;
size_t scalar_words; size_t scalar_words;
Base::stats_t statistics; size_t nodes;
size_t edges;
size_t max_edges;
std::vector<USearchIndex::stats_t> level_stats; /// for debugging, excluded from getStatistics()
String toString() const;
}; };
Statistics getStatistics() const; Statistics getStatistics() const;

View File

@ -3,8 +3,6 @@ Two or six index arguments
2nd argument (distance function) must be String and L2Distance or cosineDistance 2nd argument (distance function) must be String and L2Distance or cosineDistance
3nd argument (quantization), if given, must be String and f32, f16, ... 3nd argument (quantization), if given, must be String and f32, f16, ...
4nd argument (M), if given, must be UInt64 and > 1 4nd argument (M), if given, must be UInt64 and > 1
5nd argument (ef_construction), if given, must be UInt64 and > 0
6nd argument (ef_search), if given, must be UInt64 and > 0
Must be created on single column Must be created on single column
Must be created on Array(Float32) columns Must be created on Array(Float32) columns
Rejects INSERTs of Arrays with different sizes Rejects INSERTs of Arrays with different sizes

View File

@ -27,12 +27,6 @@ CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similar
SELECT '4nd argument (M), if given, must be UInt64 and > 1'; SELECT '4nd argument (M), if given, must be UInt64 and > 1';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 'invalid', 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 'invalid', 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT '5nd argument (ef_construction), if given, must be UInt64 and > 0';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 'invalid', 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 0, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT '6nd argument (ef_search), if given, must be UInt64 and > 0';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 'invalid')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 0)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT 'Must be created on single column'; SELECT 'Must be created on single column';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS }