mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 09:38:39 +08:00
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>
688 lines
23 KiB
C++
688 lines
23 KiB
C++
// 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
|
|
|
|
#ifndef MILVUS_SEGCORE_SEGMENT_INTERFACE_H_
|
|
#define MILVUS_SEGCORE_SEGMENT_INTERFACE_H_
|
|
|
|
#include <atomic>
|
|
#include <memory>
|
|
#include <shared_mutex>
|
|
#include <string>
|
|
#include <type_traits>
|
|
#include <utility>
|
|
#include <vector>
|
|
#include <index/ScalarIndex.h>
|
|
|
|
#include "cachinglayer/CacheSlot.h"
|
|
#include "common/EasyAssert.h"
|
|
#include "common/Json.h"
|
|
#include "common/OpContext.h"
|
|
#include "common/Schema.h"
|
|
#include "common/Span.h"
|
|
#include "common/SystemProperty.h"
|
|
#include "common/Types.h"
|
|
#include "common/LoadInfo.h"
|
|
#include "common/BitsetView.h"
|
|
#include "common/QueryResult.h"
|
|
#include "common/QueryInfo.h"
|
|
#include "folly/SharedMutex.h"
|
|
#include "common/type_c.h"
|
|
#include "mmap/ChunkedColumnInterface.h"
|
|
#include "index/Index.h"
|
|
#include "index/JsonFlatIndex.h"
|
|
#include "query/Plan.h"
|
|
#include "pb/segcore.pb.h"
|
|
#include "index/SkipIndex.h"
|
|
#include "index/TextMatchIndex.h"
|
|
#include "segcore/ConcurrentVector.h"
|
|
#include "segcore/InsertRecord.h"
|
|
#include "index/NgramInvertedIndex.h"
|
|
#include "index/json_stats/JsonKeyStats.h"
|
|
|
|
namespace milvus::segcore {
|
|
|
|
using namespace milvus::cachinglayer;
|
|
|
|
struct SegmentStats {
|
|
// we stat the memory size used by the segment,
|
|
// including the insert data and delete data.
|
|
std::atomic<size_t> mem_size{};
|
|
};
|
|
|
|
// common interface of SegmentSealed and SegmentGrowing used by C API
|
|
class SegmentInterface {
|
|
public:
|
|
virtual ~SegmentInterface() = default;
|
|
|
|
virtual void
|
|
FillPrimaryKeys(const query::Plan* plan, SearchResult& results) const = 0;
|
|
|
|
virtual void
|
|
FillTargetEntry(const query::Plan* plan, SearchResult& results) const = 0;
|
|
|
|
virtual bool
|
|
Contain(const PkType& pk) const = 0;
|
|
|
|
virtual std::unique_ptr<SearchResult>
|
|
Search(const query::Plan* Plan,
|
|
const query::PlaceholderGroup* placeholder_group,
|
|
Timestamp timestamp,
|
|
const folly::CancellationToken& cancel_token,
|
|
int32_t consistency_level,
|
|
Timestamp collection_ttl) const = 0;
|
|
|
|
// Only used for test
|
|
std::unique_ptr<SearchResult>
|
|
Search(const query::Plan* Plan,
|
|
const query::PlaceholderGroup* placeholder_group,
|
|
Timestamp timestamp) const {
|
|
return Search(Plan,
|
|
placeholder_group,
|
|
timestamp,
|
|
folly::CancellationToken(),
|
|
0,
|
|
0);
|
|
}
|
|
|
|
virtual std::unique_ptr<proto::segcore::RetrieveResults>
|
|
Retrieve(tracer::TraceContext* trace_ctx,
|
|
const query::RetrievePlan* Plan,
|
|
Timestamp timestamp,
|
|
int64_t limit_size,
|
|
bool ignore_non_pk,
|
|
const folly::CancellationToken& cancel_token,
|
|
int32_t consistency_level,
|
|
Timestamp collection_ttl) const = 0;
|
|
|
|
// Only used for test
|
|
std::unique_ptr<proto::segcore::RetrieveResults>
|
|
Retrieve(tracer::TraceContext* trace_ctx,
|
|
const query::RetrievePlan* Plan,
|
|
Timestamp timestamp,
|
|
int64_t limit_size,
|
|
bool ignore_non_pk) const {
|
|
return Retrieve(trace_ctx,
|
|
Plan,
|
|
timestamp,
|
|
limit_size,
|
|
ignore_non_pk,
|
|
folly::CancellationToken(),
|
|
0,
|
|
0);
|
|
}
|
|
|
|
virtual std::unique_ptr<proto::segcore::RetrieveResults>
|
|
Retrieve(tracer::TraceContext* trace_ctx,
|
|
const query::RetrievePlan* Plan,
|
|
const int64_t* offsets,
|
|
int64_t size) const = 0;
|
|
|
|
virtual size_t
|
|
GetMemoryUsageInBytes() const = 0;
|
|
|
|
virtual int64_t
|
|
get_row_count() const = 0;
|
|
|
|
virtual const Schema&
|
|
get_schema() const = 0;
|
|
|
|
virtual int64_t
|
|
get_deleted_count() const = 0;
|
|
|
|
virtual int64_t
|
|
get_real_count() const = 0;
|
|
|
|
virtual int64_t
|
|
get_field_avg_size(FieldId field_id) const = 0;
|
|
|
|
virtual void
|
|
set_field_avg_size(FieldId field_id,
|
|
int64_t num_rows,
|
|
int64_t field_size) = 0;
|
|
|
|
virtual SegcoreError
|
|
Delete(int64_t size, const IdArray* pks, const Timestamp* timestamps) = 0;
|
|
|
|
virtual void
|
|
LoadDeletedRecord(const LoadDeletedRecordInfo& info) = 0;
|
|
|
|
virtual void
|
|
LoadFieldData(const LoadFieldDataInfo& info) = 0;
|
|
|
|
virtual int64_t
|
|
get_segment_id() const = 0;
|
|
|
|
virtual SegmentType
|
|
type() const = 0;
|
|
|
|
virtual bool
|
|
HasRawData(int64_t field_id) const = 0;
|
|
|
|
virtual bool
|
|
HasFieldData(FieldId field_id) const = 0;
|
|
|
|
virtual bool
|
|
is_nullable(FieldId field_id) const = 0;
|
|
|
|
virtual void
|
|
CreateTextIndex(FieldId field_id) = 0;
|
|
|
|
virtual PinWrapper<index::TextMatchIndex*>
|
|
GetTextIndex(milvus::OpContext* op_ctx, FieldId field_id) const = 0;
|
|
|
|
virtual std::vector<PinWrapper<const index::IndexBase*>>
|
|
PinJsonIndex(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
const std::string& path,
|
|
DataType data_type,
|
|
bool any_type,
|
|
bool is_array) const {
|
|
return {};
|
|
}
|
|
|
|
virtual std::vector<PinWrapper<const index::IndexBase*>>
|
|
PinIndex(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
bool include_ngram) const {
|
|
return {};
|
|
};
|
|
|
|
std::vector<PinWrapper<const index::IndexBase*>>
|
|
PinIndex(milvus::OpContext* op_ctx, FieldId field_id) const {
|
|
return PinIndex(op_ctx, field_id, false);
|
|
}
|
|
|
|
virtual void
|
|
BulkGetJsonData(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
std::function<void(milvus::Json, size_t, bool)> fn,
|
|
const int64_t* offsets,
|
|
int64_t count) const = 0;
|
|
|
|
virtual PinWrapper<index::NgramInvertedIndex*>
|
|
GetNgramIndex(milvus::OpContext* op_ctx, FieldId field_id) const = 0;
|
|
|
|
virtual PinWrapper<index::NgramInvertedIndex*>
|
|
GetNgramIndexForJson(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
const std::string& nested_path) const = 0;
|
|
|
|
virtual PinWrapper<index::JsonKeyStats*>
|
|
GetJsonStats(milvus::OpContext* op_ctx, FieldId field_id) const = 0;
|
|
|
|
virtual void
|
|
LoadJsonStats(FieldId field_id, index::CacheJsonKeyStatsPtr cache_slot) = 0;
|
|
|
|
virtual void
|
|
RemoveJsonStats(FieldId field_id) = 0;
|
|
|
|
virtual void
|
|
LazyCheckSchema(SchemaPtr sch) = 0;
|
|
|
|
// reopen segment with new schema
|
|
virtual void
|
|
Reopen(SchemaPtr sch) = 0;
|
|
|
|
// FinishLoad notifies the segment that all load operation are done
|
|
// 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
|
|
// only for implementation
|
|
class SegmentInternalInterface : public SegmentInterface {
|
|
public:
|
|
virtual void
|
|
prefetch_chunks(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
const std::vector<int64_t>& chunk_ids) const {
|
|
// do nothing
|
|
}
|
|
|
|
template <typename T>
|
|
PinWrapper<Span<T>>
|
|
chunk_data(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id) const {
|
|
return chunk_data_impl(op_ctx, field_id, chunk_id)
|
|
.transform<Span<T>>([](SpanBase&& span_base) {
|
|
return static_cast<Span<T>>(span_base);
|
|
});
|
|
}
|
|
|
|
template <typename ViewType>
|
|
PinWrapper<std::pair<std::vector<ViewType>, FixedVector<bool>>>
|
|
chunk_view(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id,
|
|
std::optional<std::pair<int64_t, int64_t>> offset_len =
|
|
std::nullopt) const {
|
|
if constexpr (std::is_same_v<ViewType, std::string_view>) {
|
|
return chunk_string_view_impl(
|
|
op_ctx, field_id, chunk_id, offset_len);
|
|
} else if constexpr (std::is_same_v<ViewType, ArrayView>) {
|
|
return chunk_array_view_impl(
|
|
op_ctx, field_id, chunk_id, offset_len);
|
|
} else if constexpr (std::is_same_v<ViewType, VectorArrayView>) {
|
|
return chunk_vector_array_view_impl(
|
|
op_ctx, field_id, chunk_id, offset_len);
|
|
} else if constexpr (std::is_same_v<ViewType, Json>) {
|
|
auto pw =
|
|
chunk_string_view_impl(op_ctx, field_id, chunk_id, offset_len);
|
|
auto [string_views, valid_data] = pw.get();
|
|
std::vector<Json> res;
|
|
res.reserve(string_views.size());
|
|
for (const auto& str_view : string_views) {
|
|
res.emplace_back(Json(str_view));
|
|
}
|
|
return PinWrapper<
|
|
std::pair<std::vector<ViewType>, FixedVector<bool>>>(
|
|
pw, {std::move(res), std::move(valid_data)});
|
|
}
|
|
}
|
|
|
|
template <typename ViewType>
|
|
PinWrapper<std::pair<std::vector<ViewType>, FixedVector<bool>>>
|
|
get_batch_views(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id,
|
|
int64_t start_offset,
|
|
int64_t length) const {
|
|
if (this->type() == SegmentType::Growing) {
|
|
ThrowInfo(ErrorCode::Unsupported,
|
|
"get chunk views not supported for growing segment");
|
|
}
|
|
return chunk_view<ViewType>(
|
|
op_ctx, field_id, chunk_id, std::make_pair(start_offset, length));
|
|
}
|
|
|
|
template <typename ViewType>
|
|
PinWrapper<std::pair<std::vector<ViewType>, FixedVector<bool>>>
|
|
get_views_by_offsets(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id,
|
|
const FixedVector<int32_t>& offsets) const {
|
|
if (this->type() == SegmentType::Growing) {
|
|
ThrowInfo(ErrorCode::Unsupported,
|
|
"get chunk views not supported for growing segment");
|
|
}
|
|
if constexpr (std::is_same_v<ViewType, std::string_view>) {
|
|
return chunk_string_views_by_offsets(
|
|
op_ctx, field_id, chunk_id, offsets);
|
|
} else if constexpr (std::is_same_v<ViewType, Json>) {
|
|
auto pw = chunk_string_views_by_offsets(
|
|
op_ctx, field_id, chunk_id, offsets);
|
|
std::vector<ViewType> res;
|
|
res.reserve(pw.get().first.size());
|
|
for (const auto& view : pw.get().first) {
|
|
res.emplace_back(view);
|
|
}
|
|
return PinWrapper<
|
|
std::pair<std::vector<ViewType>, FixedVector<bool>>>(
|
|
{std::move(res), pw.get().second});
|
|
} else if constexpr (std::is_same_v<ViewType, ArrayView>) {
|
|
return chunk_array_views_by_offsets(
|
|
op_ctx, field_id, chunk_id, offsets);
|
|
}
|
|
}
|
|
|
|
// union(segment_id, field_id) as unique id
|
|
virtual std::string
|
|
GetUniqueFieldId(int64_t field_id) const {
|
|
return std::to_string(get_segment_id()) + "_" +
|
|
std::to_string(field_id);
|
|
}
|
|
|
|
// Bring in base class Search overloads to avoid name hiding
|
|
using SegmentInterface::Search;
|
|
|
|
std::unique_ptr<SearchResult>
|
|
Search(const query::Plan* Plan,
|
|
const query::PlaceholderGroup* placeholder_group,
|
|
Timestamp timestamp,
|
|
const folly::CancellationToken& cancel_token,
|
|
int32_t consistency_level,
|
|
Timestamp collection_ttl) const override;
|
|
|
|
void
|
|
FillPrimaryKeys(const query::Plan* plan,
|
|
SearchResult& results) const override;
|
|
|
|
void
|
|
FillTargetEntry(const query::Plan* plan,
|
|
SearchResult& results) const override;
|
|
|
|
// Bring in base class Retrieve overloads to avoid name hiding
|
|
using SegmentInterface::Retrieve;
|
|
|
|
std::unique_ptr<proto::segcore::RetrieveResults>
|
|
Retrieve(tracer::TraceContext* trace_ctx,
|
|
const query::RetrievePlan* Plan,
|
|
Timestamp timestamp,
|
|
int64_t limit_size,
|
|
bool ignore_non_pk,
|
|
const folly::CancellationToken& cancel_token,
|
|
int32_t consistency_level,
|
|
Timestamp collection_ttl) const override;
|
|
|
|
std::unique_ptr<proto::segcore::RetrieveResults>
|
|
Retrieve(tracer::TraceContext* trace_ctx,
|
|
const query::RetrievePlan* Plan,
|
|
const int64_t* offsets,
|
|
int64_t size) const override;
|
|
|
|
virtual bool
|
|
HasIndex(FieldId field_id) const = 0;
|
|
|
|
int64_t
|
|
get_real_count() const override;
|
|
|
|
int64_t
|
|
get_field_avg_size(FieldId field_id) const override;
|
|
|
|
void
|
|
set_field_avg_size(FieldId field_id,
|
|
int64_t num_rows,
|
|
int64_t field_size) override;
|
|
virtual bool
|
|
is_chunked() const {
|
|
return false;
|
|
}
|
|
|
|
const SkipIndex&
|
|
GetSkipIndex() const;
|
|
|
|
void
|
|
LoadSkipIndex(FieldId field_id,
|
|
DataType data_type,
|
|
std::shared_ptr<ChunkedColumnInterface> column) {
|
|
skip_index_.LoadSkip(get_segment_id(), field_id, data_type, column);
|
|
}
|
|
|
|
virtual DataType
|
|
GetFieldDataType(FieldId fieldId) const = 0;
|
|
|
|
PinWrapper<index::TextMatchIndex*>
|
|
GetTextIndex(milvus::OpContext* op_ctx, FieldId field_id) const override;
|
|
|
|
PinWrapper<index::NgramInvertedIndex*>
|
|
GetNgramIndex(milvus::OpContext* op_ctx, FieldId field_id) const override;
|
|
|
|
PinWrapper<index::NgramInvertedIndex*>
|
|
GetNgramIndexForJson(milvus::OpContext* op_ctx,
|
|
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
|
|
// of `query_offsets` is the number of queries in the search plus one (the first
|
|
// element in query_offsets is 0).
|
|
virtual void
|
|
vector_search(SearchInfo& search_info,
|
|
const void* query_data,
|
|
const size_t* query_offsets,
|
|
int64_t query_count,
|
|
Timestamp timestamp,
|
|
const BitsetView& bitset,
|
|
milvus::OpContext* op_context,
|
|
SearchResult& output) const = 0;
|
|
|
|
virtual void
|
|
mask_with_delete(BitsetTypeView& bitset,
|
|
int64_t ins_barrier,
|
|
Timestamp timestamp) const = 0;
|
|
|
|
// count of chunk that has raw data
|
|
virtual int64_t
|
|
num_chunk_data(FieldId field_id) const = 0;
|
|
|
|
virtual int64_t
|
|
num_rows_until_chunk(FieldId field_id, int64_t chunk_id) const = 0;
|
|
|
|
// bitset 1 means not hit. 0 means hit.
|
|
virtual void
|
|
mask_with_timestamps(BitsetTypeView& bitset_chunk,
|
|
Timestamp timestamp,
|
|
Timestamp collection_ttl) const = 0;
|
|
|
|
// count of chunks
|
|
virtual int64_t
|
|
num_chunk(FieldId field_id) const = 0;
|
|
|
|
virtual int64_t
|
|
chunk_size(FieldId field_id, int64_t chunk_id) const = 0;
|
|
|
|
virtual std::pair<int64_t, int64_t>
|
|
get_chunk_by_offset(FieldId field_id, int64_t offset) const = 0;
|
|
|
|
// element size in each chunk
|
|
virtual int64_t
|
|
size_per_chunk() const = 0;
|
|
|
|
virtual int64_t
|
|
get_active_count(Timestamp ts) const = 0;
|
|
|
|
/**
|
|
* search offset by possible pk values and mvcc timestamp
|
|
*
|
|
* @param bitset The final bitset after id array filtering,
|
|
* `false` means that the entity will be filtered out.
|
|
* @param id_array possible pk values
|
|
* this interface is used for internal expression calculation,
|
|
* so no need timestamp parameter, mvcc node prove the timestamp is already filtered.
|
|
*/
|
|
virtual void
|
|
search_ids(BitsetType& bitset, const IdArray& id_array) const = 0;
|
|
|
|
/**
|
|
* Apply timestamp filtering on bitset, the query can't see an entity whose
|
|
* timestamp is bigger than the timestamp of query.
|
|
*
|
|
* @param bitset The final bitset after scalar filtering and delta filtering,
|
|
* `false` means that the entity will be filtered out.
|
|
* @param timestamp The timestamp of query.
|
|
*/
|
|
void
|
|
timestamp_filter(BitsetType& bitset, Timestamp timestamp) const;
|
|
|
|
/**
|
|
* Apply timestamp filtering on bitset, the query can't see an entity whose
|
|
* timestamp is bigger than the timestamp of query. The passed offsets are
|
|
* all candidate entities.
|
|
*
|
|
* @param bitset The final bitset after scalar filtering and delta filtering,
|
|
* `true` means that the entity will be filtered out.
|
|
* @param offsets The segment offsets of all candidates.
|
|
* @param timestamp The timestamp of query.
|
|
*/
|
|
void
|
|
timestamp_filter(BitsetType& bitset,
|
|
const std::vector<int64_t>& offsets,
|
|
Timestamp timestamp) const;
|
|
|
|
/**
|
|
* Sort all candidates in ascending order, and then return the limit smallest.
|
|
* Bitset is used to check if the candidate will be filtered out. `false_filtered_out`
|
|
* determines how to filter out candidates. If `false_filtered_out` is true, we will
|
|
* filter all candidates whose related bit is false.
|
|
*
|
|
* @param limit
|
|
* @param bitset
|
|
* @param false_filtered_out
|
|
* @return All candidates offsets.
|
|
*/
|
|
virtual std::pair<std::vector<OffsetMap::OffsetType>, bool>
|
|
find_first(int64_t limit, const BitsetType& bitset) const = 0;
|
|
|
|
void
|
|
FillTargetEntry(
|
|
tracer::TraceContext* trace_ctx,
|
|
const query::RetrievePlan* plan,
|
|
const std::unique_ptr<proto::segcore::RetrieveResults>& results,
|
|
const int64_t* offsets,
|
|
int64_t size,
|
|
bool ignore_non_pk,
|
|
bool fill_ids) const;
|
|
|
|
// return whether field mmap or not
|
|
virtual bool
|
|
is_mmap_field(FieldId field_id) const = 0;
|
|
|
|
virtual std::unique_ptr<DataArray>
|
|
bulk_subscript_not_exist_field(const milvus::FieldMeta& field_meta,
|
|
int64_t count) const;
|
|
|
|
protected:
|
|
// todo: use an Unified struct for all type in growing/seal segment to store data and valid_data.
|
|
// internal API: return chunk_data in span
|
|
virtual PinWrapper<SpanBase>
|
|
chunk_data_impl(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id) const = 0;
|
|
|
|
// internal API: return chunk string views in vector
|
|
virtual PinWrapper<
|
|
std::pair<std::vector<std::string_view>, FixedVector<bool>>>
|
|
chunk_string_view_impl(
|
|
milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id,
|
|
std::optional<std::pair<int64_t, int64_t>> offset_len) const = 0;
|
|
|
|
virtual PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
|
|
chunk_array_view_impl(
|
|
milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id,
|
|
std::optional<std::pair<int64_t, int64_t>> offset_len) const = 0;
|
|
|
|
virtual PinWrapper<
|
|
std::pair<std::vector<VectorArrayView>, FixedVector<bool>>>
|
|
chunk_vector_array_view_impl(
|
|
milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id,
|
|
std::optional<std::pair<int64_t, int64_t>> offset_len) const = 0;
|
|
|
|
virtual PinWrapper<
|
|
std::pair<std::vector<std::string_view>, FixedVector<bool>>>
|
|
chunk_string_views_by_offsets(
|
|
milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id,
|
|
const FixedVector<int32_t>& offsets) const = 0;
|
|
|
|
virtual PinWrapper<std::pair<std::vector<ArrayView>, FixedVector<bool>>>
|
|
chunk_array_views_by_offsets(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
int64_t chunk_id,
|
|
const FixedVector<int32_t>& offsets) const = 0;
|
|
|
|
virtual void
|
|
check_search(const query::Plan* plan) const = 0;
|
|
|
|
virtual const ConcurrentVector<Timestamp>&
|
|
get_timestamps() const = 0;
|
|
|
|
public:
|
|
virtual bool
|
|
is_field_exist(FieldId field_id) const = 0;
|
|
// calculate output[i] = Vec[seg_offsets[i]}, where Vec binds to system_type
|
|
virtual void
|
|
bulk_subscript(milvus::OpContext* op_ctx,
|
|
SystemFieldType system_type,
|
|
const int64_t* seg_offsets,
|
|
int64_t count,
|
|
void* output) const = 0;
|
|
|
|
// calculate output[i] = Vec[seg_offsets[i]}, where Vec binds to field_offset
|
|
virtual std::unique_ptr<DataArray>
|
|
bulk_subscript(milvus::OpContext* op_ctx,
|
|
FieldId field_id,
|
|
const int64_t* seg_offsets,
|
|
int64_t count) const = 0;
|
|
|
|
virtual std::unique_ptr<DataArray>
|
|
bulk_subscript(
|
|
milvus::OpContext* op_ctx,
|
|
FieldId field_ids,
|
|
const int64_t* seg_offsets,
|
|
int64_t count,
|
|
const std::vector<std::string>& dynamic_field_names) const = 0;
|
|
|
|
virtual void
|
|
pk_range(milvus::OpContext* op_ctx,
|
|
proto::plan::OpType op,
|
|
const PkType& pk,
|
|
BitsetTypeView& bitset) const = 0;
|
|
|
|
virtual void
|
|
pk_binary_range(milvus::OpContext* op_ctx,
|
|
const PkType& lower_pk,
|
|
bool lower_inclusive,
|
|
const PkType& upper_pk,
|
|
bool upper_inclusive,
|
|
BitsetTypeView& bitset) const = 0;
|
|
|
|
virtual GEOSContextHandle_t
|
|
get_ctx() const {
|
|
return ctx_;
|
|
};
|
|
|
|
protected:
|
|
// 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>>
|
|
variable_fields_avg_size_; // bytes;
|
|
SkipIndex skip_index_;
|
|
|
|
// text-indexes used to do match.
|
|
std::unordered_map<
|
|
FieldId,
|
|
std::variant<std::unique_ptr<milvus::index::TextMatchIndex>,
|
|
std::shared_ptr<milvus::index::TextMatchIndexHolder>,
|
|
std::shared_ptr<milvus::cachinglayer::CacheSlot<
|
|
milvus::index::TextMatchIndex>>>>
|
|
text_indexes_;
|
|
|
|
// json stats cache (field_id -> CacheSlot of JsonKeyStats)
|
|
mutable folly::Synchronized<
|
|
std::unordered_map<FieldId, index::CacheJsonKeyStatsPtr>>
|
|
json_stats_;
|
|
|
|
GEOSContextHandle_t ctx_ = GEOS_init_r();
|
|
};
|
|
|
|
} // namespace milvus::segcore
|
|
|
|
#endif // MILVUS_SEGCORE_SEGMENT_INTERFACE_H_
|