feat: impl StructArray -- miscellaneous staffs for struct array (#43960)

Ref https://github.com/milvus-io/milvus/issues/42148

1. enable storage v2
2. implement some missing staffs
3. fix some bugs and add tests

---------

Signed-off-by: SpadeA <tangchenjie1210@gmail.com>
This commit is contained in:
Spade A 2025-08-26 21:35:53 +08:00 committed by GitHub
parent 5bdc593b8a
commit 8456f824be
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
39 changed files with 973 additions and 90 deletions

View File

@ -450,14 +450,42 @@ class VectorArrayChunk : public Chunk {
data_ptr, dim_, len, next_offset - offset, element_type_);
}
std::vector<VectorArrayView>
Views() const {
std::pair<std::vector<VectorArrayView>, FixedVector<bool>>
Views(std::optional<std::pair<int64_t, int64_t>> offset_len =
std::nullopt) const {
auto start_offset = 0;
auto len = row_nums_;
if (offset_len.has_value()) {
start_offset = offset_len->first;
len = offset_len->second;
AssertInfo(
start_offset >= 0 && start_offset < row_nums_,
"Retrieve vector array views with out-of-bound offset:{}, "
"len:{}, wrong",
start_offset,
len);
AssertInfo(
len > 0 && len <= row_nums_,
"Retrieve vector array views with out-of-bound offset:{}, "
"len:{}, wrong",
start_offset,
len);
AssertInfo(
start_offset + len <= row_nums_,
"Retrieve vector array views with out-of-bound offset:{}, "
"len:{}, wrong",
start_offset,
len);
}
std::vector<VectorArrayView> views;
views.reserve(row_nums_);
for (int64_t i = 0; i < row_nums_; i++) {
views.reserve(len);
auto end_offset = start_offset + len;
for (int64_t i = start_offset; i < end_offset; i++) {
views.emplace_back(View(i));
}
return views;
// vector array does not support null, so just return {}.
return {std::move(views), {}};
}
const char*

View File

@ -194,6 +194,8 @@ GetArrowDataType(DataType data_type, int dim = 1) {
return arrow::binary();
case DataType::VECTOR_INT8:
return arrow::fixed_size_binary(dim);
case DataType::VECTOR_ARRAY:
return arrow::binary();
default: {
ThrowInfo(DataTypeInvalid,
fmt::format("failed to get data type, invalid type {}",

View File

@ -28,22 +28,22 @@
namespace milvus {
#define GET_ELEM_TYPE_FOR_VECTOR_TRAIT \
using elem_type = std::conditional_t< \
std::is_same_v<TraitType, milvus::EmbListFloatVector>, \
milvus::EmbListFloatVector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::FloatVector>, \
milvus::FloatVector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::Float16Vector>, \
milvus::Float16Vector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::BFloat16Vector>, \
milvus::BFloat16Vector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::Int8Vector>, \
milvus::Int8Vector::embedded_type, \
#define GET_ELEM_TYPE_FOR_VECTOR_TRAIT \
using elem_type = std::conditional_t< \
std::is_same_v<TraitType, milvus::EmbListFloatVector>, \
milvus::EmbListFloatVector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::FloatVector>, \
milvus::FloatVector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::Float16Vector>, \
milvus::Float16Vector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::BFloat16Vector>, \
milvus::BFloat16Vector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::Int8Vector>, \
milvus::Int8Vector::embedded_type, \
milvus::BinaryVector::embedded_type>>>>>;
#define GET_SCHEMA_DATA_TYPE_FOR_VECTOR_TRAIT \

View File

@ -343,8 +343,6 @@ VectorMemIndex<T>::Build(const Config& config) {
for (auto data : field_datas) {
total_size += data->Size();
total_num_rows += data->get_num_rows();
AssertInfo(dim == 0 || dim == data->get_dim(),
"inconsistent dim value between field datas!");
// todo(SapdeA): now, vector arrays (embedding list) are serialized
// to parquet by using binary format which does not provide dim
@ -354,6 +352,9 @@ VectorMemIndex<T>::Build(const Config& config) {
"embedding list index must have elem_type");
dim = config[DIM_KEY].get<int64_t>();
} else {
AssertInfo(dim == 0 || dim == data->get_dim(),
"inconsistent dim value between field datas!");
dim = data->get_dim();
}
}
@ -585,7 +586,8 @@ VectorMemIndex<T>::GetSparseVector(const DatasetPtr dataset) const {
}
template <typename T>
void VectorMemIndex<T>::LoadFromFile(const Config& config) {
void
VectorMemIndex<T>::LoadFromFile(const Config& config) {
auto local_filepath =
GetValueFromConfig<std::string>(config, MMAP_FILE_PATH);
AssertInfo(local_filepath.has_value(),

View File

@ -266,8 +266,10 @@ class ChunkedColumnBase : public ChunkedColumnInterface {
"ArrayViews only supported for ArrayChunkedColumn");
}
PinWrapper<std::vector<VectorArrayView>>
VectorArrayViews(int64_t chunk_id) const override {
PinWrapper<std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
VectorArrayViews(
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override {
ThrowInfo(
ErrorCode::Unsupported,
"VectorArrayViews only supported for ChunkedVectorArrayColumn");
@ -620,13 +622,16 @@ class ChunkedVectorArrayColumn : public ChunkedColumnBase {
}
}
PinWrapper<std::vector<VectorArrayView>>
VectorArrayViews(int64_t chunk_id) const override {
PinWrapper<std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
VectorArrayViews(int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len =
std::nullopt) const override {
auto ca =
SemiInlineGet(slot_->PinCells({static_cast<cid_t>(chunk_id)}));
auto chunk = ca->get_cell_of(chunk_id);
return PinWrapper<std::vector<VectorArrayView>>(
ca, static_cast<VectorArrayChunk*>(chunk)->Views());
return PinWrapper<
std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>(
ca, static_cast<VectorArrayChunk*>(chunk)->Views(offset_len));
}
PinWrapper<const size_t*>

View File

@ -305,8 +305,10 @@ class ProxyChunkColumn : public ChunkedColumnInterface {
static_cast<ArrayChunk*>(chunk.get())->Views(offset_len));
}
PinWrapper<std::vector<VectorArrayView>>
VectorArrayViews(int64_t chunk_id) const override {
PinWrapper<std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
VectorArrayViews(int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len =
std::nullopt) const override {
if (!IsChunkedVectorArrayColumnDataType(data_type_)) {
ThrowInfo(ErrorCode::Unsupported,
"[StorageV2] VectorArrayViews only supported for "
@ -314,9 +316,10 @@ class ProxyChunkColumn : public ChunkedColumnInterface {
}
auto chunk_wrapper = group_->GetGroupChunk(chunk_id);
auto chunk = chunk_wrapper.get()->GetChunk(field_id_);
return PinWrapper<std::vector<VectorArrayView>>(
return PinWrapper<
std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>(
chunk_wrapper,
static_cast<VectorArrayChunk*>(chunk.get())->Views());
static_cast<VectorArrayChunk*>(chunk.get())->Views(offset_len));
}
PinWrapper<const size_t*>
@ -519,8 +522,7 @@ class ProxyChunkColumn : public ChunkedColumnInterface {
auto chunk = group_chunk->GetChunk(field_id_);
auto valid = chunk->isValid(offsets_in_chunk[i]);
auto value = static_cast<StringChunk*>(chunk.get())
->
operator[](offsets_in_chunk[i]);
->operator[](offsets_in_chunk[i]);
fn(value, i, valid);
}
}
@ -547,8 +549,7 @@ class ProxyChunkColumn : public ChunkedColumnInterface {
auto chunk = group_chunk->GetChunk(field_id_);
auto valid = chunk->isValid(offsets_in_chunk[i]);
auto str_view = static_cast<StringChunk*>(chunk.get())
->
operator[](offsets_in_chunk[i]);
->operator[](offsets_in_chunk[i]);
fn(Json(str_view.data(), str_view.size()), i, valid);
}
}

View File

@ -81,8 +81,11 @@ class ChunkedColumnInterface {
ArrayViews(int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const = 0;
virtual PinWrapper<std::vector<VectorArrayView>>
VectorArrayViews(int64_t chunk_id) const = 0;
virtual PinWrapper<
std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
VectorArrayViews(
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const = 0;
virtual PinWrapper<const size_t*>
VectorArrayLims(int64_t chunk_id) const = 0;

View File

@ -105,8 +105,14 @@ ParsePlaceholderGroup(const Plan* plan,
field_meta.get_element_type());
for (auto& line : info.values()) {
target.insert(target.end(), line.begin(), line.end());
AssertInfo(
line.size() % (dim * elem_size) == 0,
"line.size() % (dim * elem_size) == 0 assert failed, "
"line.size() = {}, dim = {}, elem_size = {}",
line.size(),
dim,
elem_size);
Assert(line.size() % (dim * elem_size) == 0);
offset += line.size() / (dim * elem_size);
lims.push_back(offset);
}

View File

@ -580,6 +580,22 @@ ChunkedSegmentSealedImpl::chunk_array_view_impl(
"chunk_array_view_impl only used for chunk column field ");
}
PinWrapper<std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
ChunkedSegmentSealedImpl::chunk_vector_array_view_impl(
FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len =
std::nullopt) const {
std::shared_lock lck(mutex_);
AssertInfo(get_bit(field_data_ready_bitset_, field_id),
"Can't get bitset element at " + std::to_string(field_id.get()));
if (auto it = fields_.find(field_id); it != fields_.end()) {
return it->second->VectorArrayViews(chunk_id, offset_len);
}
ThrowInfo(ErrorCode::UnexpectedError,
"chunk_vector_array_view_impl only used for chunk column field ");
}
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
ChunkedSegmentSealedImpl::chunk_string_view_impl(
FieldId field_id,

View File

@ -303,6 +303,12 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override;
PinWrapper<std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
chunk_vector_array_view_impl(
FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override;
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_string_views_by_offsets(
FieldId field_id,

View File

@ -644,6 +644,16 @@ SegmentGrowingImpl::chunk_array_view_impl(
"chunk array view impl not implement for growing segment");
}
PinWrapper<std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
SegmentGrowingImpl::chunk_vector_array_view_impl(
FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len =
std::nullopt) const {
ThrowInfo(ErrorCode::NotImplemented,
"chunk vector array view impl not implement for growing segment");
}
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
SegmentGrowingImpl::chunk_string_views_by_offsets(
FieldId field_id,

View File

@ -437,6 +437,12 @@ class SegmentGrowingImpl : public SegmentGrowing {
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override;
PinWrapper<std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
chunk_vector_array_view_impl(
FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>> offset_len) const override;
PinWrapper<std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_string_views_by_offsets(
FieldId field_id,

View File

@ -202,6 +202,8 @@ class SegmentInternalInterface : public SegmentInterface {
return chunk_string_view_impl(field_id, chunk_id, offset_len);
} else if constexpr (std::is_same_v<ViewType, ArrayView>) {
return chunk_array_view_impl(field_id, chunk_id, offset_len);
} else if constexpr (std::is_same_v<ViewType, VectorArrayView>) {
return chunk_vector_array_view_impl(field_id, chunk_id, offset_len);
} else if constexpr (std::is_same_v<ViewType, Json>) {
auto pw = chunk_string_view_impl(field_id, chunk_id, offset_len);
auto [string_views, valid_data] = pw.get();
@ -533,6 +535,13 @@ class SegmentInternalInterface : public SegmentInterface {
std::optional<std::pair<int64_t, int64_t>>
offset_len = std::nullopt) const = 0;
virtual PinWrapper<
std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
chunk_vector_array_view_impl(FieldId field_id,
int64_t chunk_id,
std::optional<std::pair<int64_t, int64_t>>
offset_len = std::nullopt) const = 0;
virtual PinWrapper<
std::pair<std::vector<std::string_view>, FixedVector<bool>>>
chunk_string_views_by_offsets(

View File

@ -111,6 +111,7 @@ set(MILVUS_TEST_FILES
test_vector_array.cpp
test_ngram_query.cpp
test_stlsort_index.cpp
test_vector_array_storage_v2.cpp
)
if ( INDEX_ENGINE STREQUAL "cardinal" )

View File

@ -73,7 +73,7 @@ class TestChunkSegmentStorageV2 : public testing::TestWithParam<bool> {
// Initialize file system
auto conf = milvus_storage::ArrowFileSystemConfig();
conf.storage_type = "local";
conf.root_path = "test_data";
conf.root_path = "/tmp/test_data";
milvus_storage::ArrowFileSystemSingleton::GetInstance().Init(conf);
auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance()
.GetArrowFileSystem();

View File

@ -366,7 +366,8 @@ GenerateRandomSparseFloatVector(size_t rows,
return tensor;
}
inline SchemaPtr CreateTestSchema() {
inline SchemaPtr
CreateTestSchema() {
auto schema = std::make_shared<milvus::Schema>();
auto bool_field =
schema->AddDebugField("bool", milvus::DataType::BOOL, true);

View File

@ -0,0 +1,340 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <gtest/gtest.h>
#include <algorithm>
#include <numeric>
#include <vector>
#include <memory>
#include <string>
#include <unordered_map>
#include <iostream>
#include <arrow/record_batch.h>
#include <arrow/array/builder_binary.h>
#include <arrow/array/builder_primitive.h>
#include "common/Schema.h"
#include "common/Types.h"
#include "common/VectorArray.h"
#include "common/Consts.h"
#include "milvus-storage/packed/writer.h"
#include "milvus-storage/filesystem/fs.h"
#include "milvus-storage/common/constants.h"
#include "segcore/SegmentSealed.h"
#include "segcore/ChunkedSegmentSealedImpl.h"
#include "segcore/SegcoreConfig.h"
#include "segcore/Types.h"
#include "test_utils/DataGen.h"
#include "pb/schema.pb.h"
#include "knowhere/comp/index_param.h"
#include "index/IndexFactory.h"
#include "index/VectorIndex.h"
#include "storage/Util.h"
#include "storage/ChunkManager.h"
#include "test_utils/storage_test_utils.h"
#include "common/QueryResult.h"
using namespace milvus;
using namespace milvus::segcore;
using namespace milvus::storage;
const int64_t DIM = 32;
SchemaPtr
GenVectorArrayTestSchema() {
auto schema = std::make_shared<Schema>();
auto int64_fid = schema->AddDebugField("int64", DataType::INT64);
schema->AddDebugVectorArrayField(
"vector_array", DataType::VECTOR_FLOAT, DIM, knowhere::metric::L2);
schema->AddField(FieldName("ts"),
TimestampFieldID,
DataType::INT64,
false,
std::nullopt);
schema->set_primary_field_id(int64_fid);
return schema;
}
class TestVectorArrayStorageV2 : public testing::Test {
protected:
void
SetUp() override {
schema_ = GenVectorArrayTestSchema();
segment_ = segcore::CreateSealedSegment(
schema_,
nullptr,
-1,
segcore::SegcoreConfig::default_config(),
true);
// Initialize file system
auto conf = milvus_storage::ArrowFileSystemConfig();
conf.storage_type = "local";
conf.root_path = "/tmp/test_vector_array_for_storage_v2";
milvus_storage::ArrowFileSystemSingleton::GetInstance().Init(conf);
auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance()
.GetArrowFileSystem();
// Prepare paths and column groups
std::vector<std::string> paths = {"test_data/0/10000.parquet",
"test_data/101/10001.parquet"};
// Create directories for the parquet files
for (const auto& path : paths) {
auto dir_path = path.substr(0, path.find_last_of('/'));
auto status = fs->CreateDir(dir_path);
EXPECT_TRUE(status.ok())
<< "Failed to create directory: " << dir_path;
}
std::vector<std::vector<int>> column_groups = {
{0, 2}, {1}}; // narrow columns and wide columns
auto writer_memory = 16 * 1024 * 1024;
auto storage_config = milvus_storage::StorageConfig();
// Create writer
milvus_storage::PackedRecordBatchWriter writer(
fs,
paths,
schema_->ConvertToArrowSchema(),
storage_config,
column_groups,
writer_memory);
// Generate and write data
int64_t row_count = 0;
int start_id = 0;
std::vector<std::string> str_data;
for (int i = 0; i < test_data_count_ * chunk_num_; i++) {
str_data.push_back("test" + std::to_string(i));
}
std::sort(str_data.begin(), str_data.end());
fields_ = {
{"int64", schema_->get_field_id(FieldName("int64"))},
{"ts", TimestampFieldID},
{"vector_array", schema_->get_field_id(FieldName("vector_array"))}};
auto arrow_schema = schema_->ConvertToArrowSchema();
for (int chunk_id = 0; chunk_id < chunk_num_;
chunk_id++, start_id += test_data_count_) {
std::vector<int64_t> test_data(test_data_count_);
std::iota(test_data.begin(), test_data.end(), start_id);
// Create arrow arrays for each field
std::vector<std::shared_ptr<arrow::Array>> arrays;
for (int i = 0; i < arrow_schema->fields().size(); i++) {
if (arrow_schema->fields()[i]->type()->id() ==
arrow::Type::INT64) {
arrow::Int64Builder builder;
auto status = builder.AppendValues(test_data.data(),
test_data_count_);
EXPECT_TRUE(status.ok());
std::shared_ptr<arrow::Array> array;
status = builder.Finish(&array);
EXPECT_TRUE(status.ok());
arrays.push_back(array);
} else {
// vector array
arrow::BinaryBuilder builder;
for (int row = 0; row < test_data_count_; row++) {
milvus::proto::schema::VectorField
field_float_vector_array;
field_float_vector_array.set_dim(DIM);
auto data = generate_float_vector(10, DIM);
field_float_vector_array.mutable_float_vector()
->mutable_data()
->Add(data.begin(), data.end());
std::string serialized_data;
bool success =
field_float_vector_array.SerializeToString(
&serialized_data);
EXPECT_TRUE(success);
auto status = builder.Append(serialized_data);
EXPECT_TRUE(status.ok());
}
std::shared_ptr<arrow::Array> array;
auto status = builder.Finish(&array);
EXPECT_TRUE(status.ok());
arrays.push_back(array);
}
}
// Create record batch
auto record_batch = arrow::RecordBatch::Make(
schema_->ConvertToArrowSchema(), test_data_count_, arrays);
row_count += test_data_count_;
EXPECT_TRUE(writer.Write(record_batch).ok());
}
EXPECT_TRUE(writer.Close().ok());
LoadFieldDataInfo load_info;
load_info.field_infos.emplace(
int64_t(0),
FieldBinlogInfo{
int64_t(0),
static_cast<int64_t>(row_count),
std::vector<int64_t>(chunk_num_ * test_data_count_),
std::vector<int64_t>(chunk_num_ * test_data_count_ * 4),
false,
std::vector<std::string>({paths[0]})});
load_info.field_infos.emplace(
int64_t(101),
FieldBinlogInfo{int64_t(101),
static_cast<int64_t>(row_count),
std::vector<int64_t>(chunk_num_ * test_data_count_),
std::vector<int64_t>(chunk_num_ * test_data_count_ *
10 * 4 * DIM),
false,
std::vector<std::string>({paths[1]})});
load_info.mmap_dir_path = "";
load_info.storage_version = 2;
segment_->AddFieldDataInfoForSealed(load_info);
for (auto& [id, info] : load_info.field_infos) {
LoadFieldDataInfo load_field_info;
load_field_info.storage_version = 2;
load_field_info.mmap_dir_path = "";
load_field_info.field_infos.emplace(id, info);
segment_->LoadFieldData(load_field_info);
}
}
void
TearDown() override {
// Clean up test data directory
auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance()
.GetArrowFileSystem();
fs->DeleteDir("/tmp/test_vector_array_for_storage_v2");
}
protected:
SchemaPtr schema_;
segcore::SegmentSealedUPtr segment_;
int chunk_num_ = 2;
int test_data_count_ = 100;
std::unordered_map<std::string, FieldId> fields_;
};
TEST_F(TestVectorArrayStorageV2, BuildEmbListHNSWIndex) {
ASSERT_NE(segment_, nullptr);
ASSERT_EQ(segment_->get_row_count(), test_data_count_ * chunk_num_);
auto vector_array_field_id = fields_["vector_array"];
ASSERT_TRUE(segment_->HasFieldData(vector_array_field_id));
// Get the storage v2 parquet file paths that were already written in SetUp
std::vector<std::string> paths = {"test_data/101/10001.parquet"};
// Use the existing Arrow file system from SetUp
auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance()
.GetArrowFileSystem();
// Prepare for index building
int64_t collection_id = 1;
int64_t partition_id = 2;
int64_t segment_id = 3;
int64_t index_build_id = 4000;
int64_t index_version = 4000;
auto field_meta =
milvus::segcore::gen_field_meta(collection_id,
partition_id,
segment_id,
vector_array_field_id.get(),
DataType::VECTOR_ARRAY,
DataType::VECTOR_FLOAT,
false);
auto index_meta = gen_index_meta(
segment_id, vector_array_field_id.get(), index_build_id, index_version);
// Create storage config pointing to the test data location
auto storage_config =
gen_local_storage_config("/tmp/test_vector_array_for_storage_v2");
auto cm = CreateChunkManager(storage_config);
// Create index using storage v2 config
milvus::index::CreateIndexInfo create_index_info;
create_index_info.field_type = DataType::VECTOR_ARRAY;
create_index_info.metric_type = knowhere::metric::L2;
create_index_info.index_type = knowhere::IndexEnum::INDEX_EMB_LIST_HNSW;
create_index_info.index_engine_version =
knowhere::Version::GetCurrentVersion().VersionNumber();
auto emb_list_hnsw_index =
milvus::index::IndexFactory::GetInstance().CreateIndex(
create_index_info,
storage::FileManagerContext(field_meta, index_meta, cm, fs));
// Build index with storage v2 configuration
Config config;
config[milvus::index::INDEX_TYPE] =
knowhere::IndexEnum::INDEX_EMB_LIST_HNSW;
config[knowhere::meta::METRIC_TYPE] = create_index_info.metric_type;
config[knowhere::indexparam::M] = "16";
config[knowhere::indexparam::EF] = "10";
config[DIM_KEY] = DIM;
config[INDEX_NUM_ROWS_KEY] =
test_data_count_ * chunk_num_; // Important: set row count
config[STORAGE_VERSION_KEY] = 2; // Use storage v2
config[DATA_TYPE_KEY] = DataType::VECTOR_ARRAY;
// For storage v2, we need to provide segment insert files instead of individual binlog files
milvus::SegmentInsertFiles segment_insert_files;
segment_insert_files.emplace_back(
paths); // Column group with vector array field
config[SEGMENT_INSERT_FILES_KEY] = segment_insert_files;
emb_list_hnsw_index->Build(config);
auto vec_index =
dynamic_cast<milvus::index::VectorIndex*>(emb_list_hnsw_index.get());
// Each row has 10 vectors, so total count should be rows * 10
EXPECT_EQ(vec_index->Count(), test_data_count_ * chunk_num_ * 10);
EXPECT_EQ(vec_index->GetDim(), DIM);
{
auto vec_num = 10;
std::vector<float> query_vec = generate_float_vector(vec_num, DIM);
auto query_dataset =
knowhere::GenDataSet(vec_num, DIM, query_vec.data());
std::vector<size_t> query_vec_lims;
query_vec_lims.push_back(0);
query_vec_lims.push_back(3);
query_vec_lims.push_back(10);
query_dataset->SetLims(query_vec_lims.data());
auto search_conf = knowhere::Json{{knowhere::indexparam::NPROBE, 10}};
milvus::SearchInfo searchInfo;
searchInfo.topk_ = 5;
searchInfo.metric_type_ = knowhere::metric::L2;
searchInfo.search_params_ = search_conf;
SearchResult result;
vec_index->Query(query_dataset, searchInfo, nullptr, result);
auto ref_result = SearchResultToJson(result);
std::cout << ref_result.dump(1) << std::endl;
EXPECT_EQ(result.total_nq_, 2);
EXPECT_EQ(result.distances_.size(), 2 * searchInfo.topk_);
}
}

View File

@ -117,7 +117,8 @@ func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (m
if len(pack.insertData) == 0 {
return make(map[int64]*datapb.FieldBinlog), nil
}
columnGroups := storagecommon.SplitBySchema(bw.schema.GetFields())
allFields := typeutil.GetAllFieldSchemas(bw.schema)
columnGroups := storagecommon.SplitBySchema(allFields)
rec, err := bw.serializeBinlog(ctx, pack)
if err != nil {
@ -193,9 +194,9 @@ func (bw *BulkPackWriterV2) serializeBinlog(ctx context.Context, pack *SyncPack)
}
rec := builder.NewRecord()
field2Col := make(map[storage.FieldID]int, len(bw.schema.GetFields()))
for c, field := range bw.schema.GetFields() {
allFields := typeutil.GetAllFieldSchemas(bw.schema)
field2Col := make(map[storage.FieldID]int, len(allFields))
for c, field := range allFields {
field2Col[field.FieldID] = c
}
return storage.NewSimpleArrowRecord(rec, field2Col), nil

View File

@ -89,6 +89,23 @@ func (s *PackWriterV2Suite) SetupTest() {
},
},
},
StructArrayFields: []*schemapb.StructArrayFieldSchema{
{
FieldID: 102,
Name: "struct_array",
Fields: []*schemapb.FieldSchema{
{
FieldID: 103,
Name: "vector_array",
DataType: schemapb.DataType_ArrayOfVector,
ElementType: schemapb.DataType_FloatVector,
TypeParams: []*commonpb.KeyValuePair{
{Key: common.DimKey, Value: "128"},
},
},
},
},
},
}
s.cm = storage.NewLocalChunkManager(objectstorage.RootPath(s.rootPath))
}
@ -220,10 +237,27 @@ func genInsertData(size int, schema *schemapb.CollectionSchema) []*storage.Inser
data[common.RowIDField] = int64(i + 1)
data[common.TimeStampField] = int64(i + 1)
data[100] = int64(i + 1)
vector := lo.RepeatBy(128, func(_ int) float32 {
return rand.Float32()
})
data[101] = vector
arraySize := rand.Intn(3) + 2
vectorData := lo.RepeatBy(arraySize, func(_ int) float32 {
return rand.Float32()
})
vectorArray := &schemapb.VectorField{
Dim: 128,
Data: &schemapb.VectorField_FloatVector{
FloatVector: &schemapb.FloatArray{
Data: vectorData,
},
},
}
data[103] = vectorArray
buf.Append(data)
}
return []*storage.InsertData{buf}

View File

@ -955,6 +955,12 @@ func TestProxy(t *testing.T) {
// params.Save(params.CommonCfg.SessionRetryTimes.Key, "500")
// params.Save(params.CommonCfg.GracefulStopTimeout.Key, "3600")
params.Save(params.EtcdCfg.RequestTimeout.Key, "300000")
params.Save(params.CommonCfg.SessionTTL.Key, "300")
params.Save(params.CommonCfg.SessionRetryTimes.Key, "500")
params.Save(params.CommonCfg.GracefulStopTimeout.Key, "3600")
params.Save(params.CommonCfg.EnableStorageV2.Key, "true")
params.RootCoordGrpcServerCfg.IP = "localhost"
params.QueryCoordGrpcServerCfg.IP = "localhost"
params.DataCoordGrpcServerCfg.IP = "localhost"

View File

@ -288,17 +288,17 @@ func parseSearchInfo(searchParamsPair []*commonpb.KeyValuePair, schema *schemapb
if annField != nil && annField.GetDataType() == schemapb.DataType_ArrayOfVector {
if strings.Contains(searchParamStr, radiusKey) {
return nil, merr.WrapErrParameterInvalid("", "",
"range search is not supported for vector array (embedding list) fields, fieldName: %s", annsFieldName)
"range search is not supported for vector array (embedding list) fields, fieldName:", annsFieldName)
}
if groupByFieldId > 0 {
return nil, merr.WrapErrParameterInvalid("", "",
"group by search is not supported for vector array (embedding list) fields, fieldName: %s", annsFieldName)
"group by search is not supported for vector array (embedding list) fields, fieldName:", annsFieldName)
}
if isIterator {
return nil, merr.WrapErrParameterInvalid("", "",
"search iterator is not supported for vector array (embedding list) fields, fieldName: %s", annsFieldName)
"search iterator is not supported for vector array (embedding list) fields, fieldName:", annsFieldName)
}
}
}

View File

@ -725,6 +725,60 @@ func TestTranslateOutputFields(t *testing.T) {
})
}
func TestTranslateOutputFields_StructArrayField(t *testing.T) {
const (
idFieldName = "id"
tsFieldName = "timestamp"
floatVectorFieldName = "float_vector"
binaryVectorFieldName = "binary_vector"
float16VectorFieldName = "float16_vector"
bfloat16VectorFieldName = "bfloat16_vector"
sparseFloatVectorFieldName = "sparse_float_vector"
)
var outputFields []string
var userOutputFields []string
var userDynamicFields []string
var requestedPK bool
var err error
collSchema := &schemapb.CollectionSchema{
Name: "TestTranslateOutputFields",
Description: "TestTranslateOutputFields",
AutoID: false,
Fields: []*schemapb.FieldSchema{
{Name: idFieldName, FieldID: 0, DataType: schemapb.DataType_Int64, IsPrimaryKey: true},
{Name: tsFieldName, FieldID: 1, DataType: schemapb.DataType_Int64},
{Name: floatVectorFieldName, FieldID: 100, DataType: schemapb.DataType_FloatVector},
},
StructArrayFields: []*schemapb.StructArrayFieldSchema{
{
FieldID: 101,
Name: "struct_array_field",
Fields: []*schemapb.FieldSchema{
{Name: "sub_field", FieldID: 102, DataType: schemapb.DataType_Array, ElementType: schemapb.DataType_Int64},
{Name: "sub_vector_field", FieldID: 103, DataType: schemapb.DataType_ArrayOfVector, ElementType: schemapb.DataType_FloatVector},
},
},
},
}
schema := newSchemaInfo(collSchema)
// Test struct array field
outputFields, userOutputFields, userDynamicFields, requestedPK, err = translateOutputFields([]string{"sub_vector_field"}, schema, false)
assert.NoError(t, err)
assert.ElementsMatch(t, []string{"sub_vector_field"}, outputFields)
assert.ElementsMatch(t, []string{"sub_vector_field"}, userOutputFields)
assert.ElementsMatch(t, []string{}, userDynamicFields)
assert.False(t, requestedPK)
outputFields, userOutputFields, userDynamicFields, requestedPK, err = translateOutputFields([]string{"struct_array_field", "sub_field"}, schema, false)
assert.NoError(t, err)
assert.ElementsMatch(t, []string{"sub_vector_field", "sub_field"}, outputFields)
assert.ElementsMatch(t, []string{"sub_vector_field", "sub_field"}, userOutputFields)
assert.ElementsMatch(t, []string{}, userDynamicFields)
assert.False(t, requestedPK)
}
func TestAddFieldTask(t *testing.T) {
rc := NewMixCoordMock()
ctx := context.Background()

View File

@ -285,6 +285,27 @@ func (v *validateUtil) checkAligned(data []*schemapb.FieldData, schema *typeutil
if n != numRows {
return errNumRowsMismatch(field.GetFieldName(), n)
}
case schemapb.DataType_ArrayOfVector:
f, err := schema.GetFieldFromName(field.GetFieldName())
if err != nil {
return err
}
dim, err := typeutil.GetDim(f)
if err != nil {
return err
}
dataDim := field.GetVectors().GetVectorArray().GetDim()
if dataDim != dim {
return errDimMismatch(field.GetFieldName(), dataDim, dim)
}
n := uint64(len(field.GetVectors().GetVectorArray().GetData()))
if n != numRows {
return errNumRowsMismatch(field.GetFieldName(), n)
}
default:
// error won't happen here.
n, err := funcutil.GetNumRowOfFieldDataWithSchema(field, schema)

View File

@ -6928,3 +6928,225 @@ func TestCheckArrayElementNilData(t *testing.T) {
err := v.checkArrayElement(data, fieldSchema)
assert.True(t, merr.ErrParameterInvalid.Is(err))
}
func Test_validateUtil_checkArrayOfVectorFieldData(t *testing.T) {
t.Run("nil data", func(t *testing.T) {
f := &schemapb.FieldData{
Field: &schemapb.FieldData_Vectors{
Vectors: &schemapb.VectorField{},
},
}
fieldSchema := &schemapb.FieldSchema{
ElementType: schemapb.DataType_FloatVector,
}
v := newValidateUtil()
err := v.checkArrayOfVectorFieldData(f, fieldSchema)
assert.Error(t, err)
})
t.Run("nil vector in array", func(t *testing.T) {
f := &schemapb.FieldData{
Field: &schemapb.FieldData_Vectors{
Vectors: &schemapb.VectorField{
Data: &schemapb.VectorField_VectorArray{
VectorArray: &schemapb.VectorArray{
Data: []*schemapb.VectorField{{Data: nil}},
},
},
},
},
}
fieldSchema := &schemapb.FieldSchema{
ElementType: schemapb.DataType_FloatVector,
}
v := newValidateUtil()
err := v.checkArrayOfVectorFieldData(f, fieldSchema)
assert.Error(t, err)
})
t.Run("no check", func(t *testing.T) {
f := &schemapb.FieldData{
Field: &schemapb.FieldData_Vectors{
Vectors: &schemapb.VectorField{
Data: &schemapb.VectorField_VectorArray{
VectorArray: &schemapb.VectorArray{
Data: []*schemapb.VectorField{
{
Data: &schemapb.VectorField_FloatVector{
FloatVector: &schemapb.FloatArray{
Data: []float32{1.0, 2.0, float32(math.NaN())},
},
},
},
},
},
},
},
},
}
fieldSchema := &schemapb.FieldSchema{
ElementType: schemapb.DataType_FloatVector,
}
v := newValidateUtil()
v.checkNAN = false
err := v.checkArrayOfVectorFieldData(f, fieldSchema)
assert.NoError(t, err)
})
t.Run("has nan", func(t *testing.T) {
f := &schemapb.FieldData{
Field: &schemapb.FieldData_Vectors{
Vectors: &schemapb.VectorField{
Data: &schemapb.VectorField_VectorArray{
VectorArray: &schemapb.VectorArray{
Data: []*schemapb.VectorField{
{
Data: &schemapb.VectorField_FloatVector{
FloatVector: &schemapb.FloatArray{
Data: []float32{float32(math.NaN())},
},
},
},
},
},
},
},
},
}
fieldSchema := &schemapb.FieldSchema{
ElementType: schemapb.DataType_FloatVector,
}
v := newValidateUtil(withNANCheck())
err := v.checkArrayOfVectorFieldData(f, fieldSchema)
assert.Error(t, err)
})
t.Run("normal case", func(t *testing.T) {
f := &schemapb.FieldData{
Field: &schemapb.FieldData_Vectors{
Vectors: &schemapb.VectorField{
Data: &schemapb.VectorField_VectorArray{
VectorArray: &schemapb.VectorArray{
Data: []*schemapb.VectorField{
{
Data: &schemapb.VectorField_FloatVector{
FloatVector: &schemapb.FloatArray{
Data: []float32{1.0, 2.0},
},
},
},
{
Data: &schemapb.VectorField_FloatVector{
FloatVector: &schemapb.FloatArray{
Data: []float32{3.0, 4.0},
},
},
},
},
},
},
},
},
}
fieldSchema := &schemapb.FieldSchema{
ElementType: schemapb.DataType_FloatVector,
}
v := newValidateUtil(withNANCheck())
err := v.checkArrayOfVectorFieldData(f, fieldSchema)
assert.NoError(t, err)
})
}
func Test_validateUtil_checkAligned_ArrayOfVector(t *testing.T) {
t.Run("array of vector dim mismatch", func(t *testing.T) {
data := []*schemapb.FieldData{
{
FieldName: "test",
Type: schemapb.DataType_ArrayOfVector,
Field: &schemapb.FieldData_Vectors{
Vectors: &schemapb.VectorField{
Data: &schemapb.VectorField_VectorArray{
VectorArray: &schemapb.VectorArray{
Dim: 16,
Data: []*schemapb.VectorField{
{
Data: &schemapb.VectorField_FloatVector{
FloatVector: &schemapb.FloatArray{
Data: make([]float32, 16),
},
},
},
},
},
},
},
},
},
}
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
Name: "test",
DataType: schemapb.DataType_ArrayOfVector,
ElementType: schemapb.DataType_FloatVector,
TypeParams: []*commonpb.KeyValuePair{
{Key: common.DimKey, Value: "8"},
},
},
},
}
h, err := typeutil.CreateSchemaHelper(schema)
assert.NoError(t, err)
v := newValidateUtil()
err = v.checkAligned(data, h, 1)
assert.Error(t, err)
})
t.Run("array of vector row num mismatch", func(t *testing.T) {
data := []*schemapb.FieldData{
{
FieldName: "test",
Type: schemapb.DataType_ArrayOfVector,
Field: &schemapb.FieldData_Vectors{
Vectors: &schemapb.VectorField{
Data: &schemapb.VectorField_VectorArray{
VectorArray: &schemapb.VectorArray{
Dim: 8,
Data: []*schemapb.VectorField{
{
Data: &schemapb.VectorField_FloatVector{
FloatVector: &schemapb.FloatArray{
Data: make([]float32, 8),
},
},
},
},
},
},
},
},
},
}
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
Name: "test",
DataType: schemapb.DataType_ArrayOfVector,
ElementType: schemapb.DataType_FloatVector,
TypeParams: []*commonpb.KeyValuePair{
{Key: common.DimKey, Value: "8"},
},
},
},
}
h, err := typeutil.CreateSchemaHelper(schema)
assert.NoError(t, err)
v := newValidateUtil()
err = v.checkAligned(data, h, 100)
assert.Error(t, err)
})
}

View File

@ -743,8 +743,9 @@ func separateLoadInfoV2(loadInfo *querypb.SegmentLoadInfo, schema *schemapb.Coll
fieldID := fieldBinlog.FieldID
if fieldID == storagecommon.DefaultShortColumnGroupID {
allFields := typeutil.GetAllFieldSchemas(schema)
// for short column group, we need to load all fields in the group
for _, field := range schema.GetFields() {
for _, field := range allFields {
if infos, ok := fieldID2IndexInfo[field.GetFieldID()]; ok {
for _, indexInfo := range infos {
fieldInfo := &IndexedFieldInfo{

View File

@ -366,6 +366,7 @@ func (t *SearchTask) combinePlaceHolderGroups() error {
if err := proto.Unmarshal(t.placeholderGroup, ret); err != nil {
return merr.WrapErrParameterInvalidMsg("invalid search vector placeholder: %v", err)
}
if len(ret.GetPlaceholders()) == 0 {
return merr.WrapErrParameterInvalidMsg("empty search vector is not allowed")
}

View File

@ -210,6 +210,8 @@ func (t *createCollectionTask) assignFieldAndFunctionID(schema *schemapb.Collect
for _, field := range structArrayField.GetFields() {
field.FieldID = int64(idx + StartOfUserFieldID)
idx++
// Also register sub-field names in name2id map
name2id[field.GetName()] = field.GetFieldID()
}
}

View File

@ -8,13 +8,11 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/storagev2/packed"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
func ConvertToArrowSchema(schema *schemapb.CollectionSchema) (*arrow.Schema, error) {
fieldCount := len(schema.GetFields())
for _, structField := range schema.GetStructArrayFields() {
fieldCount += len(structField.GetFields())
}
fieldCount := typeutil.GetTotalFieldsNum(schema)
arrowFields := make([]arrow.Field, 0, fieldCount)
appendArrowField := func(field *schemapb.FieldSchema) error {
if serdeMap[field.DataType].arrowType == nil {
@ -23,7 +21,7 @@ func ConvertToArrowSchema(schema *schemapb.CollectionSchema) (*arrow.Schema, err
var dim int
switch field.DataType {
case schemapb.DataType_BinaryVector, schemapb.DataType_Float16Vector, schemapb.DataType_BFloat16Vector,
schemapb.DataType_Int8Vector, schemapb.DataType_FloatVector:
schemapb.DataType_Int8Vector, schemapb.DataType_FloatVector, schemapb.DataType_ArrayOfVector:
var err error
dim, err = GetDimFromParams(field.TypeParams)
if err != nil {

View File

@ -114,7 +114,8 @@ func newPackedRecordReader(paths [][]string, schema *schemapb.CollectionSchema,
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
}
field2Col := make(map[FieldID]int)
for i, field := range schema.Fields {
allFields := typeutil.GetAllFieldSchemas(schema)
for i, field := range allFields {
field2Col[field.FieldID] = i
}
return &packedRecordReader{
@ -158,8 +159,10 @@ func (pw *packedRecordWriter) Write(r Record) error {
var rec arrow.Record
sar, ok := r.(*simpleArrowRecord)
if !ok {
arrays := make([]arrow.Array, len(pw.schema.Fields))
for i, field := range pw.schema.Fields {
// Get all fields including struct sub-fields
allFields := typeutil.GetAllFieldSchemas(pw.schema)
arrays := make([]arrow.Array, len(allFields))
for i, field := range allFields {
arrays[i] = r.Column(field.FieldID)
}
rec = array.NewRecord(pw.arrowSchema, arrays, int64(r.Len()))
@ -354,7 +357,8 @@ func (pw *PackedBinlogRecordWriter) Write(r Record) error {
func (pw *PackedBinlogRecordWriter) initWriters(r Record) error {
if pw.writer == nil {
if len(pw.columnGroups) == 0 {
pw.columnGroups = storagecommon.SplitBySchema(pw.schema.Fields)
allFields := typeutil.GetAllFieldSchemas(pw.schema)
pw.columnGroups = storagecommon.SplitBySchema(allFields)
}
logIdStart, _, err := pw.allocator.Alloc(uint32(len(pw.columnGroups)))
if err != nil {

View File

@ -55,7 +55,8 @@ func IsVectorDataType(dataType schemapb.DataType) bool {
schemapb.DataType_BFloat16Vector,
schemapb.DataType_Int8Vector,
schemapb.DataType_FloatVector,
schemapb.DataType_SparseFloatVector:
schemapb.DataType_SparseFloatVector,
schemapb.DataType_ArrayOfVector:
return true
}
return false

View File

@ -52,7 +52,7 @@ func TestInvalidUTF8(t *testing.T) {
wf, err := os.OpenFile(filePath, os.O_RDWR|os.O_CREATE, 0o666)
assert.NoError(t, err)
pqSchema, err := ConvertToArrowSchema(schema, false)
pqSchema, err := ConvertToArrowSchemaForUT(schema, false)
assert.NoError(t, err)
fw, err := pqarrow.NewFileWriter(pqSchema, wf,
parquet.NewWriterProperties(parquet.WithMaxRowGroupLength(numRows)), pqarrow.DefaultWriterProps())

View File

@ -84,7 +84,7 @@ func writeParquet(w io.Writer, schema *schemapb.CollectionSchema, numRows int, n
if nullPercent == 100 {
useNullType = true
}
pqSchema, err := ConvertToArrowSchema(schema, useNullType)
pqSchema, err := ConvertToArrowSchemaForUT(schema, useNullType)
if err != nil {
return nil, err
}
@ -667,7 +667,7 @@ func TestParquetReaderError(t *testing.T) {
wf, err := os.OpenFile(filePath, os.O_RDWR|os.O_CREATE, 0o666)
assert.NoError(t, err)
pqSchema, err := ConvertToArrowSchema(colSchema, false)
pqSchema, err := ConvertToArrowSchemaForUT(colSchema, false)
assert.NoError(t, err)
fw, err := pqarrow.NewFileWriter(pqSchema, wf, parquet.NewWriterProperties(parquet.WithMaxRowGroupLength(int64(numRows))), pqarrow.DefaultWriterProps())

View File

@ -292,7 +292,7 @@ func convertToArrowDataType(field *schemapb.FieldSchema, isArray bool) (arrow.Da
// This method is used only by import util and related tests. Returned arrow.Schema
// doesn't include function output fields.
func ConvertToArrowSchema(schema *schemapb.CollectionSchema, useNullType bool) (*arrow.Schema, error) {
func ConvertToArrowSchemaForUT(schema *schemapb.CollectionSchema, useNullType bool) (*arrow.Schema, error) {
arrFields := make([]arrow.Field, 0)
for _, field := range schema.GetFields() {
if typeutil.IsAutoPKField(field) || field.GetIsFunctionOutput() {

View File

@ -248,6 +248,29 @@ func genEmptyInt8VectorFieldData(field *schemapb.FieldSchema) (*schemapb.FieldDa
}, nil
}
func genEmptyArrayOfVectorFieldData(field *schemapb.FieldSchema) (*schemapb.FieldData, error) {
dim, err := GetDim(field)
if err != nil {
return nil, err
}
return &schemapb.FieldData{
Type: field.GetDataType(),
FieldName: field.GetName(),
Field: &schemapb.FieldData_Vectors{
Vectors: &schemapb.VectorField{
Dim: dim,
Data: &schemapb.VectorField_VectorArray{
VectorArray: &schemapb.VectorArray{
Data: []*schemapb.VectorField{},
Dim: dim,
ElementType: field.GetElementType(),
},
},
},
},
}, nil
}
func GenEmptyFieldData(field *schemapb.FieldSchema) (*schemapb.FieldData, error) {
dataType := field.GetDataType()
switch dataType {
@ -281,6 +304,8 @@ func GenEmptyFieldData(field *schemapb.FieldSchema) (*schemapb.FieldData, error)
return genEmptySparseFloatVectorFieldData(field)
case schemapb.DataType_Int8Vector:
return genEmptyInt8VectorFieldData(field)
case schemapb.DataType_ArrayOfVector:
return genEmptyArrayOfVectorFieldData(field)
default:
return nil, fmt.Errorf("unsupported data type: %s", dataType.String())
}

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/metric"
"github.com/milvus-io/milvus/tests/integration"
"github.com/stretchr/testify/suite"
)
type ArrayStructDataNodeSuite struct {
@ -90,7 +91,7 @@ func (s *ArrayStructDataNodeSuite) loadCollection(collectionName string) {
AutoID: false,
}
structF := &schemapb.StructArrayFieldSchema{
FieldID: 105,
FieldID: 102,
Name: integration.StructArrayField,
Fields: []*schemapb.FieldSchema{sId, sVec},
}
@ -156,15 +157,28 @@ func (s *ArrayStructDataNodeSuite) loadCollection(collectionName string) {
// create index
createIndexStatus, err := c.MilvusClient.CreateIndex(context.TODO(), &milvuspb.CreateIndexRequest{
DbName: dbName,
CollectionName: collectionName,
FieldName: integration.FloatVecField,
IndexName: "_default",
IndexName: "float_vector_index",
ExtraParams: integration.ConstructIndexParam(s.dim, integration.IndexFaissIvfFlat, metric.IP),
})
s.NoError(err)
err = merr.Error(createIndexStatus)
s.NoError(err)
s.WaitForIndexBuilt(context.TODO(), collectionName, integration.FloatVecField)
createIndexResult, err := c.MilvusClient.CreateIndex(context.TODO(), &milvuspb.CreateIndexRequest{
DbName: dbName,
CollectionName: collectionName,
FieldName: integration.StructSubFloatVecField,
IndexName: "array_of_vector_index",
ExtraParams: integration.ConstructIndexParam(s.dim, integration.IndexEmbListHNSW, metric.MaxSim),
})
s.NoError(err)
s.Require().Equal(createIndexResult.GetErrorCode(), commonpb.ErrorCode_Success)
s.WaitForIndexBuilt(context.TODO(), collectionName, integration.StructSubFloatVecField)
log.Info("=========================Index created=========================")
// load
@ -299,12 +313,10 @@ func (s *ArrayStructDataNodeSuite) query(collectionName string) {
nq := 10
topk := 10
roundDecimal := -1
radius := 10
params := integration.GetSearchParams(integration.IndexFaissIvfFlat, metric.IP)
params["radius"] = radius
searchReq := integration.ConstructSearchRequest("", collectionName, expr,
integration.StructSubFloatVecField, schemapb.DataType_FloatVector, nil, metric.IP, params, nq, s.dim, topk, roundDecimal)
searchReq := integration.ConstructEmbeddingListSearchRequest("", collectionName, expr,
integration.StructSubFloatVecField, schemapb.DataType_FloatVector, nil, metric.MaxSim, params, nq, s.dim, topk, roundDecimal)
searchResult, _ := c.MilvusClient.Search(context.TODO(), searchReq)
@ -323,5 +335,5 @@ func (s *ArrayStructDataNodeSuite) TestSwapQN() {
func TestArrayStructDataNodeUtil(t *testing.T) {
// skip struct array test
// suite.Run(t, new(ArrayStructDataNodeSuite))
suite.Run(t, new(ArrayStructDataNodeSuite))
}

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
"github.com/milvus-io/milvus/pkg/v2/util/metric"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/tests/integration"
)
@ -47,6 +48,11 @@ type TestArrayStructSuite struct {
}
func (s *TestArrayStructSuite) run() {
revertGuard := s.Cluster.MustModifyMilvusConfig(map[string]string{
paramtable.Get().CommonCfg.EnableStorageV2.Key: "true",
})
defer revertGuard()
ctx, cancel := context.WithCancel(s.Cluster.GetContext())
defer cancel()
@ -102,7 +108,7 @@ func (s *TestArrayStructSuite) run() {
}
structSubVec := &schemapb.FieldSchema{
FieldID: 102,
FieldID: 103,
Name: structSubVecFieldName,
IsPrimaryKey: false,
Description: "",
@ -118,7 +124,7 @@ func (s *TestArrayStructSuite) run() {
}
structField := &schemapb.StructArrayFieldSchema{
FieldID: 103,
FieldID: 102,
Name: structFieldName,
Fields: []*schemapb.FieldSchema{structSubVec},
}
@ -179,25 +185,67 @@ func (s *TestArrayStructSuite) run() {
s.Require().NoError(err)
s.Require().NotEmpty(segments)
// // create index
// createIndexResult, err := s.Cluster.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
// DbName: s.dbName,
// CollectionName: collection,
// FieldName: structSubVecFieldName,
// IndexName: "_default",
// ExtraParams: integration.ConstructIndexParam(dim, s.indexType, s.metricType),
// })
// s.Require().NoError(err)
// s.Require().Equal(createIndexResult.GetErrorCode(), commonpb.ErrorCode_Success)
// create index for float vector field
_, err = s.Cluster.MilvusClient.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
DbName: s.dbName,
CollectionName: collection,
FieldName: vecFieldName,
IndexName: "float_vector_index",
ExtraParams: integration.ConstructIndexParam(dim, integration.IndexFaissIvfFlat, metric.L2),
})
s.Require().NoError(err)
s.Require().Equal(createCollectionStatus.GetErrorCode(), commonpb.ErrorCode_Success)
// s.WaitForIndexBuiltWithDB(ctx, s.dbName, collection, structSubVecFieldName)
s.WaitForIndexBuiltWithDB(ctx, s.dbName, collection, vecFieldName)
// create index for struct sub-vector field
createIndexResult, err := s.Cluster.MilvusClient.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
DbName: s.dbName,
CollectionName: collection,
FieldName: structSubVecFieldName,
IndexName: "array_of_vector_index",
ExtraParams: integration.ConstructIndexParam(dim, s.indexType, s.metricType),
})
s.Require().NoError(err)
s.Require().Equal(createIndexResult.GetErrorCode(), commonpb.ErrorCode_Success)
s.WaitForIndexBuiltWithDB(ctx, s.dbName, collection, structSubVecFieldName)
// load
_, err = s.Cluster.MilvusClient.LoadCollection(ctx, &milvuspb.LoadCollectionRequest{
DbName: s.dbName,
CollectionName: collection,
})
s.Require().NoError(err)
s.Require().Equal(createCollectionStatus.GetErrorCode(), commonpb.ErrorCode_Success)
s.WaitForLoadWithDB(ctx, s.dbName, collection)
// search
nq := s.nq
topk := s.topK
outputFields := []string{structSubVecFieldName}
params := integration.GetSearchParams(s.indexType, s.metricType)
searchReq := integration.ConstructEmbeddingListSearchRequest(s.dbName, collection, "",
structSubVecFieldName, s.vecType, outputFields, s.metricType, params, nq, dim, topk, -1)
searchResp, err := s.Cluster.MilvusClient.Search(ctx, searchReq)
s.Require().NoError(err)
s.Require().Equal(commonpb.ErrorCode_Success, searchResp.GetStatus().GetErrorCode())
result := searchResp.GetResults()
s.Require().Len(result.GetIds().GetIntId().GetData(), nq*topk)
s.Require().Len(result.GetScores(), nq*topk)
s.Require().GreaterOrEqual(len(result.GetFieldsData()), 1)
s.Require().EqualValues(nq, result.GetNumQueries())
s.Require().EqualValues(topk, result.GetTopK())
}
func (s *TestArrayStructSuite) TestGetVector_ArrayStruct_FloatVector() {
s.nq = 10
s.topK = 10
s.indexType = integration.IndexHNSW
s.metricType = metric.L2
s.indexType = integration.IndexEmbListHNSW
s.metricType = metric.MaxSim
s.vecType = schemapb.DataType_FloatVector
s.run()
}

View File

@ -86,7 +86,7 @@ func GenerateParquetFileAndReturnInsertData(c *cluster.MiniClusterV3, schema *sc
func searilizeParquetFile(schema *schemapb.CollectionSchema, insertData *storage.InsertData, numRows int) (*bytes.Buffer, error) {
buf := bytes.NewBuffer(make([]byte, 0, 10240))
pqSchema, err := pq.ConvertToArrowSchema(schema, false)
pqSchema, err := pq.ConvertToArrowSchemaForUT(schema, false)
if err != nil {
return nil, err
}

View File

@ -43,6 +43,7 @@ const (
IndexDISKANN = "DISKANN"
IndexSparseInvertedIndex = "SPARSE_INVERTED_INDEX"
IndexSparseWand = "SPARSE_WAND"
IndexEmbListHNSW = "EMB_LIST_HNSW"
)
func (s *MiniClusterSuite) WaitForIndexBuiltWithDB(ctx context.Context, dbName, collection, field string) {
@ -168,6 +169,15 @@ func ConstructIndexParam(dim int, indexType string, metricType string) []*common
Key: "efConstruction",
Value: "200",
})
case IndexEmbListHNSW:
params = append(params, &commonpb.KeyValuePair{
Key: "M",
Value: "16",
})
params = append(params, &commonpb.KeyValuePair{
Key: "efConstruction",
Value: "200",
})
case IndexSparseInvertedIndex:
case IndexSparseWand:
case IndexDISKANN:
@ -185,6 +195,7 @@ func GetSearchParams(indexType string, metricType string) map[string]any {
case IndexFaissIvfFlat, IndexFaissBinIvfFlat, IndexFaissIvfSQ8, IndexFaissIvfPQ, IndexScaNN:
params["nprobe"] = 8
case IndexHNSW:
case IndexEmbListHNSW:
params["ef"] = 200
case IndexDISKANN:
params["search_list"] = 20

View File

@ -319,8 +319,14 @@ func constructPlaceholderGroup(nq, dim int, vectorType schemapb.DataType, isEmbe
placeholderType = commonpb.PlaceholderType_EmbListFloatVector
}
for i := 0; i < nq; i++ {
bs := make([]byte, 0, dim*4)
for j := 0; j < dim; j++ {
vecCount := dim
// generate some embedding lists with random number of vectors
if isEmbeddingList {
vecCount = vecCount * (rand.Intn(10) + 3)
}
bs := make([]byte, 0, vecCount*4)
for j := 0; j < vecCount; j++ {
var buffer bytes.Buffer
f := rand.Float32()
err := binary.Write(&buffer, common.Endian, f)