mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Cosmetics
This commit is contained in:
parent
d4bbe2e9f0
commit
b9548504d9
@ -55,6 +55,7 @@ const std::unordered_map<String, unum::usearch::scalar_kind_t> quantizationToSca
|
||||
{"f32", unum::usearch::scalar_kind_t::f32_k},
|
||||
{"f16", unum::usearch::scalar_kind_t::f16_k},
|
||||
{"i8", unum::usearch::scalar_kind_t::i8_k}};
|
||||
/// Usearch provides more quantizations but ^^ above ones seem the only ones comprehensively supported accross all distance functions.
|
||||
|
||||
template<typename T>
|
||||
concept is_set = std::same_as<T, std::set<typename T::key_type, typename T::key_compare, typename T::allocator_type>>;
|
||||
@ -253,8 +254,7 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"The provided position is not less than the number of block rows. Position: {}, Block rows: {}.",
|
||||
*pos,
|
||||
block.rows());
|
||||
*pos, block.rows());
|
||||
|
||||
size_t rows_read = std::min(limit, block.rows() - *pos);
|
||||
|
||||
@ -268,63 +268,64 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_
|
||||
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);
|
||||
const ColumnPtr & index_column = block.getByName(index_column_name).column;
|
||||
ColumnPtr column_cut = index_column->cut(*pos, rows_read);
|
||||
|
||||
if (const auto & column_array = typeid_cast<const ColumnArray *>(column_cut.get()))
|
||||
const auto * column_array = typeid_cast<const ColumnArray *>(column_cut.get());
|
||||
if (!column_array)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float32) column");
|
||||
|
||||
if (column_array->empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty");
|
||||
|
||||
/// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays
|
||||
/// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default
|
||||
/// values 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);
|
||||
|
||||
const size_t rows = column_array->size();
|
||||
|
||||
const auto & column_array_offsets = column_array->getOffsets();
|
||||
const size_t dimensions = column_array_offsets[0];
|
||||
|
||||
if (!index)
|
||||
index = std::make_shared<USearchIndexWithSerialization>(dimensions, metric_kind, scalar_kind, usearch_hnsw_params);
|
||||
|
||||
/// 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->dimensions() != dimensions)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length");
|
||||
|
||||
/// 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() + rows > std::numeric_limits<UInt32>::max())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries");
|
||||
|
||||
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();
|
||||
|
||||
/// Check all sizes are the same
|
||||
for (size_t row = 0; row < rows - 1; ++row)
|
||||
if (column_array_offsets[row + 1] - column_array_offsets[row] != dimensions)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length");
|
||||
|
||||
/// Reserving space is mandatory
|
||||
if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + rows)))
|
||||
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index");
|
||||
|
||||
for (size_t row = 0; row < rows; ++row)
|
||||
{
|
||||
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 vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays
|
||||
/// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default
|
||||
/// values 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
|
||||
const size_t dimensions = column_array_offsets[0];
|
||||
for (size_t i = 0; i < num_rows - 1; ++i)
|
||||
if (column_array_offsets[i + 1] - column_array_offsets[i] != dimensions)
|
||||
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->dimensions() != dimensions)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name);
|
||||
|
||||
if (!index)
|
||||
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
|
||||
if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows)))
|
||||
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index");
|
||||
|
||||
for (size_t row = 0; row < num_rows; ++row)
|
||||
if (auto result = index->add(static_cast<USearchIndex::vector_key_t>(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release()));
|
||||
else
|
||||
{
|
||||
if (auto result = index->add(static_cast<UInt32>(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release()));
|
||||
else
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddCount);
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members);
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances);
|
||||
}
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddCount);
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members);
|
||||
ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances);
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float32) column");
|
||||
|
||||
*pos += rows_read;
|
||||
}
|
||||
@ -483,7 +484,7 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta
|
||||
if (!quantizationToScalarKind.contains(index.arguments[2].safeGet<String>()))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind));
|
||||
|
||||
/// Call Usearche's own parameter validation method for HNSW-specific parameters
|
||||
/// Call Usearch's own parameter validation method for HNSW-specific parameters
|
||||
UInt64 m = index.arguments[3].safeGet<UInt64>();
|
||||
UInt64 ef_construction = index.arguments[4].safeGet<UInt64>();
|
||||
UInt64 ef_search = index.arguments[5].safeGet<UInt64>();
|
||||
@ -498,18 +499,14 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta
|
||||
if (index.column_names.size() != 1 || index.data_types.size() != 1)
|
||||
throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity indexes must be created on a single column");
|
||||
|
||||
/// Check data type of the indexed column:
|
||||
/// Check that the data type is Array(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 Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)");
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * data_type_array = typeid_cast<const DataTypeArray *>(data_type.get());
|
||||
if (!data_type_array)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)");
|
||||
TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId();
|
||||
if (!WhichDataType(nested_type_index).isFloat32())
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,3 +1,4 @@
|
||||
Rejects INSERTs of Arrays with different sizes
|
||||
Issue #52258: Empty Arrays or Arrays with default values are rejected
|
||||
It is possible to create parts with different Array vector sizes but there will be an error at query time
|
||||
Correctness of index with > 1 mark
|
||||
|
@ -7,6 +7,12 @@ SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make
|
||||
|
||||
DROP TABLE IF EXISTS tab;
|
||||
|
||||
SELECT 'Rejects INSERTs of Arrays with different sizes';
|
||||
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) 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;
|
||||
|
||||
SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected';
|
||||
|
||||
CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree() ORDER BY id;
|
||||
|
@ -5,4 +5,3 @@ Two or six index arguments
|
||||
4nd argument (M), if given, must be UInt64 and > 1
|
||||
Must be created on single column
|
||||
Must be created on Array(Float32) columns
|
||||
Rejects INSERTs of Arrays with different sizes
|
||||
|
@ -38,8 +38,3 @@ CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity('hn
|
||||
CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN }
|
||||
CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) 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 vector_similarity('hnsw', 'L2Distance')) 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,9 +1,7 @@
|
||||
10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block
|
||||
- ORDER-BY-type
|
||||
5 [0,2] 0
|
||||
6 [0,2.1] 0.09999990463256836
|
||||
7 [0,2.2] 0.20000004768371582
|
||||
- ORDER-BY-type, EXPLAIN
|
||||
Expression (Projection)
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
@ -20,11 +18,9 @@ Expression (Projection)
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block
|
||||
- ORDER-BY-type
|
||||
6 [0,2] 0
|
||||
7 [0,2.1] 0.09999990463256836
|
||||
8 [0,2.2] 0.20000004768371582
|
||||
- ORDER-BY-type, EXPLAIN
|
||||
Expression (Projection)
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
@ -41,11 +37,11 @@ Expression (Projection)
|
||||
Parts: 1/1
|
||||
Granules: 2/4
|
||||
Special cases
|
||||
- ORDER-BY-type
|
||||
-- Non-default metric, M, ef_construction, ef_search
|
||||
6 [1,9.3] 0.005731362878640178
|
||||
1 [2,3.2] 0.15200169244542905
|
||||
7 [5.5,4.7] 0.3503476876550442
|
||||
- Special case: setting "max_limit_for_ann_queries"
|
||||
-- Setting "max_limit_for_ann_queries"
|
||||
Expression (Projection)
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
|
@ -14,14 +14,12 @@ CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similar
|
||||
INSERT INTO tab 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 '- ORDER-BY-type';
|
||||
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;
|
||||
|
||||
SELECT '- ORDER-BY-type, EXPLAIN';
|
||||
EXPLAIN indexes = 1
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
@ -37,14 +35,12 @@ SELECT '12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexe
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
|
||||
INSERT INTO tab 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 '- ORDER-BY-type';
|
||||
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;
|
||||
|
||||
SELECT '- ORDER-BY-type, EXPLAIN';
|
||||
EXPLAIN indexes = 1
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
@ -56,19 +52,18 @@ DROP TABLE tab;
|
||||
|
||||
|
||||
SELECT 'Special cases'; -- Not a systematic test, just to check that no bad things happen.
|
||||
-- Test with non-default metric, M, ef_construction, ef_search
|
||||
|
||||
SELECT '-- Non-default metric, M, ef_construction, ef_search';
|
||||
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 42, 99, 66) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
|
||||
INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
|
||||
|
||||
SELECT '- ORDER-BY-type';
|
||||
WITH [0.0, 2.0] AS reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
FROM tab
|
||||
ORDER BY cosineDistance(vec, reference_vec)
|
||||
LIMIT 3;
|
||||
|
||||
SELECT '- Special case: setting "max_limit_for_ann_queries"';
|
||||
SELECT '-- Setting "max_limit_for_ann_queries"';
|
||||
EXPLAIN indexes=1
|
||||
WITH [0.0, 2.0] as reference_vec
|
||||
SELECT id, vec, cosineDistance(vec, reference_vec)
|
||||
|
Loading…
Reference in New Issue
Block a user