feat: [2.6] integrate Loon FFI for manifest-based segment loading and index building (#46076)

Cherry-pick from master
pr: #45061 #45488 #45803 #46017 #44991 #45132 #45723 #45726 #45798
#45897 #45918 #44998

This feature integrates the Storage V2 (Loon) FFI interface as a unified
storage layer for segment loading and index building in Milvus. It
enables
manifest-based data access, replacing the traditional binlog-based
approach
with a more efficient columnar storage format.

Key changes:

### Segment Self-Managed Loading Architecture
- Move segment loading orchestration from Go layer to C++ segcore
- Add NewSegmentWithLoadInfo() API for passing load info during segment
creation
- Implement SetLoadInfo() and Load() methods in SegmentInterface
- Support parallel loading of indexed and non-indexed fields
- Enable both sealed and growing segments to self-manage loading

### Storage V2 FFI Integration
- Integrate milvus-storage library's FFI interface for packed columnar
data
- Add manifest path support throughout the data path (SegmentInfo,
LoadInfo)
- Implement ManifestReader for generating manifests from binlogs
- Support zero-copy data exchange using Arrow C Data Interface
- Add ToCStorageConfig() for Go-to-C storage config conversion

### Manifest-Based Index Building
- Extend FileManagerContext to carry loon_ffi_properties
- Implement GetFieldDatasFromManifest() using Arrow C Stream interface
- Support manifest-based reading in DiskFileManagerImpl and
MemFileManagerImpl
- Add fallback to traditional segment insert files when manifest
unavailable

### Compaction Pipeline Updates
- Include manifest path in all compaction task builders (clustering, L0,
mix)
- Update BulkPackWriterV2 to return manifest path
- Propagate manifest metadata through compaction pipeline

### Configuration & Protocol
- Add common.storageV2.useLoonFFI config option (default: false)
- Add manifest_path field to SegmentLoadInfo and related proto messages
- Add manifest field to compaction segment messages

### Bug Fixes
- Fix mmap settings not applied during segment load (key typo fix)
- Populate index info after segment loading to prevent redundant load
tasks
- Fix memory corruption by removing premature transaction handle
destruction

Related issues: #44956, #45060, #39173

## Individual Cherry-Picked Commits

1. **e1c923b5cc** - fix: apply mmap settings correctly during segment
load (#46017)
2. **63b912370b** - enhance: use milvus-storage internal C++ Reader API
for Loon FFI (#45897)
3. **bfc192faa5** - enhance: Resolve issues integrating loon FFI
(#45918)
4. **fb18564631** - enhance: support manifest-based index building with
Loon FFI reader (#45726)
5. **b9ec2392b9** - enhance: integrate StorageV2 FFI interface for
manifest-based segment loading (#45798)
6. **66db3c32e6** - enhance: integrate Storage V2 FFI interface for
unified storage access (#45723)
7. **ae789273ac** - fix: populate index info after segment loading to
prevent redundant load tasks (#45803)
8. **49688b0be2** - enhance: Move segment loading logic from Go layer to
segcore for self-managed loading (#45488)
9. **5b2df88bac** - enhance: [StorageV2] Integrate FFI interface for
packed reader (#45132)
10. **91ff5706ac** - enhance: [StorageV2] add manifest path support for
FFI integration (#44991)
11. **2192bb4a85** - enhance: add NewSegmentWithLoadInfo API to support
segment self-managed loading (#45061)
12. **4296b01da0** - enhance: update delta log serialization APIs to
integrate storage V2 (#44998)

## Technical Details

### Architecture Changes
- **Before**: Go layer orchestrated segment loading, making multiple CGO
calls
- **After**: Segments autonomously manage loading in C++ layer with
single entry point

### Storage Access Pattern
- **Before**: Read individual binlog files through Go storage layer
- **After**: Read manifest file that references packed columnar data via
FFI

### Benefits
- Reduced cross-language call overhead
- Better resource management at C++ level
- Improved I/O performance through batched streaming reads
- Cleaner separation of concerns between Go and C++ layers
- Foundation for proactive schema evolution handling

---------

Signed-off-by: Ted Xu <ted.xu@zilliz.com>
Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
Co-authored-by: Ted Xu <ted.xu@zilliz.com>
This commit is contained in:
congqixia 2025-12-04 17:09:12 +08:00 committed by GitHub
parent 75cf028613
commit f94b04e642
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
106 changed files with 12073 additions and 5763 deletions

View File

@ -1010,6 +1010,7 @@ common:
splitByAvgSize:
enabled: false # enable split by average size policy in storage v2
threshold: 1024 # split by average size policy threshold(in bytes) in storage v2
useLoonFFI: false
# Whether to disable the internal time messaging mechanism for the system.
# If disabled (set to false), the system will not allow DML operations, including insertion, deletion, queries, and searches.
# This helps Milvus-CDC synchronize incremental data

View File

@ -31,6 +31,7 @@ type Params struct {
PreferSegmentSizeRatio float64 `json:"prefer_segment_size_ratio,omitempty"`
BloomFilterApplyBatchSize int `json:"bloom_filter_apply_batch_size,omitempty"`
StorageConfig *indexpb.StorageConfig `json:"storage_config,omitempty"`
UseLoonFFI bool `json:"use_loon_ffi,omitempty"`
}
func GenParams() Params {
@ -46,6 +47,7 @@ func GenParams() Params {
PreferSegmentSizeRatio: paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.GetAsFloat(),
BloomFilterApplyBatchSize: paramtable.Get().CommonCfg.BloomFilterApplyBatchSize.GetAsInt(),
StorageConfig: CreateStorageConfig(),
UseLoonFFI: paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool(),
}
}

View File

@ -112,6 +112,8 @@ const std::string DIM_KEY = "dim";
const std::string DATA_TYPE_KEY = "data_type";
const std::string ELEMENT_TYPE_KEY = "element_type";
const std::string INDEX_NUM_ROWS_KEY = "index_num_rows";
const std::string SEGMENT_MANIFEST_KEY = "segment_manifest";
const std::string LOON_FFI_PROPERTIES_KEY = "loon_ffi_properties";
// storage version
const int64_t STORAGE_V1 = 1;
@ -126,3 +128,5 @@ const std::string ELEMENT_TYPE_KEY_FOR_ARROW = "elementType";
// EPSILON value for comparing float numbers
const float EPSILON = 0.0000000119;
const std::string NAMESPACE_FIELD_NAME = "$namespace_id";
const std::string MMAP_ENABLED_KEY = "mmap.enabled";

View File

@ -26,7 +26,9 @@
#include "Schema.h"
#include "SystemProperty.h"
#include "arrow/util/key_value_metadata.h"
#include "common/Consts.h"
#include "milvus-storage/common/constants.h"
#include "pb/common.pb.h"
#include "protobuf_utils.h"
namespace milvus {
@ -58,6 +60,12 @@ Schema::ParseFrom(const milvus::proto::schema::CollectionSchema& schema_proto) {
"repetitive dynamic field");
schema->set_dynamic_field_id(field_id);
}
auto [has_setting, enabled] =
GetBoolFromRepeatedKVs(child.type_params(), MMAP_ENABLED_KEY);
if (has_setting) {
schema->mmap_fields_[field_id] = enabled;
}
};
for (const milvus::proto::schema::FieldSchema& child :
@ -72,6 +80,9 @@ Schema::ParseFrom(const milvus::proto::schema::CollectionSchema& schema_proto) {
}
}
std::tie(schema->has_mmap_setting_, schema->mmap_enabled_) =
GetBoolFromRepeatedKVs(schema_proto.properties(), MMAP_ENABLED_KEY);
AssertInfo(schema->get_primary_field_id().has_value(),
"primary key should be specified");
@ -147,4 +158,14 @@ Schema::AbsentFields(Schema& old_schema) const {
return std::make_unique<std::vector<FieldMeta>>(result);
}
std::pair<bool, bool>
Schema::MmapEnabled(const FieldId& field_id) const {
auto it = mmap_fields_.find(field_id);
// fallback to collection-level config
if (it == mmap_fields_.end()) {
return {has_mmap_setting_, mmap_enabled_};
}
return {true, it->second};
}
} // namespace milvus

View File

@ -364,6 +364,24 @@ class Schema {
std::unique_ptr<std::vector<FieldMeta>>
AbsentFields(Schema& old_schema) const;
/**
* @brief Determines whether the specified field should use mmap for data loading.
*
* This function checks mmap settings at the field level first. If no field-level
* setting is found, it falls back to the collection-level mmap configuration.
*
* @param field The field ID to check mmap settings for.
*
* @return A pair of booleans:
* - first: Whether an mmap setting exists (at field or collection level).
* - second: Whether mmap is enabled (only meaningful when first is true).
*
* @note If no mmap setting exists at any level, first will be false and second
* should be ignored.
*/
std::pair<bool, bool>
MmapEnabled(const FieldId& field) const;
private:
int64_t debug_id = START_USER_FIELDID;
std::vector<FieldId> field_ids_;
@ -384,6 +402,11 @@ class Schema {
// schema_version_, currently marked with update timestamp
uint64_t schema_version_;
// mmap settings
bool has_mmap_setting_ = false;
bool mmap_enabled_ = false;
std::unordered_map<FieldId, bool> mmap_fields_;
};
using SchemaPtr = std::shared_ptr<Schema>;

View File

@ -0,0 +1,212 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed 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 "common/Schema.h"
using namespace milvus;
class SchemaTest : public ::testing::Test {
protected:
void
SetUp() override {
schema_ = std::make_shared<Schema>();
}
std::shared_ptr<Schema> schema_;
};
TEST_F(SchemaTest, MmapEnabledNoSetting) {
// Add a field without any mmap setting
auto field_id = schema_->AddDebugField("test_field", DataType::INT64);
schema_->set_primary_field_id(field_id);
// When no mmap setting exists at any level, first should be false
auto [has_setting, enabled] = schema_->MmapEnabled(field_id);
EXPECT_FALSE(has_setting);
// The enabled value is undefined when has_setting is false, so we don't check it
}
TEST_F(SchemaTest, MmapEnabledCollectionLevelEnabled) {
// Create schema with collection-level mmap enabled via protobuf
milvus::proto::schema::CollectionSchema schema_proto;
auto* field = schema_proto.add_fields();
field->set_fieldid(100);
field->set_name("pk_field");
field->set_data_type(milvus::proto::schema::DataType::Int64);
field->set_is_primary_key(true);
// Set collection-level mmap enabled
auto* prop = schema_proto.add_properties();
prop->set_key("mmap.enabled");
prop->set_value("true");
auto parsed_schema = Schema::ParseFrom(schema_proto);
FieldId pk_field_id(100);
auto [has_setting, enabled] = parsed_schema->MmapEnabled(pk_field_id);
EXPECT_TRUE(has_setting);
EXPECT_TRUE(enabled);
}
TEST_F(SchemaTest, MmapEnabledCollectionLevelDisabled) {
// Create schema with collection-level mmap disabled via protobuf
milvus::proto::schema::CollectionSchema schema_proto;
auto* field = schema_proto.add_fields();
field->set_fieldid(100);
field->set_name("pk_field");
field->set_data_type(milvus::proto::schema::DataType::Int64);
field->set_is_primary_key(true);
// Set collection-level mmap disabled
auto* prop = schema_proto.add_properties();
prop->set_key("mmap.enabled");
prop->set_value("false");
auto parsed_schema = Schema::ParseFrom(schema_proto);
FieldId pk_field_id(100);
auto [has_setting, enabled] = parsed_schema->MmapEnabled(pk_field_id);
EXPECT_TRUE(has_setting);
EXPECT_FALSE(enabled);
}
TEST_F(SchemaTest, MmapEnabledCollectionLevelCaseInsensitive) {
// Test that mmap value parsing is case-insensitive
milvus::proto::schema::CollectionSchema schema_proto;
auto* field = schema_proto.add_fields();
field->set_fieldid(100);
field->set_name("pk_field");
field->set_data_type(milvus::proto::schema::DataType::Int64);
field->set_is_primary_key(true);
// Set collection-level mmap with uppercase TRUE
auto* prop = schema_proto.add_properties();
prop->set_key("mmap.enabled");
prop->set_value("TRUE");
auto parsed_schema = Schema::ParseFrom(schema_proto);
FieldId pk_field_id(100);
auto [has_setting, enabled] = parsed_schema->MmapEnabled(pk_field_id);
EXPECT_TRUE(has_setting);
EXPECT_TRUE(enabled);
}
TEST_F(SchemaTest, MmapEnabledFieldLevelOverridesCollectionLevel) {
// Test that field-level mmap setting overrides collection-level setting
milvus::proto::schema::CollectionSchema schema_proto;
auto* field = schema_proto.add_fields();
field->set_fieldid(100);
field->set_name("pk_field");
field->set_data_type(milvus::proto::schema::DataType::Int64);
field->set_is_primary_key(true);
// Set collection-level mmap enabled
auto* prop = schema_proto.add_properties();
prop->set_key("mmap.enabled");
prop->set_value("true");
// Note: Field-level mmap settings are set via schema_proto.properties()
// in the current implementation, which applies to all fields.
// This test verifies the fallback behavior when no field-level override exists.
auto parsed_schema = Schema::ParseFrom(schema_proto);
FieldId pk_field_id(100);
// Without field-level override, should use collection-level setting
auto [has_setting, enabled] = parsed_schema->MmapEnabled(pk_field_id);
EXPECT_TRUE(has_setting);
EXPECT_TRUE(enabled);
}
TEST_F(SchemaTest, MmapEnabledNonExistentField) {
// Test MmapEnabled with a field that doesn't exist in mmap_fields_
// but collection-level setting exists
milvus::proto::schema::CollectionSchema schema_proto;
auto* field1 = schema_proto.add_fields();
field1->set_fieldid(100);
field1->set_name("pk_field");
field1->set_data_type(milvus::proto::schema::DataType::Int64);
field1->set_is_primary_key(true);
auto* field2 = schema_proto.add_fields();
field2->set_fieldid(101);
field2->set_name("data_field");
field2->set_data_type(milvus::proto::schema::DataType::Float);
// Set collection-level mmap enabled
auto* prop = schema_proto.add_properties();
prop->set_key("mmap.enabled");
prop->set_value("true");
auto parsed_schema = Schema::ParseFrom(schema_proto);
// Both fields should fallback to collection-level setting
FieldId pk_field_id(100);
auto [has_setting1, enabled1] = parsed_schema->MmapEnabled(pk_field_id);
EXPECT_TRUE(has_setting1);
EXPECT_TRUE(enabled1);
FieldId data_field_id(101);
auto [has_setting2, enabled2] = parsed_schema->MmapEnabled(data_field_id);
EXPECT_TRUE(has_setting2);
EXPECT_TRUE(enabled2);
// Test with a field ID that was never added to the schema
FieldId non_existent_field_id(999);
auto [has_setting3, enabled3] =
parsed_schema->MmapEnabled(non_existent_field_id);
EXPECT_TRUE(has_setting3); // Falls back to collection-level
EXPECT_TRUE(enabled3);
}
TEST_F(SchemaTest, MmapEnabledMultipleFields) {
// Test MmapEnabled with multiple fields, all using collection-level setting
milvus::proto::schema::CollectionSchema schema_proto;
auto* pk_field = schema_proto.add_fields();
pk_field->set_fieldid(100);
pk_field->set_name("pk_field");
pk_field->set_data_type(milvus::proto::schema::DataType::Int64);
pk_field->set_is_primary_key(true);
auto* int_field = schema_proto.add_fields();
int_field->set_fieldid(101);
int_field->set_name("int_field");
int_field->set_data_type(milvus::proto::schema::DataType::Int32);
auto* float_field = schema_proto.add_fields();
float_field->set_fieldid(102);
float_field->set_name("float_field");
float_field->set_data_type(milvus::proto::schema::DataType::Float);
// Set collection-level mmap disabled
auto* prop = schema_proto.add_properties();
prop->set_key("mmap.enabled");
prop->set_value("false");
auto parsed_schema = Schema::ParseFrom(schema_proto);
// All fields should have the same collection-level setting
for (int64_t id = 100; id <= 102; ++id) {
FieldId field_id(id);
auto [has_setting, enabled] = parsed_schema->MmapEnabled(field_id);
EXPECT_TRUE(has_setting);
EXPECT_FALSE(enabled);
}
}

View File

@ -205,6 +205,8 @@ ToProtoDataType(DataType data_type) {
return proto::schema::DataType::Int8Vector;
case DataType::VECTOR_ARRAY:
return proto::schema::DataType::ArrayOfVector;
case DataType::GEOMETRY:
return proto::schema::DataType::Geometry;
// Internal-only or unsupported mappings
case DataType::ROW:

View File

@ -45,6 +45,32 @@ RepeatedKeyValToMap(
return mapping;
}
/**
* @brief Get a boolean value from repeated KeyValuePair by key.
*
* @param kvs The repeated KeyValuePair field to search.
* @param key The key to look for.
* @return std::pair<bool, bool> where:
* - first: whether the key was found.
* - second: the parsed boolean value (true if value is "true", case-insensitive).
*/
static std::pair<bool, bool>
GetBoolFromRepeatedKVs(
const google::protobuf::RepeatedPtrField<proto::common::KeyValuePair>& kvs,
const std::string& key) {
for (auto& kv : kvs) {
if (kv.key() == key) {
std::string lower;
std::transform(kv.value().begin(),
kv.value().end(),
std::back_inserter(lower),
::tolower);
return {true, lower == "true"};
}
}
return {false, false};
}
class ProtoLayout;
using ProtoLayoutPtr = std::unique_ptr<ProtoLayout>;

View File

@ -17,6 +17,7 @@
#include "indexbuilder/type_c.h"
#include "log/Log.h"
#include "storage/PluginLoader.h"
#include "storage/loon_ffi/util.h"
#ifdef __linux__
#include <malloc.h>
@ -177,6 +178,7 @@ get_config(std::unique_ptr<milvus::proto::indexcgo::BuildIndexInfo>& info) {
if (info->storage_version() == STORAGE_V2) {
config[SEGMENT_INSERT_FILES_KEY] =
get_segment_insert_files(info->segment_insert_files());
config[SEGMENT_MANIFEST_KEY] = info->manifest();
}
config[DIM_KEY] = info->dim();
config[DATA_TYPE_KEY] = info->field_schema().data_type();
@ -251,6 +253,11 @@ CreateIndex(CIndex* res_index,
milvus::storage::FileManagerContext fileManagerContext(
field_meta, index_meta, chunk_manager, fs);
if (build_index_info->manifest() != "") {
auto loon_properties = MakeInternalPropertiesFromStorageConfig(
ToCStorageConfig(storage_config));
fileManagerContext.set_loon_ffi_properties(loon_properties);
}
if (build_index_info->has_storage_plugin_context()) {
auto cipherPlugin =
@ -314,6 +321,9 @@ BuildJsonKeyIndex(ProtoLayoutInterface result,
get_storage_config(build_index_info->storage_config());
auto config = get_config(build_index_info);
auto loon_properties =
MakePropertiesFromStorageConfig(ToCStorageConfig(storage_config));
// init file manager
milvus::storage::FieldDataMeta field_meta{
build_index_info->collectionid(),
@ -350,6 +360,12 @@ BuildJsonKeyIndex(ProtoLayoutInterface result,
milvus::storage::FileManagerContext fileManagerContext(
field_meta, index_meta, chunk_manager, fs);
if (build_index_info->manifest() != "") {
auto loon_properties = MakeInternalPropertiesFromStorageConfig(
ToCStorageConfig(storage_config));
fileManagerContext.set_loon_ffi_properties(loon_properties);
}
if (build_index_info->has_storage_plugin_context()) {
auto cipherPlugin =
milvus::storage::PluginLoader::GetInstance().getCipherPlugin();
@ -435,6 +451,12 @@ BuildTextIndex(ProtoLayoutInterface result,
milvus::storage::FileManagerContext fileManagerContext(
field_meta, index_meta, chunk_manager, fs);
if (build_index_info->manifest() != "") {
auto loon_properties = MakeInternalPropertiesFromStorageConfig(
ToCStorageConfig(storage_config));
fileManagerContext.set_loon_ffi_properties(loon_properties);
}
if (build_index_info->has_storage_plugin_context()) {
auto cipherPlugin =
milvus::storage::PluginLoader::GetInstance().getCipherPlugin();

View File

@ -11,6 +11,7 @@
#include "ChunkedSegmentSealedImpl.h"
#include <arrow/c/bridge.h>
#include <arrow/record_batch.h>
#include <fcntl.h>
#include <fmt/core.h>
@ -45,6 +46,8 @@
#include "common/Types.h"
#include "common/resource_c.h"
#include "folly/Synchronized.h"
#include "milvus-storage/properties.h"
#include "milvus-storage/reader.h"
#include "monitor/scope_metric.h"
#include "google/protobuf/message_lite.h"
#include "index/Index.h"
@ -63,10 +66,14 @@
#include "mmap/ChunkedColumnGroup.h"
#include "segcore/storagev1translator/InterimSealedIndexTranslator.h"
#include "segcore/storagev1translator/TextMatchIndexTranslator.h"
#include "segcore/storagev2translator/ManifestGroupTranslator.h"
#include "storage/Util.h"
#include "storage/ThreadPools.h"
#include "storage/MmapManager.h"
#include "storage/loon_ffi/property_singleton.h"
#include "storage/loon_ffi/util.h"
#include "storage/RemoteChunkManagerSingleton.h"
#include "milvus-storage/ffi_c.h"
#include "milvus-storage/filesystem/fs.h"
#include "cachinglayer/CacheSlot.h"
#include "storage/LocalChunkManagerSingleton.h"
@ -242,6 +249,81 @@ ChunkedSegmentSealedImpl::LoadScalarIndex(const LoadIndexInfo& info) {
request.has_raw_data);
}
LoadIndexInfo
ChunkedSegmentSealedImpl::ConvertFieldIndexInfoToLoadIndexInfo(
const milvus::proto::segcore::FieldIndexInfo* field_index_info) const {
LoadIndexInfo load_index_info;
load_index_info.segment_id = id_;
// Extract field ID
auto field_id = FieldId(field_index_info->fieldid());
load_index_info.field_id = field_id.get();
// Get field type from schema
const auto& field_meta = get_schema()[field_id];
load_index_info.field_type = field_meta.get_data_type();
load_index_info.element_type = field_meta.get_element_type();
// Set index metadata
load_index_info.index_id = field_index_info->indexid();
load_index_info.index_build_id = field_index_info->buildid();
load_index_info.index_version = field_index_info->index_version();
load_index_info.index_store_version =
field_index_info->index_store_version();
load_index_info.index_engine_version =
static_cast<IndexVersion>(field_index_info->current_index_version());
load_index_info.index_size = field_index_info->index_size();
load_index_info.num_rows = field_index_info->num_rows();
load_index_info.schema = field_meta.ToProto();
// Copy index file paths, excluding indexParams file
for (const auto& file_path : field_index_info->index_file_paths()) {
size_t last_slash = file_path.find_last_of('/');
std::string filename = (last_slash != std::string::npos)
? file_path.substr(last_slash + 1)
: file_path;
if (filename != "indexParams") {
load_index_info.index_files.push_back(file_path);
}
}
auto& mmap_config = storage::MmapManager::GetInstance().GetMmapConfig();
auto use_mmap = IsVectorDataType(field_meta.get_data_type())
? mmap_config.GetVectorIndexEnableMmap()
: mmap_config.GetScalarIndexEnableMmap();
// Set index params
for (const auto& kv_pair : field_index_info->index_params()) {
if (kv_pair.key() == "mmap.enabled") {
std::string lower;
std::transform(kv_pair.value().begin(),
kv_pair.value().end(),
std::back_inserter(lower),
::tolower);
use_mmap = (lower == "true");
}
load_index_info.index_params[kv_pair.key()] = kv_pair.value();
}
size_t dim =
IsVectorDataType(field_meta.get_data_type()) &&
!IsSparseFloatVectorDataType(field_meta.get_data_type())
? field_meta.get_dim()
: 1;
load_index_info.dim = dim;
auto remote_chunk_manager =
milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
load_index_info.mmap_dir_path =
milvus::storage::LocalChunkManagerSingleton::GetInstance()
.GetChunkManager()
->GetRootPath();
load_index_info.enable_mmap = use_mmap;
return load_index_info;
}
void
ChunkedSegmentSealedImpl::LoadFieldData(const LoadFieldDataInfo& load_info) {
switch (load_info.storage_version) {
@ -255,6 +337,172 @@ ChunkedSegmentSealedImpl::LoadFieldData(const LoadFieldDataInfo& load_info) {
}
}
void
ChunkedSegmentSealedImpl::LoadColumnGroups(const std::string& manifest_path) {
LOG_INFO(
"Loading segment {} field data with manifest {}", id_, manifest_path);
auto properties = milvus::storage::LoonFFIPropertiesSingleton::GetInstance()
.GetProperties();
auto column_groups = GetColumnGroups(manifest_path, properties);
auto arrow_schema = schema_->ConvertToArrowSchema();
reader_ = milvus_storage::api::Reader::create(
column_groups, arrow_schema, nullptr, *properties);
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::LOW);
std::vector<std::future<void>> load_group_futures;
for (int64_t i = 0; i < column_groups->size(); ++i) {
auto future = pool.Submit([this, column_groups, properties, i] {
LoadColumnGroup(column_groups, properties, i);
});
load_group_futures.emplace_back(std::move(future));
}
std::vector<std::exception_ptr> load_exceptions;
for (auto& future : load_group_futures) {
try {
future.get();
} catch (...) {
load_exceptions.push_back(std::current_exception());
}
}
// If any exceptions occurred during index loading, handle them
if (!load_exceptions.empty()) {
LOG_ERROR("Failed to load {} out of {} indexes for segment {}",
load_exceptions.size(),
load_group_futures.size(),
id_);
// Rethrow the first exception
std::rethrow_exception(load_exceptions[0]);
}
}
void
ChunkedSegmentSealedImpl::LoadColumnGroup(
const std::shared_ptr<milvus_storage::api::ColumnGroups>& column_groups,
const std::shared_ptr<milvus_storage::api::Properties>& properties,
int64_t index) {
AssertInfo(index < column_groups->size(),
"load column group index out of range");
auto column_group = column_groups->get_column_group(index);
std::vector<FieldId> milvus_field_ids;
for (auto& column : column_group->columns) {
auto field_id = std::stoll(column);
milvus_field_ids.emplace_back(field_id);
}
auto field_metas = schema_->get_field_metas(milvus_field_ids);
// assumption: vector field occupies whole column group
bool is_vector = false;
bool index_has_rawdata = true;
bool has_mmap_setting = false;
bool mmap_enabled = false;
for (auto& [field_id, field_meta] : field_metas) {
if (IsVectorDataType(field_meta.get_data_type())) {
is_vector = true;
}
std::shared_lock lck(mutex_);
if (index_has_raw_data_.find(field_id) != index_has_raw_data_.end()) {
index_has_rawdata =
index_has_raw_data_.at(field_id) && index_has_rawdata;
} else {
index_has_rawdata = false;
}
// if field has mmap setting, use it
// - mmap setting at collection level, then all field are the same
// - mmap setting at field level, we define that as long as one field shall be mmap, then whole group shall be mmaped
auto [field_has_setting, field_mmap_enabled] =
schema_->MmapEnabled(field_id);
has_mmap_setting = has_mmap_setting || field_has_setting;
mmap_enabled = mmap_enabled || field_mmap_enabled;
}
if (index_has_rawdata) {
LOG_INFO(
"[StorageV2] segment {} index(es) provide all raw data for column "
"group index {}, skip loading binlog",
this->get_segment_id(),
index);
return;
}
auto& mmap_config = storage::MmapManager::GetInstance().GetMmapConfig();
bool global_use_mmap = is_vector ? mmap_config.GetVectorFieldEnableMmap()
: mmap_config.GetScalarFieldEnableMmap();
auto use_mmap = has_mmap_setting ? mmap_enabled : global_use_mmap;
auto chunk_reader_result = reader_->get_chunk_reader(index);
AssertInfo(chunk_reader_result.ok(),
"get chunk reader failed, segment {}, column group index {}",
get_segment_id(),
index);
auto chunk_reader = std::move(chunk_reader_result).ValueOrDie();
LOG_INFO("[StorageV2] segment {} loads manifest cg index {}",
this->get_segment_id(),
index);
auto translator =
std::make_unique<storagev2translator::ManifestGroupTranslator>(
get_segment_id(),
index,
std::move(chunk_reader),
field_metas,
use_mmap,
column_group->columns.size(),
segment_load_info_.priority());
auto chunked_column_group =
std::make_shared<ChunkedColumnGroup>(std::move(translator));
// Create ProxyChunkColumn for each field
for (const auto& field_id : milvus_field_ids) {
auto field_meta = field_metas.at(field_id);
auto column = std::make_shared<ProxyChunkColumn>(
chunked_column_group, field_id, field_meta);
auto data_type = field_meta.get_data_type();
load_field_data_common(field_id,
column,
segment_load_info_.num_of_rows(),
data_type,
use_mmap,
true);
if (field_id == TimestampFieldID) {
auto timestamp_proxy_column = get_column(TimestampFieldID);
AssertInfo(timestamp_proxy_column != nullptr,
"timestamp proxy column is nullptr");
// TODO check timestamp_index ready instead of check system_ready_count_
int64_t num_rows = segment_load_info_.num_of_rows();
auto all_ts_chunks = timestamp_proxy_column->GetAllChunks(nullptr);
std::vector<Timestamp> timestamps(num_rows);
int64_t offset = 0;
for (auto& all_ts_chunk : all_ts_chunks) {
auto chunk_data = all_ts_chunk.get();
auto fixed_chunk = dynamic_cast<FixedWidthChunk*>(chunk_data);
auto span = fixed_chunk->Span();
for (size_t j = 0; j < span.row_count(); j++) {
auto ts = *(int64_t*)((char*)span.data() +
j * span.element_sizeof());
timestamps[offset++] = ts;
}
}
init_timestamp_index(timestamps, num_rows);
system_ready_count_++;
AssertInfo(offset == num_rows,
"[StorageV2] timestamp total row count {} not equal "
"to expected {}",
offset,
num_rows);
}
}
}
void
ChunkedSegmentSealedImpl::load_column_group_data_internal(
const LoadFieldDataInfo& load_info) {
@ -287,7 +535,7 @@ ChunkedSegmentSealedImpl::load_column_group_data_internal(
std::vector<FieldId> milvus_field_ids;
milvus_field_ids.reserve(field_id_list.size());
for (int i = 0; i < field_id_list.size(); ++i) {
milvus_field_ids.push_back(FieldId(field_id_list.Get(i)));
milvus_field_ids.emplace_back(field_id_list.Get(i));
merged_in_load_list = merged_in_load_list ||
schema_->ShouldLoadField(milvus_field_ids[i]);
}
@ -2558,4 +2806,206 @@ ChunkedSegmentSealedImpl::LoadGeometryCache(
}
}
void
ChunkedSegmentSealedImpl::SetLoadInfo(
const proto::segcore::SegmentLoadInfo& load_info) {
std::unique_lock lck(mutex_);
segment_load_info_ = load_info;
LOG_INFO(
"SetLoadInfo for segment {}, num_rows: {}, index count: {}, "
"storage_version: {}",
id_,
segment_load_info_.num_of_rows(),
segment_load_info_.index_infos_size(),
segment_load_info_.storageversion());
}
void
ChunkedSegmentSealedImpl::Load(milvus::tracer::TraceContext& trace_ctx) {
// Get load info from segment_load_info_
auto num_rows = segment_load_info_.num_of_rows();
LOG_INFO("Loading segment {} with {} rows", id_, num_rows);
// Step 1: Separate indexed and non-indexed fields
std::map<FieldId, const proto::segcore::FieldIndexInfo*>
field_id_to_index_info;
std::set<FieldId> indexed_fields;
for (int i = 0; i < segment_load_info_.index_infos_size(); i++) {
const auto& index_info = segment_load_info_.index_infos(i);
if (index_info.index_file_paths_size() == 0) {
continue;
}
auto field_id = FieldId(index_info.fieldid());
field_id_to_index_info[field_id] = &index_info;
indexed_fields.insert(field_id);
}
// Step 2: Load indexes in parallel using thread pool
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::LOW);
std::vector<std::future<void>> load_index_futures;
for (const auto& pair : field_id_to_index_info) {
auto field_id = pair.first;
auto index_info_ptr = pair.second;
auto future = pool.Submit(
[this, trace_ctx, field_id, index_info_ptr, num_rows]() mutable
-> void {
// Convert proto FieldIndexInfo to LoadIndexInfo
auto load_index_info =
ConvertFieldIndexInfoToLoadIndexInfo(index_info_ptr);
LOG_INFO("Loading index for segment {} field {} with {} files",
id_,
field_id.get(),
load_index_info.index_files.size());
// Download & compose index
LoadIndexData(trace_ctx, &load_index_info);
// Load index into segment
LoadIndex(load_index_info);
});
load_index_futures.push_back(std::move(future));
}
// Wait for all index loading to complete and collect exceptions
std::vector<std::exception_ptr> index_exceptions;
for (auto& future : load_index_futures) {
try {
future.get();
} catch (...) {
index_exceptions.push_back(std::current_exception());
}
}
// If any exceptions occurred during index loading, handle them
if (!index_exceptions.empty()) {
LOG_ERROR("Failed to load {} out of {} indexes for segment {}",
index_exceptions.size(),
load_index_futures.size(),
id_);
// Rethrow the first exception
std::rethrow_exception(index_exceptions[0]);
}
LOG_INFO("Finished loading {} indexes for segment {}",
field_id_to_index_info.size(),
id_);
auto manifest_path = segment_load_info_.manifest_path();
if (manifest_path != "") {
LoadColumnGroups(manifest_path);
return;
}
std::map<FieldId, LoadFieldDataInfo> field_data_to_load;
for (int i = 0; i < segment_load_info_.binlog_paths_size(); i++) {
LoadFieldDataInfo load_field_data_info;
load_field_data_info.storage_version =
segment_load_info_.storageversion();
const auto& field_binlog = segment_load_info_.binlog_paths(i);
auto field_id = FieldId(field_binlog.fieldid());
// Skip if this field has an index with raw data
auto iter = index_has_raw_data_.find(field_id);
if (iter != index_has_raw_data_.end() && iter->second) {
LOG_INFO(
"Skip loading binlog for segment {} field {} because index "
"has raw data",
id_,
field_id.get());
continue;
}
// Build FieldBinlogInfo
FieldBinlogInfo field_binlog_info;
field_binlog_info.field_id = field_id.get();
// Calculate total row count and collect binlog paths
int64_t total_entries = 0;
for (const auto& binlog : field_binlog.binlogs()) {
field_binlog_info.insert_files.push_back(binlog.log_path());
field_binlog_info.entries_nums.push_back(binlog.entries_num());
field_binlog_info.memory_sizes.push_back(binlog.memory_size());
total_entries += binlog.entries_num();
}
field_binlog_info.row_count = total_entries;
bool has_mmap_setting = false;
bool mmap_enabled = false;
bool is_vector = false;
for (const auto& child_field_id : field_binlog.child_fields()) {
auto& field_meta = schema_->operator[](FieldId(child_field_id));
if (IsVectorDataType(field_meta.get_data_type())) {
is_vector = true;
}
// if field has mmap setting, use it
// - mmap setting at collection level, then all field are the same
// - mmap setting at field level, we define that as long as one field shall be mmap, then whole group shall be mmaped
auto [field_has_setting, field_mmap_enabled] =
schema_->MmapEnabled(field_id);
has_mmap_setting = has_mmap_setting || field_has_setting;
mmap_enabled = mmap_enabled || field_mmap_enabled;
}
auto& mmap_config = storage::MmapManager::GetInstance().GetMmapConfig();
auto global_use_mmap = is_vector
? mmap_config.GetVectorFieldEnableMmap()
: mmap_config.GetScalarFieldEnableMmap();
field_binlog_info.enable_mmap =
has_mmap_setting ? mmap_enabled : global_use_mmap;
// Store in map
load_field_data_info.field_infos[field_id.get()] = field_binlog_info;
field_data_to_load[field_id] = load_field_data_info;
}
// Step 4: Load field data for non-indexed fields
if (!field_data_to_load.empty()) {
LOG_INFO("Loading field data for {} fields in segment {}",
field_data_to_load.size(),
id_);
std::vector<std::future<void>> load_field_futures;
for (const auto& [field_id, load_field_data_info] :
field_data_to_load) {
// Create a local copy to capture in lambda (C++17 compatible)
const auto field_data = load_field_data_info;
auto future = pool.Submit(
[this, field_data]() -> void { LoadFieldData(field_data); });
load_field_futures.push_back(std::move(future));
}
// Wait for all field data loading to complete and collect exceptions
std::vector<std::exception_ptr> field_exceptions;
for (auto& future : load_field_futures) {
try {
future.get();
} catch (...) {
field_exceptions.push_back(std::current_exception());
}
}
// If any exceptions occurred during field data loading, handle them
if (!field_exceptions.empty()) {
LOG_ERROR("Failed to load {} out of {} field data for segment {}",
field_exceptions.size(),
load_field_futures.size(),
id_);
// Rethrow the first exception
std::rethrow_exception(field_exceptions[0]);
}
}
LOG_INFO("Successfully loaded segment {} with {} rows", id_, num_rows);
}
} // namespace milvus::segcore

View File

@ -186,6 +186,13 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
void
FinishLoad() override;
void
SetLoadInfo(
const milvus::proto::segcore::SegmentLoadInfo& load_info) override;
void
Load(milvus::tracer::TraceContext& trace_ctx) override;
public:
size_t
GetMemoryUsageInBytes() const override {
@ -922,6 +929,33 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
void
load_column_group_data_internal(const LoadFieldDataInfo& load_info);
/**
* @brief Load all column groups from a manifest file path
*
* This method reads the manifest file to retrieve column groups metadata
* and loads each column group into the segment.
*
* @param manifest_path JSON string containing base_path and version fields
*/
void
LoadColumnGroups(const std::string& manifest_path);
/**
* @brief Load a single column group at the specified index
*
* Reads a specific column group from milvus storage, converts the data
* to internal format, and stores it in the segment's field data structures.
*
* @param column_groups Metadata about all available column groups
* @param properties Storage properties for accessing the data
* @param index Index of the column group to load
*/
void
LoadColumnGroup(
const std::shared_ptr<milvus_storage::api::ColumnGroups>& column_groups,
const std::shared_ptr<milvus_storage::api::Properties>& properties,
int64_t index);
void
load_field_data_common(
FieldId field_id,
@ -931,6 +965,11 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
bool enable_mmap,
bool is_proxy_column);
// Convert proto::segcore::FieldIndexInfo to LoadIndexInfo
LoadIndexInfo
ConvertFieldIndexInfoToLoadIndexInfo(
const milvus::proto::segcore::FieldIndexInfo* field_index_info) const;
std::shared_ptr<ChunkedColumnInterface>
get_column(FieldId field_id) const {
std::shared_ptr<ChunkedColumnInterface> res;
@ -984,6 +1023,7 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
mutable DeletedRecord<true> deleted_record_;
LoadFieldDataInfo field_data_info_;
milvus::proto::segcore::SegmentLoadInfo segment_load_info_;
SchemaPtr schema_;
int64_t id_;
@ -1003,6 +1043,9 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
// whether the segment is sorted by the pk
// 1. will skip index loading for primary key field
bool is_sorted_by_pk_ = false;
// milvus storage internal api reader instance
std::unique_ptr<milvus_storage::api::Reader> reader_;
};
inline SegmentSealedUPtr

View File

@ -16,6 +16,7 @@
#include <numeric>
#include <optional>
#include <queue>
#include <string>
#include <thread>
#include <boost/iterator/counting_iterator.hpp>
#include <type_traits>
@ -40,6 +41,8 @@
#include "segcore/Utils.h"
#include "segcore/memory_planner.h"
#include "storage/RemoteChunkManagerSingleton.h"
#include "storage/loon_ffi/property_singleton.h"
#include "storage/loon_ffi/util.h"
#include "storage/Util.h"
#include "storage/ThreadPools.h"
#include "storage/KeyRetriever.h"
@ -1323,6 +1326,54 @@ SegmentGrowingImpl::Reopen(SchemaPtr sch) {
}
}
void
SegmentGrowingImpl::Load(milvus::tracer::TraceContext& trace_ctx) {
// Convert load_info_ (SegmentLoadInfo) to LoadFieldDataInfo
LoadFieldDataInfo field_data_info;
// Set storage version
field_data_info.storage_version = load_info_.storageversion();
// Set load priority
field_data_info.load_priority = load_info_.priority();
auto manifest_path = load_info_.manifest_path();
if (manifest_path != "") {
LoadColumnsGroups(manifest_path);
return;
}
// Convert binlog_paths to field_infos
for (const auto& field_binlog : load_info_.binlog_paths()) {
FieldBinlogInfo binlog_info;
binlog_info.field_id = field_binlog.fieldid();
// Process each binlog
int64_t total_row_count = 0;
for (const auto& binlog : field_binlog.binlogs()) {
binlog_info.entries_nums.push_back(binlog.entries_num());
binlog_info.insert_files.push_back(binlog.log_path());
binlog_info.memory_sizes.push_back(binlog.memory_size());
total_row_count += binlog.entries_num();
}
binlog_info.row_count = total_row_count;
// Set child field ids
for (const auto& child_field : field_binlog.child_fields()) {
binlog_info.child_field_ids.push_back(child_field);
}
// Add to field_infos map
field_data_info.field_infos[binlog_info.field_id] =
std::move(binlog_info);
}
// Call LoadFieldData with the converted info
if (!field_data_info.field_infos.empty()) {
LoadFieldData(field_data_info);
}
}
void
SegmentGrowingImpl::FinishLoad() {
for (const auto& [field_id, field_meta] : schema_->get_fields()) {
@ -1338,6 +1389,157 @@ SegmentGrowingImpl::FinishLoad() {
}
}
void
SegmentGrowingImpl::LoadColumnsGroups(std::string manifest_path) {
LOG_INFO(
"Loading segment {} field data with manifest {}", id_, manifest_path);
// size_t num_rows = storage::GetNumRowsForLoadInfo(infos);
auto num_rows = load_info_.num_of_rows();
auto primary_field_id =
schema_->get_primary_field_id().value_or(FieldId(-1));
auto properties = milvus::storage::LoonFFIPropertiesSingleton::GetInstance()
.GetProperties();
auto column_groups = GetColumnGroups(manifest_path, properties);
auto arrow_schema = schema_->ConvertToArrowSchema();
reader_ = milvus_storage::api::Reader::create(
column_groups, arrow_schema, nullptr, *properties);
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::LOW);
std::vector<
std::future<std::unordered_map<FieldId, std::vector<FieldDataPtr>>>>
load_group_futures;
for (int64_t i = 0; i < column_groups->size(); ++i) {
auto future = pool.Submit([this, column_groups, properties, i] {
return LoadColumnGroup(column_groups, properties, i);
});
load_group_futures.emplace_back(std::move(future));
}
std::vector<std::unordered_map<FieldId, std::vector<FieldDataPtr>>>
column_group_results;
std::vector<std::exception_ptr> load_exceptions;
for (auto& future : load_group_futures) {
try {
column_group_results.emplace_back(future.get());
} catch (...) {
load_exceptions.push_back(std::current_exception());
}
}
// If any exceptions occurred during index loading, handle them
if (!load_exceptions.empty()) {
LOG_ERROR("Failed to load {} out of {} indexes for segment {}",
load_exceptions.size(),
load_group_futures.size(),
id_);
// Rethrow the first exception
std::rethrow_exception(load_exceptions[0]);
}
auto reserved_offset = PreInsert(num_rows);
for (auto& column_group_result : column_group_results) {
for (auto& [field_id, field_data] : column_group_result) {
load_field_data_common(field_id,
reserved_offset,
field_data,
primary_field_id,
num_rows);
// Build geometry cache for GEOMETRY fields
if (schema_->operator[](field_id).get_data_type() ==
DataType::GEOMETRY &&
segcore_config_.get_enable_geometry_cache()) {
BuildGeometryCacheForLoad(field_id, field_data);
}
}
}
insert_record_.ack_responder_.AddSegment(reserved_offset,
reserved_offset + num_rows);
}
std::unordered_map<FieldId, std::vector<FieldDataPtr>>
SegmentGrowingImpl::LoadColumnGroup(
const std::shared_ptr<milvus_storage::api::ColumnGroups>& column_groups,
const std::shared_ptr<milvus_storage::api::Properties>& properties,
int64_t index) {
AssertInfo(index < column_groups->size(),
"load column group index out of range");
auto column_group = column_groups->get_column_group(index);
LOG_INFO("Loading segment {} column group {}", id_, index);
auto chunk_reader_result = reader_->get_chunk_reader(index);
AssertInfo(chunk_reader_result.ok(),
"get chunk reader failed, segment {}, column group index {}",
get_segment_id(),
index);
auto chunk_reader = std::move(chunk_reader_result.ValueOrDie());
auto parallel_degree =
static_cast<uint64_t>(DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
std::vector<int64_t> all_row_groups(chunk_reader->total_number_of_chunks());
std::iota(all_row_groups.begin(), all_row_groups.end(), 0);
// create parallel degree split strategy
auto strategy =
std::make_unique<ParallelDegreeSplitStrategy>(parallel_degree);
auto split_result = strategy->split(all_row_groups);
auto& thread_pool =
ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::HIGH);
auto part_futures = std::vector<
std::future<std::vector<std::shared_ptr<arrow::RecordBatch>>>>();
for (const auto& part : split_result) {
part_futures.emplace_back(
thread_pool.Submit([chunk_reader = chunk_reader.get(), part]() {
std::vector<int64_t> chunk_ids(part.count);
std::iota(chunk_ids.begin(), chunk_ids.end(), part.offset);
auto result = chunk_reader->get_chunks(chunk_ids, 1);
AssertInfo(result.ok(), "get chunks failed");
return result.ValueOrDie();
}));
}
std::unordered_map<FieldId, std::vector<FieldDataPtr>> field_data_map;
for (auto& future : part_futures) {
auto part_result = future.get();
for (auto& record_batch : part_result) {
// result->emplace_back(std::move(record_batch));
auto batch_num_rows = record_batch->num_rows();
for (auto i = 0; i < column_group->columns.size(); ++i) {
auto column = column_group->columns[i];
auto field_id = FieldId(std::stoll(column));
auto field = schema_->operator[](field_id);
auto data_type = field.get_data_type();
auto field_data = storage::CreateFieldData(
data_type,
field.get_element_type(),
field.is_nullable(),
IsVectorDataType(data_type) &&
!IsSparseFloatVectorDataType(data_type)
? field.get_dim()
: 1,
batch_num_rows);
auto array = record_batch->column(i);
field_data->FillFieldData(array);
field_data_map[FieldId(field_id)].push_back(field_data);
}
}
}
LOG_INFO("Finished loading segment {} column group {}", id_, index);
return field_data_map;
}
void
SegmentGrowingImpl::fill_empty_field(const FieldMeta& field_meta) {
auto field_id = field_meta.get_id();

View File

@ -106,6 +106,9 @@ class SegmentGrowingImpl : public SegmentGrowing {
void
FinishLoad() override;
void
Load(milvus::tracer::TraceContext& trace_ctx) override;
private:
// Build geometry cache for inserted data
void
@ -555,6 +558,34 @@ class SegmentGrowingImpl : public SegmentGrowing {
void
CreateTextIndexes();
/**
* @brief Load all column groups from a manifest file path
*
* This method parses the manifest path to retrieve column groups metadata
* and loads each column group into the growing segment.
*
* @param manifest_path JSON string containing base_path and version fields
*/
void
LoadColumnsGroups(std::string manifest_path);
/**
* @brief Load a single column group and return field data
*
* Reads a specific column group from milvus storage and converts it to
* field data format that can be inserted into the growing segment.
*
* @param column_groups Metadata about all available column groups
* @param properties Storage properties for accessing the data
* @param index Index of the column group to load
* @return Map of field IDs to their corresponding field data vectors
*/
std::unordered_map<FieldId, std::vector<FieldDataPtr>>
LoadColumnGroup(
const std::shared_ptr<milvus_storage::api::ColumnGroups>& column_groups,
const std::shared_ptr<milvus_storage::api::Properties>& properties,
int64_t index);
private:
storage::MmapChunkDescriptorPtr mmap_descriptor_ = nullptr;
SegcoreConfig segcore_config_;
@ -575,6 +606,9 @@ class SegmentGrowingImpl : public SegmentGrowing {
int64_t id_;
SegmentStats stats_{};
// milvus storage internal api reader instance
std::unique_ptr<milvus_storage::api::Reader> reader_;
};
inline SegmentGrowingPtr

View File

@ -237,6 +237,12 @@ class SegmentInterface {
// currently it's used to sync field data list with updated schema.
virtual void
FinishLoad() = 0;
virtual void
SetLoadInfo(const milvus::proto::segcore::SegmentLoadInfo& load_info) = 0;
virtual void
Load(milvus::tracer::TraceContext& trace_ctx) = 0;
};
// internal API for DSL calculation
@ -424,6 +430,12 @@ class SegmentInternalInterface : public SegmentInterface {
FieldId field_id,
const std::string& nested_path) const override;
virtual void
SetLoadInfo(
const milvus::proto::segcore::SegmentLoadInfo& load_info) override {
load_info_ = load_info;
}
public:
// `query_offsets` is not null only for vector array (embedding list) search
// where it denotes the number of vectors in each embedding list. The length
@ -645,6 +657,8 @@ class SegmentInternalInterface : public SegmentInterface {
// mutex protecting rw options on schema_
std::shared_mutex sch_mutex_;
milvus::proto::segcore::SegmentLoadInfo load_info_;
mutable std::shared_mutex mutex_;
// fieldID -> std::pair<num_rows, avg_size>
std::unordered_map<FieldId, std::pair<int64_t, int64_t>>

View File

@ -26,6 +26,9 @@
#include "common/Utils.h"
#include "index/ScalarIndex.h"
#include "log/Log.h"
#include "segcore/storagev1translator/SealedIndexTranslator.h"
#include "segcore/storagev1translator/V1SealedIndexTranslator.h"
#include "segcore/Types.h"
#include "storage/DataCodec.h"
#include "storage/RemoteChunkManagerSingleton.h"
#include "storage/ThreadPools.h"
@ -1200,4 +1203,116 @@ getCellDataType(bool is_vector, bool is_index) {
}
}
void
LoadIndexData(milvus::tracer::TraceContext& ctx,
milvus::segcore::LoadIndexInfo* load_index_info) {
auto& index_params = load_index_info->index_params;
auto field_type = load_index_info->field_type;
auto engine_version = load_index_info->index_engine_version;
milvus::index::CreateIndexInfo index_info;
index_info.field_type = load_index_info->field_type;
index_info.index_engine_version = engine_version;
auto config = milvus::index::ParseConfigFromIndexParams(
load_index_info->index_params);
auto load_priority_str = config[milvus::LOAD_PRIORITY].get<std::string>();
auto priority_for_load = milvus::PriorityForLoad(load_priority_str);
config[milvus::LOAD_PRIORITY] = priority_for_load;
// Config should have value for milvus::index::SCALAR_INDEX_ENGINE_VERSION for production calling chain.
// Use value_or(1) for unit test without setting this value
index_info.scalar_index_engine_version =
milvus::index::GetValueFromConfig<int32_t>(
config, milvus::index::SCALAR_INDEX_ENGINE_VERSION)
.value_or(1);
index_info.tantivy_index_version =
milvus::index::GetValueFromConfig<int32_t>(
config, milvus::index::TANTIVY_INDEX_VERSION)
.value_or(milvus::index::TANTIVY_INDEX_LATEST_VERSION);
LOG_INFO(
"[collection={}][segment={}][field={}][enable_mmap={}][load_"
"priority={}] load index {}, "
"mmap_dir_path={}",
load_index_info->collection_id,
load_index_info->segment_id,
load_index_info->field_id,
load_index_info->enable_mmap,
load_priority_str,
load_index_info->index_id,
load_index_info->mmap_dir_path);
// get index type
AssertInfo(index_params.find("index_type") != index_params.end(),
"index type is empty");
index_info.index_type = index_params.at("index_type");
// get metric type
if (milvus::IsVectorDataType(field_type)) {
AssertInfo(index_params.find("metric_type") != index_params.end(),
"metric type is empty for vector index");
index_info.metric_type = index_params.at("metric_type");
}
if (index_info.index_type == milvus::index::NGRAM_INDEX_TYPE) {
AssertInfo(
index_params.find(milvus::index::MIN_GRAM) != index_params.end(),
"min_gram is empty for ngram index");
AssertInfo(
index_params.find(milvus::index::MAX_GRAM) != index_params.end(),
"max_gram is empty for ngram index");
// get min_gram and max_gram and convert to uintptr_t
milvus::index::NgramParams ngram_params{};
ngram_params.loading_index = true;
ngram_params.min_gram =
std::stoul(milvus::index::GetValueFromConfig<std::string>(
config, milvus::index::MIN_GRAM)
.value());
ngram_params.max_gram =
std::stoul(milvus::index::GetValueFromConfig<std::string>(
config, milvus::index::MAX_GRAM)
.value());
index_info.ngram_params = std::make_optional(ngram_params);
}
// init file manager
milvus::storage::FieldDataMeta field_meta{load_index_info->collection_id,
load_index_info->partition_id,
load_index_info->segment_id,
load_index_info->field_id,
load_index_info->schema};
milvus::storage::IndexMeta index_meta{load_index_info->segment_id,
load_index_info->field_id,
load_index_info->index_build_id,
load_index_info->index_version};
config[milvus::index::INDEX_FILES] = load_index_info->index_files;
if (load_index_info->field_type == milvus::DataType::JSON) {
index_info.json_cast_type = milvus::JsonCastType::FromString(
config.at(JSON_CAST_TYPE).get<std::string>());
index_info.json_path = config.at(JSON_PATH).get<std::string>();
}
auto remote_chunk_manager =
milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance()
.GetArrowFileSystem();
AssertInfo(fs != nullptr, "arrow file system is nullptr");
milvus::storage::FileManagerContext file_manager_context(
field_meta, index_meta, remote_chunk_manager, fs);
file_manager_context.set_for_loading_index(true);
// use cache layer to load vector/scalar index
std::unique_ptr<milvus::cachinglayer::Translator<milvus::index::IndexBase>>
translator = std::make_unique<
milvus::segcore::storagev1translator::SealedIndexTranslator>(
index_info, load_index_info, ctx, file_manager_context, config);
load_index_info->cache_index =
milvus::cachinglayer::Manager::GetInstance().CreateCacheSlot(
std::move(translator));
}
} // namespace milvus::segcore

View File

@ -22,6 +22,7 @@
#include "index/Index.h"
#include "cachinglayer/Utils.h"
#include "segcore/ConcurrentVector.h"
#include "segcore/Types.h"
namespace milvus::segcore {
@ -143,4 +144,8 @@ getCacheWarmupPolicy(bool is_vector, bool is_index, bool in_load_list = true);
milvus::cachinglayer::CellDataType
getCellDataType(bool is_vector, bool is_index);
void
LoadIndexData(milvus::tracer::TraceContext& ctx,
milvus::segcore::LoadIndexInfo* load_index_info);
} // namespace milvus::segcore

View File

@ -18,6 +18,8 @@
#include "milvus-storage/filesystem/fs.h"
#include "common/EasyAssert.h"
#include "common/type_c.h"
#include "storage/loon_ffi/property_singleton.h"
#include "storage/loon_ffi/util.h"
CStatus
InitLocalArrowFileSystemSingleton(const char* c_path) {
@ -28,6 +30,8 @@ InitLocalArrowFileSystemSingleton(const char* c_path) {
conf.storage_type = "local";
milvus_storage::ArrowFileSystemSingleton::GetInstance().Init(conf);
milvus::storage::LoonFFIPropertiesSingleton::GetInstance().Init(c_path);
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
@ -64,6 +68,9 @@ InitRemoteArrowFileSystemSingleton(CStorageConfig c_storage_config) {
conf.max_connections = c_storage_config.max_connections;
milvus_storage::ArrowFileSystemSingleton::GetInstance().Init(conf);
milvus::storage::LoonFFIPropertiesSingleton::GetInstance().Init(
c_storage_config);
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);

View File

@ -34,6 +34,7 @@
#include "cachinglayer/Manager.h"
#include "segcore/storagev1translator/SealedIndexTranslator.h"
#include "segcore/storagev1translator/V1SealedIndexTranslator.h"
#include "segcore/Utils.h"
#include "monitor/scope_metric.h"
bool
@ -239,121 +240,14 @@ AppendIndexV2(CTraceContext c_trace, CLoadIndexInfo c_load_index_info) {
try {
auto load_index_info =
static_cast<milvus::segcore::LoadIndexInfo*>(c_load_index_info);
auto& index_params = load_index_info->index_params;
auto field_type = load_index_info->field_type;
auto engine_version = load_index_info->index_engine_version;
milvus::index::CreateIndexInfo index_info;
index_info.field_type = load_index_info->field_type;
index_info.index_engine_version = engine_version;
auto config = milvus::index::ParseConfigFromIndexParams(
load_index_info->index_params);
auto load_priority_str =
config[milvus::LOAD_PRIORITY].get<std::string>();
auto priority_for_load = milvus::PriorityForLoad(load_priority_str);
config[milvus::LOAD_PRIORITY] = priority_for_load;
// Config should have value for milvus::index::SCALAR_INDEX_ENGINE_VERSION for production calling chain.
// Use value_or(1) for unit test without setting this value
index_info.scalar_index_engine_version =
milvus::index::GetValueFromConfig<int32_t>(
config, milvus::index::SCALAR_INDEX_ENGINE_VERSION)
.value_or(1);
index_info.tantivy_index_version =
milvus::index::GetValueFromConfig<int32_t>(
config, milvus::index::TANTIVY_INDEX_VERSION)
.value_or(milvus::index::TANTIVY_INDEX_LATEST_VERSION);
auto ctx = milvus::tracer::TraceContext{
c_trace.traceID, c_trace.spanID, c_trace.traceFlags};
auto span = milvus::tracer::StartSpan("SegCoreLoadIndex", &ctx);
milvus::tracer::SetRootSpan(span);
LOG_INFO(
"[collection={}][segment={}][field={}][enable_mmap={}][load_"
"priority={}] load index {}, "
"mmap_dir_path={}",
load_index_info->collection_id,
load_index_info->segment_id,
load_index_info->field_id,
load_index_info->enable_mmap,
load_priority_str,
load_index_info->index_id,
load_index_info->mmap_dir_path);
LoadIndexData(ctx, load_index_info);
// get index type
AssertInfo(index_params.find("index_type") != index_params.end(),
"index type is empty");
index_info.index_type = index_params.at("index_type");
// get metric type
if (milvus::IsVectorDataType(field_type)) {
AssertInfo(index_params.find("metric_type") != index_params.end(),
"metric type is empty for vector index");
index_info.metric_type = index_params.at("metric_type");
}
if (index_info.index_type == milvus::index::NGRAM_INDEX_TYPE) {
AssertInfo(index_params.find(milvus::index::MIN_GRAM) !=
index_params.end(),
"min_gram is empty for ngram index");
AssertInfo(index_params.find(milvus::index::MAX_GRAM) !=
index_params.end(),
"max_gram is empty for ngram index");
// get min_gram and max_gram and convert to uintptr_t
milvus::index::NgramParams ngram_params{};
ngram_params.loading_index = true;
ngram_params.min_gram =
std::stoul(milvus::index::GetValueFromConfig<std::string>(
config, milvus::index::MIN_GRAM)
.value());
ngram_params.max_gram =
std::stoul(milvus::index::GetValueFromConfig<std::string>(
config, milvus::index::MAX_GRAM)
.value());
index_info.ngram_params = std::make_optional(ngram_params);
}
// init file manager
milvus::storage::FieldDataMeta field_meta{
load_index_info->collection_id,
load_index_info->partition_id,
load_index_info->segment_id,
load_index_info->field_id,
load_index_info->schema};
milvus::storage::IndexMeta index_meta{load_index_info->segment_id,
load_index_info->field_id,
load_index_info->index_build_id,
load_index_info->index_version};
config[milvus::index::INDEX_FILES] = load_index_info->index_files;
if (load_index_info->field_type == milvus::DataType::JSON) {
index_info.json_cast_type = milvus::JsonCastType::FromString(
config.at(JSON_CAST_TYPE).get<std::string>());
index_info.json_path = config.at(JSON_PATH).get<std::string>();
}
auto remote_chunk_manager =
milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto fs = milvus_storage::ArrowFileSystemSingleton::GetInstance()
.GetArrowFileSystem();
AssertInfo(fs != nullptr, "arrow file system is nullptr");
milvus::storage::FileManagerContext fileManagerContext(
field_meta, index_meta, remote_chunk_manager, fs);
fileManagerContext.set_for_loading_index(true);
// use cache layer to load vector/scalar index
std::unique_ptr<
milvus::cachinglayer::Translator<milvus::index::IndexBase>>
translator = std::make_unique<
milvus::segcore::storagev1translator::SealedIndexTranslator>(
index_info, load_index_info, ctx, fileManagerContext, config);
load_index_info->cache_index =
milvus::cachinglayer::Manager::GetInstance().CreateCacheSlot(
std::move(translator));
span->End();
milvus::tracer::CloseRootSpan();

View File

@ -14,6 +14,8 @@
#include <memory>
#include <limits>
#include "common/EasyAssert.h"
#include "common/common_type_c.h"
#include "pb/cgo_msg.pb.h"
#include "pb/index_cgo_msg.pb.h"
@ -27,6 +29,7 @@
#include "log/Log.h"
#include "mmap/Types.h"
#include "monitor/scope_metric.h"
#include "pb/segcore.pb.h"
#include "segcore/Collection.h"
#include "segcore/SegcoreConfig.h"
#include "segcore/SegmentGrowingImpl.h"
@ -45,17 +48,20 @@
#include "common/GeometryCache.h"
////////////////////////////// common interfaces //////////////////////////////
CStatus
NewSegment(CCollection collection,
/**
* @brief Create a segment from a collection.
* @param col The collection to create the segment from.
* @param seg_type The type of segment to create.
* @param segment_id The ID of the segment to create.
* @param is_sorted_by_pk Whether the data in the sealed segment is sorted by primary key.
* @return A unique pointer to a SegmentInterface object.
*/
std::unique_ptr<milvus::segcore::SegmentInterface>
CreateSegment(milvus::segcore::Collection* col,
SegmentType seg_type,
int64_t segment_id,
CSegmentInterface* newSegment,
bool is_sorted_by_pk) {
SCOPE_CGO_CALL_METRIC();
try {
auto col = static_cast<milvus::segcore::Collection*>(collection);
std::unique_ptr<milvus::segcore::SegmentInterface> segment;
switch (seg_type) {
case Growing: {
@ -78,11 +84,26 @@ NewSegment(CCollection collection,
break;
default:
ThrowInfo(milvus::UnexpectedError,
"invalid segment type: {}",
seg_type);
ThrowInfo(
milvus::UnexpectedError, "invalid segment type: {}", seg_type);
}
return segment;
}
CStatus
NewSegment(CCollection collection,
SegmentType seg_type,
int64_t segment_id,
CSegmentInterface* newSegment,
bool is_sorted_by_pk) {
SCOPE_CGO_CALL_METRIC();
try {
auto col = static_cast<milvus::segcore::Collection*>(collection);
auto segment =
CreateSegment(col, seg_type, segment_id, is_sorted_by_pk);
*newSegment = segment.release();
return milvus::SuccessCStatus();
} catch (std::exception& e) {
@ -90,6 +111,51 @@ NewSegment(CCollection collection,
}
}
CStatus
NewSegmentWithLoadInfo(CCollection collection,
SegmentType seg_type,
int64_t segment_id,
CSegmentInterface* newSegment,
bool is_sorted_by_pk,
const uint8_t* load_info_blob,
const int64_t load_info_length) {
SCOPE_CGO_CALL_METRIC();
try {
AssertInfo(load_info_blob, "load info is null");
milvus::proto::segcore::SegmentLoadInfo load_info;
auto suc = load_info.ParseFromArray(load_info_blob, load_info_length);
AssertInfo(suc, "unmarshal load info failed");
auto col = static_cast<milvus::segcore::Collection*>(collection);
auto segment =
CreateSegment(col, seg_type, segment_id, is_sorted_by_pk);
segment->SetLoadInfo(load_info);
*newSegment = segment.release();
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
SegmentLoad(CTraceContext c_trace, CSegmentInterface c_segment) {
SCOPE_CGO_CALL_METRIC();
try {
auto segment =
static_cast<milvus::segcore::SegmentInterface*>(c_segment);
// TODO unify trace context to op context after supported
auto trace_ctx = milvus::tracer::TraceContext{
c_trace.traceID, c_trace.spanID, c_trace.traceFlags};
segment->Load(trace_ctx);
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
void
DeleteSegment(CSegmentInterface c_segment) {
SCOPE_CGO_CALL_METRIC();

View File

@ -36,6 +36,39 @@ NewSegment(CCollection collection,
CSegmentInterface* newSegment,
bool is_sorted_by_pk);
/**
* @brief Create a new segment with pre-loaded segment information
* This function creates a segment and initializes it with serialized load info,
* which can include precomputed metadata, statistics, or configuration data
*
* @param collection: The collection that this segment belongs to
* @param seg_type: Type of the segment (growing, sealed, etc.)
* @param segment_id: Unique identifier for this segment
* @param newSegment: Output parameter for the created segment interface
* @param is_sorted_by_pk: Whether the segment data is sorted by primary key
* @param load_info_blob: Serialized load information blob
* @param load_info_length: Length of the load_info_blob in bytes
* @return CStatus indicating success or failure
*/
CStatus
NewSegmentWithLoadInfo(CCollection collection,
SegmentType seg_type,
int64_t segment_id,
CSegmentInterface* newSegment,
bool is_sorted_by_pk,
const uint8_t* load_info_blob,
const int64_t load_info_length);
/**
* @brief Dispatch a segment manage load task.
* This function make segment itself load index & field data according to load info previously set.
*
* @param c_trace: tracing context param
* @param c_segment: segment handle indicate which segment to load
* @return CStatus indicating success or failure
*/
CStatus
SegmentLoad(CTraceContext c_trace, CSegmentInterface c_segment);
void
DeleteSegment(CSegmentInterface c_segment);

View File

@ -0,0 +1,245 @@
// 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 "segcore/storagev2translator/ManifestGroupTranslator.h"
#include "common/type_c.h"
#include "milvus-storage/reader.h"
#include "segcore/storagev2translator/GroupCTMeta.h"
#include "common/GroupChunk.h"
#include "mmap/Types.h"
#include "common/Types.h"
#include "milvus-storage/common/metadata.h"
#include "milvus-storage/filesystem/fs.h"
#include "milvus-storage/common/constants.h"
#include "milvus-storage/format/parquet/file_reader.h"
#include "storage/ThreadPools.h"
#include "storage/KeyRetriever.h"
#include "segcore/memory_planner.h"
#include <cstddef>
#include <cstdint>
#include <memory>
#include <string>
#include <unordered_set>
#include <vector>
#include <unordered_map>
#include <set>
#include <algorithm>
#include "arrow/type.h"
#include "arrow/type_fwd.h"
#include "cachinglayer/Utils.h"
#include "common/ChunkWriter.h"
#include "segcore/Utils.h"
namespace milvus::segcore::storagev2translator {
ManifestGroupTranslator::ManifestGroupTranslator(
int64_t segment_id,
int64_t column_group_index,
std::unique_ptr<milvus_storage::api::ChunkReader> chunk_reader,
const std::unordered_map<FieldId, FieldMeta>& field_metas,
bool use_mmap,
int64_t num_fields,
milvus::proto::common::LoadPriority load_priority)
: segment_id_(segment_id),
column_group_index_(column_group_index),
chunk_reader_(std::move(chunk_reader)),
key_(fmt::format("seg_{}_cg_{}", segment_id, column_group_index)),
field_metas_(field_metas),
meta_(num_fields,
use_mmap ? milvus::cachinglayer::StorageType::DISK
: milvus::cachinglayer::StorageType::MEMORY,
milvus::cachinglayer::CellIdMappingMode::IDENTICAL,
milvus::segcore::getCellDataType(
/* is_vector */
[&]() {
for (const auto& [fid, field_meta] : field_metas_) {
if (IsVectorDataType(field_meta.get_data_type())) {
return true;
}
}
return false;
}(),
/* is_index */ false),
milvus::segcore::getCacheWarmupPolicy(
/* is_vector */
[&]() {
for (const auto& [fid, field_meta] : field_metas_) {
if (IsVectorDataType(field_meta.get_data_type())) {
return true;
}
}
return false;
}(),
/* is_index */ false),
/* support_eviction */ true) {
auto chunk_size_result = chunk_reader_->get_chunk_size();
if (!chunk_size_result.ok()) {
throw std::runtime_error("get chunk size failed");
}
chunk_size_ = chunk_size_result.ValueOrDie();
auto rows_result = chunk_reader_->get_chunk_rows();
if (!rows_result.ok()) {
throw std::runtime_error("get chunk rows failed");
}
auto chunk_rows = rows_result.ValueOrDie();
meta_.num_rows_until_chunk_.push_back(0);
for (int i = 0; i < chunk_reader_->total_number_of_chunks(); ++i) {
meta_.num_rows_until_chunk_.push_back(
meta_.num_rows_until_chunk_.back() +
static_cast<int64_t>(chunk_rows[i]));
meta_.chunk_memory_size_.push_back(
static_cast<int64_t>(chunk_size_[i]));
}
}
size_t
ManifestGroupTranslator::num_cells() const {
return chunk_reader_->total_number_of_chunks();
}
milvus::cachinglayer::cid_t
ManifestGroupTranslator::cell_id_of(milvus::cachinglayer::uid_t uid) const {
return uid;
}
std::pair<milvus::cachinglayer::ResourceUsage,
milvus::cachinglayer::ResourceUsage>
ManifestGroupTranslator::estimated_byte_size_of_cell(
milvus::cachinglayer::cid_t cid) const {
// return chunk_reader_->get_chunk_size()[cid];
AssertInfo(cid < chunk_size_.size(), "invalid cid");
auto cell_sz = static_cast<int64_t>(chunk_size_[cid]);
if (use_mmap_) {
// why double the disk size for loading?
// during file writing, the temporary size could be larger than the final size
// so we need to reserve more space for the disk size.
return {{0, cell_sz}, {2 * cell_sz, 2 * cell_sz}};
} else {
return {{cell_sz, 0}, {2 * cell_sz, 0}};
}
}
const std::string&
ManifestGroupTranslator::key() const {
return key_;
}
std::vector<
std::pair<milvus::cachinglayer::cid_t, std::unique_ptr<milvus::GroupChunk>>>
ManifestGroupTranslator::get_cells(
const std::vector<milvus::cachinglayer::cid_t>& cids) {
std::vector<std::pair<milvus::cachinglayer::cid_t,
std::unique_ptr<milvus::GroupChunk>>>
cells;
cells.reserve(cids.size());
auto parallel_degree =
static_cast<uint64_t>(DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
auto read_result =
chunk_reader_->get_chunks(cids, static_cast<int64_t>(parallel_degree));
if (!read_result.ok()) {
throw std::runtime_error("get chunk failed");
}
auto chunks = read_result.ValueOrDie();
for (size_t i = 0; i < chunks.size(); ++i) {
auto& chunk = chunks[i];
AssertInfo(chunk != nullptr,
"chunk is null, idx = {}, group index = {}, segment id = "
"{}, parallel degree = {}",
i,
column_group_index_,
segment_id_,
parallel_degree);
auto cid = cids[i];
auto group_chunk = load_group_chunk(chunk, cid);
cells.emplace_back(cid, std::move(group_chunk));
}
return cells;
}
std::unique_ptr<milvus::GroupChunk>
ManifestGroupTranslator::load_group_chunk(
const std::shared_ptr<arrow::RecordBatch>& record_batch,
const milvus::cachinglayer::cid_t cid) {
std::unordered_map<FieldId, std::shared_ptr<Chunk>> chunks;
// Iterate through field_id_list to get field_id and create chunk
for (int i = 0; i < record_batch->num_columns(); ++i) {
// column name here is field id
auto column_name = record_batch->column_name(i);
auto field_id = std::stoll(column_name);
auto fid = milvus::FieldId(field_id);
if (fid == RowFieldID) {
// ignore row id field
continue;
}
auto it = field_metas_.find(fid);
AssertInfo(
it != field_metas_.end(),
"[StorageV2] translator {} field id {} not found in field_metas",
key_,
fid.get());
const auto& field_meta = it->second;
const arrow::ArrayVector array_vec = {record_batch->column(i)};
std::unique_ptr<Chunk> chunk;
if (!use_mmap_) {
// Memory mode
chunk = create_chunk(field_meta, array_vec);
} else {
// Mmap mode
std::filesystem::path filepath;
if (field_meta.get_main_field_id() != INVALID_FIELD_ID) {
// json shredding mode
filepath = std::filesystem::path(mmap_dir_path_) /
std::to_string(segment_id_) /
std::to_string(field_meta.get_main_field_id()) /
std::to_string(field_id) / std::to_string(cid);
} else {
filepath = std::filesystem::path(mmap_dir_path_) /
std::to_string(segment_id_) /
std::to_string(field_id) / std::to_string(cid);
}
LOG_INFO(
"[StorageV2] translator {} mmaping field {} chunk {} to path "
"{}",
key_,
field_id,
cid,
filepath.string());
std::filesystem::create_directories(filepath.parent_path());
chunk = create_chunk(field_meta, array_vec, filepath.string());
}
chunks[fid] = std::move(chunk);
}
return std::make_unique<milvus::GroupChunk>(chunks);
}
} // namespace milvus::segcore::storagev2translator

View File

@ -0,0 +1,183 @@
// 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.
#pragma once
#include <string>
#include <vector>
#include <memory>
#include <unordered_map>
#include "cachinglayer/Translator.h"
#include "cachinglayer/Utils.h"
#include "milvus-storage/common/metadata.h"
#include "mmap/Types.h"
#include "common/Types.h"
#include "common/GroupChunk.h"
#include "parquet/metadata.h"
#include "segcore/ChunkedSegmentSealedImpl.h"
#include "segcore/InsertRecord.h"
#include "segcore/storagev2translator/GroupCTMeta.h"
namespace milvus::segcore::storagev2translator {
/**
* @brief Translator for loading column groups from milvus storage manifest
*
* This class implements the Translator interface to load column group data
* from milvus storage v2 format. It reads chunks from a ChunkReader and
* translates them into GroupChunk objects for caching.
*/
class ManifestGroupTranslator
: public milvus::cachinglayer::Translator<milvus::GroupChunk> {
public:
/**
* @brief Construct a translator for a column group
*
* @param segment_id ID of the segment being loaded
* @param column_group_index Index of the column group within the segment
* @param chunk_reader Reader for accessing chunks from storage
* @param field_metas Metadata for all fields in this column group
* @param use_mmap Whether to use memory mapping for data loading
* @param num_fields Total number of fields in the column group
* @param load_priority Priority level for loading operations
*/
ManifestGroupTranslator(
int64_t segment_id,
int64_t column_group_index,
std::unique_ptr<milvus_storage::api::ChunkReader> chunk_reader,
const std::unordered_map<FieldId, FieldMeta>& field_metas,
bool use_mmap,
int64_t num_fields,
milvus::proto::common::LoadPriority load_priority);
~ManifestGroupTranslator() = default;
/**
* @brief Get the total number of cells (chunks) in this column group
*
* @return Number of chunks available in the chunk reader
*/
size_t
num_cells() const override;
/**
* @brief Map a unit ID to its corresponding cell ID
*
* For this translator, the mapping is identical (uid == cid).
*
* @param uid Unit ID to map
* @return Corresponding cell ID
*/
milvus::cachinglayer::cid_t
cell_id_of(milvus::cachinglayer::uid_t uid) const override;
/**
* @brief Estimate memory and disk usage for a cell
*
* Returns resource usage estimates for loading a specific chunk.
* For mmap mode, reserves double the disk space for temporary files.
*
* @param cid Cell ID to estimate
* @return Pair of (memory_usage, disk_usage) for loading and storage
*/
std::pair<milvus::cachinglayer::ResourceUsage,
milvus::cachinglayer::ResourceUsage>
estimated_byte_size_of_cell(milvus::cachinglayer::cid_t cid) const override;
/**
* @brief Get the cache key for this translator
*
* @return Cache key in format "seg_{segment_id}_cg_{column_group_index}"
*/
const std::string&
key() const override;
/**
* @brief Load specified cells (chunks) from storage
*
* Reads the requested chunks from the chunk reader and converts them
* to GroupChunk objects containing field data.
*
* @param cids List of cell IDs to load
* @return Vector of (cell_id, GroupChunk) pairs
*/
std::vector<std::pair<milvus::cachinglayer::cid_t,
std::unique_ptr<milvus::GroupChunk>>>
get_cells(const std::vector<milvus::cachinglayer::cid_t>& cids) override;
/**
* @brief Get the metadata object for this translator
*
* @return Pointer to the GroupCTMeta metadata
*/
milvus::cachinglayer::Meta*
meta() override {
return &meta_;
}
/**
* @brief Calculate total storage bytes needed for loading cells
*
* Sums up the storage size for all requested cells, with a minimum
* size of 1MB per cell.
*
* @param cids List of cell IDs
* @return Total storage bytes required
*/
int64_t
cells_storage_bytes(
const std::vector<milvus::cachinglayer::cid_t>& cids) const override {
constexpr int64_t MIN_STORAGE_BYTES = 1 * 1024 * 1024;
int64_t total_size = 0;
for (auto cid : cids) {
total_size += std::max(static_cast<int64_t>(chunk_size_[cid]),
MIN_STORAGE_BYTES);
}
return total_size;
}
private:
/**
* @brief Load a single chunk from Arrow RecordBatch
*
* Converts an Arrow RecordBatch into a GroupChunk containing
* field data for all columns in the chunk.
*
* @param record_batch Arrow RecordBatch containing the chunk data
* @param cid Cell ID of the chunk being loaded
* @return GroupChunk containing the loaded field data
*/
std::unique_ptr<milvus::GroupChunk>
load_group_chunk(const std::shared_ptr<arrow::RecordBatch>& record_batch,
const milvus::cachinglayer::cid_t cid);
int64_t segment_id_;
int64_t column_group_index_;
std::string key_;
std::unordered_map<FieldId, FieldMeta> field_metas_;
std::unique_ptr<milvus_storage::api::ChunkReader> chunk_reader_;
// chunk stats from reader
std::vector<uint64_t> chunk_size_;
GroupCTMeta meta_;
bool use_mmap_;
std::string mmap_dir_path_;
milvus::proto::common::LoadPriority load_priority_{
milvus::proto::common::LoadPriority::HIGH};
};
} // namespace milvus::segcore::storagev2translator

View File

@ -32,6 +32,10 @@ if(USE_OPENDAL)
set(SOURCE_FILES ${SOURCE_FILES} opendal/OpenDALChunkManager.cpp)
endif()
# Add loon_ffi subdirectory to include loon FFI source files
add_subdirectory(loon_ffi)
include_directories(${CMAKE_CURRENT_SOURCE_DIR}/loon_ffi)
include_directories(${CMAKE_CURRENT_SOURCE_DIR}/plugin)
add_library(milvus_storage OBJECT ${SOURCE_FILES})

View File

@ -59,6 +59,7 @@ DiskFileManagerImpl::DiskFileManagerImpl(
rcm_ = fileManagerContext.chunkManagerPtr;
fs_ = fileManagerContext.fs;
plugin_context_ = fileManagerContext.plugin_context;
loon_ffi_properties_ = fileManagerContext.loon_ffi_properties;
}
DiskFileManagerImpl::~DiskFileManagerImpl() {
@ -652,12 +653,28 @@ DiskFileManagerImpl::cache_raw_data_to_disk_storage_v2(const Config& config) {
uint32_t var_dim = 0;
int64_t write_offset = sizeof(num_rows) + sizeof(var_dim);
auto field_datas = GetFieldDatasFromStorageV2(all_remote_files,
std::vector<FieldDataPtr> field_datas;
auto manifest =
index::GetValueFromConfig<std::string>(config, SEGMENT_MANIFEST_KEY);
auto manifest_path_str = manifest.value_or("");
if (manifest_path_str != "") {
AssertInfo(
loon_ffi_properties_ != nullptr,
"loon ffi properties is null when build index with manifest");
field_datas = GetFieldDatasFromManifest(manifest_path_str,
loon_ffi_properties_,
field_meta_,
data_type,
dim,
element_type);
} else {
field_datas = GetFieldDatasFromStorageV2(all_remote_files,
GetFieldDataMeta().field_id,
data_type.value(),
element_type.value(),
dim,
fs_);
}
for (auto& field_data : field_datas) {
num_rows += uint32_t(field_data->get_num_rows());
cache_raw_data_to_disk_common<T>(field_data,

View File

@ -23,6 +23,7 @@
#include "common/Consts.h"
#include "boost/filesystem/path.hpp"
#include "log/Log.h"
#include "milvus-storage/properties.h"
#include "storage/ChunkManager.h"
#include "storage/Types.h"
#include "milvus-storage/filesystem/fs.h"
@ -61,12 +62,28 @@ struct FileManagerContext {
plugin_context = context;
}
/**
* @brief Set the loon FFI properties for storage access
*
* Configures the properties used for accessing loon storage through
* the FFI interface. These properties contain storage configuration
* such as endpoints, credentials, and connection settings.
*
* @param properties Shared pointer to Properties object
*/
void
set_loon_ffi_properties(
std::shared_ptr<milvus_storage::api::Properties> properties) {
loon_ffi_properties = std::move(properties);
}
FieldDataMeta fieldDataMeta;
IndexMeta indexMeta;
ChunkManagerPtr chunkManagerPtr;
milvus_storage::ArrowFileSystemPtr fs;
bool for_loading_index{false};
std::shared_ptr<CPluginContext> plugin_context;
std::shared_ptr<milvus_storage::api::Properties> loon_ffi_properties;
};
#define FILEMANAGER_TRY try {
@ -207,6 +224,7 @@ class FileManagerImpl : public milvus::FileManager {
IndexMeta index_meta_;
ChunkManagerPtr rcm_;
milvus_storage::ArrowFileSystemPtr fs_;
std::shared_ptr<milvus_storage::api::Properties> loon_ffi_properties_;
std::shared_ptr<CPluginContext> plugin_context_;
};

View File

@ -16,15 +16,21 @@
#include "storage/MemFileManagerImpl.h"
#include <memory>
#include <string>
#include <unordered_map>
#include <arrow/c/bridge.h>
#include "common/Common.h"
#include "common/Consts.h"
#include "common/FieldData.h"
#include "common/Types.h"
#include "log/Log.h"
#include "storage/Util.h"
#include "storage/FileManager.h"
#include "storage/loon_ffi/ffi_reader_c.h"
#include "index/Utils.h"
#include "milvus-storage/ffi_c.h"
#include "util.h"
namespace milvus::storage {
@ -34,6 +40,7 @@ MemFileManagerImpl::MemFileManagerImpl(
fileManagerContext.indexMeta) {
rcm_ = fileManagerContext.chunkManagerPtr;
fs_ = fileManagerContext.fs;
loon_ffi_properties_ = fileManagerContext.loon_ffi_properties;
plugin_context_ = fileManagerContext.plugin_context;
}
@ -213,9 +220,25 @@ MemFileManagerImpl::cache_raw_data_to_memory_storage_v2(const Config& config) {
auto segment_insert_files =
index::GetValueFromConfig<std::vector<std::vector<std::string>>>(
config, SEGMENT_INSERT_FILES_KEY);
AssertInfo(segment_insert_files.has_value(),
"[StorageV2] insert file paths for storage v2 is empty when "
"build index");
auto manifest =
index::GetValueFromConfig<std::string>(config, SEGMENT_MANIFEST_KEY);
AssertInfo(segment_insert_files.has_value() || manifest.has_value(),
"[StorageV2] insert file paths and manifest for storage v2 is "
"empty when build index");
// use manifest file for storage v2
auto manifest_path_str = manifest.value_or("");
if (manifest_path_str != "") {
AssertInfo(loon_ffi_properties_ != nullptr,
"[StorageV2] loon ffi properties is null when build index "
"with manifest");
return GetFieldDatasFromManifest(manifest_path_str,
loon_ffi_properties_,
field_meta_,
data_type,
dim,
element_type);
}
auto remote_files = segment_insert_files.value();
for (auto& files : remote_files) {
SortByPath(files);

View File

@ -19,6 +19,7 @@
#include "arrow/array/builder_binary.h"
#include "arrow/array/builder_nested.h"
#include "arrow/array/builder_primitive.h"
#include <arrow/c/bridge.h>
#include "arrow/scalar.h"
#include "arrow/type_fwd.h"
#include "common/type_c.h"
@ -58,8 +59,12 @@
#include "storage/KeyRetriever.h"
#include "segcore/memory_planner.h"
#include "mmap/Types.h"
#include "storage/loon_ffi/ffi_reader_c.h"
#include "storage/loon_ffi/util.h"
#include "milvus-storage/ffi_c.h"
#include "milvus-storage/format/parquet/file_reader.h"
#include "milvus-storage/filesystem/fs.h"
#include "milvus-storage/reader.h"
namespace milvus::storage {
@ -1359,6 +1364,95 @@ GetFieldDatasFromStorageV2(std::vector<std::vector<std::string>>& remote_files,
return field_data_list;
}
std::vector<FieldDataPtr>
GetFieldDatasFromManifest(
const std::string& manifest_path,
const std::shared_ptr<milvus_storage::api::Properties>& loon_ffi_properties,
const FieldDataMeta& field_meta,
std::optional<DataType> data_type,
int64_t dim,
std::optional<DataType> element_type) {
auto column_groups = GetColumnGroups(manifest_path, loon_ffi_properties);
// ReaderHandle reader_handler = 0;
std::string field_id_str = std::to_string(field_meta.field_id);
std::vector<std::string> needed_columns = {field_id_str};
// Create arrow schema from field meta
std::shared_ptr<arrow::Schema> arrow_schema;
bool nullable = field_meta.field_schema.nullable();
if (IsVectorDataType(data_type.value())) {
if (data_type.value() == DataType::VECTOR_ARRAY) {
arrow_schema = CreateArrowSchema(
data_type.value(), static_cast<int>(dim), element_type.value());
} else if (IsSparseFloatVectorDataType(data_type.value())) {
arrow_schema = CreateArrowSchema(data_type.value(), nullable);
} else {
arrow_schema = CreateArrowSchema(
data_type.value(), static_cast<int>(dim), nullable);
}
} else if (data_type.value() == DataType::ARRAY) {
// For ARRAY types, we use binary representation
// Element type information is encoded in the data itself
arrow_schema = CreateArrowSchema(data_type.value(), nullable);
} else {
// For scalar types
arrow_schema = CreateArrowSchema(data_type.value(), nullable);
}
auto updated_schema = std::make_shared<arrow::Schema>(
arrow::Schema({arrow_schema->field(0)->WithName(
std::to_string((field_meta.field_id)))}));
auto reader = milvus_storage::api::Reader::create(
column_groups,
updated_schema,
std::make_shared<std::vector<std::string>>(needed_columns),
*loon_ffi_properties);
AssertInfo(reader != nullptr, "Failed to create reader");
// without predicate
auto reader_result = reader->get_record_batch_reader("");
AssertInfo(reader_result.ok(),
"Failed to get record batch reader: " +
reader_result.status().ToString());
auto record_batch_reader = reader_result.ValueOrDie();
// Read all record batches and convert to FieldDataPtr
std::vector<FieldDataPtr> field_datas;
while (true) {
std::shared_ptr<arrow::RecordBatch> batch;
auto status = record_batch_reader->ReadNext(&batch);
AssertInfo(status.ok(),
"Failed to read record batch: " + status.ToString());
if (batch == nullptr) {
break; // End of stream
}
// Convert record batch to FieldData
auto num_rows = batch->num_rows();
if (num_rows == 0) {
continue;
}
auto chunked_array =
std::make_shared<arrow::ChunkedArray>(batch->column(0));
auto field_data = CreateFieldData(data_type.value(),
element_type.value(),
batch->schema()->field(0)->nullable(),
dim,
num_rows);
field_data->FillFieldData(chunked_array);
field_datas.push_back(field_data);
}
return field_datas;
}
std::vector<FieldDataPtr>
CacheRawDataAndFillMissing(const MemFileManagerImplPtr& file_manager,
const Config& config) {

View File

@ -189,6 +189,15 @@ GetFieldDatasFromStorageV2(std::vector<std::vector<std::string>>& remote_files,
int64_t dim,
milvus_storage::ArrowFileSystemPtr fs);
std::vector<FieldDataPtr>
GetFieldDatasFromManifest(
const std::string& manifest_path,
const std::shared_ptr<milvus_storage::api::Properties>& loon_ffi_properties,
const FieldDataMeta& field_meta,
std::optional<DataType> data_type,
int64_t dim,
std::optional<DataType> element_type);
std::map<std::string, int64_t>
PutIndexData(ChunkManager* remote_chunk_manager,
const std::vector<const uint8_t*>& data_slices,

View File

@ -0,0 +1,22 @@
# Copyright (C) 2019-2020 Zilliz. All rights reserved.
#
# Licensed 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
# FFI Reader source files for interfacing with milvus-storage through FFI
set(FFI_SRCS
${CMAKE_CURRENT_SOURCE_DIR}/ffi_reader_c.cpp
${CMAKE_CURRENT_SOURCE_DIR}/util.cpp
)
# Add FFI Reader source files to parent's SOURCE_FILES
set(SOURCE_FILES ${SOURCE_FILES} ${FFI_SRCS} PARENT_SCOPE)
# Include directories for FFI Reader
include_directories(${CMAKE_CURRENT_SOURCE_DIR})

View File

@ -0,0 +1,174 @@
// Copyright 2023 Zilliz
//
// Licensed 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 <arrow/c/bridge.h>
#include "common/EasyAssert.h"
#include "storage/loon_ffi/ffi_reader_c.h"
#include "common/common_type_c.h"
#include "milvus-storage/ffi_c.h"
#include "milvus-storage/reader.h"
#include "storage/loon_ffi/util.h"
#include "monitor/scope_metric.h"
ReaderHandle
createFFIReader(char* manifest,
struct ArrowSchema* schema,
char** needed_columns,
int64_t needed_columns_size,
const std::shared_ptr<Properties>& properties) {
ReaderHandle reader_handler = 0;
FFIResult result = reader_new(manifest,
schema,
needed_columns,
needed_columns_size,
properties.get(),
&reader_handler);
if (!IsSuccess(&result)) {
auto message = GetErrorMessage(&result);
// Copy the error message before freeing the FFIResult
std::string error_msg = message ? message : "Unknown error";
FreeFFIResult(&result);
throw std::runtime_error(error_msg);
}
FreeFFIResult(&result);
return reader_handler;
}
std::unique_ptr<milvus_storage::api::Reader>
GetLoonReader(
std::shared_ptr<milvus_storage::api::ColumnGroups> column_groups,
struct ArrowSchema* schema,
char** needed_columns,
int64_t needed_columns_size,
const std::shared_ptr<milvus_storage::api::Properties>& properties) {
auto result = arrow::ImportSchema(schema);
AssertInfo(result.ok(), "Import arrow schema failed");
auto arrow_schema = result.ValueOrDie();
return milvus_storage::api::Reader::create(
column_groups,
arrow_schema,
std::make_shared<std::vector<std::string>>(
needed_columns, needed_columns + needed_columns_size),
*properties);
}
CStatus
NewPackedFFIReader(const char* manifest_path,
struct ArrowSchema* schema,
char** needed_columns,
int64_t needed_columns_size,
CFFIPackedReader* c_packed_reader,
CStorageConfig c_storage_config,
CPluginContext* c_plugin_context) {
SCOPE_CGO_CALL_METRIC();
try {
auto properties =
MakeInternalPropertiesFromStorageConfig(c_storage_config);
AssertInfo(properties != nullptr, "properties is nullptr");
auto column_groups = GetColumnGroups(manifest_path, properties);
AssertInfo(column_groups != nullptr, "column groups is nullptr");
auto reader = GetLoonReader(column_groups,
schema,
needed_columns,
needed_columns_size,
properties);
*c_packed_reader = static_cast<CFFIPackedReader>(reader.release());
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
NewPackedFFIReaderWithManifest(const char* manifest_content,
struct ArrowSchema* schema,
char** needed_columns,
int64_t needed_columns_size,
CFFIPackedReader* c_loon_reader,
CStorageConfig c_storage_config,
CPluginContext* c_plugin_context) {
SCOPE_CGO_CALL_METRIC();
try {
auto properties =
MakeInternalPropertiesFromStorageConfig(c_storage_config);
// Parse the column groups, the column groups is a JSON string
auto cpp_column_groups =
std::make_shared<milvus_storage::api::ColumnGroups>();
auto des_result =
cpp_column_groups->deserialize(std::string_view(manifest_content));
AssertInfo(des_result.ok(), "failed to deserialize column groups");
auto reader = GetLoonReader(cpp_column_groups,
schema,
needed_columns,
needed_columns_size,
properties);
*c_loon_reader = static_cast<CFFIPackedReader>(reader.release());
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
GetFFIReaderStream(CFFIPackedReader c_packed_reader,
int64_t buffer_size,
struct ArrowArrayStream* out_stream) {
SCOPE_CGO_CALL_METRIC();
try {
auto reader =
static_cast<milvus_storage::api::Reader*>(c_packed_reader);
// FFIResult result =
// get_record_batch_reader(reader_handle, predicate, out_stream);
auto result = reader->get_record_batch_reader();
AssertInfo(result.ok(),
"failed to get record batch reader, {}",
result.status().ToString());
auto array_stream = result.ValueOrDie();
arrow::Status status =
arrow::ExportRecordBatchReader(array_stream, out_stream);
AssertInfo(status.ok(),
"failed to export record batch reader, {}",
status.ToString());
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
CloseFFIReader(CFFIPackedReader c_packed_reader) {
SCOPE_CGO_CALL_METRIC();
try {
auto reader =
static_cast<milvus_storage::api::Reader*>(c_packed_reader);
delete reader;
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}

View File

@ -0,0 +1,151 @@
// Copyright 2023 Zilliz
//
// Licensed 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.
#pragma once
#ifdef __cplusplus
extern "C" {
#endif
#include "common/common_type_c.h"
#include "common/type_c.h"
#include <arrow/c/abi.h>
#include "milvus-storage/ffi_c.h"
/**
* @brief Handle to a packed FFI reader instance.
*
* This is an alias for ReaderHandle used to read packed columnar data
* from storage in Milvus. The reader supports Arrow-based data access
* through the FFI (Foreign Function Interface) layer.
*/
typedef void* CFFIPackedReader;
/**
* @brief Creates a new packed FFI reader from a manifest file path.
*
* This function initializes a packed reader that can read columnar data
* from storage based on the manifest file. The manifest contains metadata
* about the data layout and file locations.
*
* @param manifest_path Path to the manifest file in object storage.
* Must be a valid UTF-8 encoded null-terminated string.
* @param schema Arrow schema defining the structure of the data.
* Must be a valid ArrowSchema pointer conforming to
* the Arrow C data interface specification.
* @param needed_columns Array of column names to read. If NULL, all columns
* from the schema will be read.
* @param needed_columns_size Number of column names in the needed_columns array.
* Must be 0 if needed_columns is NULL.
* @param c_packed_reader Output parameter for the created reader handle.
* On success, will contain a valid reader handle that
* must be released by the caller when no longer needed.
* @param c_storage_config Storage configuration containing credentials and
* endpoint information for accessing object storage.
* @param c_plugin_context Plugin context for extensibility, may be NULL if
* no plugins are used.
*
* @return CStatus indicating success or failure. On failure, the error_msg
* field contains details about what went wrong.
*
* @note The caller is responsible for releasing the reader handle after use.
* @note The schema pointer must remain valid for the lifetime of the reader.
*/
CStatus
NewPackedFFIReader(const char* manifest_path,
struct ArrowSchema* schema,
char** needed_columns,
int64_t needed_columns_size,
CFFIPackedReader* c_packed_reader,
CStorageConfig c_storage_config,
CPluginContext* c_plugin_context);
/**
* @brief Creates a new packed FFI reader from manifest content directly.
*
* Similar to NewPackedFFIReader, but accepts the manifest content directly
* as a string instead of reading from a file path. This is useful when the
* manifest has already been loaded or is generated dynamically.
*
* @param manifest_content The manifest content as a null-terminated string.
* Must be valid JSON or protobuf text format containing
* the manifest data.
* @param schema Arrow schema defining the structure of the data.
* Must be a valid ArrowSchema pointer conforming to
* the Arrow C data interface specification.
* @param needed_columns Array of column names to read. If NULL, all columns
* from the schema will be read.
* @param needed_columns_size Number of column names in the needed_columns array.
* Must be 0 if needed_columns is NULL.
* @param c_packed_reader Output parameter for the created reader handle.
* On success, will contain a valid reader handle that
* must be released by the caller when no longer needed.
* @param c_storage_config Storage configuration containing credentials and
* endpoint information for accessing object storage.
* @param c_plugin_context Plugin context for extensibility, may be NULL if
* no plugins are used.
*
* @return CStatus indicating success or failure. On failure, the error_msg
* field contains details about what went wrong.
*
* @note The caller is responsible for releasing the reader handle after use.
* @note The schema pointer must remain valid for the lifetime of the reader.
* @note The manifest content is copied internally, so the input string can
* be freed after this call returns.
*/
CStatus
NewPackedFFIReaderWithManifest(const char* manifest_content,
struct ArrowSchema* schema,
char** needed_columns,
int64_t needed_columns_size,
CFFIPackedReader* c_loon_reader,
CStorageConfig c_storage_config,
CPluginContext* c_plugin_context);
/**
* @brief Gets an ArrowArrayStream from the FFI reader for streaming data access.
*
* This function returns an ArrowArrayStream that can be used to iterate through
* record batches. The stream follows the Arrow C Stream Interface specification
* and must be released by calling stream->release() when done.
*
* @param c_packed_reader The FFI reader handle.
* @param out_stream Output parameter for the ArrowArrayStream. The caller
* is responsible for calling stream->release() when done.
*
* @return CStatus indicating success or failure. On failure, the error_msg
* field contains details about what went wrong.
*
* @note The stream must be released by calling out_stream->release(out_stream)
* when no longer needed to prevent memory leaks.
* @note Each call to this function creates a new stream starting from the beginning.
*/
CStatus
GetFFIReaderStream(CFFIPackedReader c_packed_reader,
int64_t batch_size,
struct ArrowArrayStream* out_stream);
/**
* @brief Closes and releases the FFI reader.
*
* @param c_packed_reader The FFI reader handle to close.
*
* @return CStatus indicating success or failure.
*/
CStatus
CloseFFIReader(CFFIPackedReader c_packed_reader);
#ifdef __cplusplus
}
#endif

View File

@ -0,0 +1,13 @@
// Copyright 2023 Zilliz
//
// Licensed 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.

View File

@ -0,0 +1,68 @@
// Copyright 2023 Zilliz
//
// Licensed 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.
#pragma once
#include <memory>
#include <mutex>
#include <shared_mutex>
#include "common/type_c.h"
#include "milvus-storage/properties.h"
#include "util.h"
namespace milvus::storage {
class LoonFFIPropertiesSingleton {
private:
LoonFFIPropertiesSingleton() = default;
public:
static LoonFFIPropertiesSingleton&
GetInstance() {
static LoonFFIPropertiesSingleton instance;
return instance;
}
void
Init(CStorageConfig c_storage_config) {
std::unique_lock lck(mutex_);
if (properties_ == nullptr) {
properties_ =
MakeInternalPropertiesFromStorageConfig(c_storage_config);
}
}
void
Init(const char* root_path) {
std::unique_lock lck(mutex_);
if (properties_ == nullptr) {
properties_ = MakeInternalLocalProperies(root_path);
}
}
std::shared_ptr<milvus_storage::api::Properties>
GetProperties() const {
std::shared_lock lck(mutex_);
return properties_;
}
private:
mutable std::shared_mutex mutex_;
std::shared_ptr<milvus_storage::api::Properties> properties_ = nullptr;
};
} // namespace milvus::storage

View File

@ -0,0 +1,321 @@
// Copyright 2023 Zilliz
//
// Licensed 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 <log/Log.h>
#include <memory>
#include <stdexcept>
#include <string>
#include <vector>
#include <nlohmann/json.hpp>
#include "common/common_type_c.h"
#include "common/type_c.h"
#include "milvus-storage/properties.h"
#include "milvus-storage/transaction/manifest.h"
#include "milvus-storage/transaction/transaction.h"
#include "storage/loon_ffi/util.h"
using json = nlohmann::json;
std::shared_ptr<Properties>
MakePropertiesFromStorageConfig(CStorageConfig c_storage_config) {
// Prepare key-value pairs from CStorageConfig
std::vector<const char*> keys;
std::vector<const char*> values;
// Add non-null string fields
if (c_storage_config.address != nullptr) {
keys.emplace_back(PROPERTY_FS_ADDRESS);
values.emplace_back(c_storage_config.address);
}
if (c_storage_config.bucket_name != nullptr) {
keys.emplace_back(PROPERTY_FS_BUCKET_NAME);
values.emplace_back(c_storage_config.bucket_name);
}
if (c_storage_config.access_key_id != nullptr) {
keys.emplace_back(PROPERTY_FS_ACCESS_KEY_ID);
values.emplace_back(c_storage_config.access_key_id);
}
if (c_storage_config.access_key_value != nullptr) {
keys.emplace_back(PROPERTY_FS_ACCESS_KEY_VALUE);
values.emplace_back(c_storage_config.access_key_value);
}
if (c_storage_config.root_path != nullptr) {
keys.emplace_back(PROPERTY_FS_ROOT_PATH);
values.emplace_back(c_storage_config.root_path);
}
if (c_storage_config.storage_type != nullptr) {
keys.emplace_back(PROPERTY_FS_STORAGE_TYPE);
values.emplace_back(c_storage_config.storage_type);
}
if (c_storage_config.cloud_provider != nullptr) {
keys.emplace_back(PROPERTY_FS_CLOUD_PROVIDER);
values.emplace_back(c_storage_config.cloud_provider);
}
if (c_storage_config.iam_endpoint != nullptr) {
keys.emplace_back(PROPERTY_FS_IAM_ENDPOINT);
values.emplace_back(c_storage_config.iam_endpoint);
}
if (c_storage_config.log_level != nullptr) {
keys.emplace_back(PROPERTY_FS_LOG_LEVEL);
values.emplace_back("Warn");
}
if (c_storage_config.region != nullptr) {
keys.emplace_back(PROPERTY_FS_REGION);
values.emplace_back(c_storage_config.region);
}
if (c_storage_config.sslCACert != nullptr) {
keys.emplace_back(PROPERTY_FS_SSL_CA_CERT);
values.emplace_back(c_storage_config.sslCACert);
}
if (c_storage_config.gcp_credential_json != nullptr) {
keys.emplace_back(PROPERTY_FS_GCP_CREDENTIAL_JSON);
values.emplace_back(c_storage_config.gcp_credential_json);
}
// Add boolean fields
keys.emplace_back(PROPERTY_FS_USE_SSL);
values.emplace_back(c_storage_config.useSSL ? "true" : "false");
keys.emplace_back(PROPERTY_FS_USE_IAM);
values.emplace_back(c_storage_config.useIAM ? "true" : "false");
keys.emplace_back(PROPERTY_FS_USE_VIRTUAL_HOST);
values.emplace_back(c_storage_config.useVirtualHost ? "true" : "false");
keys.emplace_back(PROPERTY_FS_USE_CUSTOM_PART_UPLOAD);
values.emplace_back(c_storage_config.use_custom_part_upload ? "true"
: "false");
// Add integer field
std::string timeout_str = std::to_string(c_storage_config.requestTimeoutMs);
keys.emplace_back(PROPERTY_FS_REQUEST_TIMEOUT_MS);
values.emplace_back(timeout_str.c_str());
// Create Properties using FFI
auto properties = std::make_shared<Properties>();
FFIResult result = properties_create(
keys.data(), values.data(), keys.size(), properties.get());
if (!IsSuccess(&result)) {
auto message = GetErrorMessage(&result);
// Copy the error message before freeing the FFIResult
std::string error_msg = message ? message : "Unknown error";
FreeFFIResult(&result);
throw std::runtime_error(error_msg);
}
FreeFFIResult(&result);
return properties;
}
std::shared_ptr<milvus_storage::api::Properties>
MakeInternalPropertiesFromStorageConfig(CStorageConfig c_storage_config) {
auto properties_map = std::make_shared<milvus_storage::api::Properties>();
// Add non-null string fields
if (c_storage_config.address != nullptr) {
milvus_storage::api::SetValue(
*properties_map, PROPERTY_FS_ADDRESS, c_storage_config.address);
}
if (c_storage_config.bucket_name != nullptr) {
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_BUCKET_NAME,
c_storage_config.bucket_name);
}
if (c_storage_config.access_key_id != nullptr) {
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_ACCESS_KEY_ID,
c_storage_config.access_key_id);
}
if (c_storage_config.access_key_value != nullptr) {
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_ACCESS_KEY_VALUE,
c_storage_config.access_key_value);
}
if (c_storage_config.root_path != nullptr) {
milvus_storage::api::SetValue(
*properties_map, PROPERTY_FS_ROOT_PATH, c_storage_config.root_path);
}
if (c_storage_config.storage_type != nullptr) {
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_STORAGE_TYPE,
c_storage_config.storage_type);
}
if (c_storage_config.cloud_provider != nullptr) {
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_CLOUD_PROVIDER,
c_storage_config.cloud_provider);
}
if (c_storage_config.iam_endpoint != nullptr) {
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_IAM_ENDPOINT,
c_storage_config.iam_endpoint);
}
if (c_storage_config.log_level != nullptr) {
milvus_storage::api::SetValue(
*properties_map, PROPERTY_FS_LOG_LEVEL, "Warn");
}
if (c_storage_config.region != nullptr) {
milvus_storage::api::SetValue(
*properties_map, PROPERTY_FS_REGION, c_storage_config.region);
}
if (c_storage_config.sslCACert != nullptr) {
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_SSL_CA_CERT,
c_storage_config.sslCACert);
}
if (c_storage_config.gcp_credential_json != nullptr) {
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_GCP_CREDENTIAL_JSON,
c_storage_config.gcp_credential_json);
}
// Add boolean fields
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_USE_SSL,
c_storage_config.useSSL ? "true" : "false");
milvus_storage::api::SetValue(*properties_map,
PROPERTY_FS_USE_IAM,
c_storage_config.useIAM ? "true" : "false");
milvus_storage::api::SetValue(
*properties_map,
PROPERTY_FS_USE_VIRTUAL_HOST,
c_storage_config.useVirtualHost ? "true" : "false");
milvus_storage::api::SetValue(
*properties_map,
PROPERTY_FS_USE_CUSTOM_PART_UPLOAD,
c_storage_config.use_custom_part_upload ? "true" : "false");
// Add integer fields
milvus_storage::api::SetValue(
*properties_map,
PROPERTY_FS_REQUEST_TIMEOUT_MS,
std::to_string(c_storage_config.requestTimeoutMs).c_str());
milvus_storage::api::SetValue(
*properties_map,
PROPERTY_FS_MAX_CONNECTIONS,
std::to_string(c_storage_config.max_connections).c_str());
return properties_map;
}
std::shared_ptr<milvus_storage::api::Properties>
MakeInternalLocalProperies(const char* c_path) {
auto properties_map = std::make_shared<milvus_storage::api::Properties>();
milvus_storage::api::SetValue(
*properties_map, PROPERTY_FS_STORAGE_TYPE, "local");
milvus_storage::api::SetValue(
*properties_map, PROPERTY_FS_ROOT_PATH, c_path);
return properties_map;
}
CStorageConfig
ToCStorageConfig(const milvus::storage::StorageConfig& config) {
return CStorageConfig{config.address.c_str(),
config.bucket_name.c_str(),
config.access_key_id.c_str(),
config.access_key_value.c_str(),
config.root_path.c_str(),
config.storage_type.c_str(),
config.cloud_provider.c_str(),
config.iam_endpoint.c_str(),
config.log_level.c_str(),
config.region.c_str(),
config.useSSL,
config.sslCACert.c_str(),
config.useIAM,
config.useVirtualHost,
config.requestTimeoutMs,
config.gcp_credential_json.c_str(),
false, // this field does not exist in StorageConfig
config.max_connections};
}
std::string
GetManifest(const std::string& path,
const std::shared_ptr<Properties>& properties) {
try {
// Parse the JSON string
json j = json::parse(path);
// Extract base_path and ver fields
std::string base_path = j.at("base_path").get<std::string>();
int64_t ver = j.at("ver").get<int64_t>();
// return std::make_pair(base_path, ver);
char* out_column_groups = nullptr;
int64_t out_read_version = 0;
FFIResult result = get_latest_column_groups(base_path.c_str(),
properties.get(),
&out_column_groups,
&out_read_version);
if (!IsSuccess(&result)) {
auto message = GetErrorMessage(&result);
// Copy the error message before freeing the FFIResult
std::string error_msg = message ? message : "Unknown error";
FreeFFIResult(&result);
throw std::runtime_error(error_msg);
}
FreeFFIResult(&result);
return {out_column_groups};
} catch (const json::parse_error& e) {
throw std::runtime_error(
std::string("Failed to parse manifest JSON: ") + e.what());
} catch (const json::out_of_range& e) {
throw std::runtime_error(
std::string("Missing required field in manifest: ") + e.what());
} catch (const json::type_error& e) {
throw std::runtime_error(
std::string("Invalid field type in manifest: ") + e.what());
}
}
std::shared_ptr<milvus_storage::api::ColumnGroups>
GetColumnGroups(
const std::string& path,
const std::shared_ptr<milvus_storage::api::Properties>& properties) {
try {
// Parse the JSON string
json j = json::parse(path);
// Extract base_path and ver fields
std::string base_path = j.at("base_path").get<std::string>();
int64_t ver = j.at("ver").get<int64_t>();
// TODO fetch manifest based on version after api supported
auto transaction =
std::make_unique<milvus_storage::api::transaction::TransactionImpl<
milvus_storage::api::ColumnGroups>>(*properties, base_path);
auto latest_manifest_result = transaction->get_latest_manifest();
if (!latest_manifest_result.ok()) {
throw(
std::runtime_error(latest_manifest_result.status().ToString()));
}
auto latest_manifest = latest_manifest_result.ValueOrDie();
return latest_manifest;
} catch (const json::parse_error& e) {
throw std::runtime_error(
std::string("Failed to parse manifest JSON: ") + e.what());
} catch (const json::out_of_range& e) {
throw std::runtime_error(
std::string("Missing required field in manifest: ") + e.what());
} catch (const json::type_error& e) {
throw std::runtime_error(
std::string("Invalid field type in manifest: ") + e.what());
}
}

View File

@ -0,0 +1,108 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed 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
#pragma once
#include <memory>
#include "common/common_type_c.h"
#include "common/type_c.h"
#include "milvus-storage/ffi_c.h"
#include "milvus-storage/properties.h"
#include "milvus-storage/transaction/manifest.h"
#include "storage/Types.h"
/**
* @brief Creates a shared pointer to Properties from CStorageConfig
*
* This utility function converts a CStorageConfig structure into a Properties
* object by calling the FFI properties_create function. All configuration fields
* from CStorageConfig are mapped to corresponding key-value pairs in Properties.
*
* The following fields are converted:
* - String fields: address, bucket_name, access_key_id, access_key_value,
* root_path, storage_type, cloud_provider, iam_endpoint, log_level,
* region, ssl_ca_cert, gcp_credential_json
* - Boolean fields: use_ssl, use_iam, use_virtual_host, use_custom_part_upload
* - Integer fields: request_timeout_ms
*
* @param c_storage_config The storage configuration to convert
* @return std::shared_ptr<Properties> Shared pointer to the created Properties
* @throws std::runtime_error If properties_create fails with error message from FFI
*/
std::shared_ptr<Properties>
MakePropertiesFromStorageConfig(CStorageConfig c_storage_config);
/**
* @brief Create internal API Properties from CStorageConfig
* Similar to MakePropertiesFromStorageConfig but creates a Properties
* object using the internal milvus_storage::api interface instead of FFI.
* All configuration fields from CStorageConfig are mapped to properties.
*
* @param c_storage_config The storage configuration to convert
* @return Shared pointer to milvus_storage::api::Properties
*/
std::shared_ptr<milvus_storage::api::Properties>
MakeInternalPropertiesFromStorageConfig(CStorageConfig c_storage_config);
/**
* @brief Create Properties for local filesystem storage
*
* Creates a minimal Properties object configured for local file storage
* with the specified path as the root.
*
* @param c_path Local filesystem path to use as storage root
* @return Shared pointer to Properties configured for local storage
*/
std::shared_ptr<milvus_storage::api::Properties>
MakeInternalLocalProperies(const char* c_path);
/**
* @brief Convert StorageConfig to C-style CStorageConfig
*
* Converts the C++ StorageConfig object into a CStorageConfig structure
* suitable for passing through FFI boundaries.
*
* @param config The StorageConfig object to convert
* @return CStorageConfig struct with copied configuration values
*/
CStorageConfig
ToCStorageConfig(const milvus::storage::StorageConfig& config);
/**
* @brief Retrieve manifest/column groups from storage via FFI
*
* Parses the manifest path JSON to extract base_path and version,
* then fetches the latest column groups from storage using FFI.
*
* @param path JSON string containing "base_path" and "ver" fields
* @param properties Storage properties for accessing the manifest
* @return JSON string containing column groups information
* @throws std::runtime_error If JSON parsing fails or FFI call fails
*/
std::string
GetManifest(const std::string& path,
const std::shared_ptr<Properties>& properties);
/**
* @brief Retrieve ColumnGroups metadata from manifest path
*
* Parses the manifest path JSON and fetches the latest manifest
* containing column groups metadata from the storage.
*
* @param path JSON string containing "base_path" and "ver" fields
* @param properties Storage properties for accessing the manifest
* @return Shared pointer to ColumnGroups metadata
* @throws std::runtime_error If JSON parsing or manifest fetch fails
*/
std::shared_ptr<milvus_storage::api::ColumnGroups>
GetColumnGroups(
const std::string& path,
const std::shared_ptr<milvus_storage::api::Properties>& properties);

View File

@ -14,7 +14,7 @@
# Update milvus-storage_VERSION for the first occurrence
milvus_add_pkg_config("milvus-storage")
set_property(DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES "")
set( milvus-storage_VERSION 0883026)
set( milvus-storage_VERSION ba7df7b)
set( GIT_REPOSITORY "https://github.com/milvus-io/milvus-storage.git")
message(STATUS "milvus-storage repo: ${GIT_REPOSITORY}")
message(STATUS "milvus-storage version: ${milvus-storage_VERSION}")

View File

@ -377,6 +377,7 @@ func (t *clusteringCompactionTask) BuildCompactionRequest() (*datapb.CompactionP
Deltalogs: segInfo.GetDeltalogs(),
IsSorted: segInfo.GetIsSorted(),
StorageVersion: segInfo.GetStorageVersion(),
Manifest: segInfo.GetManifestPath(),
})
}
WrapPluginContext(taskProto.GetCollectionID(), taskProto.GetSchema().GetProperties(), plan)

View File

@ -302,6 +302,7 @@ func (t *l0CompactionTask) selectFlushedSegment() ([]*SegmentInfo, []*datapb.Com
CollectionID: info.GetCollectionID(),
PartitionID: info.GetPartitionID(),
IsSorted: info.GetIsSorted(),
Manifest: info.GetManifestPath(),
})
}
@ -341,6 +342,7 @@ func (t *l0CompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, err
InsertChannel: segInfo.GetInsertChannel(),
Deltalogs: segInfo.GetDeltalogs(),
IsSorted: segInfo.GetIsSorted(),
Manifest: segInfo.GetManifestPath(),
})
segments = append(segments, segInfo)
}

View File

@ -402,6 +402,7 @@ func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, er
Deltalogs: segInfo.GetDeltalogs(),
IsSorted: segInfo.GetIsSorted(),
StorageVersion: segInfo.GetStorageVersion(),
Manifest: segInfo.GetManifestPath(),
})
segIDMap[segID] = segInfo.GetDeltalogs()
segments = append(segments, segInfo)

View File

@ -1177,6 +1177,23 @@ func UpdateCheckPointOperator(segmentID int64, checkpoints []*datapb.CheckPoint,
}
}
func UpdateManifest(segmentID int64, manifestPath string) UpdateOperator {
return func(modPack *updateSegmentPack) bool {
segment := modPack.Get(segmentID)
if segment == nil {
log.Ctx(context.TODO()).Warn("meta update: update manifest failed - segment not found",
zap.Int64("segmentID", segmentID))
return false
}
// skip empty manifest update and same manifest
if manifestPath == "" || segment.ManifestPath == manifestPath {
return false
}
segment.ManifestPath = manifestPath
return true
}
}
func UpdateImportedRows(segmentID int64, rows int64) UpdateOperator {
return func(modPack *updateSegmentPack) bool {
segment := modPack.Get(segmentID)
@ -1693,6 +1710,7 @@ func (m *meta) completeClusterCompactionMutation(t *datapb.CompactionTask, resul
// visible after stats and index
IsInvisible: true,
StorageVersion: seg.GetStorageVersion(),
ManifestPath: seg.GetManifest(),
}
segment := NewSegmentInfo(segmentInfo)
compactToSegInfos = append(compactToSegInfos, segment)
@ -1794,6 +1812,7 @@ func (m *meta) completeMixCompactionMutation(
return info.GetDmlPosition()
})),
IsSorted: compactToSegment.GetIsSorted(),
ManifestPath: compactToSegment.GetManifest(),
})
if compactToSegmentInfo.GetNumOfRows() == 0 {
@ -2283,6 +2302,7 @@ func (m *meta) completeSortCompactionMutation(
Deltalogs: resultSegment.GetDeltalogs(),
CompactionFrom: []int64{compactFromSegID},
IsSorted: true,
ManifestPath: resultSegment.GetManifest(),
}
segment := NewSegmentInfo(segmentInfo)

View File

@ -954,6 +954,7 @@ func TestUpdateSegmentsInfo(t *testing.T) {
),
UpdateStartPosition([]*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}),
UpdateCheckPointOperator(1, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10, Position: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}, Timestamp: 100}}}, true),
UpdateManifest(1, "files/binlogs/1/2/1000/manifest_0"),
)
assert.NoError(t, err)
@ -969,6 +970,7 @@ func TestUpdateSegmentsInfo(t *testing.T) {
assert.Equal(t, len(updated.Bm25Statslogs[0].Binlogs), 1)
assert.Equal(t, updated.State, commonpb.SegmentState_Growing)
assert.Equal(t, updated.NumOfRows, int64(10))
assert.Equal(t, updated.ManifestPath, "files/binlogs/1/2/1000/manifest_0")
err = meta.UpdateSegmentsInfo(
context.TODO(),
@ -991,6 +993,7 @@ func TestUpdateSegmentsInfo(t *testing.T) {
UpdateStatusOperator(1, commonpb.SegmentState_Flushed),
UpdateStartPosition([]*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}),
UpdateCheckPointOperator(1, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 12, Position: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}, Timestamp: 101}}}, true),
UpdateManifest(1, "files/binlogs/1/2/1000/manifest_2"),
)
assert.NoError(t, err)
@ -1002,6 +1005,7 @@ func TestUpdateSegmentsInfo(t *testing.T) {
assert.Equal(t, len(updated.Deltalogs), 0)
assert.Equal(t, len(updated.Bm25Statslogs), 0)
assert.Equal(t, updated.State, commonpb.SegmentState_Flushed)
assert.Equal(t, updated.ManifestPath, "files/binlogs/1/2/1000/manifest_2")
err = meta.UpdateSegmentsInfo(
context.TODO(),
@ -1124,6 +1128,12 @@ func TestUpdateSegmentsInfo(t *testing.T) {
)
assert.NoError(t, err)
err = meta.UpdateSegmentsInfo(
context.TODO(),
UpdateManifest(1, "files/binlogs/1/2/1000/manifest_0"),
)
assert.NoError(t, err)
err = meta.UpdateSegmentsInfo(context.TODO(), UpdateAsDroppedIfEmptyWhenFlushing(1))
assert.NoError(t, err)
})

View File

@ -687,8 +687,9 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
UpdateCheckPointOperator(req.GetSegmentID(), req.GetCheckPoints()))
}
// save binlogs, start positions and checkpoints
// save manifest, start positions and checkpoints
operators = append(operators,
UpdateManifest(req.GetSegmentID(), req.GetManifestPath()),
UpdateStartPosition(req.GetStartPositions()),
UpdateAsDroppedIfEmptyWhenFlushing(req.GetSegmentID()),
)

View File

@ -327,6 +327,7 @@ func (it *indexBuildTask) prepareJobRequest(ctx context.Context, segment *Segmen
TaskSlot: it.taskSlot,
LackBinlogRows: segIndex.NumRows - totalRows,
InsertLogs: segment.GetBinlogs(),
Manifest: segment.GetManifestPath(),
}
WrapPluginContext(segment.GetCollectionID(), schema.GetProperties(), req)

View File

@ -344,7 +344,8 @@ func (t *clusteringCompactionTask) getScalarAnalyzeResult(ctx context.Context) e
t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
t.partitionID, t.collectionID, t.plan.Channel, 100,
storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig))
storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
if err != nil {
return err
}
@ -367,7 +368,8 @@ func (t *clusteringCompactionTask) getScalarAnalyzeResult(ctx context.Context) e
t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
t.partitionID, t.collectionID, t.plan.Channel, 100,
storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig))
storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
if err != nil {
return err
}
@ -427,7 +429,8 @@ func (t *clusteringCompactionTask) generatedVectorPlan(ctx context.Context, buff
t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.plan.MaxSegmentRows,
t.partitionID, t.collectionID, t.plan.Channel, 100,
storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig))
storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
if err != nil {
return err
}
@ -497,6 +500,7 @@ func (t *clusteringCompactionTask) mapping(ctx context.Context,
Deltalogs: segment.Deltalogs,
FieldBinlogs: segment.FieldBinlogs,
StorageVersion: segment.StorageVersion,
Manifest: segment.GetManifest(),
}
future := t.mappingPool.Submit(func() (any, error) {
err := t.mappingSegment(ctx, segmentClone)
@ -605,7 +609,21 @@ func (t *clusteringCompactionTask) mappingSegment(
return merr.WrapErrIllegalCompactionPlan()
}
rr, err := storage.NewBinlogRecordReader(ctx,
var rr storage.RecordReader
if segment.GetManifest() != "" {
rr, err = storage.NewManifestRecordReader(ctx,
segment.GetManifest(),
t.plan.Schema,
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
return t.binlogIO.Download(ctx, paths)
}),
storage.WithCollectionID(t.GetCollection()),
storage.WithVersion(segment.StorageVersion),
storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
)
} else {
rr, err = storage.NewBinlogRecordReader(ctx,
segment.GetFieldBinlogs(),
t.plan.Schema,
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
@ -616,6 +634,8 @@ func (t *clusteringCompactionTask) mappingSegment(
storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
)
}
if err != nil {
log.Warn("new binlog record reader wrong", zap.Error(err))
return err
@ -902,7 +922,23 @@ func (t *clusteringCompactionTask) scalarAnalyzeSegment(
log.Warn("unsupported storage version", zap.Int64("storage version", segment.GetStorageVersion()))
return nil, fmt.Errorf("unsupported storage version %d", segment.GetStorageVersion())
}
rr, err := storage.NewBinlogRecordReader(ctx,
var rr storage.RecordReader
var err error
if segment.GetManifest() != "" {
rr, err = storage.NewManifestRecordReader(ctx,
segment.GetManifest(),
t.plan.GetSchema(),
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
return t.binlogIO.Download(ctx, paths)
}),
storage.WithVersion(segment.StorageVersion),
storage.WithBufferSize(t.bufferSize),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithNeededFields(requiredFields),
storage.WithCollectionID(t.GetCollection()),
)
} else {
rr, err = storage.NewBinlogRecordReader(ctx,
binlogs,
t.plan.GetSchema(),
storage.WithDownloader(func(ctx context.Context, paths []string) ([][]byte, error) {
@ -914,6 +950,7 @@ func (t *clusteringCompactionTask) scalarAnalyzeSegment(
storage.WithNeededFields(requiredFields),
storage.WithCollectionID(t.GetCollection()),
)
}
if err != nil {
log.Warn("new binlog record reader wrong", zap.Error(err))
return make(map[interface{}]int64), err

View File

@ -86,7 +86,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal() {
func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2ToV2Format() {
var segmentID int64 = 1001
fBinlogs, deltalogs, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
fBinlogs, deltalogs, _, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
s.NoError(err)
dblobs, err := getInt64DeltaBlobs(
@ -149,7 +149,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2To
var segmentID int64 = 1001
fBinlogs, deltalogs, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
fBinlogs, deltalogs, _, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
s.NoError(err)
dblobs, err := getInt64DeltaBlobs(
@ -263,6 +263,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) initStorageV2Segments(rows int,
deltas *datapb.FieldBinlog,
stats map[int64]*datapb.FieldBinlog,
bm25Stats map[int64]*datapb.FieldBinlog,
manifest string,
size int64,
err error,
) {

View File

@ -44,7 +44,8 @@ func mergeSortMultipleSegments(ctx context.Context,
logIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedLogIDs().GetBegin(), plan.GetPreAllocatedLogIDs().GetEnd())
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
writer, err := NewMultiSegmentWriter(ctx, binlogIO, compAlloc, plan.GetMaxSize(), plan.GetSchema(), compactionParams, maxRows, partitionID, collectionID, plan.GetChannel(), 4096,
storage.WithStorageConfig(compactionParams.StorageConfig))
storage.WithStorageConfig(compactionParams.StorageConfig),
storage.WithUseLoonFFI(compactionParams.UseLoonFFI))
if err != nil {
return nil, err
}
@ -58,7 +59,18 @@ func mergeSortMultipleSegments(ctx context.Context,
segmentReaders := make([]storage.RecordReader, len(binlogs))
segmentFilters := make([]compaction.EntityFilter, len(binlogs))
for i, s := range binlogs {
reader, err := storage.NewBinlogRecordReader(ctx,
var reader storage.RecordReader
if s.GetManifest() != "" {
reader, err = storage.NewManifestRecordReader(ctx,
s.GetManifest(),
plan.GetSchema(),
storage.WithCollectionID(collectionID),
storage.WithDownloader(binlogIO.Download),
storage.WithVersion(s.StorageVersion),
storage.WithStorageConfig(compactionParams.StorageConfig),
)
} else {
reader, err = storage.NewBinlogRecordReader(ctx,
s.GetFieldBinlogs(),
plan.GetSchema(),
storage.WithCollectionID(collectionID),
@ -66,6 +78,7 @@ func mergeSortMultipleSegments(ctx context.Context,
storage.WithVersion(s.StorageVersion),
storage.WithStorageConfig(compactionParams.StorageConfig),
)
}
if err != nil {
return nil, err
}

View File

@ -150,7 +150,7 @@ func (t *mixCompactionTask) mergeSplit(
segIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedSegmentIDs().GetBegin(), t.plan.GetPreAllocatedSegmentIDs().GetEnd())
logIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedLogIDs().GetBegin(), t.plan.GetPreAllocatedLogIDs().GetEnd())
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
mWriter, err := NewMultiSegmentWriter(ctx, t.binlogIO, compAlloc, t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.maxRows, t.partitionID, t.collectionID, t.GetChannelName(), 4096, storage.WithStorageConfig(t.compactionParams.StorageConfig))
mWriter, err := NewMultiSegmentWriter(ctx, t.binlogIO, compAlloc, t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.maxRows, t.partitionID, t.collectionID, t.GetChannelName(), 4096, storage.WithStorageConfig(t.compactionParams.StorageConfig), storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
if err != nil {
return nil, err
}
@ -217,7 +217,18 @@ func (t *mixCompactionTask) writeSegment(ctx context.Context,
}
entityFilter := compaction.NewEntityFilter(delta, t.plan.GetCollectionTtl(), t.currentTime)
reader, err := storage.NewBinlogRecordReader(ctx,
var reader storage.RecordReader
if seg.GetManifest() != "" {
reader, err = storage.NewManifestRecordReader(ctx,
seg.GetManifest(),
t.plan.GetSchema(),
storage.WithCollectionID(t.collectionID),
storage.WithDownloader(t.binlogIO.Download),
storage.WithVersion(seg.GetStorageVersion()),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
)
} else {
reader, err = storage.NewBinlogRecordReader(ctx,
seg.GetFieldBinlogs(),
t.plan.GetSchema(),
storage.WithCollectionID(t.collectionID),
@ -225,6 +236,7 @@ func (t *mixCompactionTask) writeSegment(ctx context.Context,
storage.WithVersion(seg.GetStorageVersion()),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
)
}
if err != nil {
log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err))
return

View File

@ -121,7 +121,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_MixToV2Format() {
v2Segments := []int64{10, 11}
for _, segID := range v2Segments {
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
s.NoError(err)
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
CollectionID: 1,
@ -156,7 +156,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV2Format() {
v2Segments := []int64{10, 11}
for _, segID := range v2Segments {
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
s.NoError(err)
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
CollectionID: 1,
@ -192,7 +192,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV1Format() {
v2Segments := []int64{10, 11}
for _, segID := range v2Segments {
binlogs, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
s.NoError(err)
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
CollectionID: 1,
@ -307,6 +307,7 @@ func (s *MixCompactionTaskStorageV2Suite) initStorageV2Segments(rows int, seed i
deltas *datapb.FieldBinlog,
stats map[int64]*datapb.FieldBinlog,
bm25Stats map[int64]*datapb.FieldBinlog,
manifest string,
size int64,
err error,
) {

View File

@ -1112,7 +1112,6 @@ func genTestCollectionMeta() *etcdpb.CollectionMeta {
{
FieldID: Int64FieldWithDefaultValue,
Name: "field_int64_with_default_value",
IsPrimaryKey: true,
DataType: schemapb.DataType_Int64,
DefaultValue: &schemapb.ValueField{
Data: &schemapb.ValueField_LongData{

View File

@ -112,7 +112,7 @@ func (s *NamespaceCompactorTestSuite) setupSortedSegments() {
StorageType: "local",
RootPath: rootPath,
}, columnGroups)
inserts, _, _, _, _, err := bw.Write(context.Background(), pack)
inserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Require().NoError(err)
s.sortedSegments = append(s.sortedSegments, &datapb.CompactionSegmentBinlogs{
SegmentID: int64(i),

View File

@ -120,7 +120,7 @@ func (w *MultiSegmentWriter) closeWriter() error {
return err
}
fieldBinlogs, statsLog, bm25Logs := w.writer.GetLogs()
fieldBinlogs, statsLog, bm25Logs, manifest := w.writer.GetLogs()
result := &datapb.CompactionSegment{
SegmentID: w.currentSegmentID,
@ -130,6 +130,7 @@ func (w *MultiSegmentWriter) closeWriter() error {
Channel: w.channel,
Bm25Logs: lo.Values(bm25Logs),
StorageVersion: w.storageVersion,
Manifest: manifest,
}
w.res = append(w.res, result)

View File

@ -62,6 +62,8 @@ type sortCompactionTask struct {
insertLogs []*datapb.FieldBinlog
storageVersion int64
segmentStorageVersion int64
manifest string
useLoonFFI bool
done chan struct{}
tr *timerecord.TimeRecorder
@ -132,6 +134,8 @@ func (t *sortCompactionTask) preCompact() error {
t.insertLogs = segment.GetFieldBinlogs()
t.storageVersion = t.compactionParams.StorageVersion
t.segmentStorageVersion = segment.GetStorageVersion()
t.manifest = segment.GetManifest()
t.useLoonFFI = t.compactionParams.UseLoonFFI
log.Ctx(t.ctx).Info("preCompaction analyze",
zap.Int64("planID", t.GetPlanID()),
@ -139,6 +143,7 @@ func (t *sortCompactionTask) preCompact() error {
zap.Int64("partitionID", t.partitionID),
zap.Int64("segmentID", t.segmentID),
zap.Int64("storageVersion", t.storageVersion),
zap.Bool("useLoonFFI", t.useLoonFFI),
zap.Any("compactionParams", t.compactionParams),
)
@ -175,6 +180,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
}),
storage.WithVersion(t.storageVersion),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithUseLoonFFI(t.useLoonFFI),
)
if err != nil {
log.Warn("sort segment wrong, unable to init segment writer",
@ -207,12 +213,23 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
log.Warn("sort task only support int64 and varchar pk field")
}
rr, err := storage.NewBinlogRecordReader(ctx, t.insertLogs, t.plan.Schema,
var rr storage.RecordReader
// use manifest reader if manifest presents
if t.manifest != "" {
rr, err = storage.NewManifestRecordReader(ctx, t.manifest, t.plan.Schema,
storage.WithVersion(t.segmentStorageVersion),
storage.WithDownloader(t.binlogIO.Download),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithCollectionID(t.collectionID),
)
} else {
rr, err = storage.NewBinlogRecordReader(ctx, t.insertLogs, t.plan.Schema,
storage.WithVersion(t.segmentStorageVersion),
storage.WithDownloader(t.binlogIO.Download),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
storage.WithCollectionID(t.collectionID),
)
}
if err != nil {
log.Warn("error creating insert binlog reader", zap.Error(err))
return nil, err
@ -228,7 +245,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
return nil, err
}
binlogs, stats, bm25stats := srw.GetLogs()
binlogs, stats, bm25stats, manifest := srw.GetLogs()
insertLogs := storage.SortFieldBinlogs(binlogs)
if err := binlog.CompressFieldBinlogs(insertLogs); err != nil {
return nil, err
@ -264,6 +281,7 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
Channel: t.GetChannelName(),
IsSorted: true,
StorageVersion: t.storageVersion,
Manifest: manifest,
},
}
planResult := &datapb.CompactionPlanResult{

View File

@ -310,6 +310,7 @@ func (it *indexBuildTask) Execute(ctx context.Context) error {
it.req.GetCollectionID(),
it.req.GetPartitionID(),
it.req.GetSegmentID())
buildIndexParams.Manifest = it.req.GetManifest()
}
log.Info("create index", zap.Any("buildIndexParams", buildIndexParams))

View File

@ -261,7 +261,7 @@ func (st *statsTask) sort(ctx context.Context) ([]*datapb.FieldBinlog, error) {
return nil, err
}
binlogs, stats, bm25stats := srw.GetLogs()
binlogs, stats, bm25stats, _ := srw.GetLogs()
insertLogs := storage.SortFieldBinlogs(binlogs)
if err := binlog.CompressFieldBinlogs(insertLogs); err != nil {
return nil, err

View File

@ -48,6 +48,7 @@ type SegmentInfo struct {
deltalogs []*datapb.FieldBinlog
bm25logs []*datapb.FieldBinlog
currentSplit []storagecommon.ColumnGroup
manifestPath string
}
func (s *SegmentInfo) SegmentID() int64 {
@ -129,6 +130,10 @@ func (s *SegmentInfo) Bm25logs() []*datapb.FieldBinlog {
return s.bm25logs
}
func (s *SegmentInfo) ManifestPath() string {
return s.manifestPath
}
func (s *SegmentInfo) Clone() *SegmentInfo {
return &SegmentInfo{
segmentID: s.segmentID,
@ -150,6 +155,7 @@ func (s *SegmentInfo) Clone() *SegmentInfo {
deltalogs: s.deltalogs,
bm25logs: s.bm25logs,
currentSplit: s.currentSplit,
manifestPath: s.manifestPath,
}
}
@ -188,5 +194,6 @@ func NewSegmentInfo(info *datapb.SegmentInfo, bfs pkoracle.PkStat, bm25Stats *Se
deltalogs: info.GetDeltalogs(),
bm25logs: info.GetBm25Statslogs(),
currentSplit: currentSplit,
manifestPath: info.GetManifestPath(),
}
}

View File

@ -92,6 +92,7 @@ func (b *brokerMetaWriter) UpdateSync(ctx context.Context, pack *SyncTask) error
zap.Int("statslogNum", lo.SumBy(statsFieldBinlogs, getBinlogNum)),
zap.Int("deltalogNum", lo.SumBy(deltaFieldBinlogs, getBinlogNum)),
zap.Int("bm25logNum", lo.SumBy(deltaBm25StatsBinlogs, getBinlogNum)),
zap.String("manifestPath", pack.manifestPath),
zap.String("vChannelName", pack.channelName),
)
@ -118,6 +119,7 @@ func (b *brokerMetaWriter) UpdateSync(ctx context.Context, pack *SyncTask) error
SegLevel: pack.level,
StorageVersion: segment.GetStorageVersion(),
WithFullBinlogs: true,
ManifestPath: pack.manifestPath,
}
err := retry.Handle(ctx, func() (bool, error) {
err := b.broker.SaveBinlogPaths(ctx, req)

View File

@ -18,8 +18,12 @@ package syncmgr
import (
"context"
"fmt"
"path"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/apache/arrow/go/v17/arrow/memory"
"github.com/samber/lo"
"go.uber.org/zap"
@ -306,22 +310,100 @@ func (bw *BulkPackWriter) writeDelta(ctx context.Context, pack *SyncPack) (*data
if pack.deltaData == nil {
return &datapb.FieldBinlog{}, nil
}
s, err := NewStorageSerializer(bw.metaCache, bw.schema)
if err != nil {
return nil, err
pkField := func() *schemapb.FieldSchema {
for _, field := range bw.schema.Fields {
if field.IsPrimaryKey {
return field
}
deltaBlob, err := s.serializeDeltalog(pack)
}
return nil
}()
if pkField == nil {
return nil, fmt.Errorf("primary key field not found")
}
logID := bw.nextID()
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID, logID)
path := path.Join(bw.chunkManager.RootPath(), common.SegmentDeltaLogPath, k)
writer, err := storage.NewDeltalogWriter(
ctx, pack.collectionID, pack.partitionID, pack.segmentID, logID, pkField.DataType, path,
storage.WithUploader(func(ctx context.Context, kvs map[string][]byte) error {
// Get the only blob in the map
if len(kvs) != 1 {
return fmt.Errorf("expected 1 blob, got %d", len(kvs))
}
for _, blob := range kvs {
return bw.chunkManager.Write(ctx, path, blob)
}
return nil
}),
)
if err != nil {
return nil, err
}
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID, bw.nextID())
deltalog, err := bw.writeLog(ctx, deltaBlob, common.SegmentDeltaLogPath, k, pack)
pkType := func() arrow.DataType {
switch pkField.DataType {
case schemapb.DataType_Int64:
return arrow.PrimitiveTypes.Int64
case schemapb.DataType_VarChar:
return arrow.BinaryTypes.String
default:
return nil
}
}()
if pkType == nil {
return nil, fmt.Errorf("unexpected pk type %v", pkField.DataType)
}
pkBuilder := array.NewBuilder(memory.DefaultAllocator, pkType)
tsBuilder := array.NewBuilder(memory.DefaultAllocator, arrow.PrimitiveTypes.Int64)
defer pkBuilder.Release()
defer tsBuilder.Release()
for i := int64(0); i < pack.deltaData.RowCount; i++ {
switch pkField.DataType {
case schemapb.DataType_Int64:
pkBuilder.(*array.Int64Builder).Append(pack.deltaData.Pks[i].GetValue().(int64))
case schemapb.DataType_VarChar:
pkBuilder.(*array.StringBuilder).Append(pack.deltaData.Pks[i].GetValue().(string))
default:
return nil, fmt.Errorf("unexpected pk type %v", pkField.DataType)
}
tsBuilder.(*array.Int64Builder).Append(int64(pack.deltaData.Tss[i]))
}
pkArray := pkBuilder.NewArray()
tsArray := tsBuilder.NewArray()
record := storage.NewSimpleArrowRecord(array.NewRecord(arrow.NewSchema([]arrow.Field{
{Name: "pk", Type: pkType},
{Name: "ts", Type: arrow.PrimitiveTypes.Int64},
}, nil), []arrow.Array{pkArray, tsArray}, pack.deltaData.RowCount), map[storage.FieldID]int{
common.RowIDField: 0,
common.TimeStampField: 1,
})
err = writer.Write(record)
if err != nil {
return nil, err
}
err = writer.Close()
if err != nil {
return nil, err
}
deltalog := &datapb.Binlog{
EntriesNum: pack.deltaData.RowCount,
TimestampFrom: pack.tsFrom,
TimestampTo: pack.tsTo,
LogPath: path,
LogSize: pack.deltaData.Size() / 4, // Not used
MemorySize: pack.deltaData.Size(),
}
bw.sizeWritten += deltalog.LogSize
return &datapb.FieldBinlog{
FieldID: s.pkField.GetFieldID(),
FieldID: pkField.GetFieldID(),
Binlogs: []*datapb.Binlog{deltalog},
}, nil
}

View File

@ -151,14 +151,14 @@ func TestBulkPackWriter_Write(t *testing.T) {
{
EntriesNum: 10,
LogPath: "files/delta_log/123/456/789/10000",
LogSize: 592,
MemorySize: 327,
LogSize: 60,
MemorySize: 240,
},
},
},
wantStats: map[int64]*datapb.FieldBinlog{},
wantBm25Stats: map[int64]*datapb.FieldBinlog{},
wantSize: 592,
wantSize: 60,
wantErr: nil,
},
}

View File

@ -20,6 +20,7 @@ import (
"context"
"encoding/base64"
"math"
"path"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/apache/arrow/go/v17/arrow/memory"
@ -77,6 +78,7 @@ func (bw *BulkPackWriterV2) Write(ctx context.Context, pack *SyncPack) (
deltas *datapb.FieldBinlog,
stats map[int64]*datapb.FieldBinlog,
bm25Stats map[int64]*datapb.FieldBinlog,
manifest string,
size int64,
err error,
) {
@ -86,7 +88,7 @@ func (bw *BulkPackWriterV2) Write(ctx context.Context, pack *SyncPack) (
return
}
if inserts, err = bw.writeInserts(ctx, pack); err != nil {
if inserts, manifest, err = bw.writeInserts(ctx, pack); err != nil {
log.Error("failed to write insert data", zap.Error(err))
return
}
@ -125,24 +127,20 @@ func (bw *BulkPackWriterV2) getBucketName() string {
return paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue()
}
func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, error) {
func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, string, error) {
if len(pack.insertData) == 0 {
return make(map[int64]*datapb.FieldBinlog), nil
return make(map[int64]*datapb.FieldBinlog), "", nil
}
columnGroups := bw.columnGroups
rec, err := bw.serializeBinlog(ctx, pack)
if err != nil {
return nil, err
return nil, "", err
}
logs := make(map[int64]*datapb.FieldBinlog)
paths := make([]string, 0)
for _, columnGroup := range columnGroups {
path := metautil.BuildInsertLogPath(bw.getRootPath(), pack.collectionID, pack.partitionID, pack.segmentID, columnGroup.GroupID, bw.nextID())
paths = append(paths, path)
}
tsArray := rec.Column(common.TimeStampField).(*array.Int64)
rows := rec.Len()
var tsFrom uint64 = math.MaxUint64
@ -175,16 +173,24 @@ func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (m
}
}
w, err := storage.NewPackedRecordWriter(bucketName, paths, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
if err != nil {
return nil, err
}
doWrite := func(w storage.RecordWriter) error {
if err = w.Write(rec); err != nil {
return nil, err
return err
}
// close first to get compressed size
if err = w.Close(); err != nil {
return nil, err
// close first the get stats & output
return w.Close()
}
var manifestPath string
if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() {
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID)
basePath := path.Join(bw.getRootPath(), common.SegmentInsertLogPath, k)
w, err := storage.NewPackedRecordManifestWriter(bucketName, basePath, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
if err != nil {
return nil, "", err
}
if err = doWrite(w); err != nil {
return nil, "", err
}
for _, columnGroup := range columnGroups {
columnGroupID := columnGroup.GroupID
@ -203,14 +209,48 @@ func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (m
},
}
}
return logs, nil
manifestPath = w.GetWrittenManifest()
} else {
paths := make([]string, 0)
for _, columnGroup := range columnGroups {
path := metautil.BuildInsertLogPath(bw.getRootPath(), pack.collectionID, pack.partitionID, pack.segmentID, columnGroup.GroupID, bw.nextID())
paths = append(paths, path)
}
w, err := storage.NewPackedRecordWriter(bucketName, paths, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
if err != nil {
return nil, "", err
}
if err = doWrite(w); err != nil {
return nil, "", err
}
// workaround to store row num
for _, columnGroup := range columnGroups {
columnGroupID := columnGroup.GroupID
logs[columnGroupID] = &datapb.FieldBinlog{
FieldID: columnGroupID,
ChildFields: columnGroup.Fields,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(w.GetColumnGroupWrittenCompressed(columnGroup.GroupID)),
MemorySize: int64(w.GetColumnGroupWrittenUncompressed(columnGroup.GroupID)),
LogPath: w.GetWrittenPaths(columnGroupID),
EntriesNum: w.GetWrittenRowNum(),
TimestampFrom: tsFrom,
TimestampTo: tsTo,
},
},
}
}
}
func (bw *BulkPackWriterV2) serializeBinlog(ctx context.Context, pack *SyncPack) (storage.Record, error) {
return logs, manifestPath, nil
}
func (bw *BulkPackWriterV2) serializeBinlog(_ context.Context, pack *SyncPack) (storage.Record, error) {
if len(pack.insertData) == 0 {
return nil, nil
}
arrowSchema, err := storage.ConvertToArrowSchema(bw.schema)
arrowSchema, err := storage.ConvertToArrowSchema(bw.schema, true)
if err != nil {
return nil, err
}

View File

@ -145,7 +145,7 @@ func (s *PackWriterV2Suite) TestPackWriterV2_Write() {
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
gotInserts, _, _, _, _, err := bw.Write(context.Background(), pack)
gotInserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.NoError(err)
s.Equal(gotInserts[0].Binlogs[0].GetEntriesNum(), int64(rows))
s.Equal(gotInserts[0].Binlogs[0].GetLogPath(), "/tmp/insert_log/123/456/789/0/1")
@ -164,7 +164,7 @@ func (s *PackWriterV2Suite) TestWriteEmptyInsertData() {
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName)
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
_, _, _, _, _, err := bw.Write(context.Background(), pack)
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.NoError(err)
}
@ -193,7 +193,7 @@ func (s *PackWriterV2Suite) TestNoPkField() {
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
_, _, _, _, _, err := bw.Write(context.Background(), pack)
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err)
}
@ -210,7 +210,7 @@ func (s *PackWriterV2Suite) TestAllocIDExhausedError() {
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema))
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
_, _, _, _, _, err := bw.Write(context.Background(), pack)
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err)
}
@ -231,7 +231,7 @@ func (s *PackWriterV2Suite) TestWriteInsertDataError() {
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit)
_, _, _, _, _, err := bw.Write(context.Background(), pack)
_, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err)
}

View File

@ -18,7 +18,6 @@ package syncmgr
import (
"context"
"fmt"
"strconv"
"github.com/samber/lo"
@ -181,31 +180,6 @@ func (s *storageV1Serializer) serializeMergedBM25Stats(pack *SyncPack) (map[int6
return blobs, nil
}
func (s *storageV1Serializer) serializeDeltalog(pack *SyncPack) (*storage.Blob, error) {
if len(pack.deltaData.Pks) == 0 {
return &storage.Blob{}, nil
}
writer, finalizer, err := storage.CreateDeltalogWriter(pack.collectionID, pack.partitionID, pack.segmentID, pack.deltaData.Pks[0].Type(), 1024)
if err != nil {
return nil, err
}
if len(pack.deltaData.Pks) != len(pack.deltaData.Tss) {
return nil, fmt.Errorf("pk and ts should have same length in delta log, but get %d and %d", len(pack.deltaData.Pks), len(pack.deltaData.Tss))
}
for i := 0; i < len(pack.deltaData.Pks); i++ {
deleteLog := storage.NewDeleteLog(pack.deltaData.Pks[i], pack.deltaData.Tss[i])
err = writer.WriteValue(deleteLog)
if err != nil {
return nil, err
}
}
writer.Close()
return finalizer()
}
func hasBM25Function(schema *schemapb.CollectionSchema) bool {
for _, function := range schema.GetFunctions() {
if function.GetType() == schemapb.FunctionType_BM25 {

View File

@ -241,18 +241,6 @@ func (s *StorageV1SerializerSuite) TestSerializeInsert() {
})
}
func (s *StorageV1SerializerSuite) TestSerializeDelete() {
s.Run("serialize_normal", func() {
pack := s.getBasicPack()
pack.WithDeleteData(s.getDeleteBuffer())
pack.WithTimeRange(50, 100)
blob, err := s.serializer.serializeDeltalog(pack)
s.NoError(err)
s.NotNil(blob)
})
}
func (s *StorageV1SerializerSuite) TestBadSchema() {
mockCache := metacache.NewMockMetaCache(s.T())
_, err := NewStorageSerializer(mockCache, &schemapb.CollectionSchema{})

View File

@ -75,6 +75,8 @@ type SyncTask struct {
bm25Binlogs map[int64]*datapb.FieldBinlog
deltaBinlog *datapb.FieldBinlog
manifestPath string
writeRetryOpts []retry.Option
failureCallback func(err error)
@ -134,10 +136,11 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
switch segmentInfo.GetStorageVersion() {
case storage.StorageV2:
// TODO change to return manifest after integrated
// New sync task means needs to flush data immediately, so do not need to buffer data in writer again.
writer := NewBulkPackWriterV2(t.metacache, t.schema, t.chunkManager, t.allocator, 0,
packed.DefaultMultiPartUploadSize, t.storageConfig, columnGroups, t.writeRetryOpts...)
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.flushedSize, err = writer.Write(ctx, t.pack)
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.manifestPath, t.flushedSize, err = writer.Write(ctx, t.pack)
if err != nil {
log.Warn("failed to write sync data with storage v2 format", zap.Error(err))
return err

View File

@ -90,6 +90,7 @@ func PackSegmentLoadInfo(segment *datapb.SegmentInfo, channelCheckpoint *msgpb.M
IsSorted: segment.GetIsSorted(),
TextStatsLogs: segment.GetTextStatsLogs(),
JsonKeyStatsLogs: segment.GetJsonKeyStats(),
ManifestPath: segment.GetManifestPath(),
}
return loadInfo
}

View File

@ -1151,6 +1151,52 @@ func (_c *MockSegment_Level_Call) RunAndReturn(run func() datapb.SegmentLevel) *
return _c
}
// Load provides a mock function with given fields: ctx
func (_m *MockSegment) Load(ctx context.Context) error {
ret := _m.Called(ctx)
if len(ret) == 0 {
panic("no return value specified for Load")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context) error); ok {
r0 = rf(ctx)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockSegment_Load_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Load'
type MockSegment_Load_Call struct {
*mock.Call
}
// Load is a helper method to define mock.On call
// - ctx context.Context
func (_e *MockSegment_Expecter) Load(ctx interface{}) *MockSegment_Load_Call {
return &MockSegment_Load_Call{Call: _e.mock.On("Load", ctx)}
}
func (_c *MockSegment_Load_Call) Run(run func(ctx context.Context)) *MockSegment_Load_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
})
return _c
}
func (_c *MockSegment_Load_Call) Return(_a0 error) *MockSegment_Load_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockSegment_Load_Call) RunAndReturn(run func(context.Context) error) *MockSegment_Load_Call {
_c.Call.Return(run)
return _c
}
// LoadDeltaData provides a mock function with given fields: ctx, deltaData
func (_m *MockSegment) LoadDeltaData(ctx context.Context, deltaData *storage.DeltaData) error {
ret := _m.Called(ctx, deltaData)

View File

@ -370,6 +370,7 @@ func NewSegment(ctx context.Context,
SegmentID: loadInfo.GetSegmentID(),
SegmentType: segmentType,
IsSorted: loadInfo.GetIsSorted(),
LoadInfo: loadInfo,
})
return nil, err
}).Await(); err != nil {
@ -1367,6 +1368,10 @@ func (s *LocalSegment) FinishLoad() error {
return nil
}
func (s *LocalSegment) Load(ctx context.Context) error {
return s.csegment.Load(ctx)
}
type ReleaseScope int
const (

View File

@ -85,6 +85,7 @@ type Segment interface {
Delete(ctx context.Context, primaryKeys storage.PrimaryKeys, timestamps []typeutil.Timestamp) error
LoadDeltaData(ctx context.Context, deltaData *storage.DeltaData) error
LastDeltaTimestamp() uint64
Load(ctx context.Context) error
FinishLoad() error
Release(ctx context.Context, opts ...releaseOption)

View File

@ -182,6 +182,10 @@ func (s *L0Segment) FinishLoad() error {
return nil
}
func (s *L0Segment) Load(ctx context.Context) error {
return nil
}
func (s *L0Segment) Release(ctx context.Context, opts ...releaseOption) {
s.dataGuard.Lock()
defer s.dataGuard.Unlock()

View File

@ -46,6 +46,7 @@ import (
"github.com/milvus-io/milvus/internal/querynodev2/pkoracle"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/internal/util/indexparamcheck"
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
@ -56,6 +57,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/util/contextutil"
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
"github.com/milvus-io/milvus/pkg/v2/util/hardware"
"github.com/milvus-io/milvus/pkg/v2/util/indexparams"
"github.com/milvus-io/milvus/pkg/v2/util/logutil"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/metric"
@ -309,6 +311,28 @@ func (loader *segmentLoader) Load(ctx context.Context,
for _, info := range infos {
loadInfo := info
for _, indexInfo := range loadInfo.IndexInfos {
indexParams := funcutil.KeyValuePair2Map(indexInfo.IndexParams)
// some build params also exist in indexParams, which are useless during loading process
if vecindexmgr.GetVecIndexMgrInstance().IsDiskANN(indexParams["index_type"]) {
if err := indexparams.SetDiskIndexLoadParams(paramtable.Get(), indexParams, indexInfo.GetNumRows()); err != nil {
return nil, err
}
}
// set whether enable offset cache for bitmap index
if indexParams["index_type"] == indexparamcheck.IndexBitmap {
indexparams.SetBitmapIndexLoadParams(paramtable.Get(), indexParams)
}
if err := indexparams.AppendPrepareLoadParams(paramtable.Get(), indexParams); err != nil {
return nil, err
}
indexInfo.IndexParams = funcutil.Map2KeyValuePair(indexParams)
}
segment, err := NewSegment(
ctx,
collection,
@ -885,7 +909,7 @@ func (loader *segmentLoader) loadSealedSegment(ctx context.Context, loadInfo *qu
collection := segment.GetCollection()
schemaHelper, _ := typeutil.CreateSchemaHelper(collection.Schema())
indexedFieldInfos, fieldBinlogs, textIndexes, unindexedTextFields, jsonKeyStats := separateLoadInfoV2(loadInfo, collection.Schema())
indexedFieldInfos, _, textIndexes, unindexedTextFields, jsonKeyStats := separateLoadInfoV2(loadInfo, collection.Schema())
if err := segment.AddFieldDataInfo(ctx, loadInfo.GetNumOfRows(), loadInfo.GetBinlogPaths()); err != nil {
return err
}
@ -898,63 +922,25 @@ func (loader *segmentLoader) loadSealedSegment(ctx context.Context, loadInfo *qu
zap.Int64s("unindexed text fields", lo.Keys(unindexedTextFields)),
zap.Int64s("indexed json key fields", lo.Keys(jsonKeyStats)),
)
if err := loader.loadFieldsIndex(ctx, schemaHelper, segment, loadInfo.GetNumOfRows(), indexedFieldInfos); err != nil {
return err
}
loadFieldsIndexSpan := tr.RecordSpan()
metrics.QueryNodeLoadIndexLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(loadFieldsIndexSpan.Milliseconds()))
// 2. complement raw data for the scalar fields without raw data
for _, info := range indexedFieldInfos {
fieldID := info.IndexInfo.FieldID
field, err := schemaHelper.GetFieldFromID(fieldID)
if err != nil {
return err
if err = segment.Load(ctx); err != nil {
return errors.Wrap(err, "At Load")
}
if !segment.HasRawData(fieldID) || field.GetIsPrimaryKey() {
// Skip loading raw data for fields in column group when using storage v2
if loadInfo.GetStorageVersion() == storage.StorageV2 &&
!storagecommon.IsVectorDataType(field.GetDataType()) &&
field.GetDataType() != schemapb.DataType_Text {
log.Info("skip loading raw data for field in short column group",
zap.Int64("fieldID", fieldID),
zap.String("index", info.IndexInfo.GetIndexName()),
)
continue
}
log.Info("field index doesn't include raw data, load binlog...",
zap.Int64("fieldID", fieldID),
zap.String("index", info.IndexInfo.GetIndexName()),
)
// for scalar index's raw data, only load to mmap not memory
if err = segment.LoadFieldData(ctx, fieldID, loadInfo.GetNumOfRows(), info.FieldBinlog); err != nil {
log.Warn("load raw data failed", zap.Int64("fieldID", fieldID), zap.Error(err))
return err
}
}
if !storagecommon.IsVectorDataType(field.GetDataType()) &&
!segment.HasFieldData(fieldID) &&
loadInfo.GetStorageVersion() != storage.StorageV2 {
// Lazy load raw data to avoid search failure after dropping index.
// storage v2 will load all scalar fields so we don't need to load raw data for them.
if err = segment.LoadFieldData(ctx, fieldID, loadInfo.GetNumOfRows(), info.FieldBinlog, "disable"); err != nil {
log.Warn("load raw data failed", zap.Int64("fieldID", fieldID), zap.Error(err))
return err
}
}
}
complementScalarDataSpan := tr.RecordSpan()
if err := loadSealedSegmentFields(ctx, collection, segment, fieldBinlogs, loadInfo.GetNumOfRows()); err != nil {
return err
}
loadRawDataSpan := tr.RecordSpan()
if err = segment.FinishLoad(); err != nil {
return errors.Wrap(err, "At FinishLoad")
}
for _, indexInfo := range loadInfo.IndexInfos {
segment.fieldIndexes.Insert(indexInfo.GetIndexID(), &IndexedFieldInfo{
FieldBinlog: &datapb.FieldBinlog{
FieldID: indexInfo.GetFieldID(),
},
IndexInfo: indexInfo,
IsLoaded: true,
})
}
// load text indexes.
for _, info := range textIndexes {
if err := segment.LoadTextIndex(ctx, info, schemaHelper); err != nil {
@ -985,9 +971,9 @@ func (loader *segmentLoader) loadSealedSegment(ctx context.Context, loadInfo *qu
}
patchEntryNumberSpan := tr.RecordSpan()
log.Info("Finish loading segment",
zap.Duration("loadFieldsIndexSpan", loadFieldsIndexSpan),
zap.Duration("complementScalarDataSpan", complementScalarDataSpan),
zap.Duration("loadRawDataSpan", loadRawDataSpan),
// zap.Duration("loadFieldsIndexSpan", loadFieldsIndexSpan),
// zap.Duration("complementScalarDataSpan", complementScalarDataSpan),
// zap.Duration("loadRawDataSpan", loadRawDataSpan),
zap.Duration("patchEntryNumberSpan", patchEntryNumberSpan),
zap.Duration("loadTextIndexesSpan", loadTextIndexesSpan),
zap.Duration("loadJsonKeyIndexSpan", loadJSONKeyIndexesSpan),
@ -1032,7 +1018,7 @@ func (loader *segmentLoader) LoadSegment(ctx context.Context,
return err
}
} else {
if err := segment.LoadMultiFieldData(ctx); err != nil {
if err := segment.Load(ctx); err != nil {
return err
}
if err := segment.FinishLoad(); err != nil {

View File

@ -0,0 +1,456 @@
// 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.
package storage
import (
"fmt"
"path"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type BinlogRecordWriter interface {
RecordWriter
GetLogs() (
fieldBinlogs map[FieldID]*datapb.FieldBinlog,
statsLog *datapb.FieldBinlog,
bm25StatsLog map[FieldID]*datapb.FieldBinlog,
manifest string,
)
GetRowNum() int64
FlushChunk() error
GetBufferUncompressed() uint64
Schema() *schemapb.CollectionSchema
}
type packedBinlogRecordWriterBase struct {
// attributes
collectionID UniqueID
partitionID UniqueID
segmentID UniqueID
schema *schemapb.CollectionSchema
BlobsWriter ChunkedBlobsWriter
allocator allocator.Interface
maxRowNum int64
arrowSchema *arrow.Schema
bufferSize int64
multiPartUploadSize int64
columnGroups []storagecommon.ColumnGroup
storageConfig *indexpb.StorageConfig
storagePluginContext *indexcgopb.StoragePluginContext
pkCollector *PkStatsCollector
bm25Collector *Bm25StatsCollector
tsFrom typeutil.Timestamp
tsTo typeutil.Timestamp
rowNum int64
writtenUncompressed uint64
// results
fieldBinlogs map[FieldID]*datapb.FieldBinlog
statsLog *datapb.FieldBinlog
bm25StatsLog map[FieldID]*datapb.FieldBinlog
manifest string
}
func (pw *packedBinlogRecordWriterBase) getColumnStatsFromRecord(r Record, allFields []*schemapb.FieldSchema) map[int64]storagecommon.ColumnStats {
result := make(map[int64]storagecommon.ColumnStats)
for _, field := range allFields {
if arr := r.Column(field.FieldID); arr != nil {
result[field.FieldID] = storagecommon.ColumnStats{
AvgSize: int64(arr.Data().SizeInBytes()) / int64(arr.Len()),
}
}
}
return result
}
func (pw *packedBinlogRecordWriterBase) GetWrittenUncompressed() uint64 {
return pw.writtenUncompressed
}
func (pw *packedBinlogRecordWriterBase) writeStats() error {
// Write PK stats
pkStatsMap, err := pw.pkCollector.Digest(
pw.collectionID,
pw.partitionID,
pw.segmentID,
pw.storageConfig.GetRootPath(),
pw.rowNum,
pw.allocator,
pw.BlobsWriter,
)
if err != nil {
return err
}
// Extract single PK stats from map
for _, statsLog := range pkStatsMap {
pw.statsLog = statsLog
break
}
// Write BM25 stats
bm25StatsLog, err := pw.bm25Collector.Digest(
pw.collectionID,
pw.partitionID,
pw.segmentID,
pw.storageConfig.GetRootPath(),
pw.rowNum,
pw.allocator,
pw.BlobsWriter,
)
if err != nil {
return err
}
pw.bm25StatsLog = bm25StatsLog
return nil
}
func (pw *packedBinlogRecordWriterBase) GetLogs() (
fieldBinlogs map[FieldID]*datapb.FieldBinlog,
statsLog *datapb.FieldBinlog,
bm25StatsLog map[FieldID]*datapb.FieldBinlog,
manifest string,
) {
return pw.fieldBinlogs, pw.statsLog, pw.bm25StatsLog, pw.manifest
}
func (pw *packedBinlogRecordWriterBase) GetRowNum() int64 {
return pw.rowNum
}
func (pw *packedBinlogRecordWriterBase) FlushChunk() error {
return nil // do nothing
}
func (pw *packedBinlogRecordWriterBase) Schema() *schemapb.CollectionSchema {
return pw.schema
}
func (pw *packedBinlogRecordWriterBase) GetBufferUncompressed() uint64 {
return uint64(pw.multiPartUploadSize)
}
var _ BinlogRecordWriter = (*PackedBinlogRecordWriter)(nil)
type PackedBinlogRecordWriter struct {
packedBinlogRecordWriterBase
writer *packedRecordWriter
}
func (pw *PackedBinlogRecordWriter) Write(r Record) error {
if err := pw.initWriters(r); err != nil {
return err
}
// Track timestamps
tsArray := r.Column(common.TimeStampField).(*array.Int64)
rows := r.Len()
for i := 0; i < rows; i++ {
ts := typeutil.Timestamp(tsArray.Value(i))
if ts < pw.tsFrom {
pw.tsFrom = ts
}
if ts > pw.tsTo {
pw.tsTo = ts
}
}
// Collect statistics
if err := pw.pkCollector.Collect(r); err != nil {
return err
}
if err := pw.bm25Collector.Collect(r); err != nil {
return err
}
err := pw.writer.Write(r)
if err != nil {
return merr.WrapErrServiceInternal(fmt.Sprintf("write record batch error: %s", err.Error()))
}
pw.writtenUncompressed = pw.writer.GetWrittenUncompressed()
return nil
}
func (pw *PackedBinlogRecordWriter) initWriters(r Record) error {
if pw.writer == nil {
if len(pw.columnGroups) == 0 {
allFields := typeutil.GetAllFieldSchemas(pw.schema)
pw.columnGroups = storagecommon.SplitColumns(allFields, pw.getColumnStatsFromRecord(r, allFields), storagecommon.DefaultPolicies()...)
}
logIdStart, _, err := pw.allocator.Alloc(uint32(len(pw.columnGroups)))
if err != nil {
return err
}
paths := []string{}
for _, columnGroup := range pw.columnGroups {
path := metautil.BuildInsertLogPath(pw.storageConfig.GetRootPath(), pw.collectionID, pw.partitionID, pw.segmentID, columnGroup.GroupID, logIdStart)
paths = append(paths, path)
logIdStart++
}
pw.writer, err = NewPackedRecordWriter(pw.storageConfig.GetBucketName(), paths, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
if err != nil {
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
}
}
return nil
}
func (pw *PackedBinlogRecordWriter) finalizeBinlogs() {
if pw.writer == nil {
return
}
pw.rowNum = pw.writer.GetWrittenRowNum()
if pw.fieldBinlogs == nil {
pw.fieldBinlogs = make(map[FieldID]*datapb.FieldBinlog, len(pw.columnGroups))
}
for _, columnGroup := range pw.columnGroups {
columnGroupID := columnGroup.GroupID
if _, exists := pw.fieldBinlogs[columnGroupID]; !exists {
pw.fieldBinlogs[columnGroupID] = &datapb.FieldBinlog{
FieldID: columnGroupID,
ChildFields: columnGroup.Fields,
}
}
pw.fieldBinlogs[columnGroupID].Binlogs = append(pw.fieldBinlogs[columnGroupID].Binlogs, &datapb.Binlog{
LogSize: int64(pw.writer.GetColumnGroupWrittenCompressed(columnGroupID)),
MemorySize: int64(pw.writer.GetColumnGroupWrittenUncompressed(columnGroupID)),
LogPath: pw.writer.GetWrittenPaths(columnGroupID),
EntriesNum: pw.writer.GetWrittenRowNum(),
TimestampFrom: pw.tsFrom,
TimestampTo: pw.tsTo,
})
}
pw.manifest = pw.writer.GetWrittenManifest()
}
func (pw *PackedBinlogRecordWriter) Close() error {
if pw.writer != nil {
if err := pw.writer.Close(); err != nil {
return err
}
}
pw.finalizeBinlogs()
if err := pw.writeStats(); err != nil {
return err
}
return nil
}
func newPackedBinlogRecordWriter(collectionID, partitionID, segmentID UniqueID, schema *schemapb.CollectionSchema,
blobsWriter ChunkedBlobsWriter, allocator allocator.Interface, maxRowNum int64, bufferSize, multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (*PackedBinlogRecordWriter, error) {
arrowSchema, err := ConvertToArrowSchema(schema, true)
if err != nil {
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
}
writer := &PackedBinlogRecordWriter{
packedBinlogRecordWriterBase: packedBinlogRecordWriterBase{
collectionID: collectionID,
partitionID: partitionID,
segmentID: segmentID,
schema: schema,
arrowSchema: arrowSchema,
BlobsWriter: blobsWriter,
allocator: allocator,
maxRowNum: maxRowNum,
bufferSize: bufferSize,
multiPartUploadSize: multiPartUploadSize,
columnGroups: columnGroups,
storageConfig: storageConfig,
storagePluginContext: storagePluginContext,
tsFrom: typeutil.MaxTimestamp,
tsTo: 0,
},
}
// Create stats collectors
writer.pkCollector, err = NewPkStatsCollector(
collectionID,
schema,
maxRowNum,
)
if err != nil {
return nil, err
}
writer.bm25Collector = NewBm25StatsCollector(schema)
return writer, nil
}
var _ BinlogRecordWriter = (*PackedManifestRecordWriter)(nil)
type PackedManifestRecordWriter struct {
packedBinlogRecordWriterBase
// writer and stats generated at runtime
writer *packedRecordManifestWriter
}
func (pw *PackedManifestRecordWriter) Write(r Record) error {
if err := pw.initWriters(r); err != nil {
return err
}
// Track timestamps
tsArray := r.Column(common.TimeStampField).(*array.Int64)
rows := r.Len()
for i := 0; i < rows; i++ {
ts := typeutil.Timestamp(tsArray.Value(i))
if ts < pw.tsFrom {
pw.tsFrom = ts
}
if ts > pw.tsTo {
pw.tsTo = ts
}
}
// Collect statistics
if err := pw.pkCollector.Collect(r); err != nil {
return err
}
if err := pw.bm25Collector.Collect(r); err != nil {
return err
}
err := pw.writer.Write(r)
if err != nil {
return merr.WrapErrServiceInternal(fmt.Sprintf("write record batch error: %s", err.Error()))
}
pw.writtenUncompressed = pw.writer.GetWrittenUncompressed()
return nil
}
func (pw *PackedManifestRecordWriter) initWriters(r Record) error {
if pw.writer == nil {
if len(pw.columnGroups) == 0 {
allFields := typeutil.GetAllFieldSchemas(pw.schema)
pw.columnGroups = storagecommon.SplitColumns(allFields, pw.getColumnStatsFromRecord(r, allFields), storagecommon.DefaultPolicies()...)
}
var err error
k := metautil.JoinIDPath(pw.collectionID, pw.partitionID, pw.segmentID)
basePath := path.Join(pw.storageConfig.GetRootPath(), common.SegmentInsertLogPath, k)
pw.writer, err = NewPackedRecordManifestWriter(pw.storageConfig.GetBucketName(), basePath, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
if err != nil {
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
}
}
return nil
}
func (pw *PackedManifestRecordWriter) finalizeBinlogs() {
if pw.writer == nil {
return
}
pw.rowNum = pw.writer.GetWrittenRowNum()
if pw.fieldBinlogs == nil {
pw.fieldBinlogs = make(map[FieldID]*datapb.FieldBinlog, len(pw.columnGroups))
}
for _, columnGroup := range pw.columnGroups {
columnGroupID := columnGroup.GroupID
if _, exists := pw.fieldBinlogs[columnGroupID]; !exists {
pw.fieldBinlogs[columnGroupID] = &datapb.FieldBinlog{
FieldID: columnGroupID,
ChildFields: columnGroup.Fields,
}
}
pw.fieldBinlogs[columnGroupID].Binlogs = append(pw.fieldBinlogs[columnGroupID].Binlogs, &datapb.Binlog{
LogSize: int64(pw.writer.GetColumnGroupWrittenCompressed(columnGroupID)),
MemorySize: int64(pw.writer.GetColumnGroupWrittenUncompressed(columnGroupID)),
LogPath: pw.writer.GetWrittenPaths(columnGroupID),
EntriesNum: pw.writer.GetWrittenRowNum(),
TimestampFrom: pw.tsFrom,
TimestampTo: pw.tsTo,
})
}
pw.manifest = pw.writer.GetWrittenManifest()
}
func (pw *PackedManifestRecordWriter) Close() error {
if pw.writer != nil {
if err := pw.writer.Close(); err != nil {
return err
}
}
pw.finalizeBinlogs()
if err := pw.writeStats(); err != nil {
return err
}
return nil
}
func newPackedManifestRecordWriter(collectionID, partitionID, segmentID UniqueID, schema *schemapb.CollectionSchema,
blobsWriter ChunkedBlobsWriter, allocator allocator.Interface, maxRowNum int64, bufferSize, multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (*PackedManifestRecordWriter, error) {
arrowSchema, err := ConvertToArrowSchema(schema, true)
if err != nil {
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
}
writer := &PackedManifestRecordWriter{
packedBinlogRecordWriterBase: packedBinlogRecordWriterBase{
collectionID: collectionID,
partitionID: partitionID,
segmentID: segmentID,
schema: schema,
arrowSchema: arrowSchema,
BlobsWriter: blobsWriter,
allocator: allocator,
maxRowNum: maxRowNum,
bufferSize: bufferSize,
multiPartUploadSize: multiPartUploadSize,
columnGroups: columnGroups,
storageConfig: storageConfig,
storagePluginContext: storagePluginContext,
tsFrom: typeutil.MaxTimestamp,
tsTo: 0,
},
}
// Create stats collectors
writer.pkCollector, err = NewPkStatsCollector(
collectionID,
schema,
maxRowNum,
)
if err != nil {
return nil, err
}
writer.bm25Collector = NewBm25StatsCollector(schema)
return writer, nil
}

View File

@ -0,0 +1,336 @@
package storage
import (
"fmt"
"io"
"strconv"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/array"
"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/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type RecordReader interface {
Next() (Record, error)
Close() error
}
type packedRecordReader struct {
reader *packed.PackedReader
field2Col map[FieldID]int
}
var _ RecordReader = (*packedRecordReader)(nil)
func (pr *packedRecordReader) Next() (Record, error) {
rec, err := pr.reader.ReadNext()
if err != nil {
return nil, err
}
return NewSimpleArrowRecord(rec, pr.field2Col), nil
}
func (pr *packedRecordReader) Close() error {
if pr.reader != nil {
return pr.reader.Close()
}
return nil
}
func newPackedRecordReader(
paths []string,
schema *schemapb.CollectionSchema,
bufferSize int64,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (*packedRecordReader, error) {
arrowSchema, err := ConvertToArrowSchema(schema, true)
if err != nil {
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
}
field2Col := make(map[FieldID]int)
allFields := typeutil.GetAllFieldSchemas(schema)
for i, field := range allFields {
field2Col[field.FieldID] = i
}
reader, err := packed.NewPackedReader(paths, arrowSchema, bufferSize, storageConfig, storagePluginContext)
if err != nil {
return nil, err
}
return &packedRecordReader{
reader: reader,
field2Col: field2Col,
}, nil
}
func NewRecordReaderFromManifest(manifest string,
schema *schemapb.CollectionSchema,
bufferSize int64,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (RecordReader, error) {
return NewManifestReader(manifest, schema, bufferSize, storageConfig, storagePluginContext)
}
var _ RecordReader = (*IterativeRecordReader)(nil)
type IterativeRecordReader struct {
cur RecordReader
iterate func() (RecordReader, error)
}
// Close implements RecordReader.
func (ir *IterativeRecordReader) Close() error {
if ir.cur != nil {
return ir.cur.Close()
}
return nil
}
func (ir *IterativeRecordReader) Next() (Record, error) {
if ir.cur == nil {
r, err := ir.iterate()
if err != nil {
return nil, err
}
ir.cur = r
}
rec, err := ir.cur.Next()
if err == io.EOF {
closeErr := ir.cur.Close()
if closeErr != nil {
return nil, closeErr
}
ir.cur, err = ir.iterate()
if err != nil {
return nil, err
}
rec, err = ir.cur.Next()
}
return rec, err
}
func newIterativePackedRecordReader(
paths [][]string,
schema *schemapb.CollectionSchema,
bufferSize int64,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) *IterativeRecordReader {
chunk := 0
return &IterativeRecordReader{
iterate: func() (RecordReader, error) {
if chunk >= len(paths) {
return nil, io.EOF
}
currentPaths := paths[chunk]
chunk++
return newPackedRecordReader(currentPaths, schema, bufferSize, storageConfig, storagePluginContext)
},
}
}
type ManifestReader struct {
fieldBinlogs []*datapb.FieldBinlog
manifest string
reader *packed.FFIPackedReader
bufferSize int64
arrowSchema *arrow.Schema
schema *schemapb.CollectionSchema
schemaHelper *typeutil.SchemaHelper
field2Col map[FieldID]int
storageConfig *indexpb.StorageConfig
storagePluginContext *indexcgopb.StoragePluginContext
neededColumns []string
}
// NewManifestReaderFromBinlogs creates a ManifestReader from binlogs
func NewManifestReaderFromBinlogs(fieldBinlogs []*datapb.FieldBinlog,
schema *schemapb.CollectionSchema,
bufferSize int64,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (*ManifestReader, error) {
arrowSchema, err := ConvertToArrowSchema(schema, false)
if err != nil {
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
}
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
if err != nil {
return nil, err
}
field2Col := make(map[FieldID]int)
allFields := typeutil.GetAllFieldSchemas(schema)
neededColumns := make([]string, 0, len(allFields))
for i, field := range allFields {
field2Col[field.FieldID] = i
neededColumns = append(neededColumns, field.Name)
}
prr := &ManifestReader{
fieldBinlogs: fieldBinlogs,
bufferSize: bufferSize,
arrowSchema: arrowSchema,
schema: schema,
schemaHelper: schemaHelper,
field2Col: field2Col,
storageConfig: storageConfig,
storagePluginContext: storagePluginContext,
neededColumns: neededColumns,
}
err = prr.init()
if err != nil {
return nil, err
}
return prr, nil
}
func NewManifestReader(manifest string,
schema *schemapb.CollectionSchema,
bufferSize int64,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (*ManifestReader, error) {
arrowSchema, err := ConvertToArrowSchema(schema, true)
if err != nil {
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
}
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
if err != nil {
return nil, err
}
field2Col := make(map[FieldID]int)
allFields := typeutil.GetAllFieldSchemas(schema)
neededColumns := make([]string, 0, len(allFields))
for i, field := range allFields {
field2Col[field.FieldID] = i
// Use field id here
neededColumns = append(neededColumns, strconv.FormatInt(field.FieldID, 10))
}
prr := &ManifestReader{
manifest: manifest,
bufferSize: bufferSize,
arrowSchema: arrowSchema,
schema: schema,
schemaHelper: schemaHelper,
field2Col: field2Col,
storageConfig: storageConfig,
storagePluginContext: storagePluginContext,
neededColumns: neededColumns,
}
err = prr.init()
if err != nil {
return nil, err
}
return prr, nil
}
func (mr *ManifestReader) init() error {
// TODO add needed column option
manifest, err := packed.GetManifest(mr.manifest, mr.storageConfig)
if err != nil {
return err
}
reader, err := packed.NewFFIPackedReader(manifest, mr.arrowSchema, mr.neededColumns, mr.bufferSize, mr.storageConfig, mr.storagePluginContext)
if err != nil {
return err
}
mr.reader = reader
return nil
}
func (mr ManifestReader) Next() (Record, error) {
rec, err := mr.reader.ReadNext()
if err != nil {
return nil, err
}
return NewSimpleArrowRecord(rec, mr.field2Col), nil
}
func (mr ManifestReader) Close() error {
if mr.reader != nil {
return mr.reader.Close()
}
return nil
}
// ChunkedBlobsReader returns a chunk composed of blobs, or io.EOF if no more data
type ChunkedBlobsReader func() ([]*Blob, error)
type CompositeBinlogRecordReader struct {
fields map[FieldID]*schemapb.FieldSchema
index map[FieldID]int16
brs []*BinlogReader
rrs []array.RecordReader
}
var _ RecordReader = (*CompositeBinlogRecordReader)(nil)
func (crr *CompositeBinlogRecordReader) Next() (Record, error) {
recs := make([]arrow.Array, len(crr.fields))
nonExistingFields := make([]*schemapb.FieldSchema, 0)
nRows := 0
for _, f := range crr.fields {
idx := crr.index[f.FieldID]
if crr.rrs[idx] != nil {
if ok := crr.rrs[idx].Next(); !ok {
return nil, io.EOF
}
r := crr.rrs[idx].Record()
recs[idx] = r.Column(0)
if nRows == 0 {
nRows = int(r.NumRows())
}
if nRows != int(r.NumRows()) {
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("number of rows mismatch for field %d", f.FieldID))
}
} else {
nonExistingFields = append(nonExistingFields, f)
}
}
for _, f := range nonExistingFields {
// If the field is not in the current batch, fill with null array
arr, err := GenerateEmptyArrayFromSchema(f, nRows)
if err != nil {
return nil, err
}
recs[crr.index[f.FieldID]] = arr
}
return &compositeRecord{
index: crr.index,
recs: recs,
}, nil
}
func (crr *CompositeBinlogRecordReader) Close() error {
if crr.brs != nil {
for _, er := range crr.brs {
if er != nil {
er.Close()
}
}
}
if crr.rrs != nil {
for _, rr := range crr.rrs {
if rr != nil {
rr.Release()
}
}
}
return nil
}

View File

@ -0,0 +1,370 @@
// 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.
package storage
import (
"fmt"
"path"
"strconv"
"time"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/samber/lo"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/internal/storagev2/packed"
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
var _ RecordWriter = (*packedRecordWriter)(nil)
type packedRecordWriter struct {
writer *packed.PackedWriter
bufferSize int64
columnGroups []storagecommon.ColumnGroup
bucketName string
pathsMap map[typeutil.UniqueID]string
schema *schemapb.CollectionSchema
arrowSchema *arrow.Schema
rowNum int64
writtenUncompressed uint64
columnGroupUncompressed map[typeutil.UniqueID]uint64
columnGroupCompressed map[typeutil.UniqueID]uint64
outputManifest string
storageConfig *indexpb.StorageConfig
}
func (pw *packedRecordWriter) Write(r Record) error {
var rec arrow.Record
sar, ok := r.(*simpleArrowRecord)
if !ok {
// 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()))
} else {
rec = sar.r
}
pw.rowNum += int64(r.Len())
for col, arr := range rec.Columns() {
// size := arr.Data().SizeInBytes()
size := calculateActualDataSize(arr)
pw.writtenUncompressed += size
for _, columnGroup := range pw.columnGroups {
if lo.Contains(columnGroup.Columns, col) {
pw.columnGroupUncompressed[columnGroup.GroupID] += size
break
}
}
}
defer rec.Release()
return pw.writer.WriteRecordBatch(rec)
}
func (pw *packedRecordWriter) GetWrittenUncompressed() uint64 {
return pw.writtenUncompressed
}
func (pw *packedRecordWriter) GetColumnGroupWrittenUncompressed(columnGroup typeutil.UniqueID) uint64 {
if size, ok := pw.columnGroupUncompressed[columnGroup]; ok {
return size
}
return 0
}
func (pw *packedRecordWriter) GetColumnGroupWrittenCompressed(columnGroup typeutil.UniqueID) uint64 {
if size, ok := pw.columnGroupCompressed[columnGroup]; ok {
return size
}
return 0
}
func (pw *packedRecordWriter) GetWrittenPaths(columnGroup typeutil.UniqueID) string {
if path, ok := pw.pathsMap[columnGroup]; ok {
return path
}
return ""
}
func (pw *packedRecordWriter) GetWrittenManifest() string {
return pw.outputManifest
}
func (pw *packedRecordWriter) GetWrittenRowNum() int64 {
return pw.rowNum
}
func (pw *packedRecordWriter) Close() error {
if pw.writer != nil {
err := pw.writer.Close()
if err != nil {
return err
}
for id, fpath := range pw.pathsMap {
truePath := path.Join(pw.bucketName, fpath)
size, err := packed.GetFileSize(truePath, pw.storageConfig)
if err != nil {
return err
}
pw.columnGroupCompressed[id] = uint64(size)
}
}
return nil
}
func NewPackedRecordWriter(
bucketName string,
paths []string,
schema *schemapb.CollectionSchema,
bufferSize int64,
multiPartUploadSize int64,
columnGroups []storagecommon.ColumnGroup,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (*packedRecordWriter, error) {
// Validate PK field exists before proceeding
_, err := typeutil.GetPrimaryFieldSchema(schema)
if err != nil {
return nil, err
}
arrowSchema, err := ConvertToArrowSchema(schema, false)
if err != nil {
return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
}
// if storage config is not passed, use common config
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
if storageConfig != nil {
storageType = storageConfig.GetStorageType()
}
// compose true path before create packed writer here
// and returned writtenPaths shall remain untouched
truePaths := lo.Map(paths, func(p string, _ int) string {
if storageType == "local" {
return p
}
return path.Join(bucketName, p)
})
writer, err := packed.NewPackedWriter(truePaths, arrowSchema, bufferSize, multiPartUploadSize, columnGroups, storageConfig, storagePluginContext)
if err != nil {
return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not new packed record writer %s", err.Error()))
}
columnGroupUncompressed := make(map[typeutil.UniqueID]uint64)
columnGroupCompressed := make(map[typeutil.UniqueID]uint64)
pathsMap := make(map[typeutil.UniqueID]string)
if len(paths) != len(columnGroups) {
return nil, merr.WrapErrParameterInvalid(len(paths), len(columnGroups),
"paths length is not equal to column groups length for packed record writer")
}
for i, columnGroup := range columnGroups {
columnGroupUncompressed[columnGroup.GroupID] = 0
columnGroupCompressed[columnGroup.GroupID] = 0
pathsMap[columnGroup.GroupID] = paths[i]
}
return &packedRecordWriter{
writer: writer,
schema: schema,
arrowSchema: arrowSchema,
bufferSize: bufferSize,
bucketName: bucketName,
pathsMap: pathsMap,
columnGroups: columnGroups,
columnGroupUncompressed: columnGroupUncompressed,
columnGroupCompressed: columnGroupCompressed,
storageConfig: storageConfig,
}, nil
}
type packedRecordManifestWriter struct {
writer *packed.FFIPackedWriter
bufferSize int64
columnGroups []storagecommon.ColumnGroup
bucketName string
pathsMap map[typeutil.UniqueID]string
schema *schemapb.CollectionSchema
arrowSchema *arrow.Schema
rowNum int64
writtenUncompressed uint64
columnGroupUncompressed map[typeutil.UniqueID]uint64
columnGroupCompressed map[typeutil.UniqueID]uint64
outputManifest string
storageConfig *indexpb.StorageConfig
}
func (pw *packedRecordManifestWriter) Write(r Record) error {
var rec arrow.Record
sar, ok := r.(*simpleArrowRecord)
if !ok {
// 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()))
} else {
rec = sar.r
}
pw.rowNum += int64(r.Len())
for col, arr := range rec.Columns() {
// size := arr.Data().SizeInBytes()
size := calculateActualDataSize(arr)
pw.writtenUncompressed += size
for _, columnGroup := range pw.columnGroups {
if lo.Contains(columnGroup.Columns, col) {
pw.columnGroupUncompressed[columnGroup.GroupID] += size
break
}
}
}
defer rec.Release()
return pw.writer.WriteRecordBatch(rec)
}
func (pw *packedRecordManifestWriter) GetWrittenUncompressed() uint64 {
return pw.writtenUncompressed
}
func (pw *packedRecordManifestWriter) GetColumnGroupWrittenUncompressed(columnGroup typeutil.UniqueID) uint64 {
if size, ok := pw.columnGroupUncompressed[columnGroup]; ok {
return size
}
return 0
}
func (pw *packedRecordManifestWriter) GetColumnGroupWrittenCompressed(columnGroup typeutil.UniqueID) uint64 {
if size, ok := pw.columnGroupCompressed[columnGroup]; ok {
return size
}
return 0
}
func (pw *packedRecordManifestWriter) GetWrittenPaths(columnGroup typeutil.UniqueID) string {
if path, ok := pw.pathsMap[columnGroup]; ok {
return path
}
return ""
}
func (pw *packedRecordManifestWriter) GetWrittenManifest() string {
return pw.outputManifest
}
func (pw *packedRecordManifestWriter) GetWrittenRowNum() int64 {
return pw.rowNum
}
func (pw *packedRecordManifestWriter) Close() error {
if pw.writer != nil {
manifest, err := pw.writer.Close()
if err != nil {
return err
}
pw.outputManifest = manifest
for id := range pw.pathsMap {
pw.columnGroupCompressed[id] = uint64(0)
}
}
return nil
}
func NewPackedRecordManifestWriter(
bucketName string,
basePath string,
schema *schemapb.CollectionSchema,
bufferSize int64,
multiPartUploadSize int64,
columnGroups []storagecommon.ColumnGroup,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (*packedRecordManifestWriter, error) {
// Validate PK field exists before proceeding
_, err := typeutil.GetPrimaryFieldSchema(schema)
if err != nil {
return nil, err
}
arrowSchema, err := ConvertToArrowSchema(schema, true)
if err != nil {
return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
}
// if storage config is not passed, use common config
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
if storageConfig != nil {
storageType = storageConfig.GetStorageType()
}
ffiBasePath := basePath
if storageType != "local" {
ffiBasePath = path.Join(bucketName, basePath)
}
writer, err := packed.NewFFIPackedWriter(ffiBasePath, arrowSchema, columnGroups, storageConfig, storagePluginContext)
if err != nil {
return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not new packed record writer %s", err.Error()))
}
columnGroupUncompressed := make(map[typeutil.UniqueID]uint64)
columnGroupCompressed := make(map[typeutil.UniqueID]uint64)
// provide mock path
pathsMap := make(map[typeutil.UniqueID]string)
start := time.Now().UnixNano()
for _, columnGroup := range columnGroups {
columnGroupUncompressed[columnGroup.GroupID] = 0
columnGroupCompressed[columnGroup.GroupID] = 0
start++
pathsMap[columnGroup.GroupID] = path.Join(basePath, strconv.FormatInt(columnGroup.GroupID, 10), strconv.FormatInt(start, 10))
}
return &packedRecordManifestWriter{
writer: writer,
schema: schema,
arrowSchema: arrowSchema,
bufferSize: bufferSize,
bucketName: bucketName,
pathsMap: pathsMap,
columnGroups: columnGroups,
columnGroupUncompressed: columnGroupUncompressed,
columnGroupCompressed: columnGroupCompressed,
storageConfig: storageConfig,
}, nil
}
// Deprecated, todo remove
func NewPackedSerializeWriter(bucketName string, paths []string, schema *schemapb.CollectionSchema, bufferSize int64,
multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup, batchSize int,
) (*SerializeWriterImpl[*Value], error) {
packedRecordWriter, err := NewPackedRecordWriter(bucketName, paths, schema, bufferSize, multiPartUploadSize, columnGroups, nil, nil)
if err != nil {
return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not new packed record writer %s", err.Error()))
}
return NewSerializeRecordWriter(packedRecordWriter, func(v []*Value) (Record, error) {
return ValueSerializer(v, schema)
}, batchSize), nil
}

View File

@ -68,12 +68,10 @@ type rwOptions struct {
collectionID int64
storageConfig *indexpb.StorageConfig
neededFields typeutil.Set[int64]
useLoonFFI bool
}
func (o *rwOptions) validate() error {
if o.storageConfig == nil {
return merr.WrapErrServiceInternal("storage config is nil")
}
if o.collectionID == 0 {
log.Warn("storage config collection id is empty when init BinlogReader")
// return merr.WrapErrServiceInternal("storage config collection id is empty")
@ -87,6 +85,9 @@ func (o *rwOptions) validate() error {
return merr.WrapErrServiceInternal("downloader is nil for v1 reader")
}
case StorageV2:
if o.storageConfig == nil {
return merr.WrapErrServiceInternal("storage config is nil")
}
default:
return merr.WrapErrServiceInternal(fmt.Sprintf("unsupported storage version %d", o.version))
}
@ -164,6 +165,12 @@ func WithNeededFields(neededFields typeutil.Set[int64]) RwOption {
}
}
func WithUseLoonFFI(useLoonFFI bool) RwOption {
return func(options *rwOptions) {
options.useLoonFFI = useLoonFFI
}
}
func makeBlobsReader(ctx context.Context, binlogs []*datapb.FieldBinlog, downloader downloaderFn) (ChunkedBlobsReader, error) {
if len(binlogs) == 0 {
return func() ([]*Blob, error) {
@ -267,7 +274,7 @@ func NewBinlogRecordReader(ctx context.Context, binlogs []*datapb.FieldBinlog, s
if err != nil {
return nil, err
}
rr, err = newCompositeBinlogRecordReader(schema, blobsReader, binlogReaderOpts...)
rr = newIterativeCompositeBinlogRecordReader(schema, rwOptions.neededFields, blobsReader, binlogReaderOpts...)
case StorageV2:
if len(binlogs) <= 0 {
return nil, sio.EOF
@ -275,6 +282,7 @@ func NewBinlogRecordReader(ctx context.Context, binlogs []*datapb.FieldBinlog, s
sort.Slice(binlogs, func(i, j int) bool {
return binlogs[i].GetFieldID() < binlogs[j].GetFieldID()
})
binlogLists := lo.Map(binlogs, func(fieldBinlog *datapb.FieldBinlog, _ int) []*datapb.Binlog {
return fieldBinlog.GetBinlogs()
})
@ -289,19 +297,42 @@ func NewBinlogRecordReader(ctx context.Context, binlogs []*datapb.FieldBinlog, s
paths[j] = append(paths[j], logPath)
}
}
rr, err = newPackedRecordReader(paths, schema, rwOptions.bufferSize, rwOptions.storageConfig, pluginContext)
// FIXME: add needed fields support
rr = newIterativePackedRecordReader(paths, schema, rwOptions.bufferSize, rwOptions.storageConfig, pluginContext)
default:
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("unsupported storage version %d", rwOptions.version))
}
if err != nil {
return nil, err
}
if rwOptions.neededFields != nil {
rr.SetNeededFields(rwOptions.neededFields)
}
return rr, nil
}
func NewManifestRecordReader(ctx context.Context, manifestPath string, schema *schemapb.CollectionSchema, option ...RwOption) (rr RecordReader, err error) {
rwOptions := DefaultReaderOptions()
for _, opt := range option {
opt(rwOptions)
}
if err := rwOptions.validate(); err != nil {
return nil, err
}
var pluginContext *indexcgopb.StoragePluginContext
if hookutil.IsClusterEncyptionEnabled() {
if ez := hookutil.GetEzByCollProperties(schema.GetProperties(), rwOptions.collectionID); ez != nil {
unsafe := hookutil.GetCipher().GetUnsafeKey(ez.EzID, ez.CollectionID)
if len(unsafe) > 0 {
pluginContext = &indexcgopb.StoragePluginContext{
EncryptionZoneId: ez.EzID,
CollectionId: ez.CollectionID,
EncryptionKey: string(unsafe),
}
}
}
}
return NewRecordReaderFromManifest(manifestPath, schema, rwOptions.bufferSize, rwOptions.storageConfig, pluginContext)
}
func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segmentID UniqueID,
schema *schemapb.CollectionSchema, allocator allocator.Interface, chunkSize uint64, maxRowNum int64,
option ...RwOption,
@ -353,6 +384,13 @@ func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segme
blobsWriter, allocator, chunkSize, rootPath, maxRowNum, opts...,
)
case StorageV2:
if rwOptions.useLoonFFI {
return newPackedManifestRecordWriter(collectionID, partitionID, segmentID, schema,
blobsWriter, allocator, maxRowNum,
rwOptions.bufferSize, rwOptions.multiPartUploadSize, rwOptions.columnGroups,
rwOptions.storageConfig,
pluginContext)
} else {
return newPackedBinlogRecordWriter(collectionID, partitionID, segmentID, schema,
blobsWriter, allocator, maxRowNum,
rwOptions.bufferSize, rwOptions.multiPartUploadSize, rwOptions.columnGroups,
@ -360,5 +398,39 @@ func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segme
pluginContext,
)
}
}
return nil, merr.WrapErrServiceInternal(fmt.Sprintf("unsupported storage version %d", rwOptions.version))
}
func NewDeltalogWriter(
ctx context.Context,
collectionID, partitionID, segmentID, logID UniqueID,
pkType schemapb.DataType,
path string,
option ...RwOption,
) (RecordWriter, error) {
rwOptions := DefaultWriterOptions()
for _, opt := range option {
opt(rwOptions)
}
if err := rwOptions.validate(); err != nil {
return nil, err
}
return NewLegacyDeltalogWriter(collectionID, partitionID, segmentID, logID, pkType, rwOptions.uploader, path)
}
func NewDeltalogReader(
pkField *schemapb.FieldSchema,
paths []string,
option ...RwOption,
) (RecordReader, error) {
rwOptions := DefaultReaderOptions()
for _, opt := range option {
opt(rwOptions)
}
if err := rwOptions.validate(); err != nil {
return nil, err
}
return NewLegacyDeltalogReader(pkField, rwOptions.downloader, paths)
}

View File

@ -96,26 +96,32 @@ func (s *PackedBinlogRecordSuite) TestPackedBinlogRecordIntegration() {
{
GroupID: 0,
Columns: []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12},
Fields: []int64{0, 1, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 101},
},
{
GroupID: 102,
Columns: []int{13},
Fields: []int64{102},
},
{
GroupID: 103,
Columns: []int{14},
Fields: []int64{103},
},
{
GroupID: 104,
Columns: []int{15},
Fields: []int64{104},
},
{
GroupID: 105,
Columns: []int{16},
Fields: []int64{105},
},
{
GroupID: 106,
Columns: []int{17},
Fields: []int64{106},
},
}
wOption := []RwOption{
@ -155,7 +161,7 @@ func (s *PackedBinlogRecordSuite) TestPackedBinlogRecordIntegration() {
rowNum := w.GetRowNum()
s.Equal(rowNum, int64(rows))
fieldBinlogs, statsLog, bm25StatsLog := w.GetLogs()
fieldBinlogs, statsLog, bm25StatsLog, _ := w.GetLogs()
s.Equal(len(fieldBinlogs), len(columnGroups))
for _, columnGroup := range fieldBinlogs {
s.Equal(len(columnGroup.Binlogs), 1)
@ -234,7 +240,7 @@ func (s *PackedBinlogRecordSuite) TestGenerateBM25Stats() {
s.NoError(err)
err = w.Close()
s.NoError(err)
fieldBinlogs, statsLog, bm25StatsLog := w.GetLogs()
fieldBinlogs, statsLog, bm25StatsLog, _ := w.GetLogs()
s.Equal(len(fieldBinlogs), len(columnGroups))
s.Equal(statsLog.Binlogs[0].EntriesNum, int64(1))

View File

@ -1,6 +1,7 @@
package storage
import (
"fmt"
"strconv"
"github.com/apache/arrow/go/v17/arrow"
@ -11,7 +12,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
func ConvertToArrowSchema(schema *schemapb.CollectionSchema) (*arrow.Schema, error) {
func ConvertToArrowSchema(schema *schemapb.CollectionSchema, useFieldID bool) (*arrow.Schema, error) {
fieldCount := typeutil.GetTotalFieldsNum(schema)
arrowFields := make([]arrow.Field, 0, fieldCount)
appendArrowField := func(field *schemapb.FieldSchema) error {
@ -37,7 +38,7 @@ func ConvertToArrowSchema(schema *schemapb.CollectionSchema) (*arrow.Schema, err
}
arrowType := serdeMap[field.DataType].arrowType(dim, elementType)
arrowField := ConvertToArrowField(field, arrowType)
arrowField := ConvertToArrowField(field, arrowType, useFieldID)
// Add extra metadata for ArrayOfVector
if field.DataType == schemapb.DataType_ArrayOfVector {
@ -67,11 +68,16 @@ func ConvertToArrowSchema(schema *schemapb.CollectionSchema) (*arrow.Schema, err
return arrow.NewSchema(arrowFields, nil), nil
}
func ConvertToArrowField(field *schemapb.FieldSchema, dataType arrow.DataType) arrow.Field {
return arrow.Field{
Name: field.GetName(),
func ConvertToArrowField(field *schemapb.FieldSchema, dataType arrow.DataType, useFieldID bool) arrow.Field {
f := arrow.Field{
Type: dataType,
Metadata: arrow.NewMetadata([]string{packed.ArrowFieldIdMetadataKey}, []string{strconv.Itoa(int(field.GetFieldID()))}),
Nullable: field.GetNullable(),
}
if useFieldID { // use fieldID as name when specified
f.Name = fmt.Sprintf("%d", field.GetFieldID())
} else {
f.Name = field.GetName()
}
return f
}

View File

@ -56,7 +56,7 @@ func TestConvertArrowSchema(t *testing.T) {
Fields: fieldSchemas,
StructArrayFields: StructArrayFieldSchemas,
}
arrowSchema, err := ConvertToArrowSchema(schema)
arrowSchema, err := ConvertToArrowSchema(schema, false)
assert.NoError(t, err)
assert.Equal(t, len(fieldSchemas)+len(StructArrayFieldSchemas[0].Fields), len(arrowSchema.Fields()))
}
@ -84,6 +84,6 @@ func TestConvertArrowSchemaWithoutDim(t *testing.T) {
schema := &schemapb.CollectionSchema{
Fields: fieldSchemas,
}
_, err := ConvertToArrowSchema(schema)
_, err := ConvertToArrowSchema(schema, false)
assert.Error(t, err)
}

View File

@ -45,12 +45,6 @@ type Record interface {
Retain()
}
type RecordReader interface {
Next() (Record, error)
SetNeededFields(fields typeutil.Set[int64])
Close() error
}
type RecordWriter interface {
Write(r Record) error
GetWrittenUncompressed() uint64

View File

@ -0,0 +1,627 @@
// 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.
package storage
import (
"bytes"
"context"
"encoding/binary"
"encoding/json"
"fmt"
"io"
"strconv"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/apache/arrow/go/v17/arrow/memory"
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
// newDeltalogOneFieldReader creates a reader for the old single-field deltalog format
func newDeltalogOneFieldReader(blobs []*Blob) (*DeserializeReaderImpl[*DeleteLog], error) {
reader := newIterativeCompositeBinlogRecordReader(
&schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
DataType: schemapb.DataType_VarChar,
},
},
},
nil,
MakeBlobsReader(blobs))
return NewDeserializeReader(reader, func(r Record, v []*DeleteLog) error {
for i := 0; i < r.Len(); i++ {
if v[i] == nil {
v[i] = &DeleteLog{}
}
// retrieve the only field
a := r.(*compositeRecord).recs[0].(*array.String)
strVal := a.Value(i)
if err := v[i].Parse(strVal); err != nil {
return err
}
}
return nil
}), nil
}
// DeltalogStreamWriter writes deltalog in the old JSON format
type DeltalogStreamWriter struct {
collectionID UniqueID
partitionID UniqueID
segmentID UniqueID
fieldSchema *schemapb.FieldSchema
buf bytes.Buffer
rw *singleFieldRecordWriter
}
func (dsw *DeltalogStreamWriter) GetRecordWriter() (RecordWriter, error) {
if dsw.rw != nil {
return dsw.rw, nil
}
rw, err := newSingleFieldRecordWriter(dsw.fieldSchema, &dsw.buf, WithRecordWriterProps(getFieldWriterProps(dsw.fieldSchema)))
if err != nil {
return nil, err
}
dsw.rw = rw
return rw, nil
}
func (dsw *DeltalogStreamWriter) Finalize() (*Blob, error) {
if dsw.rw == nil {
return nil, io.ErrUnexpectedEOF
}
dsw.rw.Close()
var b bytes.Buffer
if err := dsw.writeDeltalogHeaders(&b); err != nil {
return nil, err
}
if _, err := b.Write(dsw.buf.Bytes()); err != nil {
return nil, err
}
return &Blob{
Value: b.Bytes(),
RowNum: int64(dsw.rw.numRows),
MemorySize: int64(dsw.rw.writtenUncompressed),
}, nil
}
func (dsw *DeltalogStreamWriter) writeDeltalogHeaders(w io.Writer) error {
// Write magic number
if err := binary.Write(w, common.Endian, MagicNumber); err != nil {
return err
}
// Write descriptor
de := NewBaseDescriptorEvent(dsw.collectionID, dsw.partitionID, dsw.segmentID)
de.PayloadDataType = dsw.fieldSchema.DataType
de.descriptorEventData.AddExtra(originalSizeKey, strconv.Itoa(int(dsw.rw.writtenUncompressed)))
if err := de.Write(w); err != nil {
return err
}
// Write event header
eh := newEventHeader(DeleteEventType)
// Write event data
ev := newDeleteEventData()
ev.StartTimestamp = 1
ev.EndTimestamp = 1
eh.EventLength = int32(dsw.buf.Len()) + eh.GetMemoryUsageInBytes() + int32(binary.Size(ev))
// eh.NextPosition = eh.EventLength + w.Offset()
if err := eh.Write(w); err != nil {
return err
}
if err := ev.WriteEventData(w); err != nil {
return err
}
return nil
}
func newDeltalogStreamWriter(collectionID, partitionID, segmentID UniqueID) *DeltalogStreamWriter {
return &DeltalogStreamWriter{
collectionID: collectionID,
partitionID: partitionID,
segmentID: segmentID,
fieldSchema: &schemapb.FieldSchema{
FieldID: common.RowIDField,
Name: "delta",
DataType: schemapb.DataType_String,
},
}
}
func newDeltalogSerializeWriter(eventWriter *DeltalogStreamWriter, batchSize int) (*SerializeWriterImpl[*DeleteLog], error) {
rws := make(map[FieldID]RecordWriter, 1)
rw, err := eventWriter.GetRecordWriter()
if err != nil {
return nil, err
}
rws[0] = rw
compositeRecordWriter := NewCompositeRecordWriter(rws)
return NewSerializeRecordWriter(compositeRecordWriter, func(v []*DeleteLog) (Record, error) {
builder := array.NewBuilder(memory.DefaultAllocator, arrow.BinaryTypes.String)
for _, vv := range v {
strVal, err := json.Marshal(vv)
if err != nil {
return nil, err
}
builder.AppendValueFromString(string(strVal))
}
arr := []arrow.Array{builder.NewArray()}
field := []arrow.Field{{
Name: "delta",
Type: arrow.BinaryTypes.String,
Nullable: false,
}}
field2Col := map[FieldID]int{
0: 0,
}
return NewSimpleArrowRecord(array.NewRecord(arrow.NewSchema(field, nil), arr, int64(len(v))), field2Col), nil
}, batchSize), nil
}
var _ RecordReader = (*simpleArrowRecordReader)(nil)
// simpleArrowRecordReader reads simple arrow records from blobs
type simpleArrowRecordReader struct {
blobs []*Blob
blobPos int
rr array.RecordReader
closer func()
r simpleArrowRecord
}
func (crr *simpleArrowRecordReader) iterateNextBatch() error {
if crr.closer != nil {
crr.closer()
}
crr.blobPos++
if crr.blobPos >= len(crr.blobs) {
return io.EOF
}
reader, err := NewBinlogReader(crr.blobs[crr.blobPos].Value)
if err != nil {
return err
}
er, err := reader.NextEventReader()
if err != nil {
return err
}
rr, err := er.GetArrowRecordReader()
if err != nil {
return err
}
crr.rr = rr
crr.closer = func() {
crr.rr.Release()
er.Close()
reader.Close()
}
return nil
}
func (crr *simpleArrowRecordReader) Next() (Record, error) {
if crr.rr == nil {
if len(crr.blobs) == 0 {
return nil, io.EOF
}
crr.blobPos = -1
crr.r = simpleArrowRecord{
field2Col: make(map[FieldID]int),
}
if err := crr.iterateNextBatch(); err != nil {
return nil, err
}
}
composeRecord := func() bool {
if ok := crr.rr.Next(); !ok {
return false
}
record := crr.rr.Record()
for i := range record.Schema().Fields() {
crr.r.field2Col[FieldID(i)] = i
}
crr.r.r = record
return true
}
if ok := composeRecord(); !ok {
if err := crr.iterateNextBatch(); err != nil {
return nil, err
}
if ok := composeRecord(); !ok {
return nil, io.EOF
}
}
return &crr.r, nil
}
func (crr *simpleArrowRecordReader) SetNeededFields(_ typeutil.Set[int64]) {
// no-op for simple arrow record reader
}
func (crr *simpleArrowRecordReader) Close() error {
if crr.closer != nil {
crr.closer()
}
return nil
}
func newSimpleArrowRecordReader(blobs []*Blob) (*simpleArrowRecordReader, error) {
return &simpleArrowRecordReader{
blobs: blobs,
}, nil
}
// MultiFieldDeltalogStreamWriter writes deltalog in the new multi-field parquet format
type MultiFieldDeltalogStreamWriter struct {
collectionID UniqueID
partitionID UniqueID
segmentID UniqueID
pkType schemapb.DataType
buf bytes.Buffer
rw *multiFieldRecordWriter
}
func newMultiFieldDeltalogStreamWriter(collectionID, partitionID, segmentID UniqueID, pkType schemapb.DataType) *MultiFieldDeltalogStreamWriter {
return &MultiFieldDeltalogStreamWriter{
collectionID: collectionID,
partitionID: partitionID,
segmentID: segmentID,
pkType: pkType,
}
}
func (dsw *MultiFieldDeltalogStreamWriter) GetRecordWriter() (RecordWriter, error) {
if dsw.rw != nil {
return dsw.rw, nil
}
fieldIDs := []FieldID{common.RowIDField, common.TimeStampField} // Not used.
fields := []arrow.Field{
{
Name: "pk",
Type: serdeMap[dsw.pkType].arrowType(0, schemapb.DataType_None),
Nullable: false,
},
{
Name: "ts",
Type: arrow.PrimitiveTypes.Int64,
Nullable: false,
},
}
rw, err := newMultiFieldRecordWriter(fieldIDs, fields, &dsw.buf)
if err != nil {
return nil, err
}
dsw.rw = rw
return rw, nil
}
func (dsw *MultiFieldDeltalogStreamWriter) Finalize() (*Blob, error) {
if dsw.rw == nil {
return nil, io.ErrUnexpectedEOF
}
dsw.rw.Close()
var b bytes.Buffer
if err := dsw.writeDeltalogHeaders(&b); err != nil {
return nil, err
}
if _, err := b.Write(dsw.buf.Bytes()); err != nil {
return nil, err
}
return &Blob{
Value: b.Bytes(),
RowNum: int64(dsw.rw.numRows),
MemorySize: int64(dsw.rw.writtenUncompressed),
}, nil
}
func (dsw *MultiFieldDeltalogStreamWriter) writeDeltalogHeaders(w io.Writer) error {
// Write magic number
if err := binary.Write(w, common.Endian, MagicNumber); err != nil {
return err
}
// Write descriptor
de := NewBaseDescriptorEvent(dsw.collectionID, dsw.partitionID, dsw.segmentID)
de.PayloadDataType = schemapb.DataType_Int64
de.descriptorEventData.AddExtra(originalSizeKey, strconv.Itoa(int(dsw.rw.writtenUncompressed)))
de.descriptorEventData.AddExtra(version, MultiField)
if err := de.Write(w); err != nil {
return err
}
// Write event header
eh := newEventHeader(DeleteEventType)
// Write event data
ev := newDeleteEventData()
ev.StartTimestamp = 1
ev.EndTimestamp = 1
eh.EventLength = int32(dsw.buf.Len()) + eh.GetMemoryUsageInBytes() + int32(binary.Size(ev))
// eh.NextPosition = eh.EventLength + w.Offset()
if err := eh.Write(w); err != nil {
return err
}
if err := ev.WriteEventData(w); err != nil {
return err
}
return nil
}
func newDeltalogMultiFieldWriter(eventWriter *MultiFieldDeltalogStreamWriter, batchSize int) (*SerializeWriterImpl[*DeleteLog], error) {
rw, err := eventWriter.GetRecordWriter()
if err != nil {
return nil, err
}
return NewSerializeRecordWriter[*DeleteLog](rw, func(v []*DeleteLog) (Record, error) {
fields := []arrow.Field{
{
Name: "pk",
Type: serdeMap[schemapb.DataType(v[0].PkType)].arrowType(0, schemapb.DataType_None),
Nullable: false,
},
{
Name: "ts",
Type: arrow.PrimitiveTypes.Int64,
Nullable: false,
},
}
arrowSchema := arrow.NewSchema(fields, nil)
builder := array.NewRecordBuilder(memory.DefaultAllocator, arrowSchema)
defer builder.Release()
pkType := schemapb.DataType(v[0].PkType)
switch pkType {
case schemapb.DataType_Int64:
pb := builder.Field(0).(*array.Int64Builder)
for _, vv := range v {
pk := vv.Pk.GetValue().(int64)
pb.Append(pk)
}
case schemapb.DataType_VarChar:
pb := builder.Field(0).(*array.StringBuilder)
for _, vv := range v {
pk := vv.Pk.GetValue().(string)
pb.Append(pk)
}
default:
return nil, fmt.Errorf("unexpected pk type %v", v[0].PkType)
}
for _, vv := range v {
builder.Field(1).(*array.Int64Builder).Append(int64(vv.Ts))
}
arr := []arrow.Array{builder.Field(0).NewArray(), builder.Field(1).NewArray()}
field2Col := map[FieldID]int{
common.RowIDField: 0,
common.TimeStampField: 1,
}
return NewSimpleArrowRecord(array.NewRecord(arrowSchema, arr, int64(len(v))), field2Col), nil
}, batchSize), nil
}
func newDeltalogMultiFieldReader(blobs []*Blob) (*DeserializeReaderImpl[*DeleteLog], error) {
reader, err := newSimpleArrowRecordReader(blobs)
if err != nil {
return nil, err
}
return NewDeserializeReader(reader, func(r Record, v []*DeleteLog) error {
rec, ok := r.(*simpleArrowRecord)
if !ok {
return errors.New("can not cast to simple arrow record")
}
fields := rec.r.Schema().Fields()
switch fields[0].Type.ID() {
case arrow.INT64:
arr := r.Column(0).(*array.Int64)
for j := 0; j < r.Len(); j++ {
if v[j] == nil {
v[j] = &DeleteLog{}
}
v[j].Pk = NewInt64PrimaryKey(arr.Value(j))
}
case arrow.STRING:
arr := r.Column(0).(*array.String)
for j := 0; j < r.Len(); j++ {
if v[j] == nil {
v[j] = &DeleteLog{}
}
v[j].Pk = NewVarCharPrimaryKey(arr.Value(j))
}
default:
return fmt.Errorf("unexpected delta log pkType %v", fields[0].Type.Name())
}
arr := r.Column(1).(*array.Int64)
for j := 0; j < r.Len(); j++ {
v[j].Ts = uint64(arr.Value(j))
}
return nil
}), nil
}
// newDeltalogDeserializeReader is the entry point for the delta log reader.
// It includes newDeltalogOneFieldReader, which uses the existing log format with only one column in a log file,
// and newDeltalogMultiFieldReader, which uses the new format and supports multiple fields in a log file.
func newDeltalogDeserializeReader(blobs []*Blob) (*DeserializeReaderImpl[*DeleteLog], error) {
if supportMultiFieldFormat(blobs) {
return newDeltalogMultiFieldReader(blobs)
}
return newDeltalogOneFieldReader(blobs)
}
// supportMultiFieldFormat checks delta log description data to see if it is the format with
// pk and ts column separately
func supportMultiFieldFormat(blobs []*Blob) bool {
if len(blobs) > 0 {
reader, err := NewBinlogReader(blobs[0].Value)
if err != nil {
return false
}
defer reader.Close()
version := reader.descriptorEventData.Extras[version]
return version != nil && version.(string) == MultiField
}
return false
}
// CreateDeltalogReader creates a deltalog reader based on the format version
func CreateDeltalogReader(blobs []*Blob) (*DeserializeReaderImpl[*DeleteLog], error) {
return newDeltalogDeserializeReader(blobs)
}
// createDeltalogWriter creates a deltalog writer based on the configured format
func createDeltalogWriter(collectionID, partitionID, segmentID UniqueID, pkType schemapb.DataType, batchSize int,
) (*SerializeWriterImpl[*DeleteLog], func() (*Blob, error), error) {
format := paramtable.Get().DataNodeCfg.DeltalogFormat.GetValue()
switch format {
case "json":
eventWriter := newDeltalogStreamWriter(collectionID, partitionID, segmentID)
writer, err := newDeltalogSerializeWriter(eventWriter, batchSize)
return writer, eventWriter.Finalize, err
case "parquet":
eventWriter := newMultiFieldDeltalogStreamWriter(collectionID, partitionID, segmentID, pkType)
writer, err := newDeltalogMultiFieldWriter(eventWriter, batchSize)
return writer, eventWriter.Finalize, err
default:
return nil, nil, merr.WrapErrParameterInvalid("unsupported deltalog format %s", format)
}
}
type LegacyDeltalogWriter struct {
path string
pkType schemapb.DataType
writer *SerializeWriterImpl[*DeleteLog]
finalizer func() (*Blob, error)
writtenUncompressed uint64
uploader uploaderFn
}
var _ RecordWriter = (*LegacyDeltalogWriter)(nil)
func NewLegacyDeltalogWriter(
collectionID, partitionID, segmentID, logID UniqueID, pkType schemapb.DataType, uploader uploaderFn, path string,
) (*LegacyDeltalogWriter, error) {
writer, finalizer, err := createDeltalogWriter(collectionID, partitionID, segmentID, pkType, 4096)
if err != nil {
return nil, err
}
return &LegacyDeltalogWriter{
path: path,
pkType: pkType,
writer: writer,
finalizer: finalizer,
uploader: uploader,
}, nil
}
func (w *LegacyDeltalogWriter) Write(rec Record) error {
newDeleteLog := func(i int) (*DeleteLog, error) {
ts := Timestamp(rec.Column(1).(*array.Int64).Value(i))
switch w.pkType {
case schemapb.DataType_Int64:
pk := NewInt64PrimaryKey(rec.Column(0).(*array.Int64).Value(i))
return NewDeleteLog(pk, ts), nil
case schemapb.DataType_VarChar:
pk := NewVarCharPrimaryKey(rec.Column(0).(*array.String).Value(i))
return NewDeleteLog(pk, ts), nil
default:
return nil, fmt.Errorf("unexpected pk type %v", w.pkType)
}
}
for i := range rec.Len() {
deleteLog, err := newDeleteLog(i)
if err != nil {
return err
}
err = w.writer.WriteValue(deleteLog)
if err != nil {
return err
}
}
w.writtenUncompressed += (rec.Column(0).Data().SizeInBytes() + rec.Column(1).Data().SizeInBytes())
return nil
}
func (w *LegacyDeltalogWriter) Close() error {
err := w.writer.Close()
if err != nil {
return err
}
blob, err := w.finalizer()
if err != nil {
return err
}
return w.uploader(context.Background(), map[string][]byte{blob.Key: blob.Value})
}
func (w *LegacyDeltalogWriter) GetWrittenUncompressed() uint64 {
return w.writtenUncompressed
}
func NewLegacyDeltalogReader(pkField *schemapb.FieldSchema, downloader downloaderFn, paths []string) (RecordReader, error) {
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
pkField,
{
FieldID: common.TimeStampField,
DataType: schemapb.DataType_Int64,
},
},
}
chunkPos := 0
blobsReader := func() ([]*Blob, error) {
path := paths[chunkPos]
chunkPos++
blobs, err := downloader(context.Background(), []string{path})
if err != nil {
return nil, err
}
return []*Blob{{Key: path, Value: blobs[0]}}, nil
}
return newIterativeCompositeBinlogRecordReader(
schema,
nil,
blobsReader,
nil,
), nil
}

View File

@ -0,0 +1,155 @@
// 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.
package storage
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
func TestDeltalogReaderWriter(t *testing.T) {
const (
testCollectionID = int64(1)
testPartitionID = int64(2)
testSegmentID = int64(3)
testBatchSize = 1024
testNumLogs = 100
)
type deleteLogGenerator func(i int) *DeleteLog
tests := []struct {
name string
format string
pkType schemapb.DataType
logGenerator deleteLogGenerator
wantErr bool
}{
{
name: "Int64 PK - JSON format",
format: "json",
pkType: schemapb.DataType_Int64,
logGenerator: func(i int) *DeleteLog {
return NewDeleteLog(NewInt64PrimaryKey(int64(i)), uint64(100+i))
},
wantErr: false,
},
{
name: "VarChar PK - JSON format",
format: "json",
pkType: schemapb.DataType_VarChar,
logGenerator: func(i int) *DeleteLog {
return NewDeleteLog(NewVarCharPrimaryKey("key_"+string(rune(i))), uint64(100+i))
},
wantErr: false,
},
{
name: "Int64 PK - Parquet format",
format: "parquet",
pkType: schemapb.DataType_Int64,
logGenerator: func(i int) *DeleteLog {
return NewDeleteLog(NewInt64PrimaryKey(int64(i)), uint64(100+i))
},
wantErr: false,
},
{
name: "VarChar PK - Parquet format",
format: "parquet",
pkType: schemapb.DataType_VarChar,
logGenerator: func(i int) *DeleteLog {
return NewDeleteLog(NewVarCharPrimaryKey("key_"+string(rune(i))), uint64(100+i))
},
wantErr: false,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
// Set deltalog format
originalFormat := paramtable.Get().DataNodeCfg.DeltalogFormat.GetValue()
paramtable.Get().Save(paramtable.Get().DataNodeCfg.DeltalogFormat.Key, tt.format)
defer paramtable.Get().Save(paramtable.Get().DataNodeCfg.DeltalogFormat.Key, originalFormat)
writer, finalizer, err := createDeltalogWriter(testCollectionID, testPartitionID, testSegmentID, tt.pkType, testBatchSize)
if tt.wantErr {
assert.Error(t, err)
return
}
require.NoError(t, err)
assert.NotNil(t, writer)
assert.NotNil(t, finalizer)
// Write delete logs
expectedLogs := make([]*DeleteLog, 0, testNumLogs)
for i := 0; i < testNumLogs; i++ {
deleteLog := tt.logGenerator(i)
expectedLogs = append(expectedLogs, deleteLog)
err = writer.WriteValue(deleteLog)
require.NoError(t, err)
}
err = writer.Close()
require.NoError(t, err)
blob, err := finalizer()
require.NoError(t, err)
assert.NotNil(t, blob)
assert.Greater(t, len(blob.Value), 0)
// Test round trip
reader, err := CreateDeltalogReader([]*Blob{blob})
require.NoError(t, err)
require.NotNil(t, reader)
// Read and verify contents
readLogs := make([]*DeleteLog, 0)
for {
log, err := reader.NextValue()
if err != nil {
break
}
if log != nil {
readLogs = append(readLogs, *log)
}
}
assert.Equal(t, len(expectedLogs), len(readLogs))
for i := 0; i < len(expectedLogs); i++ {
assert.Equal(t, expectedLogs[i].Ts, readLogs[i].Ts)
assert.Equal(t, expectedLogs[i].Pk.GetValue(), readLogs[i].Pk.GetValue())
}
err = reader.Close()
assert.NoError(t, err)
})
}
}
func TestDeltalogStreamWriter_NoRecordWriter(t *testing.T) {
writer := newDeltalogStreamWriter(1, 2, 3)
assert.NotNil(t, writer)
// Finalize without getting record writer should return error
blob, err := writer.Finalize()
assert.Error(t, err)
assert.Nil(t, blob)
}

File diff suppressed because it is too large Load Diff

View File

@ -177,7 +177,7 @@ func TestBinlogSerializeWriter(t *testing.T) {
err = writer.Close()
assert.NoError(t, err)
logs, _, _ := writer.GetLogs()
logs, _, _, _ := writer.GetLogs()
assert.Equal(t, 18, len(logs))
assert.Equal(t, 5, len(logs[0].Binlogs))
})

View File

@ -15,617 +15,3 @@
// limitations under the License.
package storage
import (
"fmt"
"io"
"path"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/internal/storagev2/packed"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/etcdpb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type packedRecordReader struct {
paths [][]string
chunk int
reader *packed.PackedReader
bufferSize int64
arrowSchema *arrow.Schema
field2Col map[FieldID]int
storageConfig *indexpb.StorageConfig
storagePluginContext *indexcgopb.StoragePluginContext
}
var _ RecordReader = (*packedRecordReader)(nil)
func (pr *packedRecordReader) iterateNextBatch() error {
if pr.reader != nil {
if err := pr.reader.Close(); err != nil {
return err
}
}
if pr.chunk >= len(pr.paths) {
return io.EOF
}
reader, err := packed.NewPackedReader(pr.paths[pr.chunk], pr.arrowSchema, pr.bufferSize, pr.storageConfig, pr.storagePluginContext)
pr.chunk++
if err != nil {
return errors.Newf("New binlog record packed reader error: %w", err)
}
pr.reader = reader
return nil
}
func (pr *packedRecordReader) Next() (Record, error) {
if pr.reader == nil {
if err := pr.iterateNextBatch(); err != nil {
return nil, err
}
}
for {
rec, err := pr.reader.ReadNext()
if err == io.EOF {
if err := pr.iterateNextBatch(); err != nil {
return nil, err
}
continue
} else if err != nil {
return nil, err
}
return NewSimpleArrowRecord(rec, pr.field2Col), nil
}
}
func (pr *packedRecordReader) SetNeededFields(fields typeutil.Set[int64]) {
// TODO, push down SetNeededFields to packedReader after implemented
// no-op for now
}
func (pr *packedRecordReader) Close() error {
if pr.reader != nil {
return pr.reader.Close()
}
return nil
}
func newPackedRecordReader(paths [][]string, schema *schemapb.CollectionSchema, bufferSize int64, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext,
) (*packedRecordReader, error) {
arrowSchema, err := ConvertToArrowSchema(schema)
if err != nil {
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
}
field2Col := make(map[FieldID]int)
allFields := typeutil.GetAllFieldSchemas(schema)
for i, field := range allFields {
field2Col[field.FieldID] = i
}
return &packedRecordReader{
paths: paths,
bufferSize: bufferSize,
arrowSchema: arrowSchema,
field2Col: field2Col,
storageConfig: storageConfig,
storagePluginContext: storagePluginContext,
}, nil
}
// Deprecated
func NewPackedDeserializeReader(paths [][]string, schema *schemapb.CollectionSchema,
bufferSize int64, shouldCopy bool,
) (*DeserializeReaderImpl[*Value], error) {
reader, err := newPackedRecordReader(paths, schema, bufferSize, nil, nil)
if err != nil {
return nil, err
}
return NewDeserializeReader(reader, func(r Record, v []*Value) error {
return ValueDeserializerWithSchema(r, v, schema, shouldCopy)
}), nil
}
var _ RecordWriter = (*packedRecordWriter)(nil)
type packedRecordWriter struct {
writer *packed.PackedWriter
bufferSize int64
columnGroups []storagecommon.ColumnGroup
bucketName string
pathsMap map[typeutil.UniqueID]string
schema *schemapb.CollectionSchema
arrowSchema *arrow.Schema
rowNum int64
writtenUncompressed uint64
columnGroupUncompressed map[typeutil.UniqueID]uint64
columnGroupCompressed map[typeutil.UniqueID]uint64
storageConfig *indexpb.StorageConfig
}
func (pw *packedRecordWriter) Write(r Record) error {
var rec arrow.Record
sar, ok := r.(*simpleArrowRecord)
if !ok {
// 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()))
} else {
rec = sar.r
}
pw.rowNum += int64(r.Len())
for col, arr := range rec.Columns() {
// size := arr.Data().SizeInBytes()
size := calculateActualDataSize(arr)
pw.writtenUncompressed += size
for _, columnGroup := range pw.columnGroups {
if lo.Contains(columnGroup.Columns, col) {
pw.columnGroupUncompressed[columnGroup.GroupID] += size
break
}
}
}
defer rec.Release()
return pw.writer.WriteRecordBatch(rec)
}
func (pw *packedRecordWriter) GetWrittenUncompressed() uint64 {
return pw.writtenUncompressed
}
func (pw *packedRecordWriter) GetColumnGroupWrittenUncompressed(columnGroup typeutil.UniqueID) uint64 {
if size, ok := pw.columnGroupUncompressed[columnGroup]; ok {
return size
}
return 0
}
func (pw *packedRecordWriter) GetColumnGroupWrittenCompressed(columnGroup typeutil.UniqueID) uint64 {
if size, ok := pw.columnGroupCompressed[columnGroup]; ok {
return size
}
return 0
}
func (pw *packedRecordWriter) GetWrittenPaths(columnGroup typeutil.UniqueID) string {
if path, ok := pw.pathsMap[columnGroup]; ok {
return path
}
return ""
}
func (pw *packedRecordWriter) GetWrittenRowNum() int64 {
return pw.rowNum
}
func (pw *packedRecordWriter) Close() error {
if pw.writer != nil {
err := pw.writer.Close()
if err != nil {
return err
}
for id, fpath := range pw.pathsMap {
truePath := path.Join(pw.bucketName, fpath)
size, err := packed.GetFileSize(truePath, pw.storageConfig)
if err != nil {
return err
}
pw.columnGroupCompressed[id] = uint64(size)
}
}
return nil
}
func NewPackedRecordWriter(bucketName string, paths []string, schema *schemapb.CollectionSchema, bufferSize int64, multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext) (*packedRecordWriter, error) {
arrowSchema, err := ConvertToArrowSchema(schema)
if err != nil {
return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
}
// if storage config is not passed, use common config
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
if storageConfig != nil {
storageType = storageConfig.GetStorageType()
}
// compose true path before create packed writer here
// and returned writtenPaths shall remain untouched
truePaths := lo.Map(paths, func(p string, _ int) string {
if storageType == "local" {
return p
}
return path.Join(bucketName, p)
})
writer, err := packed.NewPackedWriter(truePaths, arrowSchema, bufferSize, multiPartUploadSize, columnGroups, storageConfig, storagePluginContext)
if err != nil {
return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not new packed record writer %s", err.Error()))
}
columnGroupUncompressed := make(map[typeutil.UniqueID]uint64)
columnGroupCompressed := make(map[typeutil.UniqueID]uint64)
pathsMap := make(map[typeutil.UniqueID]string)
if len(paths) != len(columnGroups) {
return nil, merr.WrapErrParameterInvalid(len(paths), len(columnGroups),
"paths length is not equal to column groups length for packed record writer")
}
for i, columnGroup := range columnGroups {
columnGroupUncompressed[columnGroup.GroupID] = 0
columnGroupCompressed[columnGroup.GroupID] = 0
pathsMap[columnGroup.GroupID] = paths[i]
}
return &packedRecordWriter{
writer: writer,
schema: schema,
arrowSchema: arrowSchema,
bufferSize: bufferSize,
bucketName: bucketName,
pathsMap: pathsMap,
columnGroups: columnGroups,
columnGroupUncompressed: columnGroupUncompressed,
columnGroupCompressed: columnGroupCompressed,
storageConfig: storageConfig,
}, nil
}
// Deprecated, todo remove
func NewPackedSerializeWriter(bucketName string, paths []string, schema *schemapb.CollectionSchema, bufferSize int64,
multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup, batchSize int,
) (*SerializeWriterImpl[*Value], error) {
packedRecordWriter, err := NewPackedRecordWriter(bucketName, paths, schema, bufferSize, multiPartUploadSize, columnGroups, nil, nil)
if err != nil {
return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not new packed record writer %s", err.Error()))
}
return NewSerializeRecordWriter(packedRecordWriter, func(v []*Value) (Record, error) {
return ValueSerializer(v, schema)
}, batchSize), nil
}
var _ BinlogRecordWriter = (*PackedBinlogRecordWriter)(nil)
type PackedBinlogRecordWriter struct {
// attributes
collectionID UniqueID
partitionID UniqueID
segmentID UniqueID
schema *schemapb.CollectionSchema
BlobsWriter ChunkedBlobsWriter
allocator allocator.Interface
maxRowNum int64
arrowSchema *arrow.Schema
bufferSize int64
multiPartUploadSize int64
columnGroups []storagecommon.ColumnGroup
storageConfig *indexpb.StorageConfig
storagePluginContext *indexcgopb.StoragePluginContext
// writer and stats generated at runtime
writer *packedRecordWriter
pkstats *PrimaryKeyStats
bm25Stats map[int64]*BM25Stats
tsFrom typeutil.Timestamp
tsTo typeutil.Timestamp
rowNum int64
writtenUncompressed uint64
// results
fieldBinlogs map[FieldID]*datapb.FieldBinlog
statsLog *datapb.FieldBinlog
bm25StatsLog map[FieldID]*datapb.FieldBinlog
}
func (pw *PackedBinlogRecordWriter) Write(r Record) error {
if err := pw.initWriters(r); err != nil {
return err
}
tsArray := r.Column(common.TimeStampField).(*array.Int64)
rows := r.Len()
for i := 0; i < rows; i++ {
ts := typeutil.Timestamp(tsArray.Value(i))
if ts < pw.tsFrom {
pw.tsFrom = ts
}
if ts > pw.tsTo {
pw.tsTo = ts
}
switch schemapb.DataType(pw.pkstats.PkType) {
case schemapb.DataType_Int64:
pkArray := r.Column(pw.pkstats.FieldID).(*array.Int64)
pk := &Int64PrimaryKey{
Value: pkArray.Value(i),
}
pw.pkstats.Update(pk)
case schemapb.DataType_VarChar:
pkArray := r.Column(pw.pkstats.FieldID).(*array.String)
pk := &VarCharPrimaryKey{
Value: pkArray.Value(i),
}
pw.pkstats.Update(pk)
default:
panic("invalid data type")
}
for fieldID, stats := range pw.bm25Stats {
field, ok := r.Column(fieldID).(*array.Binary)
if !ok {
return errors.New("bm25 field value not found")
}
stats.AppendBytes(field.Value(i))
}
}
err := pw.writer.Write(r)
if err != nil {
return merr.WrapErrServiceInternal(fmt.Sprintf("write record batch error: %s", err.Error()))
}
pw.writtenUncompressed = pw.writer.GetWrittenUncompressed()
return nil
}
func (pw *PackedBinlogRecordWriter) initWriters(r Record) error {
if pw.writer == nil {
if len(pw.columnGroups) == 0 {
allFields := typeutil.GetAllFieldSchemas(pw.schema)
pw.columnGroups = storagecommon.SplitColumns(allFields, pw.getColumnStatsFromRecord(r, allFields), storagecommon.DefaultPolicies()...)
}
logIdStart, _, err := pw.allocator.Alloc(uint32(len(pw.columnGroups)))
if err != nil {
return err
}
paths := []string{}
for _, columnGroup := range pw.columnGroups {
path := metautil.BuildInsertLogPath(pw.storageConfig.GetRootPath(), pw.collectionID, pw.partitionID, pw.segmentID, columnGroup.GroupID, logIdStart)
paths = append(paths, path)
logIdStart++
}
pw.writer, err = NewPackedRecordWriter(pw.storageConfig.GetBucketName(), paths, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
if err != nil {
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
}
}
return nil
}
func (pw *PackedBinlogRecordWriter) getColumnStatsFromRecord(r Record, allFields []*schemapb.FieldSchema) map[int64]storagecommon.ColumnStats {
result := make(map[int64]storagecommon.ColumnStats)
for _, field := range allFields {
if arr := r.Column(field.FieldID); arr != nil {
result[field.FieldID] = storagecommon.ColumnStats{
AvgSize: int64(arr.Data().SizeInBytes()) / int64(arr.Len()),
}
}
}
return result
}
func (pw *PackedBinlogRecordWriter) GetWrittenUncompressed() uint64 {
return pw.writtenUncompressed
}
func (pw *PackedBinlogRecordWriter) Close() error {
if pw.writer != nil {
if err := pw.writer.Close(); err != nil {
return err
}
}
pw.finalizeBinlogs()
if err := pw.writeStats(); err != nil {
return err
}
if err := pw.writeBm25Stats(); err != nil {
return err
}
return nil
}
func (pw *PackedBinlogRecordWriter) finalizeBinlogs() {
if pw.writer == nil {
return
}
pw.rowNum = pw.writer.GetWrittenRowNum()
if pw.fieldBinlogs == nil {
pw.fieldBinlogs = make(map[FieldID]*datapb.FieldBinlog, len(pw.columnGroups))
}
for _, columnGroup := range pw.columnGroups {
columnGroupID := columnGroup.GroupID
if _, exists := pw.fieldBinlogs[columnGroupID]; !exists {
pw.fieldBinlogs[columnGroupID] = &datapb.FieldBinlog{
FieldID: columnGroupID,
ChildFields: columnGroup.Fields,
}
}
pw.fieldBinlogs[columnGroupID].Binlogs = append(pw.fieldBinlogs[columnGroupID].Binlogs, &datapb.Binlog{
LogSize: int64(pw.writer.GetColumnGroupWrittenCompressed(columnGroupID)),
MemorySize: int64(pw.writer.GetColumnGroupWrittenUncompressed(columnGroupID)),
LogPath: pw.writer.GetWrittenPaths(columnGroupID),
EntriesNum: pw.writer.GetWrittenRowNum(),
TimestampFrom: pw.tsFrom,
TimestampTo: pw.tsTo,
})
}
}
func (pw *PackedBinlogRecordWriter) writeStats() error {
if pw.pkstats == nil {
return nil
}
id, err := pw.allocator.AllocOne()
if err != nil {
return err
}
codec := NewInsertCodecWithSchema(&etcdpb.CollectionMeta{
ID: pw.collectionID,
Schema: pw.schema,
})
sblob, err := codec.SerializePkStats(pw.pkstats, pw.rowNum)
if err != nil {
return err
}
sblob.Key = metautil.BuildStatsLogPath(pw.storageConfig.GetRootPath(),
pw.collectionID, pw.partitionID, pw.segmentID, pw.pkstats.FieldID, id)
if err := pw.BlobsWriter([]*Blob{sblob}); err != nil {
return err
}
pw.statsLog = &datapb.FieldBinlog{
FieldID: pw.pkstats.FieldID,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(len(sblob.GetValue())),
MemorySize: int64(len(sblob.GetValue())),
LogPath: sblob.Key,
EntriesNum: pw.rowNum,
},
},
}
return nil
}
func (pw *PackedBinlogRecordWriter) writeBm25Stats() error {
if len(pw.bm25Stats) == 0 {
return nil
}
id, _, err := pw.allocator.Alloc(uint32(len(pw.bm25Stats)))
if err != nil {
return err
}
if pw.bm25StatsLog == nil {
pw.bm25StatsLog = make(map[FieldID]*datapb.FieldBinlog)
}
for fid, stats := range pw.bm25Stats {
bytes, err := stats.Serialize()
if err != nil {
return err
}
key := metautil.BuildBm25LogPath(pw.storageConfig.GetRootPath(),
pw.collectionID, pw.partitionID, pw.segmentID, fid, id)
blob := &Blob{
Key: key,
Value: bytes,
RowNum: stats.NumRow(),
MemorySize: int64(len(bytes)),
}
if err := pw.BlobsWriter([]*Blob{blob}); err != nil {
return err
}
fieldLog := &datapb.FieldBinlog{
FieldID: fid,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(len(blob.GetValue())),
MemorySize: int64(len(blob.GetValue())),
LogPath: key,
EntriesNum: pw.rowNum,
},
},
}
pw.bm25StatsLog[fid] = fieldLog
id++
}
return nil
}
func (pw *PackedBinlogRecordWriter) GetLogs() (
fieldBinlogs map[FieldID]*datapb.FieldBinlog,
statsLog *datapb.FieldBinlog,
bm25StatsLog map[FieldID]*datapb.FieldBinlog,
) {
return pw.fieldBinlogs, pw.statsLog, pw.bm25StatsLog
}
func (pw *PackedBinlogRecordWriter) GetRowNum() int64 {
return pw.rowNum
}
func (pw *PackedBinlogRecordWriter) FlushChunk() error {
return nil // do nothing
}
func (pw *PackedBinlogRecordWriter) Schema() *schemapb.CollectionSchema {
return pw.schema
}
func (pw *PackedBinlogRecordWriter) GetBufferUncompressed() uint64 {
return uint64(pw.multiPartUploadSize)
}
func newPackedBinlogRecordWriter(collectionID, partitionID, segmentID UniqueID, schema *schemapb.CollectionSchema,
blobsWriter ChunkedBlobsWriter, allocator allocator.Interface, maxRowNum int64, bufferSize, multiPartUploadSize int64, columnGroups []storagecommon.ColumnGroup,
storageConfig *indexpb.StorageConfig,
storagePluginContext *indexcgopb.StoragePluginContext,
) (*PackedBinlogRecordWriter, error) {
arrowSchema, err := ConvertToArrowSchema(schema)
if err != nil {
return nil, merr.WrapErrParameterInvalid("convert collection schema [%s] to arrow schema error: %s", schema.Name, err.Error())
}
pkField, err := typeutil.GetPrimaryFieldSchema(schema)
if err != nil {
log.Warn("failed to get pk field from schema")
return nil, err
}
stats, err := NewPrimaryKeyStats(pkField.GetFieldID(), int64(pkField.GetDataType()), maxRowNum)
if err != nil {
return nil, err
}
bm25FieldIDs := lo.FilterMap(schema.GetFunctions(), func(function *schemapb.FunctionSchema, _ int) (int64, bool) {
if function.GetType() == schemapb.FunctionType_BM25 {
return function.GetOutputFieldIds()[0], true
}
return 0, false
})
bm25Stats := make(map[int64]*BM25Stats, len(bm25FieldIDs))
for _, fid := range bm25FieldIDs {
bm25Stats[fid] = NewBM25Stats()
}
return &PackedBinlogRecordWriter{
collectionID: collectionID,
partitionID: partitionID,
segmentID: segmentID,
schema: schema,
arrowSchema: arrowSchema,
BlobsWriter: blobsWriter,
allocator: allocator,
maxRowNum: maxRowNum,
bufferSize: bufferSize,
multiPartUploadSize: multiPartUploadSize,
columnGroups: columnGroups,
pkstats: stats,
bm25Stats: bm25Stats,
storageConfig: storageConfig,
storagePluginContext: storagePluginContext,
tsFrom: typeutil.MaxTimestamp,
tsTo: 0,
}, nil
}

View File

@ -24,14 +24,15 @@ import (
"github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/internal/util/initcore"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
func TestPackedSerde(t *testing.T) {
t.Run("test binlog packed serde v2", func(t *testing.T) {
t.Skip("storage v2 cgo not ready yet")
paramtable.Get().Save(paramtable.Get().CommonCfg.StorageType.Key, "local")
initcore.InitLocalArrowFileSystem("/tmp")
size := 10
bucketName := "a-bucket"
bucketName := ""
paths := [][]string{{"/tmp/0"}, {"/tmp/1"}}
bufferSize := int64(10 * 1024 * 1024) // 10MB
schema := generateTestSchema()
@ -70,16 +71,18 @@ func TestPackedSerde(t *testing.T) {
prepareChunkData(chunkPaths, size)
}
reader, err := NewPackedDeserializeReader(paths, schema, bufferSize, false)
assert.NoError(t, err)
reader := newIterativePackedRecordReader(paths, schema, bufferSize, nil, nil)
defer reader.Close()
for i := 0; i < size*len(paths); i++ {
value, err := reader.NextValue()
assert.NoError(t, err)
assertTestData(t, i%10+1, *value)
nRows := 0
for {
rec, err := reader.Next()
if err == io.EOF {
break
}
_, err = reader.NextValue()
assert.Equal(t, err, io.EOF)
assert.NoError(t, err)
nRows += rec.Len()
}
assert.Equal(t, size*len(paths), nRows)
})
}

View File

@ -436,7 +436,7 @@ func TestArrayOfVectorIntegration(t *testing.T) {
},
}
arrowSchema, err := ConvertToArrowSchema(schema)
arrowSchema, err := ConvertToArrowSchema(schema, false)
assert.NoError(t, err)
assert.NotNil(t, arrowSchema)

View File

@ -31,12 +31,10 @@ func TestSort(t *testing.T) {
getReaders := func() []RecordReader {
blobs, err := generateTestDataWithSeed(10, 3)
assert.NoError(t, err)
reader10, err := newCompositeBinlogRecordReader(generateTestSchema(), MakeBlobsReader(blobs))
assert.NoError(t, err)
reader10 := newIterativeCompositeBinlogRecordReader(generateTestSchema(), nil, MakeBlobsReader(blobs))
blobs, err = generateTestDataWithSeed(20, 3)
assert.NoError(t, err)
reader20, err := newCompositeBinlogRecordReader(generateTestSchema(), MakeBlobsReader(blobs))
assert.NoError(t, err)
reader20 := newIterativeCompositeBinlogRecordReader(generateTestSchema(), nil, MakeBlobsReader(blobs))
rr := []RecordReader{reader20, reader10}
return rr
}
@ -82,12 +80,10 @@ func TestMergeSort(t *testing.T) {
getReaders := func() []RecordReader {
blobs, err := generateTestDataWithSeed(1000, 5000)
assert.NoError(t, err)
reader10, err := newCompositeBinlogRecordReader(generateTestSchema(), MakeBlobsReader(blobs))
assert.NoError(t, err)
reader10 := newIterativeCompositeBinlogRecordReader(generateTestSchema(), nil, MakeBlobsReader(blobs))
blobs, err = generateTestDataWithSeed(4000, 5000)
assert.NoError(t, err)
reader20, err := newCompositeBinlogRecordReader(generateTestSchema(), MakeBlobsReader(blobs))
assert.NoError(t, err)
reader20 := newIterativeCompositeBinlogRecordReader(generateTestSchema(), nil, MakeBlobsReader(blobs))
rr := []RecordReader{reader20, reader10}
return rr
}
@ -138,12 +134,10 @@ func BenchmarkSort(b *testing.B) {
batch := 500000
blobs, err := generateTestDataWithSeed(batch, batch)
assert.NoError(b, err)
reader10, err := newCompositeBinlogRecordReader(generateTestSchema(), MakeBlobsReader(blobs))
assert.NoError(b, err)
reader10 := newIterativeCompositeBinlogRecordReader(generateTestSchema(), nil, MakeBlobsReader(blobs))
blobs, err = generateTestDataWithSeed(batch*2+1, batch)
assert.NoError(b, err)
reader20, err := newCompositeBinlogRecordReader(generateTestSchema(), MakeBlobsReader(blobs))
assert.NoError(b, err)
reader20 := newIterativeCompositeBinlogRecordReader(generateTestSchema(), nil, MakeBlobsReader(blobs))
rr := []RecordReader{reader20, reader10}
rw := &MockRecordWriter{
@ -174,12 +168,10 @@ func TestSortByMoreThanOneField(t *testing.T) {
blobs, err := generateTestDataWithSeed(10, batchSize)
assert.NoError(t, err)
reader10, err := newCompositeBinlogRecordReader(generateTestSchema(), MakeBlobsReader(blobs))
assert.NoError(t, err)
reader10 := newIterativeCompositeBinlogRecordReader(generateTestSchema(), nil, MakeBlobsReader(blobs))
blobs, err = generateTestDataWithSeed(20, batchSize)
assert.NoError(t, err)
reader20, err := newCompositeBinlogRecordReader(generateTestSchema(), MakeBlobsReader(blobs))
assert.NoError(t, err)
reader20 := newIterativeCompositeBinlogRecordReader(generateTestSchema(), nil, MakeBlobsReader(blobs))
rr := []RecordReader{reader20, reader10}
lastPK := int64(-1)

View File

@ -0,0 +1,278 @@
// 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.
package storage
import (
"strconv"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/etcdpb"
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
// StatsCollector collects statistics from records
type StatsCollector interface {
// Collect collects statistics from a record
Collect(r Record) error
// Digest serializes the collected statistics, writes them to storage,
// and returns the field binlog metadata
Digest(
collectionID, partitionID, segmentID UniqueID,
rootPath string,
rowNum int64,
allocator allocator.Interface,
blobsWriter ChunkedBlobsWriter,
) (map[FieldID]*datapb.FieldBinlog, error)
}
// PkStatsCollector collects primary key statistics
type PkStatsCollector struct {
pkstats *PrimaryKeyStats
collectionID UniqueID // needed for initializing codecs, TODO: remove this
schema *schemapb.CollectionSchema
}
// Collect collects primary key stats from the record
func (c *PkStatsCollector) Collect(r Record) error {
if c.pkstats == nil {
return nil
}
rows := r.Len()
for i := 0; i < rows; i++ {
switch schemapb.DataType(c.pkstats.PkType) {
case schemapb.DataType_Int64:
pkArray := r.Column(c.pkstats.FieldID).(*array.Int64)
pk := &Int64PrimaryKey{
Value: pkArray.Value(i),
}
c.pkstats.Update(pk)
case schemapb.DataType_VarChar:
pkArray := r.Column(c.pkstats.FieldID).(*array.String)
pk := &VarCharPrimaryKey{
Value: pkArray.Value(i),
}
c.pkstats.Update(pk)
default:
panic("invalid data type")
}
}
return nil
}
// Digest serializes the collected primary key statistics, writes them to storage,
// and returns the field binlog metadata
func (c *PkStatsCollector) Digest(
collectionID, partitionID, segmentID UniqueID,
rootPath string,
rowNum int64,
allocator allocator.Interface,
blobsWriter ChunkedBlobsWriter,
) (map[FieldID]*datapb.FieldBinlog, error) {
if c.pkstats == nil {
return nil, nil
}
// Serialize PK stats
codec := NewInsertCodecWithSchema(&etcdpb.CollectionMeta{
ID: c.collectionID,
Schema: c.schema,
})
sblob, err := codec.SerializePkStats(c.pkstats, rowNum)
if err != nil {
return nil, err
}
// Get pk field ID
pkField, err := typeutil.GetPrimaryFieldSchema(c.schema)
if err != nil {
return nil, err
}
// Allocate ID for stats blob
id, err := allocator.AllocOne()
if err != nil {
return nil, err
}
// Assign proper path to the blob
fieldID := pkField.GetFieldID()
sblob.Key = metautil.BuildStatsLogPath(rootPath,
c.collectionID, partitionID, segmentID, fieldID, id)
// Write the blob
if err := blobsWriter([]*Blob{sblob}); err != nil {
return nil, err
}
// Return as map for interface consistency
return map[FieldID]*datapb.FieldBinlog{
fieldID: {
FieldID: fieldID,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(len(sblob.GetValue())),
MemorySize: int64(len(sblob.GetValue())),
LogPath: sblob.Key,
EntriesNum: rowNum,
},
},
},
}, nil
}
// NewPkStatsCollector creates a new primary key stats collector
func NewPkStatsCollector(
collectionID UniqueID,
schema *schemapb.CollectionSchema,
maxRowNum int64,
) (*PkStatsCollector, error) {
pkField, err := typeutil.GetPrimaryFieldSchema(schema)
if err != nil {
return nil, err
}
stats, err := NewPrimaryKeyStats(pkField.GetFieldID(), int64(pkField.GetDataType()), maxRowNum)
if err != nil {
return nil, err
}
return &PkStatsCollector{
pkstats: stats,
collectionID: collectionID,
schema: schema,
}, nil
}
// Bm25StatsCollector collects BM25 statistics
type Bm25StatsCollector struct {
bm25Stats map[int64]*BM25Stats
}
// Collect collects BM25 statistics from the record
func (c *Bm25StatsCollector) Collect(r Record) error {
if len(c.bm25Stats) == 0 {
return nil
}
rows := r.Len()
for fieldID, stats := range c.bm25Stats {
field, ok := r.Column(fieldID).(*array.Binary)
if !ok {
return errors.New("bm25 field value not found")
}
for i := 0; i < rows; i++ {
stats.AppendBytes(field.Value(i))
}
}
return nil
}
// Digest serializes the collected BM25 statistics, writes them to storage,
// and returns the field binlog metadata
func (c *Bm25StatsCollector) Digest(
collectionID, partitionID, segmentID UniqueID,
rootPath string,
rowNum int64,
allocator allocator.Interface,
blobsWriter ChunkedBlobsWriter,
) (map[FieldID]*datapb.FieldBinlog, error) {
if len(c.bm25Stats) == 0 {
return nil, nil
}
// Serialize BM25 stats into blobs
blobs := make([]*Blob, 0, len(c.bm25Stats))
for fid, stats := range c.bm25Stats {
bytes, err := stats.Serialize()
if err != nil {
return nil, err
}
blob := &Blob{
Key: strconv.FormatInt(fid, 10), // temporary key, will be replaced below
Value: bytes,
RowNum: stats.NumRow(),
MemorySize: int64(len(bytes)),
}
blobs = append(blobs, blob)
}
// Allocate IDs for stats blobs
id, _, err := allocator.Alloc(uint32(len(blobs)))
if err != nil {
return nil, err
}
result := make(map[FieldID]*datapb.FieldBinlog)
// Process each blob and assign proper paths
for _, blob := range blobs {
// Parse the field ID from the temporary key
fieldID, parseErr := strconv.ParseInt(blob.Key, 10, 64)
if parseErr != nil {
// This should not happen for BM25 blobs
continue
}
blob.Key = metautil.BuildBm25LogPath(rootPath,
collectionID, partitionID, segmentID, fieldID, id)
result[fieldID] = &datapb.FieldBinlog{
FieldID: fieldID,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(len(blob.GetValue())),
MemorySize: int64(len(blob.GetValue())),
LogPath: blob.Key,
EntriesNum: rowNum,
},
},
}
id++
}
// Write all blobs
if err := blobsWriter(blobs); err != nil {
return nil, err
}
return result, nil
}
// NewBm25StatsCollector creates a new BM25 stats collector
func NewBm25StatsCollector(schema *schemapb.CollectionSchema) *Bm25StatsCollector {
bm25FieldIDs := lo.FilterMap(schema.GetFunctions(), func(function *schemapb.FunctionSchema, _ int) (int64, bool) {
if function.GetType() == schemapb.FunctionType_BM25 {
return function.GetOutputFieldIds()[0], true
}
return 0, false
})
bm25Stats := make(map[int64]*BM25Stats, len(bm25FieldIDs))
for _, fid := range bm25FieldIDs {
bm25Stats[fid] = NewBM25Stats()
}
return &Bm25StatsCollector{
bm25Stats: bm25Stats,
}
}

View File

@ -0,0 +1,261 @@
// 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.
package storage
import (
"fmt"
"testing"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/apache/arrow/go/v17/arrow/memory"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/pkg/v2/common"
)
func TestPkStatsCollector(t *testing.T) {
collectionID := int64(1)
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, DataType: schemapb.DataType_Int64},
{FieldID: common.TimeStampField, DataType: schemapb.DataType_Int64},
{
FieldID: 100,
Name: "pk",
DataType: schemapb.DataType_Int64,
IsPrimaryKey: true,
},
},
}
t.Run("collect and digest int64 pk", func(t *testing.T) {
collector, err := NewPkStatsCollector(collectionID, schema, 100)
require.NoError(t, err)
require.NotNil(t, collector)
// Create test record
fields := []arrow.Field{
{Name: "pk", Type: arrow.PrimitiveTypes.Int64},
}
arrowSchema := arrow.NewSchema(fields, nil)
builder := array.NewRecordBuilder(memory.DefaultAllocator, arrowSchema)
defer builder.Release()
pkBuilder := builder.Field(0).(*array.Int64Builder)
for i := 0; i < 10; i++ {
pkBuilder.Append(int64(i))
}
rec := builder.NewRecord()
field2Col := map[FieldID]int{100: 0}
record := NewSimpleArrowRecord(rec, field2Col)
// Collect stats
err = collector.Collect(record)
assert.NoError(t, err)
// Digest stats
alloc := allocator.NewLocalAllocator(1, 100)
writer := func(blobs []*Blob) error { return nil }
resultMap, err := collector.Digest(collectionID, 1, 2, "/tmp", 10, alloc, writer)
assert.NoError(t, err)
assert.NotNil(t, resultMap)
assert.Len(t, resultMap, 1)
})
t.Run("varchar pk", func(t *testing.T) {
varcharSchema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, DataType: schemapb.DataType_Int64},
{FieldID: common.TimeStampField, DataType: schemapb.DataType_Int64},
{
FieldID: 100,
Name: "pk",
DataType: schemapb.DataType_VarChar,
IsPrimaryKey: true,
},
},
}
collector, err := NewPkStatsCollector(collectionID, varcharSchema, 100)
require.NoError(t, err)
// Create test record with varchar pk
fields := []arrow.Field{
{Name: "pk", Type: arrow.BinaryTypes.String},
}
arrowSchema := arrow.NewSchema(fields, nil)
builder := array.NewRecordBuilder(memory.DefaultAllocator, arrowSchema)
defer builder.Release()
pkBuilder := builder.Field(0).(*array.StringBuilder)
for i := 0; i < 10; i++ {
pkBuilder.Append(fmt.Sprintf("key_%d", i))
}
rec := builder.NewRecord()
field2Col := map[FieldID]int{100: 0}
record := NewSimpleArrowRecord(rec, field2Col)
err = collector.Collect(record)
assert.NoError(t, err)
})
}
func TestBm25StatsCollector(t *testing.T) {
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, DataType: schemapb.DataType_Int64},
{FieldID: common.TimeStampField, DataType: schemapb.DataType_Int64},
{
FieldID: 100,
Name: "text",
DataType: schemapb.DataType_VarChar,
},
},
Functions: []*schemapb.FunctionSchema{
{
Name: "bm25_function",
Type: schemapb.FunctionType_BM25,
InputFieldIds: []int64{100},
OutputFieldIds: []int64{101},
OutputFieldNames: []string{"bm25_field"},
},
},
}
t.Run("collect bm25 stats", func(t *testing.T) {
collector := NewBm25StatsCollector(schema)
assert.NotNil(t, collector)
assert.NotNil(t, collector.bm25Stats)
})
t.Run("digest with empty stats", func(t *testing.T) {
collector := NewBm25StatsCollector(schema)
alloc := allocator.NewLocalAllocator(1, 100)
writer := func(blobs []*Blob) error { return nil }
_, err := collector.Digest(1, 1, 2, "/tmp", 10, alloc, writer)
assert.NoError(t, err)
})
}
func TestNewPkStatsCollector_NoPkField(t *testing.T) {
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, DataType: schemapb.DataType_Int64},
{FieldID: common.TimeStampField, DataType: schemapb.DataType_Int64},
},
}
collector, err := NewPkStatsCollector(1, schema, 100)
assert.Error(t, err)
assert.Nil(t, collector)
}
func TestPkStatsCollector_DigestEndToEnd(t *testing.T) {
collectionID := int64(1)
partitionID := int64(2)
segmentID := int64(3)
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
FieldID: 100,
Name: "pk",
DataType: schemapb.DataType_Int64,
IsPrimaryKey: true,
},
},
}
collector, err := NewPkStatsCollector(collectionID, schema, 100)
require.NoError(t, err)
// Create test record
fields := []arrow.Field{
{Name: "pk", Type: arrow.PrimitiveTypes.Int64},
}
arrowSchema := arrow.NewSchema(fields, nil)
builder := array.NewRecordBuilder(memory.DefaultAllocator, arrowSchema)
defer builder.Release()
pkBuilder := builder.Field(0).(*array.Int64Builder)
for i := 0; i < 10; i++ {
pkBuilder.Append(int64(i))
}
rec := builder.NewRecord()
field2Col := map[FieldID]int{100: 0}
record := NewSimpleArrowRecord(rec, field2Col)
err = collector.Collect(record)
require.NoError(t, err)
alloc := allocator.NewLocalAllocator(1, 100)
var writtenBlobs []*Blob
writer := func(blobs []*Blob) error {
writtenBlobs = blobs
return nil
}
// Test Digest which includes writing
binlogMap, err := collector.Digest(collectionID, partitionID, segmentID,
"/tmp", 10, alloc, writer)
assert.NoError(t, err)
assert.NotNil(t, binlogMap)
assert.Len(t, binlogMap, 1)
binlog := binlogMap[100]
assert.NotNil(t, binlog)
assert.Equal(t, int64(100), binlog.FieldID)
assert.Len(t, binlog.Binlogs, 1)
assert.Contains(t, binlog.Binlogs[0].LogPath, "stats_log")
assert.NotNil(t, writtenBlobs)
assert.Len(t, writtenBlobs, 1)
}
func TestBm25StatsCollector_DigestEndToEnd(t *testing.T) {
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{FieldID: 100, Name: "text", DataType: schemapb.DataType_VarChar},
},
Functions: []*schemapb.FunctionSchema{
{
Name: "bm25_function",
Type: schemapb.FunctionType_BM25,
InputFieldIds: []int64{100},
OutputFieldIds: []int64{101},
OutputFieldNames: []string{"bm25_field"},
},
},
}
collector := NewBm25StatsCollector(schema)
alloc := allocator.NewLocalAllocator(1, 100)
writer := func(blobs []*Blob) error { return nil }
// Test with empty stats
_, err := collector.Digest(1, 2, 3, "/tmp", 10, alloc, writer)
assert.NoError(t, err)
}

View File

@ -0,0 +1,215 @@
package packed
/*
#cgo pkg-config: milvus_core milvus-storage
#include <stdlib.h>
#include "milvus-storage/ffi_c.h"
#include "arrow/c/abi.h"
#include "arrow/c/helpers.h"
*/
import "C"
import (
"encoding/json"
"fmt"
"strconv"
"unsafe"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
)
// Property keys - matching milvus-storage/properties.h
const (
PropertyFSAddress = "fs.address"
PropertyFSBucketName = "fs.bucket_name"
PropertyFSAccessKeyID = "fs.access_key_id"
PropertyFSAccessKeyValue = "fs.access_key_value"
PropertyFSRootPath = "fs.root_path"
PropertyFSStorageType = "fs.storage_type"
PropertyFSCloudProvider = "fs.cloud_provider"
PropertyFSIAMEndpoint = "fs.iam_endpoint"
PropertyFSLogLevel = "fs.log_level"
PropertyFSRegion = "fs.region"
PropertyFSUseSSL = "fs.use_ssl"
PropertyFSSSLCACert = "fs.ssl_ca_cert"
PropertyFSUseIAM = "fs.use_iam"
PropertyFSUseVirtualHost = "fs.use_virtual_host"
PropertyFSRequestTimeoutMS = "fs.request_timeout_ms"
PropertyFSGCPCredentialJSON = "fs.gcp_credential_json"
PropertyFSUseCustomPartUpload = "fs.use_custom_part_upload"
PropertyWriterPolicy = "writer.policy"
PropertyWriterSchemaBasedPattern = "writer.split.schema_based.patterns"
)
// MakePropertiesFromStorageConfig creates a Properties object from StorageConfig
// This function converts a StorageConfig structure into a Properties object by
// calling the FFI properties_create function. All configuration fields from
// StorageConfig are mapped to corresponding key-value pairs in Properties.
func MakePropertiesFromStorageConfig(storageConfig *indexpb.StorageConfig, extraKVs map[string]string) (*C.Properties, error) {
if storageConfig == nil {
return nil, fmt.Errorf("storageConfig is required")
}
// Prepare key-value pairs from StorageConfig
var keys []string
var values []string
// Add non-empty string fields
if storageConfig.GetAddress() != "" {
keys = append(keys, PropertyFSAddress)
values = append(values, storageConfig.GetAddress())
}
if storageConfig.GetBucketName() != "" {
keys = append(keys, PropertyFSBucketName)
values = append(values, storageConfig.GetBucketName())
}
if storageConfig.GetAccessKeyID() != "" {
keys = append(keys, PropertyFSAccessKeyID)
values = append(values, storageConfig.GetAccessKeyID())
}
if storageConfig.GetSecretAccessKey() != "" {
keys = append(keys, PropertyFSAccessKeyValue)
values = append(values, storageConfig.GetSecretAccessKey())
}
if storageConfig.GetRootPath() != "" {
keys = append(keys, PropertyFSRootPath)
values = append(values, storageConfig.GetRootPath())
}
if storageConfig.GetStorageType() != "" {
keys = append(keys, PropertyFSStorageType)
values = append(values, storageConfig.GetStorageType())
}
if storageConfig.GetCloudProvider() != "" {
keys = append(keys, PropertyFSCloudProvider)
values = append(values, storageConfig.GetCloudProvider())
}
if storageConfig.GetIAMEndpoint() != "" {
keys = append(keys, PropertyFSIAMEndpoint)
values = append(values, storageConfig.GetIAMEndpoint())
}
// Always add log level if any string field is set (matching C++ behavior)
keys = append(keys, PropertyFSLogLevel)
values = append(values, "Warn")
if storageConfig.GetRegion() != "" {
keys = append(keys, PropertyFSRegion)
values = append(values, storageConfig.GetRegion())
}
if storageConfig.GetSslCACert() != "" {
keys = append(keys, PropertyFSSSLCACert)
values = append(values, storageConfig.GetSslCACert())
}
if storageConfig.GetGcpCredentialJSON() != "" {
keys = append(keys, PropertyFSGCPCredentialJSON)
values = append(values, storageConfig.GetGcpCredentialJSON())
}
// Add boolean fields
keys = append(keys, PropertyFSUseSSL)
if storageConfig.GetUseSSL() {
values = append(values, "true")
} else {
values = append(values, "false")
}
keys = append(keys, PropertyFSUseIAM)
if storageConfig.GetUseIAM() {
values = append(values, "true")
} else {
values = append(values, "false")
}
keys = append(keys, PropertyFSUseVirtualHost)
if storageConfig.GetUseVirtualHost() {
values = append(values, "true")
} else {
values = append(values, "false")
}
keys = append(keys, PropertyFSUseCustomPartUpload)
values = append(values, "true") // hardcoded to true as in the original code
// Add integer field
keys = append(keys, PropertyFSRequestTimeoutMS)
values = append(values, strconv.FormatInt(storageConfig.GetRequestTimeoutMs(), 10))
// Add extra kvs
for k, v := range extraKVs {
keys = append(keys, k)
values = append(values, v)
}
// Convert to C arrays
cKeys := make([]*C.char, len(keys))
cValues := make([]*C.char, len(values))
for i := range keys {
cKeys[i] = C.CString(keys[i])
cValues[i] = C.CString(values[i])
}
// Defer cleanup of all C strings
defer func() {
for i := range cKeys {
C.free(unsafe.Pointer(cKeys[i]))
C.free(unsafe.Pointer(cValues[i]))
}
}()
// Create Properties using FFI
properties := &C.Properties{}
var cKeysPtr **C.char
var cValuesPtr **C.char
if len(cKeys) > 0 {
cKeysPtr = &cKeys[0]
cValuesPtr = &cValues[0]
}
result := C.properties_create(
(**C.char)(unsafe.Pointer(cKeysPtr)),
(**C.char)(unsafe.Pointer(cValuesPtr)),
C.size_t(len(keys)),
properties,
)
err := HandleFFIResult(result)
if err != nil {
return nil, err
}
return properties, nil
}
func HandleFFIResult(ffiResult C.FFIResult) error {
defer C.FreeFFIResult(&ffiResult)
if C.IsSuccess(&ffiResult) == 0 {
errMsg := C.GetErrorMessage(&ffiResult)
errStr := "Unknown error"
if errMsg != nil {
errStr = C.GoString(errMsg)
}
return fmt.Errorf("failed to create properties: %s", errStr)
}
return nil
}
type ManifestJSON struct {
ManifestVersion int64 `json:"ver"`
BasePath string `json:"base_path"`
}
func MarshalManifestPath(basePath string, version int64) string {
bs, _ := json.Marshal(ManifestJSON{
ManifestVersion: version,
BasePath: basePath,
})
return string(bs)
}
func UnmarshalManfestPath(manifestPath string) (string, int64, error) {
var manifestJSON ManifestJSON
err := json.Unmarshal([]byte(manifestPath), &manifestJSON)
if err != nil {
return "", 0, err
}
return manifestJSON.BasePath, manifestJSON.ManifestVersion, nil
}

View File

@ -0,0 +1,213 @@
// Copyright 2023 Zilliz
//
// Licensed 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.
package packed
/*
#cgo pkg-config: milvus_core
#include <stdlib.h>
#include "storage/loon_ffi/ffi_reader_c.h"
#include "arrow/c/abi.h"
#include "arrow/c/helpers.h"
*/
import "C"
import (
"fmt"
"io"
"unsafe"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/cdata"
"go.uber.org/zap"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
)
func NewFFIPackedReader(manifest string, schema *arrow.Schema, neededColumns []string, bufferSize int64, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext) (*FFIPackedReader, error) {
cManifest := C.CString(manifest)
defer C.free(unsafe.Pointer(cManifest))
var cas cdata.CArrowSchema
cdata.ExportArrowSchema(schema, &cas)
cSchema := (*C.struct_ArrowSchema)(unsafe.Pointer(&cas))
defer cdata.ReleaseCArrowSchema(&cas)
var cPackedReader C.CFFIPackedReader
var status C.CStatus
var pluginContextPtr *C.CPluginContext
if storagePluginContext != nil {
ckey := C.CString(storagePluginContext.EncryptionKey)
defer C.free(unsafe.Pointer(ckey))
var pluginContext C.CPluginContext
pluginContext.ez_id = C.int64_t(storagePluginContext.EncryptionZoneId)
pluginContext.collection_id = C.int64_t(storagePluginContext.CollectionId)
pluginContext.key = ckey
pluginContextPtr = &pluginContext
}
if storageConfig != nil {
cStorageConfig := C.CStorageConfig{
address: C.CString(storageConfig.GetAddress()),
bucket_name: C.CString(storageConfig.GetBucketName()),
access_key_id: C.CString(storageConfig.GetAccessKeyID()),
access_key_value: C.CString(storageConfig.GetSecretAccessKey()),
root_path: C.CString(storageConfig.GetRootPath()),
storage_type: C.CString(storageConfig.GetStorageType()),
cloud_provider: C.CString(storageConfig.GetCloudProvider()),
iam_endpoint: C.CString(storageConfig.GetIAMEndpoint()),
log_level: C.CString("Warn"), // TODO use config after storage support lower case configuration
useSSL: C.bool(storageConfig.GetUseSSL()),
sslCACert: C.CString(storageConfig.GetSslCACert()),
useIAM: C.bool(storageConfig.GetUseIAM()),
region: C.CString(storageConfig.GetRegion()),
useVirtualHost: C.bool(storageConfig.GetUseVirtualHost()),
requestTimeoutMs: C.int64_t(storageConfig.GetRequestTimeoutMs()),
gcp_credential_json: C.CString(storageConfig.GetGcpCredentialJSON()),
use_custom_part_upload: true,
}
defer C.free(unsafe.Pointer(cStorageConfig.address))
defer C.free(unsafe.Pointer(cStorageConfig.bucket_name))
defer C.free(unsafe.Pointer(cStorageConfig.access_key_id))
defer C.free(unsafe.Pointer(cStorageConfig.access_key_value))
defer C.free(unsafe.Pointer(cStorageConfig.root_path))
defer C.free(unsafe.Pointer(cStorageConfig.storage_type))
defer C.free(unsafe.Pointer(cStorageConfig.cloud_provider))
defer C.free(unsafe.Pointer(cStorageConfig.iam_endpoint))
defer C.free(unsafe.Pointer(cStorageConfig.log_level))
defer C.free(unsafe.Pointer(cStorageConfig.sslCACert))
defer C.free(unsafe.Pointer(cStorageConfig.region))
defer C.free(unsafe.Pointer(cStorageConfig.gcp_credential_json))
cNeededColumn := make([]*C.char, len(neededColumns))
for i, columnName := range neededColumns {
cNeededColumn[i] = C.CString(columnName)
defer C.free(unsafe.Pointer(cNeededColumn[i]))
}
cNeededColumnArray := (**C.char)(unsafe.Pointer(&cNeededColumn[0]))
cNumColumns := C.int64_t(len(neededColumns))
status = C.NewPackedFFIReaderWithManifest(cManifest, cSchema, cNeededColumnArray, cNumColumns, &cPackedReader, cStorageConfig, pluginContextPtr)
} else {
return nil, fmt.Errorf("storageConfig is required")
}
if err := ConsumeCStatusIntoError(&status); err != nil {
return nil, err
}
// Get the ArrowArrayStream
var cStream cdata.CArrowArrayStream
status = C.GetFFIReaderStream(cPackedReader, C.int64_t(8196), (*C.struct_ArrowArrayStream)(unsafe.Pointer(&cStream)))
if err := ConsumeCStatusIntoError(&status); err != nil {
C.CloseFFIReader(cPackedReader)
return nil, fmt.Errorf("failed to get reader stream: %w", err)
}
// Import the stream as a RecordReader
recordReader, err := cdata.ImportCRecordReader(&cStream, schema)
if err != nil {
C.CloseFFIReader(cPackedReader)
return nil, fmt.Errorf("failed to import record reader: %w", err)
}
return &FFIPackedReader{
cPackedReader: cPackedReader,
recordReader: recordReader,
schema: schema,
}, nil
}
// ReadNext reads the next record batch from the reader
func (r *FFIPackedReader) ReadNext() (arrow.Record, error) {
if r.recordReader == nil {
return nil, io.EOF
}
// no need to manual release
// stream reader will release previous one
// Read next record from the stream
rec, err := r.recordReader.Read()
if err != nil {
if err == io.EOF {
return nil, io.EOF
}
return nil, fmt.Errorf("failed to read next record: %w", err)
}
return rec, nil
}
// Close closes the FFI reader
func (r *FFIPackedReader) Close() error {
// no need to manual release current batch
// stream reader handles it
if r.recordReader != nil {
r.recordReader = nil
}
status := C.CloseFFIReader(r.cPackedReader)
return ConsumeCStatusIntoError(&status)
}
// Schema returns the schema of the reader
func (r *FFIPackedReader) Schema() *arrow.Schema {
return r.schema
}
// Retain increases the reference count
func (r *FFIPackedReader) Retain() {
// if r.recordReader != nil {
// r.recordReader.Retain()
// }
}
// Release decreases the reference count
func (r *FFIPackedReader) Release() {
r.Close()
}
func GetManifest(manifestPath string, storageConfig *indexpb.StorageConfig) (manifest string, err error) {
basePath, version, err := UnmarshalManfestPath(manifestPath)
if err != nil {
return "", err
}
log.Info("GetManifest", zap.String("manifestPath", manifestPath), zap.String("basePath", basePath), zap.Int64("version", version))
cProperties, err := MakePropertiesFromStorageConfig(storageConfig, nil)
if err != nil {
return "", err
}
cBasePath := C.CString(basePath)
defer C.free(unsafe.Pointer(cBasePath))
var cManifest *C.char
var cVersion C.int64_t
result := C.get_latest_column_groups(cBasePath, cProperties, &cManifest, &cVersion)
err = HandleFFIResult(result)
if err != nil {
return "", err
}
manifest = C.GoString(cManifest)
return manifest, nil
}
// Ensure FFIPackedReader implements array.RecordReader interface
// var _ array.RecordReader = (*FFIPackedReader)(nil)

View File

@ -0,0 +1,179 @@
// Copyright 2023 Zilliz
//
// Licensed 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.
package packed
/*
#cgo pkg-config: milvus_core milvus-storage
#include <stdlib.h>
#include "milvus-storage/ffi_c.h"
#include "segcore/packed_writer_c.h"
#include "segcore/column_groups_c.h"
#include "arrow/c/abi.h"
#include "arrow/c/helpers.h"
*/
import "C"
import (
"strings"
"unsafe"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/cdata"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
func createStorageConfig() *indexpb.StorageConfig {
var storageConfig *indexpb.StorageConfig
if paramtable.Get().CommonCfg.StorageType.GetValue() == "local" {
storageConfig = &indexpb.StorageConfig{
RootPath: paramtable.Get().LocalStorageCfg.Path.GetValue(),
StorageType: paramtable.Get().CommonCfg.StorageType.GetValue(),
}
} else {
storageConfig = &indexpb.StorageConfig{
Address: paramtable.Get().MinioCfg.Address.GetValue(),
AccessKeyID: paramtable.Get().MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: paramtable.Get().MinioCfg.SecretAccessKey.GetValue(),
UseSSL: paramtable.Get().MinioCfg.UseSSL.GetAsBool(),
SslCACert: paramtable.Get().MinioCfg.SslCACert.GetValue(),
BucketName: paramtable.Get().MinioCfg.BucketName.GetValue(),
RootPath: paramtable.Get().MinioCfg.RootPath.GetValue(),
UseIAM: paramtable.Get().MinioCfg.UseIAM.GetAsBool(),
IAMEndpoint: paramtable.Get().MinioCfg.IAMEndpoint.GetValue(),
StorageType: paramtable.Get().CommonCfg.StorageType.GetValue(),
Region: paramtable.Get().MinioCfg.Region.GetValue(),
UseVirtualHost: paramtable.Get().MinioCfg.UseVirtualHost.GetAsBool(),
CloudProvider: paramtable.Get().MinioCfg.CloudProvider.GetValue(),
RequestTimeoutMs: paramtable.Get().MinioCfg.RequestTimeoutMs.GetAsInt64(),
GcpCredentialJSON: paramtable.Get().MinioCfg.GcpCredentialJSON.GetValue(),
}
}
return storageConfig
}
func NewFFIPackedWriter(basePath string, schema *arrow.Schema, columnGroups []storagecommon.ColumnGroup, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext) (*FFIPackedWriter, error) {
cBasePath := C.CString(basePath)
defer C.free(unsafe.Pointer(cBasePath))
var cas cdata.CArrowSchema
cdata.ExportArrowSchema(schema, &cas)
cSchema := (*C.struct_ArrowSchema)(unsafe.Pointer(&cas))
defer cdata.ReleaseCArrowSchema(&cas)
if storageConfig == nil {
storageConfig = createStorageConfig()
}
pattern := strings.Join(lo.Map(columnGroups, func(columnGroup storagecommon.ColumnGroup, _ int) string {
return strings.Join(lo.Map(columnGroup.Columns, func(index int, _ int) string {
return schema.Field(index).Name
}), "|")
}), ",")
cProperties, err := MakePropertiesFromStorageConfig(storageConfig, map[string]string{
PropertyWriterPolicy: "schema_based",
PropertyWriterSchemaBasedPattern: pattern,
})
if err != nil {
return nil, err
}
var writerHandle C.WriterHandle
result := C.writer_new(cBasePath, cSchema, cProperties, &writerHandle)
err = HandleFFIResult(result)
if err != nil {
return nil, err
}
return &FFIPackedWriter{
basePath: basePath,
cWriterHandle: writerHandle,
cProperties: cProperties,
}, nil
}
func (pw *FFIPackedWriter) WriteRecordBatch(recordBatch arrow.Record) error {
var caa cdata.CArrowArray
var cas cdata.CArrowSchema
// Export the record batch to C Arrow format
cdata.ExportArrowRecordBatch(recordBatch, &caa, &cas)
defer cdata.ReleaseCArrowArray(&caa)
defer cdata.ReleaseCArrowSchema(&cas)
// Convert to C struct
cArray := (*C.struct_ArrowArray)(unsafe.Pointer(&caa))
result := C.writer_write(pw.cWriterHandle, cArray)
return HandleFFIResult(result)
}
func (pw *FFIPackedWriter) Close() (string, error) {
var manifest *C.char
result := C.writer_close(pw.cWriterHandle, nil, nil, 0, &manifest)
if err := HandleFFIResult(result); err != nil {
return "", err
}
cBasePath := C.CString(pw.basePath)
defer C.free(unsafe.Pointer(cBasePath))
var transationHandle C.TransactionHandle
result = C.transaction_begin(cBasePath, pw.cProperties, &transationHandle)
if err := HandleFFIResult(result); err != nil {
return "", err
}
defer C.transaction_destroy(transationHandle)
// #define LOON_TRANSACTION_UPDATE_ADDFILES 0
// #define LOON_TRANSACTION_UPDATE_ADDFEILD 1
// #define LOON_TRANSACTION_UPDATE_MAX 2
// #define LOON_TRANSACTION_RESOLVE_FAIL 0
// #define LOON_TRANSACTION_RESOLVE_MERGE 1
// #define LOON_TRANSACTION_RESOLVE_MAX 2
var commitResult C.bool
result = C.transaction_commit(transationHandle, C.int16_t(0), C.int16_t(0), manifest, &commitResult)
if err := HandleFFIResult(result); err != nil {
return "", err
}
var readVersion C.int64_t
// TODO: not atomic, need to get version from transaction
var cOutManifest *C.char
result = C.get_latest_column_groups(cBasePath, pw.cProperties, &cOutManifest, &readVersion)
if err := HandleFFIResult(result); err != nil {
return "", err
}
outManifest := C.GoString(cOutManifest)
log.Info("FFI writer closed with output manifest", zap.String("manifest", outManifest), zap.Int64("version", int64(readVersion)))
defer C.properties_free(pw.cProperties)
return MarshalManifestPath(pw.basePath, int64(readVersion)), nil
}

View File

@ -18,6 +18,7 @@ package packed
#include <stdlib.h>
#include "arrow/c/abi.h"
#include "arrow/c/helpers.h"
#include "storage/loon_ffi/ffi_reader_c.h"
#include "segcore/packed_reader_c.h"
#include "segcore/packed_writer_c.h"
*/
@ -25,6 +26,7 @@ import "C"
import (
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/arrio"
"github.com/apache/arrow/go/v17/arrow/cdata"
)
@ -32,6 +34,12 @@ type PackedWriter struct {
cPackedWriter C.CPackedWriter
}
type FFIPackedWriter struct {
basePath string
cWriterHandle C.WriterHandle
cProperties *C.Properties
}
type PackedReader struct {
cPackedReader C.CPackedReader
arr *cdata.CArrowArray
@ -39,6 +47,12 @@ type PackedReader struct {
currentBatch arrow.Record
}
type FFIPackedReader struct {
cPackedReader C.CFFIPackedReader
recordReader arrio.Reader
schema *arrow.Schema
}
type (
// CArrowSchema is the C Data Interface for ArrowSchemas
CArrowSchema = C.struct_ArrowSchema

View File

@ -23,6 +23,9 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/cgo"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
"github.com/milvus-io/milvus/pkg/v2/proto/segcorepb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
)
@ -42,6 +45,7 @@ type CreateCSegmentRequest struct {
SegmentID int64
SegmentType SegmentType
IsSorted bool
LoadInfo *querypb.SegmentLoadInfo
}
func (req *CreateCSegmentRequest) getCSegmentType() C.SegmentType {
@ -60,7 +64,18 @@ func (req *CreateCSegmentRequest) getCSegmentType() C.SegmentType {
// CreateCSegment creates a segment from a CreateCSegmentRequest.
func CreateCSegment(req *CreateCSegmentRequest) (CSegment, error) {
var ptr C.CSegmentInterface
status := C.NewSegment(req.Collection.rawPointer(), req.getCSegmentType(), C.int64_t(req.SegmentID), &ptr, C.bool(req.IsSorted))
var status C.CStatus
if req.LoadInfo != nil {
segLoadInfo := ConvertToSegcoreSegmentLoadInfo(req.LoadInfo)
loadInfoBlob, err := proto.Marshal(segLoadInfo)
if err != nil {
return nil, err
}
status = C.NewSegmentWithLoadInfo(req.Collection.rawPointer(), req.getCSegmentType(), C.int64_t(req.SegmentID), &ptr, C.bool(req.IsSorted), (*C.uint8_t)(unsafe.Pointer(&loadInfoBlob[0])), C.int64_t(len(loadInfoBlob)))
} else {
status = C.NewSegment(req.Collection.rawPointer(), req.getCSegmentType(), C.int64_t(req.SegmentID), &ptr, C.bool(req.IsSorted))
}
if err := ConsumeCStatusIntoError(&status); err != nil {
return nil, err
}
@ -297,6 +312,13 @@ func (s *cSegmentImpl) FinishLoad() error {
return nil
}
func (s *cSegmentImpl) Load(ctx context.Context) error {
traceCtx := ParseCTraceContext(ctx)
defer runtime.KeepAlive(traceCtx)
status := C.SegmentLoad(traceCtx.ctx, s.ptr)
return ConsumeCStatusIntoError(&status)
}
func (s *cSegmentImpl) DropIndex(ctx context.Context, fieldID int64) error {
status := C.DropSealedSegmentIndex(s.ptr, C.int64_t(fieldID))
if err := ConsumeCStatusIntoError(&status); err != nil {
@ -317,3 +339,161 @@ func (s *cSegmentImpl) DropJSONIndex(ctx context.Context, fieldID int64, nestedP
func (s *cSegmentImpl) Release() {
C.DeleteSegment(s.ptr)
}
// ConvertToSegcoreSegmentLoadInfo converts querypb.SegmentLoadInfo to segcorepb.SegmentLoadInfo.
// This function is needed because segcorepb.SegmentLoadInfo is a simplified version that doesn't
// depend on data_coord.proto and excludes fields like start_position, delta_position, and level.
func ConvertToSegcoreSegmentLoadInfo(src *querypb.SegmentLoadInfo) *segcorepb.SegmentLoadInfo {
if src == nil {
return nil
}
return &segcorepb.SegmentLoadInfo{
SegmentID: src.GetSegmentID(),
PartitionID: src.GetPartitionID(),
CollectionID: src.GetCollectionID(),
DbID: src.GetDbID(),
FlushTime: src.GetFlushTime(),
BinlogPaths: convertFieldBinlogs(src.GetBinlogPaths()),
NumOfRows: src.GetNumOfRows(),
Statslogs: convertFieldBinlogs(src.GetStatslogs()),
Deltalogs: convertFieldBinlogs(src.GetDeltalogs()),
CompactionFrom: src.GetCompactionFrom(),
IndexInfos: convertFieldIndexInfos(src.GetIndexInfos()),
SegmentSize: src.GetSegmentSize(),
InsertChannel: src.GetInsertChannel(),
ReadableVersion: src.GetReadableVersion(),
StorageVersion: src.GetStorageVersion(),
IsSorted: src.GetIsSorted(),
TextStatsLogs: convertTextIndexStats(src.GetTextStatsLogs()),
Bm25Logs: convertFieldBinlogs(src.GetBm25Logs()),
JsonKeyStatsLogs: convertJSONKeyStats(src.GetJsonKeyStatsLogs()),
Priority: src.GetPriority(),
ManifestPath: src.GetManifestPath(),
}
}
// convertFieldBinlogs converts datapb.FieldBinlog to segcorepb.FieldBinlog.
func convertFieldBinlogs(src []*datapb.FieldBinlog) []*segcorepb.FieldBinlog {
if src == nil {
return nil
}
result := make([]*segcorepb.FieldBinlog, 0, len(src))
for _, fb := range src {
if fb == nil {
continue
}
result = append(result, &segcorepb.FieldBinlog{
FieldID: fb.GetFieldID(),
Binlogs: convertBinlogs(fb.GetBinlogs()),
ChildFields: fb.GetChildFields(),
})
}
return result
}
// convertBinlogs converts datapb.Binlog to segcorepb.Binlog.
func convertBinlogs(src []*datapb.Binlog) []*segcorepb.Binlog {
if src == nil {
return nil
}
result := make([]*segcorepb.Binlog, 0, len(src))
for _, b := range src {
if b == nil {
continue
}
result = append(result, &segcorepb.Binlog{
EntriesNum: b.GetEntriesNum(),
TimestampFrom: b.GetTimestampFrom(),
TimestampTo: b.GetTimestampTo(),
LogPath: b.GetLogPath(),
LogSize: b.GetLogSize(),
LogID: b.GetLogID(),
MemorySize: b.GetMemorySize(),
})
}
return result
}
// convertFieldIndexInfos converts querypb.FieldIndexInfo to segcorepb.FieldIndexInfo.
func convertFieldIndexInfos(src []*querypb.FieldIndexInfo) []*segcorepb.FieldIndexInfo {
if src == nil {
return nil
}
result := make([]*segcorepb.FieldIndexInfo, 0, len(src))
for _, fii := range src {
if fii == nil {
continue
}
result = append(result, &segcorepb.FieldIndexInfo{
FieldID: fii.GetFieldID(),
EnableIndex: fii.GetEnableIndex(),
IndexName: fii.GetIndexName(),
IndexID: fii.GetIndexID(),
BuildID: fii.GetBuildID(),
IndexParams: fii.GetIndexParams(),
IndexFilePaths: fii.GetIndexFilePaths(),
IndexSize: fii.GetIndexSize(),
IndexVersion: fii.GetIndexVersion(),
NumRows: fii.GetNumRows(),
CurrentIndexVersion: fii.GetCurrentIndexVersion(),
IndexStoreVersion: fii.GetIndexStoreVersion(),
})
}
return result
}
// convertTextIndexStats converts datapb.TextIndexStats to segcorepb.TextIndexStats.
func convertTextIndexStats(src map[int64]*datapb.TextIndexStats) map[int64]*segcorepb.TextIndexStats {
if src == nil {
return nil
}
result := make(map[int64]*segcorepb.TextIndexStats, len(src))
for k, v := range src {
if v == nil {
continue
}
result[k] = &segcorepb.TextIndexStats{
FieldID: v.GetFieldID(),
Version: v.GetVersion(),
Files: v.GetFiles(),
LogSize: v.GetLogSize(),
MemorySize: v.GetMemorySize(),
BuildID: v.GetBuildID(),
}
}
return result
}
// convertJSONKeyStats converts datapb.JsonKeyStats to segcorepb.JsonKeyStats.
func convertJSONKeyStats(src map[int64]*datapb.JsonKeyStats) map[int64]*segcorepb.JsonKeyStats {
if src == nil {
return nil
}
result := make(map[int64]*segcorepb.JsonKeyStats, len(src))
for k, v := range src {
if v == nil {
continue
}
result[k] = &segcorepb.JsonKeyStats{
FieldID: v.GetFieldID(),
Version: v.GetVersion(),
Files: v.GetFiles(),
LogSize: v.GetLogSize(),
MemorySize: v.GetMemorySize(),
BuildID: v.GetBuildID(),
JsonKeyStatsDataFormat: v.GetJsonKeyStatsDataFormat(),
}
}
return result
}

View File

@ -80,6 +80,9 @@ type basicSegmentMethodSet interface {
// FinishLoad wraps up the load process and let segcore do the leftover jobs.
FinishLoad() error
// Load invokes segment managed loading.
Load(ctx context.Context) error
// Release releases the segment.
Release()
}

View File

@ -8,12 +8,15 @@ import (
"github.com/stretchr/testify/assert"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/mocks/util/mock_segcore"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/initcore"
"github.com/milvus-io/milvus/internal/util/segcore"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/planpb"
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
"github.com/milvus-io/milvus/pkg/v2/proto/segcorepb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
@ -139,3 +142,260 @@ func assertEqualCount(
assert.NotNil(t, retrieveResult2)
retrieveResult2.Release()
}
func TestConvertToSegcoreSegmentLoadInfo(t *testing.T) {
t.Run("nil input", func(t *testing.T) {
result := segcore.ConvertToSegcoreSegmentLoadInfo(nil)
assert.Nil(t, result)
})
t.Run("empty input", func(t *testing.T) {
src := &querypb.SegmentLoadInfo{}
result := segcore.ConvertToSegcoreSegmentLoadInfo(src)
assert.NotNil(t, result)
assert.Equal(t, int64(0), result.SegmentID)
assert.Equal(t, int64(0), result.PartitionID)
assert.Equal(t, int64(0), result.CollectionID)
})
t.Run("full conversion", func(t *testing.T) {
// Create source querypb.SegmentLoadInfo with all fields populated
src := &querypb.SegmentLoadInfo{
SegmentID: 1001,
PartitionID: 2001,
CollectionID: 3001,
DbID: 4001,
FlushTime: 5001,
BinlogPaths: []*datapb.FieldBinlog{
{
FieldID: 100,
Binlogs: []*datapb.Binlog{
{
EntriesNum: 10,
TimestampFrom: 1000,
TimestampTo: 2000,
LogPath: "/path/to/binlog",
LogSize: 1024,
LogID: 9001,
MemorySize: 2048,
},
},
ChildFields: []int64{101, 102},
},
},
NumOfRows: 1000,
Statslogs: []*datapb.FieldBinlog{
{
FieldID: 200,
Binlogs: []*datapb.Binlog{
{
EntriesNum: 5,
TimestampFrom: 1500,
TimestampTo: 2500,
LogPath: "/path/to/statslog",
LogSize: 512,
LogID: 9002,
MemorySize: 1024,
},
},
},
},
Deltalogs: []*datapb.FieldBinlog{
{
FieldID: 300,
Binlogs: []*datapb.Binlog{
{
EntriesNum: 3,
TimestampFrom: 2000,
TimestampTo: 3000,
LogPath: "/path/to/deltalog",
LogSize: 256,
LogID: 9003,
MemorySize: 512,
},
},
},
},
CompactionFrom: []int64{8001, 8002},
IndexInfos: []*querypb.FieldIndexInfo{
{
FieldID: 100,
EnableIndex: true,
IndexName: "test_index",
IndexID: 7001,
BuildID: 7002,
IndexParams: []*commonpb.KeyValuePair{{Key: "index_type", Value: "HNSW"}},
IndexFilePaths: []string{"/path/to/index"},
IndexSize: 4096,
IndexVersion: 1,
NumRows: 1000,
CurrentIndexVersion: 2,
IndexStoreVersion: 3,
},
},
SegmentSize: 8192,
InsertChannel: "insert_channel_1",
ReadableVersion: 6001,
StorageVersion: 7001,
IsSorted: true,
TextStatsLogs: map[int64]*datapb.TextIndexStats{
400: {
FieldID: 400,
Version: 1,
Files: []string{"/path/to/text/stats1", "/path/to/text/stats2"},
LogSize: 2048,
MemorySize: 4096,
BuildID: 9101,
},
},
Bm25Logs: []*datapb.FieldBinlog{
{
FieldID: 500,
Binlogs: []*datapb.Binlog{
{
EntriesNum: 7,
TimestampFrom: 3000,
TimestampTo: 4000,
LogPath: "/path/to/bm25log",
LogSize: 768,
LogID: 9004,
MemorySize: 1536,
},
},
},
},
JsonKeyStatsLogs: map[int64]*datapb.JsonKeyStats{
600: {
FieldID: 600,
Version: 2,
Files: []string{"/path/to/json/stats"},
LogSize: 1024,
MemorySize: 2048,
BuildID: 9201,
JsonKeyStatsDataFormat: 1,
},
},
Priority: commonpb.LoadPriority_HIGH,
}
// Convert to segcorepb.SegmentLoadInfo
result := segcore.ConvertToSegcoreSegmentLoadInfo(src)
// Validate basic fields
assert.NotNil(t, result)
assert.Equal(t, src.SegmentID, result.SegmentID)
assert.Equal(t, src.PartitionID, result.PartitionID)
assert.Equal(t, src.CollectionID, result.CollectionID)
assert.Equal(t, src.DbID, result.DbID)
assert.Equal(t, src.FlushTime, result.FlushTime)
assert.Equal(t, src.NumOfRows, result.NumOfRows)
assert.Equal(t, src.SegmentSize, result.SegmentSize)
assert.Equal(t, src.InsertChannel, result.InsertChannel)
assert.Equal(t, src.ReadableVersion, result.ReadableVersion)
assert.Equal(t, src.StorageVersion, result.StorageVersion)
assert.Equal(t, src.IsSorted, result.IsSorted)
assert.Equal(t, src.Priority, result.Priority)
assert.Equal(t, src.CompactionFrom, result.CompactionFrom)
// Validate BinlogPaths conversion
assert.Equal(t, len(src.BinlogPaths), len(result.BinlogPaths))
assert.Equal(t, src.BinlogPaths[0].FieldID, result.BinlogPaths[0].FieldID)
assert.Equal(t, len(src.BinlogPaths[0].Binlogs), len(result.BinlogPaths[0].Binlogs))
assert.Equal(t, src.BinlogPaths[0].Binlogs[0].EntriesNum, result.BinlogPaths[0].Binlogs[0].EntriesNum)
assert.Equal(t, src.BinlogPaths[0].Binlogs[0].TimestampFrom, result.BinlogPaths[0].Binlogs[0].TimestampFrom)
assert.Equal(t, src.BinlogPaths[0].Binlogs[0].TimestampTo, result.BinlogPaths[0].Binlogs[0].TimestampTo)
assert.Equal(t, src.BinlogPaths[0].Binlogs[0].LogPath, result.BinlogPaths[0].Binlogs[0].LogPath)
assert.Equal(t, src.BinlogPaths[0].Binlogs[0].LogSize, result.BinlogPaths[0].Binlogs[0].LogSize)
assert.Equal(t, src.BinlogPaths[0].Binlogs[0].LogID, result.BinlogPaths[0].Binlogs[0].LogID)
assert.Equal(t, src.BinlogPaths[0].Binlogs[0].MemorySize, result.BinlogPaths[0].Binlogs[0].MemorySize)
assert.Equal(t, src.BinlogPaths[0].ChildFields, result.BinlogPaths[0].ChildFields)
// Validate Statslogs conversion
assert.Equal(t, len(src.Statslogs), len(result.Statslogs))
assert.Equal(t, src.Statslogs[0].FieldID, result.Statslogs[0].FieldID)
// Validate Deltalogs conversion
assert.Equal(t, len(src.Deltalogs), len(result.Deltalogs))
assert.Equal(t, src.Deltalogs[0].FieldID, result.Deltalogs[0].FieldID)
// Validate IndexInfos conversion
assert.Equal(t, len(src.IndexInfos), len(result.IndexInfos))
assert.Equal(t, src.IndexInfos[0].FieldID, result.IndexInfos[0].FieldID)
assert.Equal(t, src.IndexInfos[0].EnableIndex, result.IndexInfos[0].EnableIndex)
assert.Equal(t, src.IndexInfos[0].IndexName, result.IndexInfos[0].IndexName)
assert.Equal(t, src.IndexInfos[0].IndexID, result.IndexInfos[0].IndexID)
assert.Equal(t, src.IndexInfos[0].BuildID, result.IndexInfos[0].BuildID)
assert.Equal(t, len(src.IndexInfos[0].IndexParams), len(result.IndexInfos[0].IndexParams))
assert.Equal(t, src.IndexInfos[0].IndexFilePaths, result.IndexInfos[0].IndexFilePaths)
assert.Equal(t, src.IndexInfos[0].IndexSize, result.IndexInfos[0].IndexSize)
assert.Equal(t, src.IndexInfos[0].IndexVersion, result.IndexInfos[0].IndexVersion)
assert.Equal(t, src.IndexInfos[0].NumRows, result.IndexInfos[0].NumRows)
assert.Equal(t, src.IndexInfos[0].CurrentIndexVersion, result.IndexInfos[0].CurrentIndexVersion)
assert.Equal(t, src.IndexInfos[0].IndexStoreVersion, result.IndexInfos[0].IndexStoreVersion)
// Validate TextStatsLogs conversion
assert.Equal(t, len(src.TextStatsLogs), len(result.TextStatsLogs))
textStats := result.TextStatsLogs[400]
assert.NotNil(t, textStats)
assert.Equal(t, src.TextStatsLogs[400].FieldID, textStats.FieldID)
assert.Equal(t, src.TextStatsLogs[400].Version, textStats.Version)
assert.Equal(t, src.TextStatsLogs[400].Files, textStats.Files)
assert.Equal(t, src.TextStatsLogs[400].LogSize, textStats.LogSize)
assert.Equal(t, src.TextStatsLogs[400].MemorySize, textStats.MemorySize)
assert.Equal(t, src.TextStatsLogs[400].BuildID, textStats.BuildID)
// Validate Bm25Logs conversion
assert.Equal(t, len(src.Bm25Logs), len(result.Bm25Logs))
assert.Equal(t, src.Bm25Logs[0].FieldID, result.Bm25Logs[0].FieldID)
// Validate JsonKeyStatsLogs conversion
assert.Equal(t, len(src.JsonKeyStatsLogs), len(result.JsonKeyStatsLogs))
jsonStats := result.JsonKeyStatsLogs[600]
assert.NotNil(t, jsonStats)
assert.Equal(t, src.JsonKeyStatsLogs[600].FieldID, jsonStats.FieldID)
assert.Equal(t, src.JsonKeyStatsLogs[600].Version, jsonStats.Version)
assert.Equal(t, src.JsonKeyStatsLogs[600].Files, jsonStats.Files)
assert.Equal(t, src.JsonKeyStatsLogs[600].LogSize, jsonStats.LogSize)
assert.Equal(t, src.JsonKeyStatsLogs[600].MemorySize, jsonStats.MemorySize)
assert.Equal(t, src.JsonKeyStatsLogs[600].BuildID, jsonStats.BuildID)
assert.Equal(t, src.JsonKeyStatsLogs[600].JsonKeyStatsDataFormat, jsonStats.JsonKeyStatsDataFormat)
})
t.Run("nil elements in arrays and maps", func(t *testing.T) {
src := &querypb.SegmentLoadInfo{
SegmentID: 1001,
BinlogPaths: []*datapb.FieldBinlog{
nil, // nil element should be skipped
{FieldID: 100},
},
Statslogs: []*datapb.FieldBinlog{
nil,
},
IndexInfos: []*querypb.FieldIndexInfo{
nil,
{FieldID: 200},
},
TextStatsLogs: map[int64]*datapb.TextIndexStats{
100: nil, // nil value should be skipped
200: {FieldID: 200},
},
JsonKeyStatsLogs: map[int64]*datapb.JsonKeyStats{
300: nil,
400: {FieldID: 400},
},
}
result := segcore.ConvertToSegcoreSegmentLoadInfo(src)
assert.NotNil(t, result)
assert.Equal(t, 1, len(result.BinlogPaths))
assert.Equal(t, int64(100), result.BinlogPaths[0].FieldID)
assert.Equal(t, 0, len(result.Statslogs))
assert.Equal(t, 1, len(result.IndexInfos))
assert.Equal(t, int64(200), result.IndexInfos[0].FieldID)
assert.Equal(t, 1, len(result.TextStatsLogs))
assert.NotNil(t, result.TextStatsLogs[200])
assert.Equal(t, 1, len(result.JsonKeyStatsLogs))
assert.NotNil(t, result.JsonKeyStatsLogs[400])
})
}

View File

@ -419,6 +419,12 @@ message SegmentInfo {
// A segment generated by datacoord of old arch, will be false.
// After the growing segment is full managed by streamingnode, the true value can never be seen at coordinator.
bool is_created_by_streaming = 30;
bool is_partition_key_sorted = 31;
// manifest_path stores the fullpath of LOON manifest file of segemnt data files.
// we could keep the fullpath since one segment shall only have one active manifest
// and we could keep the possiblity that manifest stores out side of collection/partition/segment path
string manifest_path = 32;
}
message SegmentStartPosition {
@ -444,6 +450,7 @@ message SaveBinlogPathsRequest {
int64 storageVersion = 15;
repeated FieldBinlog field2Bm25logPaths = 16;
bool with_full_binlogs = 17; // report with full data for verification.
string manifest_path = 18; //
}
message CheckPoint {
@ -673,6 +680,7 @@ message CompactionSegmentBinlogs {
int64 partitionID = 8;
bool is_sorted = 9;
int64 storage_version = 10;
string manifest = 11;
}
message CompactionPlan {
@ -716,6 +724,7 @@ message CompactionSegment {
repeated FieldBinlog bm25logs = 9;
int64 storage_version = 10;
map<int64, data.TextIndexStats> text_stats_logs = 11;
string manifest = 12;
}
message CompactionPlanResult {

File diff suppressed because it is too large Load Diff

View File

@ -101,6 +101,7 @@ message BuildIndexInfo {
int64 json_stats_max_shredding_columns = 27;
double json_stats_shredding_ratio_threshold = 28;
int64 json_stats_write_batch_size = 29;
string manifest = 30;
}
message StoragePluginContext {

View File

@ -691,6 +691,7 @@ type BuildIndexInfo struct {
JsonStatsMaxShreddingColumns int64 `protobuf:"varint,27,opt,name=json_stats_max_shredding_columns,json=jsonStatsMaxShreddingColumns,proto3" json:"json_stats_max_shredding_columns,omitempty"`
JsonStatsShreddingRatioThreshold float64 `protobuf:"fixed64,28,opt,name=json_stats_shredding_ratio_threshold,json=jsonStatsShreddingRatioThreshold,proto3" json:"json_stats_shredding_ratio_threshold,omitempty"`
JsonStatsWriteBatchSize int64 `protobuf:"varint,29,opt,name=json_stats_write_batch_size,json=jsonStatsWriteBatchSize,proto3" json:"json_stats_write_batch_size,omitempty"`
Manifest string `protobuf:"bytes,30,opt,name=manifest,proto3" json:"manifest,omitempty"`
}
func (x *BuildIndexInfo) Reset() {
@ -928,6 +929,13 @@ func (x *BuildIndexInfo) GetJsonStatsWriteBatchSize() int64 {
return 0
}
func (x *BuildIndexInfo) GetManifest() string {
if x != nil {
return x.Manifest
}
return ""
}
type StoragePluginContext struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -1331,7 +1339,7 @@ var file_index_cgo_msg_proto_rawDesc = []byte{
0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x64, 0x61, 0x74, 0x61, 0x50, 0x61, 0x74, 0x68,
0x73, 0x12, 0x21, 0x0a, 0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70,
0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74,
0x54, 0x79, 0x70, 0x65, 0x22, 0xf8, 0x0b, 0x0a, 0x0e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x6e,
0x54, 0x79, 0x70, 0x65, 0x22, 0x94, 0x0c, 0x0a, 0x0e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x6e,
0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74,
0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73,
0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44,
@ -1426,71 +1434,73 @@ var file_index_cgo_msg_proto_rawDesc = []byte{
0x64, 0x12, 0x3c, 0x0a, 0x1b, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f,
0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65,
0x18, 0x1d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74,
0x73, 0x57, 0x72, 0x69, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22,
0x90, 0x01, 0x0a, 0x14, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x50, 0x6c, 0x75, 0x67, 0x69,
0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x2c, 0x0a, 0x12, 0x65, 0x6e, 0x63, 0x72,
0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x7a, 0x6f, 0x6e, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01,
0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e,
0x5a, 0x6f, 0x6e, 0x65, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x65,
0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20,
0x01, 0x28, 0x09, 0x52, 0x0d, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4b,
0x65, 0x79, 0x22, 0xff, 0x02, 0x0a, 0x11, 0x4c, 0x6f, 0x61, 0x64, 0x54, 0x65, 0x78, 0x74, 0x49,
0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x46, 0x69, 0x65, 0x6c,
0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x46, 0x69, 0x65, 0x6c, 0x64,
0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20,
0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07,
0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62,
0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18,
0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x06,
0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65,
0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06,
0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f,
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61,
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52,
0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x46, 0x0a, 0x0d,
0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x08, 0x20,
0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x72,
0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x52, 0x0c, 0x6c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f,
0x72, 0x69, 0x74, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6d,
0x6d, 0x61, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c,
0x65, 0x4d, 0x6d, 0x61, 0x70, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73,
0x69, 0x7a, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x53, 0x69, 0x7a, 0x65, 0x22, 0xa6, 0x03, 0x0a, 0x14, 0x4c, 0x6f, 0x61, 0x64, 0x4a, 0x73, 0x6f,
0x6e, 0x4b, 0x65, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a,
0x07, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07,
0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69,
0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f,
0x6e, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01,
0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x66,
0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65,
0x73, 0x12, 0x38, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68,
0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x22, 0x0a, 0x0c, 0x63,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28,
0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12,
0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x07,
0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49,
0x44, 0x12, 0x46, 0x0a, 0x0d, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69,
0x74, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c,
0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x52, 0x0c, 0x6c, 0x6f, 0x61,
0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61,
0x62, 0x6c, 0x65, 0x5f, 0x6d, 0x6d, 0x61, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a,
0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x6d, 0x61, 0x70, 0x12, 0x22, 0x0a, 0x0d, 0x6d, 0x6d,
0x61, 0x70, 0x5f, 0x64, 0x69, 0x72, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0a, 0x20, 0x01, 0x28,
0x09, 0x52, 0x0b, 0x6d, 0x6d, 0x61, 0x70, 0x44, 0x69, 0x72, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1d,
0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0b, 0x20, 0x01,
0x28, 0x03, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x73, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x35, 0x5a,
0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67,
0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63,
0x67, 0x6f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x73, 0x57, 0x72, 0x69, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12,
0x1a, 0x0a, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x1e, 0x20, 0x01, 0x28,
0x09, 0x52, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x22, 0x90, 0x01, 0x0a, 0x14,
0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e,
0x74, 0x65, 0x78, 0x74, 0x12, 0x2c, 0x0a, 0x12, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x69,
0x6f, 0x6e, 0x5f, 0x7a, 0x6f, 0x6e, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03,
0x52, 0x10, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5a, 0x6f, 0x6e, 0x65,
0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x6e, 0x63, 0x72, 0x79,
0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52,
0x0d, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x22, 0xff,
0x02, 0x0a, 0x11, 0x4c, 0x6f, 0x61, 0x64, 0x54, 0x65, 0x78, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78,
0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18,
0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x18,
0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52,
0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c,
0x64, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64,
0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28,
0x09, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65,
0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x46,
0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65,
0x6d, 0x61, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72,
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x46, 0x0a, 0x0d, 0x6c, 0x6f, 0x61, 0x64,
0x5f, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32,
0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63,
0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69,
0x74, 0x79, 0x52, 0x0c, 0x6c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79,
0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6d, 0x6d, 0x61, 0x70, 0x18,
0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x6d, 0x61,
0x70, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18,
0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x69, 0x7a, 0x65,
0x22, 0xa6, 0x03, 0x0a, 0x14, 0x4c, 0x6f, 0x61, 0x64, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65, 0x79,
0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x46, 0x69, 0x65,
0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x46, 0x69, 0x65, 0x6c,
0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02,
0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a,
0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07,
0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73,
0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x38, 0x0a,
0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68,
0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52,
0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03,
0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x46, 0x0a,
0x0d, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x08,
0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50,
0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x52, 0x0c, 0x6c, 0x6f, 0x61, 0x64, 0x50, 0x72, 0x69,
0x6f, 0x72, 0x69, 0x74, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f,
0x6d, 0x6d, 0x61, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x65, 0x6e, 0x61, 0x62,
0x6c, 0x65, 0x4d, 0x6d, 0x61, 0x70, 0x12, 0x22, 0x0a, 0x0d, 0x6d, 0x6d, 0x61, 0x70, 0x5f, 0x64,
0x69, 0x72, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6d,
0x6d, 0x61, 0x70, 0x44, 0x69, 0x72, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74,
0x61, 0x74, 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09,
0x73, 0x74, 0x61, 0x74, 0x73, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74,
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69,
0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x63, 0x67, 0x6f, 0x70, 0x62,
0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

@ -381,6 +381,7 @@ message SegmentLoadInfo {
repeated data.FieldBinlog bm25logs = 21;
map<int64, data.JsonKeyStats> jsonKeyStatsLogs = 22;
common.LoadPriority priority = 23;
string manifest_path = 24;
}
message FieldIndexInfo {

Some files were not shown because too many files have changed in this diff Show More