enhance: support json stats with shredding design (#42534)

#42533

Co-authored-by: luzhang <luzhang@zilliz.com>
This commit is contained in:
zhagnlu 2025-09-01 10:49:52 +08:00 committed by GitHub
parent 3327df72e4
commit fc876639cf
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
115 changed files with 11569 additions and 5987 deletions

View File

@ -501,7 +501,6 @@ queryNode:
# If set to 0, time based eviction is disabled.
cacheTtl: 0
knowhereScoreConsistency: false # Enable knowhere strong consistency score computation logic
jsonKeyStatsCommitInterval: 200 # the commit interval for the JSON key Stats to commit
loadMemoryUsageFactor: 1 # The multiply factor of calculating the memory usage while loading segments
enableDisk: false # enable querynode load disk index, and search on disk index
maxDiskUsagePercentage: 95
@ -513,6 +512,7 @@ queryNode:
vectorIndex: false # Enable mmap for loading vector index
scalarField: false # Enable mmap for loading scalar data
scalarIndex: false # Enable mmap for loading scalar index
jsonStats: true # Enable mmap for loading json stats
# Enable memory mapping (mmap) to optimize the handling of growing raw data.
# By activating this feature, the memory overhead associated with newly added or modified data will be significantly minimized.
# However, this optimization may come at the cost of a slight decrease in query latency for the affected data segments.
@ -732,7 +732,9 @@ dataCoord:
analyzeTaskSlotUsage: 65535 # slot usage of analyze task
jsonStatsTriggerCount: 10 # jsonkey stats task count per trigger
jsonStatsTriggerInterval: 10 # jsonkey task interval per trigger
jsonKeyStatsMemoryBudgetInTantivy: 16777216 # the memory budget for the JSON index In Tantivy, the unit is bytes
jsonStatsMaxShreddingColumns: 1024 # the max number of columns to shred
jsonStatsShreddingRatioThreshold: 0.3 # the ratio threshold to shred
jsonStatsWriteBatchSize: 819200 # the batch size to write
ip: # TCP/IP address of dataCoord. If not specified, use the first unicastable address
port: 13333 # TCP port of dataCoord
grpc:
@ -1017,6 +1019,7 @@ common:
enabledGrowingSegmentJSONKeyStats: false # Indicates growingsegment whether to enable JSON key stats
enableConfigParamTypeCheck: true # Indicates whether to enable config param type check
enablePosixMode: false # Specifies whether to run in POSIX mode for enhanced file system compatibility
UsingJSONStatsForQuery: true # Indicates whether to use json stats when query
# QuotaConfig, configurations of Milvus quota and limits.
# By default, we enable:

View File

@ -48,6 +48,7 @@ class MilvusConan(ConanFile):
"simde/0.8.2#5e1edfd5cba92f25d79bf6ef4616b972",
"xxhash/0.8.3#199e63ab9800302c232d030b27accec0",
"unordered_dense/4.4.0#6a855c992618cc4c63019109a2e47298",
"mongo-cxx-driver/3.11.0#ae206de0e90fb8cb2fb95465fb8b2f01"
)
generators = ("cmake", "cmake_find_package")
default_options = {

View File

@ -38,7 +38,8 @@ StringChunkWriter::write(const arrow::ArrayVector& array_vec) {
// tuple <data, size, offset>
std::vector<std::tuple<const uint8_t*, int64_t, int64_t>> null_bitmaps;
for (const auto& data : array_vec) {
auto array = std::dynamic_pointer_cast<arrow::StringArray>(data);
// for bson, we use binary array to store the string
auto array = std::dynamic_pointer_cast<arrow::BinaryArray>(data);
for (int i = 0; i < array->length(); i++) {
auto str = array->GetView(i);
strs.emplace_back(str);

View File

@ -25,8 +25,6 @@ std::atomic<int64_t> EXEC_EVAL_EXPR_BATCH_SIZE(
DEFAULT_EXEC_EVAL_EXPR_BATCH_SIZE);
std::atomic<bool> OPTIMIZE_EXPR_ENABLED(DEFAULT_OPTIMIZE_EXPR_ENABLED);
std::atomic<int64_t> JSON_KEY_STATS_COMMIT_INTERVAL(
DEFAULT_JSON_KEY_STATS_COMMIT_INTERVAL);
std::atomic<bool> GROWING_JSON_KEY_STATS_ENABLED(
DEFAULT_GROWING_JSON_KEY_STATS_ENABLED);
std::atomic<bool> CONFIG_PARAM_TYPE_CHECK_ENABLED(
@ -52,13 +50,6 @@ SetDefaultOptimizeExprEnable(bool val) {
OPTIMIZE_EXPR_ENABLED.load());
}
void
SetDefaultJSONKeyStatsCommitInterval(int64_t val) {
JSON_KEY_STATS_COMMIT_INTERVAL.store(val);
LOG_INFO("set default json key Stats commit interval: {}",
JSON_KEY_STATS_COMMIT_INTERVAL.load());
}
void
SetDefaultGrowingJSONKeyStatsEnable(bool val) {
GROWING_JSON_KEY_STATS_ENABLED.store(val);

View File

@ -27,7 +27,6 @@ namespace milvus {
extern std::atomic<int64_t> FILE_SLICE_SIZE;
extern std::atomic<int64_t> EXEC_EVAL_EXPR_BATCH_SIZE;
extern std::atomic<int64_t> JSON_KEY_STATS_COMMIT_INTERVAL;
extern std::atomic<bool> OPTIMIZE_EXPR_ENABLED;
extern std::atomic<bool> GROWING_JSON_KEY_STATS_ENABLED;
extern std::atomic<bool> CONFIG_PARAM_TYPE_CHECK_ENABLED;
@ -41,9 +40,6 @@ SetDefaultExecEvalExprBatchSize(int64_t val);
void
SetDefaultOptimizeExprEnable(bool val);
void
SetDefaultJSONKeyStatsCommitInterval(int64_t val);
void
SetDefaultGrowingJSONKeyStatsEnable(bool val);

View File

@ -23,6 +23,7 @@
const int64_t INVALID_FIELD_ID = -1;
const int64_t INVALID_SEG_OFFSET = -1;
const int64_t INVALID_ARRAY_INDEX = -1;
const milvus::PkType INVALID_PK; // of std::monostate if not set.
// TODO: default field start id, could get from config.yaml
const int64_t START_USER_FIELDID = 100;
@ -51,8 +52,20 @@ const char PAGE_RETAIN_ORDER[] = "page_retain_order";
const char TEXT_LOG_ROOT_PATH[] = "text_log";
const char ITERATIVE_FILTER[] = "iterative_filter";
const char HINTS[] = "hints";
// json stats related
const char JSON_KEY_INDEX_LOG_ROOT_PATH[] = "json_key_index_log";
const char NGRAM_LOG_ROOT_PATH[] = "ngram_log";
constexpr const char* JSON_STATS_ROOT_PATH = "json_stats";
constexpr const char* JSON_STATS_DATA_FORMAT_VERSION = "2";
constexpr const char* JSON_STATS_SHARED_INDEX_PATH = "shared_key_index";
constexpr const char* JSON_STATS_SHREDDING_DATA_PATH = "shredding_data";
constexpr const char* JSON_KEY_STATS_SHARED_FIELD_NAME = "__shared";
// store key layout type in parquet file metadata
inline constexpr const char* JSON_STATS_META_KEY_LAYOUT_TYPE_MAP =
"key_layout_type_map";
// start json stats field id for mock column
const int64_t START_JSON_STATS_FIELD_ID = 1000;
const int64_t END_JSON_STATS_FIELD_ID = 10000;
const char DEFAULT_PLANNODE_ID[] = "0";
const char DEAFULT_QUERY_ID[] = "0";
@ -84,7 +97,6 @@ const bool DEFAULT_OPTIMIZE_EXPR_ENABLED = true;
const int64_t DEFAULT_CONVERT_OR_TO_IN_NUMERIC_LIMIT = 150;
const int64_t DEFAULT_JSON_INDEX_MEMORY_BUDGET = 16777216; // bytes, 16MB
const bool DEFAULT_GROWING_JSON_KEY_STATS_ENABLED = false;
const int64_t DEFAULT_JSON_KEY_STATS_COMMIT_INTERVAL = 200;
const bool DEFAULT_CONFIG_PARAM_TYPE_CHECK_ENABLED = true;
// index config related

View File

@ -0,0 +1,75 @@
// 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 <chrono>
#include <functional>
#include <string>
#include <string_view>
#include <log/Log.h>
namespace milvus {
class ScopedTimer {
public:
enum class LogLevel {
Trace,
Debug,
Info,
Warn,
Error,
};
ScopedTimer(std::string_view name,
std::function<void(double /*ms*/)> reporter,
LogLevel level = LogLevel::Debug)
: name_(name),
reporter_(std::move(reporter)),
start_(std::chrono::steady_clock::now()),
level_(level) {
}
~ScopedTimer() {
auto end = std::chrono::steady_clock::now();
auto duration_us =
std::chrono::duration<double, std::micro>(end - start_).count();
reporter_(duration_us / 1000.0); // report in milliseconds
switch (level_) {
case LogLevel::Trace:
LOG_TRACE("{} time: {} ms", name_, duration_us / 1000.0);
break;
case LogLevel::Debug:
LOG_DEBUG("{} time: {} ms", name_, duration_us / 1000.0);
break;
case LogLevel::Info:
LOG_INFO("{} time: {} ms", name_, duration_us / 1000.0);
break;
case LogLevel::Warn:
LOG_WARN("{} time: {} ms", name_, duration_us / 1000.0);
break;
case LogLevel::Error:
LOG_ERROR("{} time: {} ms", name_, duration_us / 1000.0);
break;
}
}
private:
std::string name_;
std::function<void(double)> reporter_;
std::chrono::steady_clock::time_point start_;
LogLevel level_;
};
} // namespace milvus

View File

@ -24,7 +24,6 @@
#include <boost/container/vector.hpp>
#include <boost/dynamic_bitset.hpp>
#include <folly/FBVector.h>
#include <arrow/type.h>
#include <limits>
#include <memory>

View File

@ -0,0 +1,537 @@
// 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 <bsoncxx/builder/basic/document.hpp>
#include <bsoncxx/builder/basic/kvp.hpp>
#include <bsoncxx/document/view.hpp>
#include <bsoncxx/types/bson_value/view.hpp>
#include <bsoncxx/json.hpp>
#include <bsoncxx/types.hpp>
#include <string>
#include <vector>
#include <optional>
#include "fmt/format.h"
#include "log/Log.h"
#include "common/EasyAssert.h"
template <>
struct fmt::formatter<bsoncxx::type> : fmt::formatter<std::string> {
auto
format(bsoncxx::type type, fmt::format_context& ctx) const {
std::string name;
switch (type) {
case bsoncxx::type::k_int32:
name = "int32";
break;
case bsoncxx::type::k_int64:
name = "int64";
break;
case bsoncxx::type::k_double:
name = "double";
break;
case bsoncxx::type::k_string:
name = "string";
break;
case bsoncxx::type::k_bool:
name = "bool";
break;
case bsoncxx::type::k_null:
name = "null";
break;
case bsoncxx::type::k_document:
name = "document";
break;
case bsoncxx::type::k_array:
name = "array";
break;
default:
name = "Unknown";
}
return fmt::formatter<std::string>::format(name, ctx);
}
};
namespace milvus {
struct BsonRawField {
bsoncxx::type type;
std::string key;
const uint8_t* value_ptr; // points to value (not including type/key)
};
inline int32_t
ReadInt32(const uint8_t* ptr) {
return *reinterpret_cast<const int32_t*>(ptr);
}
inline int64_t
ReadInt64(const uint8_t* ptr) {
return *reinterpret_cast<const int64_t*>(ptr);
}
inline double
ReadDouble(const uint8_t* ptr) {
return *reinterpret_cast<const double*>(ptr);
}
inline std::string
ReadUtf8(const uint8_t* ptr) {
int32_t len = *reinterpret_cast<const int32_t*>(ptr);
return std::string(reinterpret_cast<const char*>(ptr + 4),
len - 1); // exclude trailing '\0'
}
inline std::string_view
ReadUtf8View(const uint8_t* ptr) {
int32_t len = *reinterpret_cast<const int32_t*>(ptr);
return std::string_view(reinterpret_cast<const char*>(ptr + 4),
len - 1); // exclude trailing '\0'
}
inline bool
ReadBool(const uint8_t* ptr) {
return *ptr != 0;
}
inline std::vector<uint8_t>
ReadRawDocOrArray(const uint8_t* ptr) {
int32_t len = *reinterpret_cast<const int32_t*>(ptr);
return std::vector<uint8_t>(ptr, ptr + len);
}
inline bsoncxx::document::view
ParseAsDocument(const uint8_t* ptr) {
int32_t len = *reinterpret_cast<const int32_t*>(ptr);
return bsoncxx::document::view(ptr, len);
}
inline bsoncxx::array::view
ParseAsArray(const uint8_t* ptr) {
int32_t len = *reinterpret_cast<const int32_t*>(ptr);
return bsoncxx::array::view(ptr, len);
}
template <typename T>
T
GetValue(const uint8_t* ptr);
template <>
inline int32_t
GetValue<int32_t>(const uint8_t* ptr) {
return ReadInt32(ptr);
}
template <>
inline int64_t
GetValue<int64_t>(const uint8_t* ptr) {
return ReadInt64(ptr);
}
template <>
inline double
GetValue<double>(const uint8_t* ptr) {
return ReadDouble(ptr);
}
template <>
inline bool
GetValue<bool>(const uint8_t* ptr) {
return ReadBool(ptr);
}
template <>
inline std::string
GetValue<std::string>(const uint8_t* ptr) {
return ReadUtf8(ptr);
}
template <>
inline std::string_view
GetValue<std::string_view>(const uint8_t* ptr) {
return ReadUtf8View(ptr);
}
template <>
inline std::vector<uint8_t>
GetValue<std::vector<uint8_t>>(const uint8_t* ptr) {
return ReadRawDocOrArray(ptr);
}
inline std::string
BsonHexDebugString(const uint8_t* data, size_t size) {
std::ostringstream oss;
oss << "BSON hex dump (" << size << " bytes): ";
for (size_t i = 0; i < size; ++i) {
oss << std::hex << std::setw(2) << std::setfill('0')
<< static_cast<int>(data[i]) << " ";
}
return oss.str();
}
// may lose precision for big int64_t more than 2^53
inline bool
CanConvertToInt64(double x) {
return std::trunc(x) == x && x >= std::numeric_limits<int64_t>::min() &&
x <= std::numeric_limits<int64_t>::max();
}
class BsonView {
public:
explicit BsonView(const std::vector<uint8_t>& data)
: data_(data.data()), size_(data.size()) {
}
explicit BsonView(const uint8_t* data, size_t size)
: data_(data), size_(size) {
}
~BsonView() {
}
std::string
ToString() const {
bsoncxx::document::view view(data_, size_);
return bsoncxx::to_json(view);
}
bool
IsNullAtOffset(size_t offset) {
const uint8_t* ptr = data_ + offset;
AssertInfo(offset < size_, "bson offset out of range");
auto type_tag = static_cast<bsoncxx::type>(*ptr++);
return type_tag == bsoncxx::type::k_null;
}
template <typename T>
std::optional<T>
ParseAsValueAtOffset(size_t offset) {
LOG_TRACE("bson hex: {}",
BsonHexDebugString(data_ + offset, size_ - offset));
const uint8_t* ptr = data_ + offset;
// check offset
AssertInfo(
offset < size_, "bson offset:{} out of range:{}", offset, size_);
// parse type tag
auto type_tag = static_cast<bsoncxx::type>(*ptr++);
// parse key
const char* key_cstr = reinterpret_cast<const char*>(ptr);
size_t key_len = strlen(key_cstr);
ptr += key_len + 1; // +1 for null terminator
// parse value
switch (type_tag) {
case bsoncxx::type::k_int32:
if constexpr (std::is_same_v<T, int32_t>) {
return GetValue<int32_t>(ptr);
}
if constexpr (std::is_same_v<T, int64_t>) {
return static_cast<int64_t>(GetValue<int32_t>(ptr));
}
break;
case bsoncxx::type::k_int64:
if constexpr (std::is_same_v<T, int64_t>) {
return GetValue<int64_t>(ptr);
} else if constexpr (std::is_same_v<T, double>) {
return static_cast<double>(GetValue<int64_t>(ptr));
}
break;
case bsoncxx::type::k_double:
if constexpr (std::is_same_v<T, double>) {
return GetValue<double>(ptr);
}
// if constexpr (std::is_same_v<T, int64_t>) {
// if (CanConvertToInt64(GetValue<double>(ptr))) {
// return static_cast<int64_t>(GetValue<double>(ptr));
// }
// }
break;
case bsoncxx::type::k_bool:
if constexpr (std::is_same_v<T, bool>) {
return GetValue<bool>(ptr);
}
break;
case bsoncxx::type::k_string:
if (ptr + 4 > data_ + size_) {
return std::nullopt;
}
if constexpr (std::is_same_v<T, std::string>) {
return GetValue<std::string>(ptr);
}
if constexpr (std::is_same_v<T, std::string_view>) {
return GetValue<std::string_view>(ptr);
}
break;
case bsoncxx::type::k_null:
case bsoncxx::type::k_document:
case bsoncxx::type::k_array:
break;
default:
ThrowInfo(
ErrorCode::Unsupported, "unknown BSON type {}", type_tag);
}
return std::nullopt;
}
std::optional<bsoncxx::array::view>
ParseAsArrayAtOffset(size_t offset) {
if (offset == 0) {
// if offset is 0, it means the array is the whole bson_view
return bsoncxx::array::view(data_, size_);
}
// check offset
AssertInfo(offset < size_, "bson offset out of range");
const uint8_t* ptr = data_ + offset;
// check type
AssertInfo(static_cast<bsoncxx::type>(*ptr) == bsoncxx::type::k_array,
"ParseAsArrayAtOffset expects an array at offset {}",
offset);
ptr++;
// skip key
const char* key_start = reinterpret_cast<const char*>(ptr);
size_t key_len = strlen(key_start);
ptr += key_len + 1; // +1 for null terminator
const uint8_t* view_start = ptr;
// parse length
int32_t len = *reinterpret_cast<const int32_t*>(ptr);
if (ptr + len > data_ + size_) {
ThrowInfo(ErrorCode::UnexpectedError,
"ParseAsArrayAtOffset out of range");
}
return bsoncxx::array::view(view_start, len);
}
inline std::optional<bsoncxx::types::bson_value::view>
FindByPath(const bsoncxx::document::view& doc_view,
const std::vector<std::string>& path,
size_t idx = 0) {
if (idx >= path.size())
return std::nullopt;
for (const auto& elem : doc_view) {
const std::string_view key{elem.key().data(), elem.key().length()};
if (key != path[idx])
continue;
const auto& value = elem.get_value();
if (idx == path.size() - 1)
return value; // found the target
// Recursively process nested structures
switch (value.type()) {
case bsoncxx::type::k_document: {
auto sub_doc = value.get_document();
return FindByPath(sub_doc.view(), path, idx + 1);
}
case bsoncxx::type::k_array:
// TODO: may support array index from parent json for now
break;
default:
break;
}
}
return std::nullopt;
}
// extract pointer to the N-th value in a BSON array
template <typename T>
static std::optional<T>
GetNthElementInArray(const uint8_t* array_ptr, size_t index) {
const uint8_t* ptr = array_ptr + 4;
for (size_t i = 0; i <= index; ++i) {
auto type_tag = *ptr++;
if (type_tag == 0x00)
break;
while (*ptr != '\0') ++ptr;
++ptr;
if (i == index) {
switch (static_cast<bsoncxx::type>(type_tag)) {
case bsoncxx::type::k_int32:
if constexpr (std::is_same_v<T, int32_t>) {
return ReadInt32(ptr);
}
break;
case bsoncxx::type::k_int64:
if constexpr (std::is_same_v<T, int64_t>) {
return ReadInt64(ptr);
}
break;
case bsoncxx::type::k_double:
if constexpr (std::is_same_v<T, double>) {
return ReadDouble(ptr);
}
break;
case bsoncxx::type::k_bool:
if constexpr (std::is_same_v<T, bool>) {
return ReadBool(ptr);
}
break;
case bsoncxx::type::k_string:
if constexpr (std::is_same_v<T, std::string>) {
return ReadUtf8(ptr);
}
if constexpr (std::is_same_v<T, std::string_view>) {
return ReadUtf8View(ptr);
}
break;
case bsoncxx::type::k_array:
if constexpr (std::is_same_v<T, bsoncxx::array::view>) {
return ParseAsArray(ptr);
}
break;
default:
return std::nullopt;
}
}
switch (static_cast<bsoncxx::type>(type_tag)) {
case bsoncxx::type::k_string: {
int32_t len = *reinterpret_cast<const int32_t*>(ptr);
ptr += 4 + len;
break;
}
case bsoncxx::type::k_int32:
ptr += 4;
break;
case bsoncxx::type::k_int64:
ptr += 8;
break;
case bsoncxx::type::k_double:
ptr += 8;
break;
case bsoncxx::type::k_bool:
ptr += 1;
break;
case bsoncxx::type::k_array:
case bsoncxx::type::k_document: {
int32_t len = *reinterpret_cast<const int32_t*>(ptr);
ptr += len;
break;
}
default:
return std::nullopt;
}
}
return std::nullopt;
}
inline BsonRawField
ParseBsonField(const uint8_t* bson_data, size_t offset) {
const uint8_t* ptr = bson_data + offset;
auto type_tag = static_cast<bsoncxx::type>(*ptr++);
const char* key_cstr = reinterpret_cast<const char*>(ptr);
size_t key_len = strlen(key_cstr);
ptr += key_len + 1;
return BsonRawField{
.type = type_tag, .key = key_cstr, .value_ptr = ptr};
}
template <typename T>
static std::optional<T>
GetValueFromElement(const bsoncxx::document::element& element) {
if constexpr (std::is_same_v<T, int32_t>) {
if (element.type() == bsoncxx::type::k_int32) {
return element.get_int32().value;
}
} else if constexpr (std::is_same_v<T, int64_t>) {
if (element.type() == bsoncxx::type::k_int64) {
return element.get_int64().value;
}
} else if constexpr (std::is_same_v<T, double>) {
if (element.type() == bsoncxx::type::k_double) {
return element.get_double().value;
}
} else if constexpr (std::is_same_v<T, bool>) {
if (element.type() == bsoncxx::type::k_bool) {
return element.get_bool().value;
}
} else if constexpr (std::is_same_v<T, std::string>) {
if (element.type() == bsoncxx::type::k_string) {
return std::string(element.get_string().value.data(),
element.get_string().value.size());
}
} else if constexpr (std::is_same_v<T, std::string_view>) {
if (element.type() == bsoncxx::type::k_string) {
return std::string_view(element.get_string().value.data(),
element.get_string().value.size());
}
}
return std::nullopt;
}
template <typename T>
static std::optional<T>
GetValueFromBsonView(const bsoncxx::types::bson_value::view& value_view) {
switch (value_view.type()) {
case bsoncxx::type::k_int32:
if constexpr (std::is_same_v<T, int32_t>)
return value_view.get_int32().value;
break;
case bsoncxx::type::k_int64:
if constexpr (std::is_same_v<T, int64_t>)
return value_view.get_int64().value;
if constexpr (std::is_same_v<T, double>)
return static_cast<double>(value_view.get_int64().value);
break;
case bsoncxx::type::k_double:
if constexpr (std::is_same_v<T, double>)
return value_view.get_double().value;
break;
case bsoncxx::type::k_bool:
if constexpr (std::is_same_v<T, bool>)
return value_view.get_bool().value;
break;
case bsoncxx::type::k_string:
if constexpr (std::is_same_v<T, std::string>) {
return std::string(value_view.get_string().value.data(),
value_view.get_string().value.size());
} else if constexpr (std::is_same_v<T, std::string_view>) {
return std::string_view(
value_view.get_string().value.data(),
value_view.get_string().value.size());
}
break;
case bsoncxx::type::k_array:
if constexpr (std::is_same_v<T, bsoncxx::array::view>) {
return value_view.get_array().value;
}
break;
// other types...
default:
break;
}
return std::nullopt;
}
const uint8_t* data_;
size_t size_;
};
} // namespace milvus

View File

@ -60,11 +60,6 @@ SetDefaultOptimizeExprEnable(bool val) {
milvus::SetDefaultOptimizeExprEnable(val);
}
void
SetDefaultJSONKeyStatsCommitInterval(int64_t val) {
milvus::SetDefaultJSONKeyStatsCommitInterval(val);
}
void
SetDefaultGrowingJSONKeyStatsEnable(bool val) {
milvus::SetDefaultGrowingJSONKeyStatsEnable(val);

View File

@ -45,9 +45,6 @@ SetDefaultExprEvalBatchSize(int64_t val);
void
SetDefaultOptimizeExprEnable(bool val);
void
SetDefaultJSONKeyStatsCommitInterval(int64_t val);
void
SetDefaultGrowingJSONKeyStatsEnable(bool val);

View File

@ -178,6 +178,7 @@ class QueryContext : public Context {
milvus::Timestamp timestamp,
milvus::Timestamp collection_ttl = 0,
int32_t consistency_level = 0,
const query::PlanOptions& plan_options = query::PlanOptions(),
std::shared_ptr<QueryConfig> query_config =
std::make_shared<QueryConfig>(),
folly::Executor* executor = nullptr,
@ -191,7 +192,8 @@ class QueryContext : public Context {
collection_ttl_timestamp_(collection_ttl),
query_config_(query_config),
executor_(executor),
consistency_level_(consistency_level) {
consistency_level_(consistency_level),
plan_options_(plan_options) {
}
folly::Executor*
@ -284,6 +286,11 @@ class QueryContext : public Context {
return consistency_level_;
}
const query::PlanOptions&
get_plan_options() const {
return plan_options_;
}
private:
folly::Executor* executor_;
//folly::Executor::KeepAlive<> executor_keepalive_;
@ -307,6 +314,8 @@ class QueryContext : public Context {
milvus::RetrieveResult retrieve_result_;
int32_t consistency_level_ = 0;
query::PlanOptions plan_options_;
};
// Represent the state of one thread of query execution.

View File

@ -18,7 +18,7 @@
#include <utility>
#include "query/Utils.h"
#include "index/json_stats/JsonKeyStats.h"
namespace milvus {
namespace exec {
@ -421,8 +421,8 @@ PhyBinaryRangeFilterExpr::ExecRangeVisitorImplForJson(EvalCtx& context) {
const auto& bitmap_input = context.get_bitmap_input();
auto* input = context.get_offset_input();
FieldId field_id = expr_->column_.field_id_;
if (CanUseJsonKeyIndex(field_id) && !has_offset_input_) {
return ExecRangeVisitorImplForJsonForIndex<ValueType>();
if (!has_offset_input_ && CanUseJsonStats(context, field_id)) {
return ExecRangeVisitorImplForJsonStats<ValueType>();
}
auto real_batch_size =
has_offset_input_ ? input->size() : GetNextBatchSize();
@ -546,281 +546,176 @@ PhyBinaryRangeFilterExpr::ExecRangeVisitorImplForJson(EvalCtx& context) {
template <typename ValueType>
VectorPtr
PhyBinaryRangeFilterExpr::ExecRangeVisitorImplForJsonForIndex() {
PhyBinaryRangeFilterExpr::ExecRangeVisitorImplForJsonStats() {
using GetType = std::conditional_t<std::is_same_v<ValueType, std::string>,
std::string_view,
ValueType>;
auto real_batch_size =
(current_data_chunk_pos_ + batch_size_ > active_count_)
? active_count_ - current_data_chunk_pos_
: batch_size_;
auto pointer = milvus::Json::pointer(expr_->column_.nested_path_);
#define BinaryRangeJSONIndexCompare(cmp) \
do { \
auto val = json.at<GetType>(offset, size); \
if (val.error()) { \
if constexpr (std::is_same_v<GetType, int64_t>) { \
auto val = json.at<double>(offset, size); \
return !val.error() && (cmp); \
} \
return false; \
} \
return (cmp); \
} while (false)
#define BinaryRangeJSONTypeCompare(cmp) \
do { \
if constexpr (std::is_same_v<GetType, std::string_view>) { \
if (type == uint8_t(milvus::index::JSONType::STRING)) { \
auto val = json.at_string(offset, size); \
return (cmp); \
} else { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, double>) { \
if (type == uint8_t(milvus::index::JSONType::INT64)) { \
auto val = \
std::stoll(std::string(json.at_string(offset, size))); \
return (cmp); \
} else if (type == uint8_t(milvus::index::JSONType::DOUBLE)) { \
auto val = \
std::stod(std::string(json.at_string(offset, size))); \
return (cmp); \
} else { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, int64_t>) { \
if (type == uint8_t(milvus::index::JSONType::INT64)) { \
auto val = \
std::stoll(std::string(json.at_string(offset, size))); \
return (cmp); \
} else if (type == uint8_t(milvus::index::JSONType::DOUBLE)) { \
auto val = \
std::stod(std::string(json.at_string(offset, size))); \
return (cmp); \
} else { \
return false; \
} \
} \
} while (false)
#define BinaryRangeJSONTypeCompareWithValue(cmp) \
do { \
if constexpr (std::is_same_v<GetType, int64_t>) { \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
float val = *reinterpret_cast<float*>(&value); \
return (cmp); \
} else { \
int64_t val = value; \
return (cmp); \
} \
} else if constexpr (std::is_same_v<GetType, double>) { \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
float val = *reinterpret_cast<float*>(&value); \
return (cmp); \
} else { \
int64_t val = value; \
return (cmp); \
} \
} else if constexpr (std::is_same_v<GetType, bool>) { \
bool val = *reinterpret_cast<bool*>(&value); \
return (cmp); \
} \
} while (false)
auto real_batch_size = current_data_chunk_pos_ + batch_size_ > active_count_
? active_count_ - current_data_chunk_pos_
: batch_size_;
auto pointer = milvus::index::JsonPointer(expr_->column_.nested_path_);
bool lower_inclusive = expr_->lower_inclusive_;
bool upper_inclusive = expr_->upper_inclusive_;
ValueType val1 = GetValueFromProto<ValueType>(expr_->lower_val_);
ValueType val2 = GetValueFromProto<ValueType>(expr_->upper_val_);
if (cached_index_chunk_id_ != 0) {
const segcore::SegmentInternalInterface* segment = nullptr;
if (segment_->type() == SegmentType::Growing) {
segment =
dynamic_cast<const segcore::SegmentGrowingImpl*>(segment_);
} else if (segment_->type() == SegmentType::Sealed) {
segment = dynamic_cast<const segcore::SegmentSealed*>(segment_);
}
auto field_id = expr_->column_.field_id_;
auto* index = segment->GetJsonKeyIndex(field_id);
Assert(index != nullptr);
auto filter_func = [segment,
&field_id,
val1,
val2,
lower_inclusive,
upper_inclusive](const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
std::vector<int64_t> invalid_row_ids;
std::vector<int64_t> invalid_offset;
std::vector<int64_t> invalid_type;
std::vector<int64_t> invalid_size;
for (size_t i = 0; i < n; i++) {
auto valid = valid_array[i];
auto type = type_array[i];
auto row_id = row_id_array[i];
auto offset = offset_array[i];
auto size = size_array[i];
auto value = value_array[i];
if (!valid) {
invalid_row_ids.push_back(row_id_array[i]);
invalid_offset.push_back(offset_array[i]);
invalid_type.push_back(type_array[i]);
invalid_size.push_back(size_array[i]);
continue;
}
auto f = [&]() {
if constexpr (std::is_same_v<GetType, int64_t>) {
if (type != uint8_t(milvus::index::JSONType::INT32) &&
type != uint8_t(milvus::index::JSONType::INT64) &&
type != uint8_t(milvus::index::JSONType::FLOAT) &&
type != uint8_t(milvus::index::JSONType::DOUBLE)) {
return false;
}
} else if constexpr (std::is_same_v<GetType,
std::string_view>) {
if (type != uint8_t(milvus::index::JSONType::STRING) &&
type !=
uint8_t(
milvus::index::JSONType::STRING_ESCAPE)) {
return false;
}
} else if constexpr (std::is_same_v<GetType, double>) {
if (type != uint8_t(milvus::index::JSONType::INT32) &&
type != uint8_t(milvus::index::JSONType::INT64) &&
type != uint8_t(milvus::index::JSONType::FLOAT) &&
type != uint8_t(milvus::index::JSONType::DOUBLE)) {
return false;
}
} else if constexpr (std::is_same_v<GetType, bool>) {
if (type != uint8_t(milvus::index::JSONType::BOOL)) {
return false;
if (cached_index_chunk_id_ != 0 &&
segment_->type() == SegmentType::Sealed) {
auto* segment = dynamic_cast<const segcore::SegmentSealed*>(segment_);
auto field_id = expr_->column_.field_id_;
auto* index = segment->GetJsonStats(field_id);
Assert(index != nullptr);
cached_index_chunk_res_ = std::make_shared<TargetBitmap>(active_count_);
cached_index_chunk_valid_res_ =
std::make_shared<TargetBitmap>(active_count_, true);
TargetBitmapView res_view(*cached_index_chunk_res_);
TargetBitmapView valid_res_view(*cached_index_chunk_valid_res_);
// process shredding data
auto try_execute = [&](milvus::index::JSONType json_type,
TargetBitmapView& res_view,
TargetBitmapView& valid_res_view,
auto GetType) {
auto target_field = index->GetShreddingField(pointer, json_type);
if (!target_field.empty()) {
using ColType = decltype(GetType);
auto shredding_executor =
[val1, val2, lower_inclusive, upper_inclusive](
const ColType* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
if (lower_inclusive && upper_inclusive) {
res[i] = src[i] >= val1 && src[i] <= val2;
} else if (lower_inclusive && !upper_inclusive) {
res[i] = src[i] >= val1 && src[i] < val2;
} else if (!lower_inclusive && upper_inclusive) {
res[i] = src[i] > val1 && src[i] <= val2;
} else {
res[i] = src[i] > val1 && src[i] < val2;
}
}
};
index->ExecutorForShreddingData<ColType>(target_field,
shredding_executor,
nullptr,
res_view,
valid_res_view);
LOG_DEBUG("using shredding data's field: {} count {}",
target_field,
res_view.count());
}
};
if constexpr (std::is_same_v<GetType, int64_t>) {
// int64 compare
try_execute(milvus::index::JSONType::INT64,
res_view,
valid_res_view,
int64_t{});
// and double compare
TargetBitmap res_double(active_count_, false);
TargetBitmapView res_double_view(res_double);
TargetBitmap res_double_valid(active_count_, true);
TargetBitmapView valid_res_double_view(res_double_valid);
try_execute(milvus::index::JSONType::DOUBLE,
res_double_view,
valid_res_double_view,
double{});
res_view.inplace_or_with_count(res_double_view, active_count_);
valid_res_view.inplace_or_with_count(valid_res_double_view,
active_count_);
} else if constexpr (std::is_same_v<GetType, double>) {
try_execute(milvus::index::JSONType::DOUBLE,
res_view,
valid_res_view,
double{});
// and int64 compare
TargetBitmap res_int64(active_count_, false);
TargetBitmapView res_int64_view(res_int64);
TargetBitmap res_int64_valid(active_count_, true);
TargetBitmapView valid_res_int64_view(res_int64_valid);
try_execute(milvus::index::JSONType::INT64,
res_int64_view,
valid_res_int64_view,
int64_t{});
res_view.inplace_or_with_count(res_int64_view, active_count_);
valid_res_view.inplace_or_with_count(valid_res_int64_view,
active_count_);
} else if constexpr (std::is_same_v<GetType, std::string_view> ||
std::is_same_v<GetType, std::string>) {
try_execute(milvus::index::JSONType::STRING,
res_view,
valid_res_view,
std::string_view{});
}
// process shared data
auto shared_executor =
[val1, val2, lower_inclusive, upper_inclusive, &res_view](
milvus::BsonView bson, uint32_t row_id, uint32_t value_offset) {
if constexpr (std::is_same_v<GetType, int64_t> ||
std::is_same_v<GetType, double>) {
auto val = bson.ParseAsValueAtOffset<double>(value_offset);
if (!val.has_value()) {
res_view[row_id] = false;
return;
}
if (lower_inclusive && upper_inclusive) {
if (type == uint8_t(milvus::index::JSONType::FLOAT)) {
BinaryRangeJSONTypeCompareWithValue(
static_cast<float>(val1) <= val &&
val <= static_cast<float>(val2));
} else {
BinaryRangeJSONTypeCompareWithValue(val1 <= val &&
val <= val2);
}
res_view[row_id] =
val.value() >= val1 && val.value() <= val2;
} else if (lower_inclusive && !upper_inclusive) {
if (type == uint8_t(milvus::index::JSONType::FLOAT)) {
BinaryRangeJSONTypeCompareWithValue(
static_cast<float>(val1) <= val &&
val < static_cast<float>(val2));
} else {
BinaryRangeJSONTypeCompareWithValue(val1 <= val &&
val < val2);
}
res_view[row_id] =
val.value() >= val1 && val.value() < val2;
} else if (!lower_inclusive && upper_inclusive) {
if (type == uint8_t(milvus::index::JSONType::FLOAT)) {
BinaryRangeJSONTypeCompareWithValue(
static_cast<float>(val1) < val &&
val <= static_cast<float>(val2));
} else {
BinaryRangeJSONTypeCompareWithValue(val1 < val &&
val <= val2);
}
res_view[row_id] =
val.value() > val1 && val.value() <= val2;
} else {
if (type == uint8_t(milvus::index::JSONType::FLOAT)) {
BinaryRangeJSONTypeCompareWithValue(
static_cast<float>(val1) < val &&
val < static_cast<float>(val2));
} else {
BinaryRangeJSONTypeCompareWithValue(val1 < val &&
val < val2);
}
}
};
bitset[row_id] = f();
}
auto f = [&](const milvus::Json& json,
uint8_t type,
uint16_t offset,
uint16_t size,
bool is_valid) {
if (!is_valid) {
return false;
}
if (lower_inclusive && upper_inclusive) {
if (type == uint8_t(milvus::index::JSONType::STRING) ||
type == uint8_t(milvus::index::JSONType::DOUBLE) ||
type == uint8_t(milvus::index::JSONType::INT64)) {
BinaryRangeJSONTypeCompare(val1 <= val && val <= val2);
} else {
BinaryRangeJSONIndexCompare(
val1 <= ValueType(val.value()) &&
ValueType(val.value()) <= val2);
}
} else if (lower_inclusive && !upper_inclusive) {
if (type == uint8_t(milvus::index::JSONType::STRING) ||
type == uint8_t(milvus::index::JSONType::DOUBLE) ||
type == uint8_t(milvus::index::JSONType::INT64)) {
BinaryRangeJSONTypeCompare(val1 <= val && val < val2);
} else {
BinaryRangeJSONIndexCompare(
val1 <= ValueType(val.value()) &&
ValueType(val.value()) < val2);
}
} else if (!lower_inclusive && upper_inclusive) {
if (type == uint8_t(milvus::index::JSONType::STRING) ||
type == uint8_t(milvus::index::JSONType::DOUBLE) ||
type == uint8_t(milvus::index::JSONType::INT64)) {
BinaryRangeJSONTypeCompare(val1 < val && val <= val2);
} else {
BinaryRangeJSONIndexCompare(
val1 < ValueType(val.value()) &&
ValueType(val.value()) <= val2);
res_view[row_id] =
val.value() > val1 && val.value() < val2;
}
} else {
if (type == uint8_t(milvus::index::JSONType::STRING) ||
type == uint8_t(milvus::index::JSONType::DOUBLE) ||
type == uint8_t(milvus::index::JSONType::INT64)) {
BinaryRangeJSONTypeCompare(val1 < val && val < val2);
auto val = bson.ParseAsValueAtOffset<GetType>(value_offset);
if (!val.has_value()) {
res_view[row_id] = false;
return;
}
if (lower_inclusive && upper_inclusive) {
res_view[row_id] =
val.value() >= val1 && val.value() <= val2;
} else if (lower_inclusive && !upper_inclusive) {
res_view[row_id] =
val.value() >= val1 && val.value() < val2;
} else if (!lower_inclusive && upper_inclusive) {
res_view[row_id] =
val.value() > val1 && val.value() <= val2;
} else {
BinaryRangeJSONIndexCompare(
val1 < ValueType(val.value()) &&
ValueType(val.value()) < val2);
res_view[row_id] =
val.value() > val1 && val.value() < val2;
}
}
};
segment->BulkGetJsonData(
field_id,
[&](const milvus::Json& json, size_t i, bool is_valid) {
auto row_id = invalid_row_ids[i];
auto type = invalid_type[i];
auto offset = invalid_offset[i];
auto size = invalid_size[i];
bitset[row_id] = f(json, type, offset, size, is_valid);
},
invalid_row_ids.data(),
invalid_row_ids.size());
};
bool is_growing = segment_->type() == SegmentType::Growing;
bool is_strong_consistency = consistency_level_ == 0;
cached_index_chunk_res_ = std::make_shared<TargetBitmap>(
std::move(index->FilterByPath(pointer,
active_count_,
is_growing,
is_strong_consistency,
filter_func)));
if (!index->CanSkipShared(pointer)) {
index->ExecuteForSharedData(pointer, shared_executor);
}
cached_index_chunk_id_ = 0;
}
TargetBitmap result;
result.append(
*cached_index_chunk_res_, current_data_global_pos_, real_batch_size);
MoveCursor();
return std::make_shared<ColumnVector>(std::move(result),
TargetBitmap(real_batch_size, true));
}
} // namespace exec
template <typename ValueType>
VectorPtr
@ -960,5 +855,5 @@ PhyBinaryRangeFilterExpr::ExecRangeVisitorImplForArray(EvalCtx& context) {
return res_vec;
}
} //namespace exec
} // namespace exec
} // namespace milvus

View File

@ -312,7 +312,7 @@ class PhyBinaryRangeFilterExpr : public SegmentExpr {
template <typename ValueType>
VectorPtr
ExecRangeVisitorImplForJsonForIndex();
ExecRangeVisitorImplForJsonStats();
template <typename ValueType>
VectorPtr

View File

@ -150,7 +150,21 @@ class SortVectorElement : public MultiElement {
bool
In(const ValueType& value) const override {
AssertInfo(sorted_, "In() should be sorted before");
if (std::holds_alternative<T>(value)) {
if constexpr (std::is_same_v<T, std::string>) {
if (std::holds_alternative<std::string>(value)) {
return std::binary_search(values_.begin(),
values_.end(),
std::get<std::string>(value));
} else if (std::holds_alternative<std::string_view>(value)) {
return std::binary_search(
values_.begin(),
values_.end(),
std::string(std::get<std::string_view>(value)),
[](const std::string& a, std::string_view b) {
return a < b;
});
}
} else if (std::holds_alternative<T>(value)) {
return std::binary_search(
values_.begin(), values_.end(), std::get<T>(value));
}
@ -168,6 +182,11 @@ class SortVectorElement : public MultiElement {
values_.push_back(value);
}
std::vector<T>
GetElements() const {
return values_;
}
public:
std::vector<T> values_;
bool sorted_{false};
@ -257,6 +276,11 @@ class SetElement : public MultiElement {
return values_.size();
}
std::vector<T>
GetElements() const {
return std::vector<T>(values_.begin(), values_.end());
}
public:
ankerl::unordered_dense::set<T> values_;
};
@ -313,6 +337,11 @@ class SetElement<bool> : public MultiElement {
return (contains_true ? 1 : 0) + (contains_false ? 1 : 0);
}
std::vector<bool>
GetElements() const {
return {contains_true, contains_false};
}
private:
bool contains_true = false;
bool contains_false = false;

View File

@ -20,6 +20,7 @@
#include "common/Types.h"
#include "common/Vector.h"
#include "index/JsonInvertedIndex.h"
#include "index/json_stats/JsonKeyStats.h"
namespace milvus {
namespace exec {
@ -113,8 +114,8 @@ PhyExistsFilterExpr::EvalJsonExistsForDataSegment(EvalCtx& context) {
auto* input = context.get_offset_input();
const auto& bitmap_input = context.get_bitmap_input();
FieldId field_id = expr_->column_.field_id_;
if (CanUseJsonKeyIndex(field_id) && !has_offset_input_) {
return EvalJsonExistsForDataSegmentForIndex();
if (CanUseJsonStats(context, field_id) && !has_offset_input_) {
return EvalJsonExistsForDataSegmentByStats();
}
auto real_batch_size =
has_offset_input_ ? input->size() : GetNextBatchSize();
@ -178,48 +179,41 @@ PhyExistsFilterExpr::EvalJsonExistsForDataSegment(EvalCtx& context) {
}
VectorPtr
PhyExistsFilterExpr::EvalJsonExistsForDataSegmentForIndex() {
PhyExistsFilterExpr::EvalJsonExistsForDataSegmentByStats() {
auto real_batch_size = GetNextBatchSize();
if (real_batch_size == 0) {
return nullptr;
}
auto pointer = milvus::Json::pointer(expr_->column_.nested_path_);
if (cached_index_chunk_id_ != 0) {
auto pointer = milvus::index::JsonPointer(expr_->column_.nested_path_);
if (cached_index_chunk_id_ != 0 &&
segment_->type() == SegmentType::Sealed) {
cached_index_chunk_id_ = 0;
const segcore::SegmentInternalInterface* segment = nullptr;
if (segment_->type() == SegmentType::Growing) {
segment =
dynamic_cast<const segcore::SegmentGrowingImpl*>(segment_);
} else if (segment_->type() == SegmentType::Sealed) {
segment = dynamic_cast<const segcore::SegmentSealed*>(segment_);
}
auto segment = static_cast<const segcore::SegmentSealed*>(segment_);
auto field_id = expr_->column_.field_id_;
auto* index = segment->GetJsonKeyIndex(field_id);
auto* index = segment->GetJsonStats(field_id);
Assert(index != nullptr);
auto filter_func = [segment, field_id, pointer](
const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
for (size_t i = 0; i < n; i++) {
auto row_id = row_id_array[i];
bitset[row_id] = true;
}
};
bool is_growing = segment_->type() == SegmentType::Growing;
bool is_strong_consistency = consistency_level_ == 0;
cached_index_chunk_res_ = std::make_shared<TargetBitmap>(
std::move(index->FilterByPath(pointer,
active_count_,
is_growing,
is_strong_consistency,
filter_func)));
cached_index_chunk_res_ = std::make_shared<TargetBitmap>(active_count_);
cached_index_chunk_valid_res_ =
std::make_shared<TargetBitmap>(active_count_, true);
TargetBitmapView res_view(*cached_index_chunk_res_);
TargetBitmapView valid_res_view(*cached_index_chunk_valid_res_);
// process shredding data
auto shredding_fields = index->GetShreddingFields(pointer);
for (const auto& field : shredding_fields) {
index->ExecutorForGettingValid(field, valid_res_view);
res_view |= valid_res_view;
}
if (!index->CanSkipShared(pointer)) {
// process shared data
index->ExecuteExistsPathForSharedData(pointer, res_view);
}
cached_index_chunk_id_ = 0;
}
TargetBitmap result;
result.append(
*cached_index_chunk_res_, current_data_global_pos_, real_batch_size);

View File

@ -83,7 +83,7 @@ class PhyExistsFilterExpr : public SegmentExpr {
EvalJsonExistsForIndex();
VectorPtr
EvalJsonExistsForDataSegmentForIndex();
EvalJsonExistsForDataSegmentByStats();
private:
std::shared_ptr<const milvus::expr::ExistsExpr> expr_;

View File

@ -1272,23 +1272,6 @@ class SegmentExpr : public Expr {
use_index_ = false;
}
bool
CanUseJsonKeyIndex(FieldId field_id) const {
if (segment_->type() == SegmentType::Sealed) {
auto sealed_seg =
dynamic_cast<const segcore::SegmentSealed*>(segment_);
Assert(sealed_seg != nullptr);
if (sealed_seg->GetJsonKeyIndex(field_id) != nullptr) {
return true;
}
} else if (segment_->type() == SegmentType ::Growing) {
if (segment_->GetJsonKeyIndex(field_id) != nullptr) {
return true;
}
}
return false;
}
bool
CanUseNgramIndex(FieldId field_id) const {
return segment_->HasNgramIndex(field_id);
@ -1300,6 +1283,26 @@ class SegmentExpr : public Expr {
return segment_->HasNgramIndexForJson(field_id, nested_path);
}
bool
PlanUseJsonStats(EvalCtx& context) const {
return context.get_exec_context()
->get_query_context()
->get_plan_options()
.expr_use_json_stats;
}
bool
HasJsonStats(FieldId field_id) const {
return segment_->type() == SegmentType::Sealed &&
static_cast<const segcore::SegmentSealed*>(segment_)
->GetJsonStats(field_id) != nullptr;
}
bool
CanUseJsonStats(EvalCtx& context, FieldId field_id) const {
return PlanUseJsonStats(context) && HasJsonStats(field_id);
}
protected:
const segcore::SegmentInternalInterface* segment_;
const FieldId field_id_;

File diff suppressed because it is too large Load Diff

View File

@ -24,10 +24,426 @@
#include "exec/expression/Expr.h"
#include "exec/expression/Element.h"
#include "segcore/SegmentInterface.h"
#include "common/bson_view.h"
#include "exec/expression/Utils.h"
namespace milvus {
namespace exec {
class ShreddingArrayBsonContainsArrayExecutor {
public:
explicit ShreddingArrayBsonContainsArrayExecutor(
const std::vector<proto::plan::Array>& elems)
: elements_(elems) {
}
void
operator()(const std::string_view* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
milvus::BsonView bson(
reinterpret_cast<const uint8_t*>(src[i].data()), src[i].size());
auto array_view = bson.ParseAsArrayAtOffset(0);
if (!array_view.has_value()) {
res[i] = false;
continue;
}
bool matched = false;
for (const auto& sub_value : array_view.value()) {
auto sub_array = milvus::BsonView::GetValueFromBsonView<
bsoncxx::array::view>(sub_value.get_value());
if (!sub_array.has_value())
continue;
for (const auto& element : elements_) {
if (CompareTwoJsonArray(sub_array.value(), element)) {
matched = true;
break;
}
}
if (matched)
break;
}
res[i] = matched;
}
}
private:
const std::vector<proto::plan::Array> elements_;
};
class ShreddingArrayBsonContainsAllArrayExecutor {
public:
explicit ShreddingArrayBsonContainsAllArrayExecutor(
const std::vector<proto::plan::Array>& elems)
: elements_(elems) {
}
void
operator()(const std::string_view* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
milvus::BsonView bson(
reinterpret_cast<const uint8_t*>(src[i].data()), src[i].size());
auto array_view = bson.ParseAsArrayAtOffset(0);
if (!array_view.has_value()) {
res[i] = false;
continue;
}
std::set<int> exist_elements_index;
for (const auto& sub_value : array_view.value()) {
auto sub_array = milvus::BsonView::GetValueFromBsonView<
bsoncxx::array::view>(sub_value.get_value());
if (!sub_array.has_value())
continue;
for (int idx = 0; idx < static_cast<int>(elements_.size());
++idx) {
if (CompareTwoJsonArray(sub_array.value(),
elements_[idx])) {
exist_elements_index.insert(idx);
}
}
if (exist_elements_index.size() == elements_.size()) {
break;
}
}
res[i] = exist_elements_index.size() == elements_.size();
}
}
private:
const std::vector<proto::plan::Array> elements_;
};
template <typename GetType>
class ShreddingArrayBsonContainsAnyExecutor {
public:
ShreddingArrayBsonContainsAnyExecutor(
std::shared_ptr<MultiElement> arg_set,
std::shared_ptr<MultiElement> arg_set_double)
: arg_set_(std::move(arg_set)),
arg_set_double_(std::move(arg_set_double)) {
}
void
operator()(const std::string_view* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
milvus::BsonView bson(
reinterpret_cast<const uint8_t*>(src[i].data()), src[i].size());
auto array_view = bson.ParseAsArrayAtOffset(0);
if (!array_view.has_value()) {
res[i] = false;
continue;
}
bool matched = false;
for (const auto& element : array_view.value()) {
if constexpr (std::is_same_v<GetType, int64_t> ||
std::is_same_v<GetType, double>) {
auto value = milvus::BsonView::GetValueFromBsonView<double>(
element.get_value());
if (value.has_value() &&
arg_set_double_->In(value.value())) {
matched = true;
break;
}
} else {
auto value =
milvus::BsonView::GetValueFromBsonView<GetType>(
element.get_value());
if (value.has_value() && arg_set_->In(value.value())) {
matched = true;
break;
}
}
}
res[i] = matched;
}
}
private:
std::shared_ptr<MultiElement> arg_set_;
std::shared_ptr<MultiElement> arg_set_double_;
};
template <typename GetType>
class ShreddingArrayBsonContainsAllExecutor {
public:
explicit ShreddingArrayBsonContainsAllExecutor(
const std::set<GetType>& elements)
: elements_(elements) {
}
void
operator()(const std::string_view* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
milvus::BsonView bson(
reinterpret_cast<const uint8_t*>(src[i].data()), src[i].size());
auto array_view = bson.ParseAsArrayAtOffset(0);
if (!array_view.has_value()) {
res[i] = false;
continue;
}
std::set<GetType> tmp_elements(elements_);
for (const auto& element : array_view.value()) {
auto value = milvus::BsonView::GetValueFromBsonView<GetType>(
element.get_value());
if (!value.has_value()) {
continue;
}
tmp_elements.erase(value.value());
if (tmp_elements.empty()) {
break;
}
}
res[i] = tmp_elements.empty();
}
}
private:
std::set<GetType> elements_;
};
class ShreddingArrayBsonContainsAllWithDiffTypeExecutor {
public:
ShreddingArrayBsonContainsAllWithDiffTypeExecutor(
std::vector<proto::plan::GenericValue> elements,
std::set<int> elements_index)
: elements_(std::move(elements)),
elements_index_(std::move(elements_index)) {
}
void
operator()(const std::string_view* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
milvus::BsonView bson(
reinterpret_cast<const uint8_t*>(src[i].data()), src[i].size());
auto array = bson.ParseAsArrayAtOffset(0);
if (!array.has_value()) {
res[i] = false;
continue;
}
std::set<int> tmp_elements_index(elements_index_);
for (const auto& sub_value : array.value()) {
int idx = -1;
for (auto& element : elements_) {
idx++;
switch (element.val_case()) {
case proto::plan::GenericValue::kBoolVal: {
auto val =
milvus::BsonView::GetValueFromBsonView<bool>(
sub_value.get_value());
if (!val.has_value()) {
continue;
}
if (val.value() == element.bool_val()) {
tmp_elements_index.erase(idx);
}
break;
}
case proto::plan::GenericValue::kInt64Val: {
auto val =
milvus::BsonView::GetValueFromBsonView<int64_t>(
sub_value.get_value());
if (!val.has_value()) {
continue;
}
if (val.value() == element.int64_val()) {
tmp_elements_index.erase(idx);
}
break;
}
case proto::plan::GenericValue::kFloatVal: {
auto val =
milvus::BsonView::GetValueFromBsonView<double>(
sub_value.get_value());
if (!val.has_value()) {
continue;
}
if (val.value() == element.float_val()) {
tmp_elements_index.erase(idx);
}
break;
}
case proto::plan::GenericValue::kStringVal: {
auto val = milvus::BsonView::GetValueFromBsonView<
std::string>(sub_value.get_value());
if (!val.has_value()) {
continue;
}
if (val.value() == element.string_val()) {
tmp_elements_index.erase(idx);
}
break;
}
case proto::plan::GenericValue::kArrayVal: {
auto val = milvus::BsonView::GetValueFromBsonView<
bsoncxx::array::view>(sub_value.get_value());
if (!val.has_value()) {
continue;
}
if (CompareTwoJsonArray(val.value(),
element.array_val())) {
tmp_elements_index.erase(idx);
}
break;
}
default:
ThrowInfo(DataTypeInvalid,
fmt::format("unsupported data type {}",
element.val_case()));
}
if (tmp_elements_index.size() == 0) {
break;
}
}
if (tmp_elements_index.size() == 0) {
break;
}
}
res[i] = tmp_elements_index.size() == 0;
}
}
private:
std::vector<proto::plan::GenericValue> elements_;
std::set<int> elements_index_;
};
class ShreddingArrayBsonContainsAnyWithDiffTypeExecutor {
public:
explicit ShreddingArrayBsonContainsAnyWithDiffTypeExecutor(
std::vector<proto::plan::GenericValue> elements)
: elements_(std::move(elements)) {
}
void
operator()(const std::string_view* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
milvus::BsonView bson(
reinterpret_cast<const uint8_t*>(src[i].data()), src[i].size());
auto array = bson.ParseAsArrayAtOffset(0);
if (!array.has_value()) {
res[i] = false;
continue;
}
bool matched = false;
for (const auto& sub_value : array.value()) {
for (auto const& element : elements_) {
switch (element.val_case()) {
case proto::plan::GenericValue::kBoolVal: {
auto val =
milvus::BsonView::GetValueFromBsonView<bool>(
sub_value.get_value());
if (val.has_value() &&
val.value() == element.bool_val()) {
matched = true;
}
break;
}
case proto::plan::GenericValue::kInt64Val: {
auto val =
milvus::BsonView::GetValueFromBsonView<int64_t>(
sub_value.get_value());
if (val.has_value() &&
val.value() == element.int64_val()) {
matched = true;
}
break;
}
case proto::plan::GenericValue::kFloatVal: {
auto val =
milvus::BsonView::GetValueFromBsonView<double>(
sub_value.get_value());
if (val.has_value() &&
val.value() == element.float_val()) {
matched = true;
}
break;
}
case proto::plan::GenericValue::kStringVal: {
auto val = milvus::BsonView::GetValueFromBsonView<
std::string>(sub_value.get_value());
if (val.has_value() &&
val.value() == element.string_val()) {
matched = true;
}
break;
}
case proto::plan::GenericValue::kArrayVal: {
auto val = milvus::BsonView::GetValueFromBsonView<
bsoncxx::array::view>(sub_value.get_value());
if (val.has_value() &&
CompareTwoJsonArray(val.value(),
element.array_val())) {
matched = true;
}
break;
}
default:
ThrowInfo(DataTypeInvalid,
fmt::format("unsupported data type {}",
element.val_case()));
}
if (matched)
break;
}
if (matched)
break;
}
res[i] = matched;
}
}
private:
std::vector<proto::plan::GenericValue> elements_;
};
class PhyJsonContainsFilterExpr : public SegmentExpr {
public:
PhyJsonContainsFilterExpr(
@ -82,7 +498,7 @@ class PhyJsonContainsFilterExpr : public SegmentExpr {
template <typename ExprValueType>
VectorPtr
ExecJsonContainsByKeyIndex();
ExecJsonContainsByStats();
template <typename ExprValueType>
VectorPtr
@ -94,7 +510,7 @@ class PhyJsonContainsFilterExpr : public SegmentExpr {
template <typename ExprValueType>
VectorPtr
ExecJsonContainsAllByKeyIndex();
ExecJsonContainsAllByStats();
template <typename ExprValueType>
VectorPtr
@ -104,25 +520,25 @@ class PhyJsonContainsFilterExpr : public SegmentExpr {
ExecJsonContainsArray(EvalCtx& context);
VectorPtr
ExecJsonContainsArrayByKeyIndex();
ExecJsonContainsArrayByStats();
VectorPtr
ExecJsonContainsAllArray(EvalCtx& context);
VectorPtr
ExecJsonContainsAllArrayByKeyIndex();
ExecJsonContainsAllArrayByStats();
VectorPtr
ExecJsonContainsAllWithDiffType(EvalCtx& context);
VectorPtr
ExecJsonContainsAllWithDiffTypeByKeyIndex();
ExecJsonContainsAllWithDiffTypeByStats();
VectorPtr
ExecJsonContainsWithDiffType(EvalCtx& context);
VectorPtr
ExecJsonContainsWithDiffTypeByKeyIndex();
ExecJsonContainsWithDiffTypeByStats();
VectorPtr
EvalArrayContainsForIndexSegment(DataType data_type);
@ -135,6 +551,7 @@ class PhyJsonContainsFilterExpr : public SegmentExpr {
std::shared_ptr<const milvus::expr::JsonContainsExpr> expr_;
bool arg_inited_{false};
std::shared_ptr<MultiElement> arg_set_;
std::shared_ptr<MultiElement> arg_set_double_;
};
} //namespace exec
} // namespace milvus

View File

@ -529,17 +529,26 @@ PhyTermFilterExpr::ExecTermJsonVariableInField(EvalCtx& context) {
template <typename ValueType>
VectorPtr
PhyTermFilterExpr::ExecJsonInVariableByKeyIndex() {
PhyTermFilterExpr::ExecJsonInVariableByStats() {
using GetType = std::conditional_t<std::is_same_v<ValueType, std::string>,
std::string_view,
ValueType>;
auto real_batch_size = GetNextBatchSize();
auto pointer = milvus::Json::pointer(expr_->column_.nested_path_);
auto pointer = milvus::index::JsonPointer(expr_->column_.nested_path_);
if (!arg_inited_) {
arg_set_ = std::make_shared<SetElement<ValueType>>(expr_->vals_);
if constexpr (std::is_same_v<GetType, double>) {
arg_set_float_ = std::make_shared<SetElement<float>>(expr_->vals_);
if constexpr (std::is_same_v<GetType, int64_t>) {
// for int64_t, we need to a double vector to store the values
auto int_arg_set =
std::static_pointer_cast<SetElement<int64_t>>(arg_set_);
std::vector<double> double_vals;
for (const auto& val : int_arg_set->GetElements()) {
double_vals.emplace_back(static_cast<double>(val));
}
arg_set_double_ = std::make_shared<SetElement<double>>(double_vals);
} else if constexpr (std::is_same_v<GetType, double>) {
arg_set_double_ = arg_set_;
}
arg_inited_ = true;
}
@ -551,154 +560,132 @@ PhyTermFilterExpr::ExecJsonInVariableByKeyIndex() {
TargetBitmap(real_batch_size, true));
}
if (cached_index_chunk_id_ != 0) {
const segcore::SegmentInternalInterface* segment = nullptr;
if (segment_->type() == SegmentType::Growing) {
segment =
dynamic_cast<const segcore::SegmentGrowingImpl*>(segment_);
} else if (segment_->type() == SegmentType::Sealed) {
segment = dynamic_cast<const segcore::SegmentSealed*>(segment_);
}
if (cached_index_chunk_id_ != 0 &&
segment_->type() == SegmentType::Sealed) {
auto segment = dynamic_cast<const segcore::SegmentSealed*>(segment_);
auto field_id = expr_->column_.field_id_;
auto* index = segment->GetJsonKeyIndex(field_id);
auto vals = expr_->vals_;
auto* index = segment->GetJsonStats(field_id);
Assert(index != nullptr);
auto filter_func = [this, segment, &field_id](
const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
std::vector<int64_t> invalid_row_ids;
std::vector<int64_t> invalid_offset;
std::vector<int64_t> invalid_type;
std::vector<int64_t> invalid_size;
for (size_t i = 0; i < n; i++) {
auto valid = valid_array[i];
auto type = type_array[i];
auto row_id = row_id_array[i];
auto offset = offset_array[i];
auto size = size_array[i];
auto value = value_array[i];
if (!valid) {
invalid_row_ids.push_back(row_id);
invalid_offset.push_back(offset);
invalid_type.push_back(type);
invalid_size.push_back(size);
continue;
}
auto f = [&]() {
if constexpr (std::is_same_v<GetType, int64_t>) {
if (type != uint8_t(milvus::index::JSONType::INT32) &&
type != uint8_t(milvus::index::JSONType::INT64) &&
type != uint8_t(milvus::index::JSONType::FLOAT) &&
type != uint8_t(milvus::index::JSONType::DOUBLE)) {
return false;
cached_index_chunk_res_ = std::make_shared<TargetBitmap>(active_count_);
cached_index_chunk_valid_res_ =
std::make_shared<TargetBitmap>(active_count_, true);
TargetBitmapView res_view(*cached_index_chunk_res_);
TargetBitmapView valid_res_view(*cached_index_chunk_valid_res_);
// process shredding data
auto try_execute = [&](milvus::index::JSONType json_type,
TargetBitmapView& res_view,
TargetBitmapView& valid_res_view,
auto GetType) {
auto target_field = index->GetShreddingField(pointer, json_type);
if (!target_field.empty()) {
using ColType = decltype(GetType);
auto shredding_executor = [this](const ColType* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
} else if constexpr (std::is_same_v<GetType,
std::string_view>) {
if (type != uint8_t(milvus::index::JSONType::STRING) &&
type !=
uint8_t(
milvus::index::JSONType::STRING_ESCAPE)) {
return false;
if constexpr (std::is_same_v<ColType, double>) {
res[i] = this->arg_set_double_->In(src[i]);
} else {
res[i] = this->arg_set_->In(src[i]);
}
} else if constexpr (std::is_same_v<GetType, double>) {
if (type != uint8_t(milvus::index::JSONType::INT32) &&
type != uint8_t(milvus::index::JSONType::INT64) &&
type != uint8_t(milvus::index::JSONType::FLOAT) &&
type != uint8_t(milvus::index::JSONType::DOUBLE)) {
return false;
}
} else if constexpr (std::is_same_v<GetType, bool>) {
if (type != uint8_t(milvus::index::JSONType::BOOL)) {
return false;
}
}
if constexpr (std::is_same_v<GetType, int64_t>) {
return this->arg_set_->In(value);
} else if constexpr (std::is_same_v<GetType, double>) {
float restoredValue = *reinterpret_cast<float*>(&value);
return this->arg_set_float_->In(restoredValue);
} else if constexpr (std::is_same_v<GetType, bool>) {
bool restoredValue = *reinterpret_cast<bool*>(&value);
return this->arg_set_->In(restoredValue);
}
};
bitset[row_id] = f();
index->ExecutorForShreddingData<ColType>(target_field,
shredding_executor,
nullptr,
res_view,
valid_res_view);
LOG_DEBUG("using shredding data's field: {} count {}",
target_field,
res_view.count());
}
auto f = [&](const milvus::Json& json,
uint8_t type,
uint16_t offset,
uint16_t size,
bool is_valid) {
if (!is_valid) {
return false;
}
if (type == uint8_t(milvus::index::JSONType::STRING) ||
type == uint8_t(milvus::index::JSONType::DOUBLE) ||
type == uint8_t(milvus::index::JSONType::INT64)) {
if (type == uint8_t(milvus::index::JSONType::STRING)) {
if constexpr (std::is_same_v<GetType,
std::string_view>) {
auto val = json.at_string(offset, size);
return this->arg_set_->In(ValueType(val));
} else {
return false;
}
} else if (type ==
uint8_t(milvus::index::JSONType::DOUBLE)) {
if constexpr (std::is_same_v<GetType, double>) {
auto val = std::stod(
std::string(json.at_string(offset, size)));
return this->arg_set_->In(ValueType(val));
} else {
return false;
}
} else if (type ==
uint8_t(milvus::index::JSONType::INT64)) {
if constexpr (std::is_same_v<GetType, int64_t>) {
auto val = std::stoll(
std::string(json.at_string(offset, size)));
return this->arg_set_->In(ValueType(val));
} else {
return false;
}
}
} else {
auto val = json.at<GetType>(offset, size);
if (val.error()) {
return false;
}
return this->arg_set_->In(ValueType(val.value()));
}
};
segment->BulkGetJsonData(
field_id,
[&](const milvus::Json& json, size_t i, bool is_valid) {
auto row_id = invalid_row_ids[i];
auto type = invalid_type[i];
auto offset = invalid_offset[i];
auto size = invalid_size[i];
bitset[row_id] = f(json, type, offset, size, is_valid);
},
invalid_row_ids.data(),
invalid_row_ids.size());
};
bool is_growing = segment_->type() == SegmentType::Growing;
bool is_strong_consistency = consistency_level_ == 0;
cached_index_chunk_res_ = std::make_shared<TargetBitmap>(
std::move(index->FilterByPath(pointer,
active_count_,
is_growing,
is_strong_consistency,
filter_func)));
if constexpr (std::is_same_v<GetType, bool>) {
try_execute(milvus::index::JSONType::BOOL,
res_view,
valid_res_view,
bool{});
} else if constexpr (std::is_same_v<GetType, int64_t>) {
try_execute(milvus::index::JSONType::INT64,
res_view,
valid_res_view,
int64_t{});
// and double compare
TargetBitmap res_double(active_count_, false);
TargetBitmapView res_double_view(res_double);
TargetBitmap res_double_valid(active_count_, true);
TargetBitmapView valid_res_double_view(res_double_valid);
try_execute(milvus::index::JSONType::DOUBLE,
res_double_view,
valid_res_double_view,
double{});
res_view.inplace_or_with_count(res_double_view, active_count_);
valid_res_view.inplace_or_with_count(valid_res_double_view,
active_count_);
} else if constexpr (std::is_same_v<GetType, double>) {
try_execute(milvus::index::JSONType::DOUBLE,
res_view,
valid_res_view,
double{});
// and int64 compare
TargetBitmap res_int64(active_count_, false);
TargetBitmapView res_int64_view(res_int64);
TargetBitmap res_int64_valid(active_count_, true);
TargetBitmapView valid_res_int64_view(res_int64_valid);
try_execute(milvus::index::JSONType::INT64,
res_int64_view,
valid_res_int64_view,
int64_t{});
res_view.inplace_or_with_count(res_int64_view, active_count_);
valid_res_view.inplace_or_with_count(valid_res_int64_view,
active_count_);
} else if constexpr (std::is_same_v<GetType, std::string_view> ||
std::is_same_v<GetType, std::string>) {
try_execute(milvus::index::JSONType::STRING,
res_view,
valid_res_view,
std::string_view{});
}
// process shared data
auto shared_executor = [this, &res_view](milvus::BsonView bson,
uint32_t row_offset,
uint32_t value_offset) {
auto get_value = bson.ParseAsValueAtOffset<GetType>(value_offset);
if constexpr (std::is_same_v<GetType, int64_t> ||
std::is_same_v<GetType, double>) {
auto get_value =
bson.ParseAsValueAtOffset<double>(value_offset);
if (get_value.has_value()) {
res_view[row_offset] =
this->arg_set_double_->In(get_value.value());
}
return;
} else {
auto get_value =
bson.ParseAsValueAtOffset<GetType>(value_offset);
if (get_value.has_value()) {
res_view[row_offset] =
this->arg_set_->In(get_value.value());
}
return;
}
};
if (!index->CanSkipShared(pointer)) {
index->ExecuteForSharedData(pointer, shared_executor);
}
cached_index_chunk_id_ = 0;
}
@ -719,8 +706,8 @@ PhyTermFilterExpr::ExecTermJsonFieldInVariable(EvalCtx& context) {
auto* input = context.get_offset_input();
const auto& bitmap_input = context.get_bitmap_input();
FieldId field_id = expr_->column_.field_id_;
if (CanUseJsonKeyIndex(field_id) && !has_offset_input_) {
return ExecJsonInVariableByKeyIndex<ValueType>();
if (!has_offset_input_ && CanUseJsonStats(context, field_id)) {
return ExecJsonInVariableByStats<ValueType>();
}
auto real_batch_size =

View File

@ -141,7 +141,7 @@ class PhyTermFilterExpr : public SegmentExpr {
template <typename ValueType>
VectorPtr
ExecJsonInVariableByKeyIndex();
ExecJsonInVariableByStats();
private:
std::shared_ptr<const milvus::expr::TermFilterExpr> expr_;
@ -150,7 +150,7 @@ class PhyTermFilterExpr : public SegmentExpr {
TargetBitmap cached_bits_;
bool arg_inited_{false};
std::shared_ptr<MultiElement> arg_set_;
std::shared_ptr<MultiElement> arg_set_float_;
std::shared_ptr<MultiElement> arg_set_double_;
SingleElement arg_val_;
int32_t consistency_level_ = 0;
};

View File

@ -16,14 +16,16 @@
#include "UnaryExpr.h"
#include <optional>
#include <boost/regex.hpp>
#include "common/EasyAssert.h"
#include "common/Json.h"
#include "common/Types.h"
#include "exec/expression/ExprCache.h"
#include "common/type_c.h"
#include "log/Log.h"
#include "monitor/Monitor.h"
#include "common/ScopedTimer.h"
#include <boost/regex.hpp>
namespace milvus {
namespace exec {
template <typename T>
@ -663,8 +665,8 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplJson(EvalCtx& context) {
const auto& bitmap_input = context.get_bitmap_input();
FieldId field_id = expr_->column_.field_id_;
if (CanUseJsonKeyIndex(field_id) && !has_offset_input_) {
return ExecRangeVisitorImplJsonForIndex<ExprValueType>();
if (!has_offset_input_ && CanUseJsonStats(context, field_id)) {
return ExecRangeVisitorImplJsonByStats<ExprValueType>();
}
auto real_batch_size =
@ -965,7 +967,7 @@ PhyUnaryRangeFilterExpr::SplitAtFirstSlashDigit(std::string input) {
template <typename ExprValueType>
VectorPtr
PhyUnaryRangeFilterExpr::ExecRangeVisitorImplJsonForIndex() {
PhyUnaryRangeFilterExpr::ExecRangeVisitorImplJsonByStats() {
using GetType =
std::conditional_t<std::is_same_v<ExprValueType, std::string>,
std::string_view,
@ -974,509 +976,261 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplJsonForIndex() {
if (real_batch_size == 0) {
return nullptr;
}
auto pointerpath = milvus::Json::pointer(expr_->column_.nested_path_);
auto pointerpair = SplitAtFirstSlashDigit(pointerpath);
std::string pointer = pointerpair.first;
std::string arrayIndex = pointerpair.second;
#define UnaryRangeJSONIndexCompare(cmp) \
do { \
auto x = json.at<GetType>(offset, size); \
if (x.error()) { \
if constexpr (std::is_same_v<GetType, int64_t>) { \
auto x = json.at<double>(offset, size); \
return !x.error() && (cmp); \
} \
return false; \
} \
return (cmp); \
} while (false)
if (cached_index_chunk_id_ != 0 &&
segment_->type() == SegmentType::Sealed) {
auto pointerpath = milvus::Json::pointer(expr_->column_.nested_path_);
auto pointerpair = SplitAtFirstSlashDigit(pointerpath);
std::string pointer = pointerpair.first;
size_t array_index = pointerpair.second.empty()
? INVALID_ARRAY_INDEX
: std::stoi(pointerpair.second);
#define UnaryJSONTypeCompare(cmp) \
do { \
if constexpr (std::is_same_v<GetType, std::string_view>) { \
if (type == uint8_t(milvus::index::JSONType::STRING)) { \
auto x = json.at_string(offset, size); \
return (cmp); \
} else { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, double>) { \
if (type == uint8_t(milvus::index::JSONType::INT64)) { \
auto x = \
std::stoll(std::string(json.at_string(offset, size))); \
return (cmp); \
} else if (type == uint8_t(milvus::index::JSONType::DOUBLE)) { \
auto x = std::stod(std::string(json.at_string(offset, size))); \
return (cmp); \
} else { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, int64_t>) { \
if (type == uint8_t(milvus::index::JSONType::INT64)) { \
auto x = \
std::stoll(std::string(json.at_string(offset, size))); \
return (cmp); \
} else if (type == uint8_t(milvus::index::JSONType::DOUBLE)) { \
auto x = std::stod(std::string(json.at_string(offset, size))); \
return (cmp); \
} else { \
return false; \
} \
} \
} while (false)
ExprValueType val = GetValueFromProto<ExprValueType>(expr_->val_);
auto op_type = expr_->op_type_;
#define UnaryJSONTypeCompareWithValue(cmp) \
do { \
if constexpr (std::is_same_v<GetType, int64_t>) { \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
float x = *reinterpret_cast<float*>(&value); \
return (cmp); \
} else { \
int64_t x = value; \
return (cmp); \
} \
} else if constexpr (std::is_same_v<GetType, double>) { \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
float x = *reinterpret_cast<float*>(&value); \
return (cmp); \
} else { \
int64_t x = value; \
return (cmp); \
} \
} else if constexpr (std::is_same_v<GetType, bool>) { \
bool x = *reinterpret_cast<bool*>(&value); \
return (cmp); \
} \
} while (false)
#define CompareValueWithOpType(type, value, val, op_type) \
switch (op_type) { \
case proto::plan::GreaterThan: \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
UnaryJSONTypeCompareWithValue(x > static_cast<float>(val)); \
} else { \
UnaryJSONTypeCompareWithValue(x > val); \
} \
break; \
case proto::plan::GreaterEqual: \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
UnaryJSONTypeCompareWithValue(x >= static_cast<float>(val)); \
} else { \
UnaryJSONTypeCompareWithValue(x >= val); \
} \
break; \
case proto::plan::LessThan: \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
UnaryJSONTypeCompareWithValue(x < static_cast<float>(val)); \
} else { \
UnaryJSONTypeCompareWithValue(x < val); \
} \
break; \
case proto::plan::LessEqual: \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
UnaryJSONTypeCompareWithValue(x <= static_cast<float>(val)); \
} else { \
UnaryJSONTypeCompareWithValue(x <= val); \
} \
break; \
case proto::plan::Equal: \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
UnaryJSONTypeCompareWithValue(x == static_cast<float>(val)); \
} else { \
UnaryJSONTypeCompareWithValue(x == val); \
} \
break; \
case proto::plan::NotEqual: \
if (type == uint8_t(milvus::index::JSONType::FLOAT)) { \
UnaryJSONTypeCompareWithValue(x != static_cast<float>(val)); \
} else { \
UnaryJSONTypeCompareWithValue(x != val); \
} \
break; \
default: \
return false; \
}
#define UnaryRangeJSONIndexCompareWithArrayIndex(cmp) \
do { \
if (type != uint8_t(milvus::index::JSONType::UNKNOWN)) { \
return false; \
} \
auto array = json.array_at(offset, size); \
if (array.error()) { \
return false; \
} \
auto value = array.at_pointer(arrayIndex); \
if (value.error()) { \
return false; \
} \
if constexpr (std::is_same_v<GetType, int64_t> || \
std::is_same_v<GetType, double>) { \
if (!value.is_number()) { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, std::string_view>) { \
if (!value.is_string()) { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, bool>) { \
if (!value.is_bool()) { \
return false; \
} \
} \
auto x = value.get<GetType>(); \
if (x.error()) { \
if constexpr (std::is_same_v<GetType, int64_t>) { \
auto x = value.get<double>(); \
return !x.error() && (cmp); \
} \
} \
return (cmp); \
} while (false)
#define UnaryRangeJSONIndexCompareNotEqual(cmp) \
do { \
auto x = json.at<GetType>(offset, size); \
if (x.error()) { \
if constexpr (std::is_same_v<GetType, int64_t>) { \
auto x = json.at<double>(offset, size); \
return x.error() || (cmp); \
} \
return true; \
} \
return (cmp); \
} while (false)
#define UnaryRangeJSONIndexCompareNotEqualWithArrayIndex(cmp) \
do { \
auto array = json.array_at(offset, size); \
if (array.error()) { \
return false; \
} \
auto value = array.at_pointer(arrayIndex); \
if (value.error()) { \
return false; \
} \
if constexpr (std::is_same_v<GetType, int64_t> || \
std::is_same_v<GetType, double>) { \
if (!value.is_number()) { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, std::string_view>) { \
if (!value.is_string()) { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, bool>) { \
if (!value.is_bool()) { \
return false; \
} \
} \
auto x = value.get<GetType>(); \
if (x.error()) { \
if constexpr (std::is_same_v<GetType, int64_t>) { \
auto x = value.get<double>(); \
return x.error() || (cmp); \
} \
} \
return (cmp); \
} while (false)
#define CHECKISJSONTYPEWITHOFFSET(type) \
(type == uint8_t(milvus::index::JSONType::STRING) || \
type == uint8_t(milvus::index::JSONType::DOUBLE) || \
type == uint8_t(milvus::index::JSONType::INT64))
#define CHECKJSONTYPEISNUMBER(type) \
if ((type != uint8_t(milvus::index::JSONType::INT32)) && \
(type != uint8_t(milvus::index::JSONType::INT64)) && \
(type != uint8_t(milvus::index::JSONType::FLOAT)) && \
(type != uint8_t(milvus::index::JSONType::DOUBLE))) { \
return false; \
}
#define ISVALIDJSONTYPE(type, GetType) \
if constexpr (std::is_same_v<GetType, int64_t>) { \
CHECKJSONTYPEISNUMBER(type) \
} else if constexpr (std::is_same_v<GetType, std::string_view>) { \
if ((type != uint8_t(milvus::index::JSONType::STRING)) && \
(type != uint8_t(milvus::index::JSONType::STRING_ESCAPE))) { \
return false; \
} \
} else if constexpr (std::is_same_v<GetType, double>) { \
CHECKJSONTYPEISNUMBER(type) \
} else if constexpr (std::is_same_v<GetType, bool>) { \
if (type != uint8_t(milvus::index::JSONType::BOOL)) { \
return false; \
} \
}
ExprValueType val = GetValueFromProto<ExprValueType>(expr_->val_);
auto op_type = expr_->op_type_;
if (cached_index_chunk_id_ != 0) {
cached_index_chunk_id_ = 0;
const segcore::SegmentInternalInterface* segment = nullptr;
if (segment_->type() == SegmentType::Growing) {
segment =
dynamic_cast<const segcore::SegmentGrowingImpl*>(segment_);
} else if (segment_->type() == SegmentType::Sealed) {
segment = dynamic_cast<const segcore::SegmentSealed*>(segment_);
}
auto segment = static_cast<const segcore::SegmentSealed*>(segment_);
auto field_id = expr_->column_.field_id_;
auto* index = segment->GetJsonKeyIndex(field_id);
auto* index = segment->GetJsonStats(field_id);
Assert(index != nullptr);
Assert(segment != nullptr);
auto filter_func = [segment,
field_id,
op_type,
val,
arrayIndex,
pointer](const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
std::vector<int64_t> invalid_row_ids;
std::vector<int64_t> invalid_offset;
std::vector<int64_t> invalid_type;
std::vector<int64_t> invalid_size;
for (size_t i = 0; i < n; i++) {
auto valid = valid_array[i];
auto type = type_array[i];
auto row_id = row_id_array[i];
auto offset = offset_array[i];
auto size = size_array[i];
auto value = value_array[i];
if (!valid) {
invalid_row_ids.push_back(row_id);
invalid_offset.push_back(offset);
invalid_type.push_back(type);
invalid_size.push_back(size);
continue;
}
auto f = [&]() {
if (type == uint8_t(milvus::index::JSONType::UNKNOWN) ||
!arrayIndex.empty()) {
return false;
}
ISVALIDJSONTYPE(type, GetType);
switch (op_type) {
case proto::plan::GreaterThan:
CompareValueWithOpType(type, value, val, op_type);
case proto::plan::GreaterEqual:
CompareValueWithOpType(type, value, val, op_type);
case proto::plan::LessThan:
CompareValueWithOpType(type, value, val, op_type);
case proto::plan::LessEqual:
CompareValueWithOpType(type, value, val, op_type);
case proto::plan::Equal:
CompareValueWithOpType(type, value, val, op_type);
case proto::plan::NotEqual:
CompareValueWithOpType(type, value, val, op_type);
default:
return false;
}
};
bitset[row_id] = f();
}
auto f = [&](const milvus::Json& json,
uint8_t type,
uint16_t offset,
uint16_t size,
bool is_valid) {
if (!is_valid) {
return false;
}
switch (op_type) {
case proto::plan::GreaterThan:
if constexpr (std::is_same_v<GetType,
proto::plan::Array>) {
return false;
} else {
if (!arrayIndex.empty()) {
UnaryRangeJSONIndexCompareWithArrayIndex(
ExprValueType(x.value()) > val);
} else {
if (CHECKISJSONTYPEWITHOFFSET(type)) {
UnaryJSONTypeCompare(x > val);
} else {
UnaryRangeJSONIndexCompare(
ExprValueType(x.value()) > val);
}
}
}
case proto::plan::GreaterEqual:
if constexpr (std::is_same_v<GetType,
proto::plan::Array>) {
return false;
} else {
if (!arrayIndex.empty()) {
UnaryRangeJSONIndexCompareWithArrayIndex(
ExprValueType(x.value()) >= val);
} else {
if (CHECKISJSONTYPEWITHOFFSET(type)) {
UnaryJSONTypeCompare(x >= val);
} else {
UnaryRangeJSONIndexCompare(
ExprValueType(x.value()) >= val);
}
}
}
case proto::plan::LessThan:
if constexpr (std::is_same_v<GetType,
proto::plan::Array>) {
return false;
} else {
if (!arrayIndex.empty()) {
UnaryRangeJSONIndexCompareWithArrayIndex(
ExprValueType(x.value()) < val);
} else {
if (CHECKISJSONTYPEWITHOFFSET(type)) {
UnaryJSONTypeCompare(x < val);
} else {
UnaryRangeJSONIndexCompare(
ExprValueType(x.value()) < val);
}
}
}
case proto::plan::LessEqual:
if constexpr (std::is_same_v<GetType,
proto::plan::Array>) {
return false;
} else {
if (!arrayIndex.empty()) {
UnaryRangeJSONIndexCompareWithArrayIndex(
ExprValueType(x.value()) <= val);
} else {
if (CHECKISJSONTYPEWITHOFFSET(type)) {
UnaryJSONTypeCompare(x <= val);
} else {
UnaryRangeJSONIndexCompare(
ExprValueType(x.value()) <= val);
}
}
}
cached_index_chunk_res_ = std::make_shared<TargetBitmap>(active_count_);
cached_index_chunk_valid_res_ =
std::make_shared<TargetBitmap>(active_count_, true);
TargetBitmapView res_view(*cached_index_chunk_res_);
TargetBitmapView valid_res_view(*cached_index_chunk_valid_res_);
case proto::plan::Equal:
if constexpr (std::is_same_v<GetType,
proto::plan::Array>) {
if (type !=
uint8_t(milvus::index::JSONType::UNKNOWN)) {
return false;
}
auto array = json.array_at(offset, size);
if (array.error()) {
return false;
}
return CompareTwoJsonArray(array.value(), val);
} else {
if (!arrayIndex.empty()) {
UnaryRangeJSONIndexCompareWithArrayIndex(
ExprValueType(x.value()) == val);
} else {
if (CHECKISJSONTYPEWITHOFFSET(type)) {
UnaryJSONTypeCompare(x == val);
} else {
UnaryRangeJSONIndexCompare(
ExprValueType(x.value()) == val);
}
}
}
case proto::plan::NotEqual:
if constexpr (std::is_same_v<GetType,
proto::plan::Array>) {
if (type !=
uint8_t(milvus::index::JSONType::UNKNOWN)) {
return false;
}
auto array = json.array_at(offset, size);
if (array.error()) {
return false;
}
return !CompareTwoJsonArray(array.value(), val);
} else {
if (!arrayIndex.empty()) {
UnaryRangeJSONIndexCompareNotEqualWithArrayIndex(
ExprValueType(x.value()) != val);
} else {
if (CHECKISJSONTYPEWITHOFFSET(type)) {
UnaryJSONTypeCompare(x != val);
} else {
UnaryRangeJSONIndexCompareNotEqual(
ExprValueType(x.value()) != val);
}
}
}
case proto::plan::InnerMatch:
case proto::plan::PostfixMatch:
case proto::plan::PrefixMatch:
if constexpr (std::is_same_v<GetType,
proto::plan::Array>) {
return false;
} else {
if (!arrayIndex.empty()) {
UnaryRangeJSONIndexCompareWithArrayIndex(
milvus::query::Match(
ExprValueType(x.value()),
val,
op_type));
} else {
if (CHECKISJSONTYPEWITHOFFSET(type)) {
UnaryJSONTypeCompare(
milvus::query::Match(x, val, op_type));
} else {
UnaryRangeJSONIndexCompare(
milvus::query::Match(
ExprValueType(x.value()),
val,
op_type));
}
}
}
case proto::plan::Match:
if constexpr (std::is_same_v<GetType,
proto::plan::Array>) {
return false;
} else {
PatternMatchTranslator translator;
auto regex_pattern = translator(val);
RegexMatcher matcher(regex_pattern);
if (!arrayIndex.empty()) {
UnaryRangeJSONIndexCompareWithArrayIndex(
matcher(ExprValueType(x.value())));
} else {
UnaryRangeJSONIndexCompare(
matcher(ExprValueType(x.value())));
}
}
default:
return false;
}
};
segment->BulkGetJsonData(
field_id,
[&](const milvus::Json& json, size_t i, bool is_valid) {
auto row_id = invalid_row_ids[i];
auto type = invalid_type[i];
auto offset = invalid_offset[i];
auto size = invalid_size[i];
bitset[row_id] = f(json, type, offset, size, is_valid);
},
invalid_row_ids.data(),
invalid_row_ids.size());
// process shredding data
auto try_execute = [&](milvus::index::JSONType json_type,
TargetBitmapView& res_view,
TargetBitmapView& valid_res_view,
auto GetType,
auto ValType) {
auto target_field = index->GetShreddingField(pointer, json_type);
if (!target_field.empty()) {
using ColType = decltype(GetType);
using ValType = decltype(ValType);
ShreddingExecutor<ColType, ValType> executor(
op_type, pointer, val);
index->ExecutorForShreddingData<ColType>(
target_field, executor, nullptr, res_view, valid_res_view);
LOG_DEBUG(
"using shredding data's field: {} with value {}, count {}",
target_field,
val,
res_view.count());
}
};
bool is_growing = segment_->type() == SegmentType::Growing;
bool is_strong_consistency = consistency_level_ == 0;
cached_index_chunk_res_ = std::make_shared<TargetBitmap>(
std::move(index->FilterByPath(pointer,
active_count_,
is_growing,
is_strong_consistency,
filter_func)));
{
milvus::ScopedTimer timer(
"unary_json_stats_shredding_data", [](double ms) {
milvus::monitor::internal_json_stats_latency_shredding
.Observe(ms);
});
if constexpr (std::is_same_v<GetType, bool>) {
try_execute(milvus::index::JSONType::BOOL,
res_view,
valid_res_view,
bool{},
bool{});
} else if constexpr (std::is_same_v<GetType, int64_t>) {
try_execute(milvus::index::JSONType::INT64,
res_view,
valid_res_view,
int64_t{},
int64_t{});
// and double compare
TargetBitmap res_double(active_count_, false);
TargetBitmapView res_double_view(res_double);
TargetBitmap res_double_valid(active_count_, true);
TargetBitmapView valid_res_double_view(res_double_valid);
try_execute(milvus::index::JSONType::DOUBLE,
res_double_view,
valid_res_double_view,
double{},
int64_t{});
res_view.inplace_or_with_count(res_double_view, active_count_);
valid_res_view.inplace_or_with_count(valid_res_double_view,
active_count_);
} else if constexpr (std::is_same_v<GetType, double>) {
try_execute(milvus::index::JSONType::DOUBLE,
res_view,
valid_res_view,
double{},
double{});
// add int64 compare
TargetBitmap res_int64(active_count_, false);
TargetBitmapView res_int64_view(res_int64);
TargetBitmap res_int64_valid(active_count_, true);
TargetBitmapView valid_res_int64_view(res_int64_valid);
try_execute(milvus::index::JSONType::INT64,
res_int64_view,
valid_res_int64_view,
int64_t{},
double{});
res_view.inplace_or_with_count(res_int64_view, active_count_);
valid_res_view.inplace_or_with_count(valid_res_int64_view,
active_count_);
} else if constexpr (std::is_same_v<GetType, std::string> ||
std::is_same_v<GetType, std::string_view>) {
try_execute(milvus::index::JSONType::STRING,
res_view,
valid_res_view,
GetType{},
GetType{});
} else if constexpr (std::is_same_v<GetType, proto::plan::Array>) {
// ARRAY shredding data: stored as BSON binary in binary column
auto target_field = index->GetShreddingField(
pointer, milvus::index::JSONType::ARRAY);
if (!target_field.empty()) {
ShreddingArrayBsonExecutor executor(op_type, pointer, val);
index->ExecutorForShreddingData<std::string_view>(
target_field,
executor,
nullptr,
res_view,
valid_res_view);
LOG_DEBUG("using shredding array field: {}, count {}",
target_field,
res_view.count());
}
}
}
// process shared data
auto shared_executor = [op_type, val, array_index, &res_view](
milvus::BsonView bson,
uint32_t row_id,
uint32_t value_offset) {
if constexpr (std::is_same_v<GetType, proto::plan::Array>) {
Assert(op_type == proto::plan::OpType::Equal ||
op_type == proto::plan::OpType::NotEqual);
if (array_index != INVALID_ARRAY_INDEX) {
auto array_value = bson.ParseAsArrayAtOffset(value_offset);
if (!array_value.has_value()) {
res_view[row_id] = false;
return;
}
auto sub_array = milvus::BsonView::GetNthElementInArray<
bsoncxx::array::view>(array_value.value().data(),
array_index);
if (!sub_array.has_value()) {
res_view[row_id] = false;
return;
}
res_view[row_id] =
op_type == proto::plan::OpType::Equal
? CompareTwoJsonArray(sub_array.value(), val)
: !CompareTwoJsonArray(sub_array.value(), val);
} else {
auto array_value = bson.ParseAsArrayAtOffset(value_offset);
if (!array_value.has_value()) {
res_view[row_id] = false;
return;
}
res_view[row_id] =
op_type == proto::plan::OpType::Equal
? CompareTwoJsonArray(array_value.value(), val)
: !CompareTwoJsonArray(array_value.value(), val);
}
} else {
std::optional<GetType> get_value;
if (array_index != INVALID_ARRAY_INDEX) {
auto array_value = bson.ParseAsArrayAtOffset(value_offset);
if (!array_value.has_value()) {
res_view[row_id] = false;
return;
}
get_value = milvus::BsonView::GetNthElementInArray<GetType>(
array_value.value().data(), array_index);
// If GetType is int and value is not found, try double
if constexpr (std::is_same_v<GetType, int64_t>) {
if (!get_value.has_value()) {
auto get_value =
milvus::BsonView::GetNthElementInArray<double>(
array_value.value().data(), array_index);
if (get_value.has_value()) {
res_view[row_id] = UnaryCompare(
get_value.value(), val, op_type);
}
return;
}
} else if constexpr (std::is_same_v<GetType, double>) {
if (!get_value.has_value()) {
auto get_value =
milvus::BsonView::GetNthElementInArray<int64_t>(
array_value.value().data(), array_index);
if (get_value.has_value()) {
res_view[row_id] = UnaryCompare(
get_value.value(), val, op_type);
}
return;
}
}
} else {
get_value =
bson.ParseAsValueAtOffset<GetType>(value_offset);
// If GetType is int and value is not found, try double
if constexpr (std::is_same_v<GetType, int64_t>) {
if (!get_value.has_value()) {
auto get_value =
bson.ParseAsValueAtOffset<double>(value_offset);
if (get_value.has_value()) {
res_view[row_id] = UnaryCompare(
get_value.value(), val, op_type);
}
return;
}
} else if constexpr (std::is_same_v<GetType, double>) {
if (!get_value.has_value()) {
auto get_value = bson.ParseAsValueAtOffset<int64_t>(
value_offset);
if (get_value.has_value()) {
res_view[row_id] = UnaryCompare(
get_value.value(), val, op_type);
}
return;
}
}
}
if (!get_value.has_value()) {
res_view[row_id] = false;
return;
}
res_view[row_id] =
UnaryCompare(get_value.value(), val, op_type);
}
};
std::set<milvus::index::JSONType> target_types;
if constexpr (std::is_same_v<GetType, std::string>) {
target_types.insert(milvus::index::JSONType::STRING);
} else if constexpr (std::is_same_v<GetType, int64_t> ||
std::is_same_v<GetType, double>) {
target_types.insert(milvus::index::JSONType::INT64);
target_types.insert(milvus::index::JSONType::DOUBLE);
} else if constexpr (std::is_same_v<GetType, bool>) {
target_types.insert(milvus::index::JSONType::BOOL);
}
{
milvus::ScopedTimer timer(
"unary_json_stats_shared_data", [](double ms) {
milvus::monitor::internal_json_stats_latency_shared.Observe(
ms);
});
if (!index->CanSkipShared(pointer, target_types)) {
index->ExecuteForSharedData(pointer, shared_executor);
}
}
cached_index_chunk_id_ = 0;
}
TargetBitmap result;
result.append(
*cached_index_chunk_res_, current_data_global_pos_, real_batch_size);

View File

@ -31,15 +31,76 @@
#include "segcore/SegmentInterface.h"
#include "query/Utils.h"
#include "common/RegexQuery.h"
#include "exec/expression/Utils.h"
#include "common/bson_view.h"
namespace milvus {
namespace exec {
template <typename T, FilterType filter_type>
template <typename T, typename U>
bool
UnaryCompare(const T& get_value, const U& val, proto::plan::OpType op_type) {
switch (op_type) {
case proto::plan::GreaterThan:
return get_value > val;
case proto::plan::GreaterEqual:
return get_value >= val;
case proto::plan::LessThan:
return get_value < val;
case proto::plan::LessEqual:
return get_value <= val;
case proto::plan::Equal:
return get_value == val;
case proto::plan::NotEqual:
return get_value != val;
case proto::plan::InnerMatch:
case proto::plan::PostfixMatch:
case proto::plan::PrefixMatch:
if constexpr (std::is_same_v<U, std::string> ||
std::is_same_v<U, std::string_view>) {
return milvus::query::Match(get_value, val, op_type);
} else {
return false;
}
case proto::plan::Match:
if constexpr (std::is_same_v<U, std::string> ||
std::is_same_v<U, std::string_view>) {
PatternMatchTranslator translator;
auto regex_pattern = translator(val);
RegexMatcher matcher(regex_pattern);
return matcher(get_value);
} else {
return false;
}
default:
return false;
}
}
template <typename T, FilterType filter_type = FilterType::sequential>
struct UnaryElementFuncForMatch {
using IndexInnerType =
std::conditional_t<std::is_same_v<T, std::string_view>, std::string, T>;
void
operator()(const T* src,
size_t size,
IndexInnerType val,
TargetBitmapView res) {
static_assert(
filter_type == FilterType::sequential,
"this override operator() of UnaryElementFuncForMatch does "
"not support FilterType::random");
PatternMatchTranslator translator;
auto regex_pattern = translator(val);
RegexMatcher matcher(regex_pattern);
for (int i = 0; i < size; ++i) {
res[i] = matcher(src[i]);
}
}
void
operator()(const T* src,
size_t size,
@ -65,11 +126,81 @@ struct UnaryElementFuncForMatch {
}
};
template <typename T, proto::plan::OpType op, FilterType filter_type>
template <typename T,
proto::plan::OpType op,
FilterType filter_type = FilterType::sequential>
struct UnaryElementFunc {
using IndexInnerType =
std::conditional_t<std::is_same_v<T, std::string_view>, std::string, T>;
void
operator()(const T* src,
size_t size,
TargetBitmapView res,
IndexInnerType val) {
static_assert(filter_type == FilterType::sequential,
"this override operator() of UnaryElementFunc does not "
"support FilterType::random");
if constexpr (op == proto::plan::OpType::Match) {
UnaryElementFuncForMatch<T> func;
func(src, size, val, res);
return;
}
if constexpr (std::is_same_v<T, std::string_view> ||
std::is_same_v<T, std::string>) {
for (int i = 0; i < size; ++i) {
if constexpr (op == proto::plan::OpType::Equal) {
res[i] = src[i] == val;
} else if constexpr (op == proto::plan::OpType::NotEqual) {
res[i] = src[i] != val;
} else if constexpr (op == proto::plan::OpType::GreaterThan) {
res[i] = src[i] > val;
} else if constexpr (op == proto::plan::OpType::LessThan) {
res[i] = src[i] < val;
} else if constexpr (op == proto::plan::OpType::GreaterEqual) {
res[i] = src[i] >= val;
} else if constexpr (op == proto::plan::OpType::LessEqual) {
res[i] = src[i] <= val;
} else if constexpr (op == proto::plan::OpType::PrefixMatch ||
op == proto::plan::OpType::PostfixMatch ||
op == proto::plan::OpType::InnerMatch) {
res[i] = milvus::query::Match(src[i], val, op);
} else {
ThrowInfo(
OpTypeInvalid,
fmt::format(
"unsupported op_type:{} for UnaryElementFunc", op));
}
}
return;
}
if constexpr (op == proto::plan::OpType::Equal) {
res.inplace_compare_val<T, milvus::bitset::CompareOpType::EQ>(
src, size, val);
} else if constexpr (op == proto::plan::OpType::NotEqual) {
res.inplace_compare_val<T, milvus::bitset::CompareOpType::NE>(
src, size, val);
} else if constexpr (op == proto::plan::OpType::GreaterThan) {
res.inplace_compare_val<T, milvus::bitset::CompareOpType::GT>(
src, size, val);
} else if constexpr (op == proto::plan::OpType::LessThan) {
res.inplace_compare_val<T, milvus::bitset::CompareOpType::LT>(
src, size, val);
} else if constexpr (op == proto::plan::OpType::GreaterEqual) {
res.inplace_compare_val<T, milvus::bitset::CompareOpType::GE>(
src, size, val);
} else if constexpr (op == proto::plan::OpType::LessEqual) {
res.inplace_compare_val<T, milvus::bitset::CompareOpType::LE>(
src, size, val);
} else {
ThrowInfo(
OpTypeInvalid,
fmt::format("unsupported op_type:{} for UnaryElementFunc", op));
}
}
void
operator()(const T* src,
size_t size,
@ -385,6 +516,222 @@ struct UnaryIndexFunc {
}
};
template <typename T, typename U>
void
BatchUnaryCompare(const T* src,
size_t size,
U& val,
proto::plan::OpType op_type,
TargetBitmapView res) {
if constexpr (std::is_integral_v<T> || std::is_floating_point_v<T>) {
using milvus::bitset::CompareOpType;
switch (op_type) {
case proto::plan::GreaterThan: {
res.inplace_compare_val<T>(
src, size, static_cast<T>(val), CompareOpType::GT);
return;
}
case proto::plan::GreaterEqual: {
res.inplace_compare_val<T>(
src, size, static_cast<T>(val), CompareOpType::GE);
return;
}
case proto::plan::LessThan: {
res.inplace_compare_val<T>(
src, size, static_cast<T>(val), CompareOpType::LT);
return;
}
case proto::plan::LessEqual: {
res.inplace_compare_val<T>(
src, size, static_cast<T>(val), CompareOpType::LE);
return;
}
case proto::plan::Equal: {
res.inplace_compare_val<T>(
src, size, static_cast<T>(val), CompareOpType::EQ);
return;
}
case proto::plan::NotEqual: {
res.inplace_compare_val<T>(
src, size, static_cast<T>(val), CompareOpType::NE);
return;
}
default:
break;
}
}
switch (op_type) {
case proto::plan::GreaterThan: {
for (int i = 0; i < size; ++i) {
res[i] = src[i] > val;
}
break;
}
case proto::plan::GreaterEqual: {
for (int i = 0; i < size; ++i) {
res[i] = src[i] >= val;
}
break;
}
case proto::plan::LessThan: {
for (int i = 0; i < size; ++i) {
res[i] = src[i] < val;
}
break;
}
case proto::plan::LessEqual: {
for (int i = 0; i < size; ++i) {
res[i] = src[i] <= val;
}
break;
}
case proto::plan::Equal: {
for (int i = 0; i < size; ++i) {
res[i] = src[i] == val;
}
break;
}
case proto::plan::NotEqual: {
for (int i = 0; i < size; ++i) {
res[i] = src[i] != val;
}
break;
}
case proto::plan::InnerMatch:
case proto::plan::PostfixMatch:
case proto::plan::PrefixMatch: {
for (int i = 0; i < size; ++i) {
res[i] = milvus::query::Match(src[i], val, op_type);
}
break;
}
case proto::plan::Match: {
if constexpr (std::is_same_v<U, std::string> ||
std::is_same_v<U, std::string_view>) {
PatternMatchTranslator translator;
auto regex_pattern = translator(val);
RegexMatcher matcher(regex_pattern);
for (int i = 0; i < size; ++i) {
res[i] = matcher(src[i]);
}
break;
}
}
default: {
ThrowInfo(
OpTypeInvalid,
fmt::format("unsupported op_type:{} for BatchUnaryCompare",
op_type));
}
}
}
template <typename GetType, typename ValType>
class ShreddingExecutor {
using IndexInnerType =
std::conditional_t<std::is_same_v<GetType, std::string_view>,
std::string,
GetType>;
public:
ShreddingExecutor(proto::plan::OpType op_type,
const std::string& pointer,
ValType val)
: op_type_(op_type), val_(val), pointer_(pointer) {
}
void
operator()(const GetType* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
if constexpr (std::is_same_v<GetType, proto::plan::Array>) {
ThrowInfo(ErrorCode::UnexpectedError,
"need using ShreddingArrayBsonExecutor for array type in "
"shredding data");
} else {
ExecuteOperation(src, size, res);
HandleValidData(valid, size, res, valid_res);
}
}
private:
void
ExecuteOperation(const GetType* src, size_t size, TargetBitmapView res) {
BatchUnaryCompare<GetType, ValType>(src, size, val_, op_type_, res);
}
void
HandleValidData(const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
if (valid != nullptr) {
for (int i = 0; i < size; ++i) {
if (!valid[i]) {
res[i] = valid_res[i] = false;
}
}
}
}
proto::plan::OpType op_type_;
ValType val_;
std::string pointer_;
};
// Executor for shredding ARRAY type stored as BSON binary in variable-length
// columns (std::string_view). Only supports Equal/NotEqual.
class ShreddingArrayBsonExecutor {
public:
ShreddingArrayBsonExecutor(proto::plan::OpType op_type,
const std::string& pointer,
const proto::plan::Array& val)
: op_type_(op_type), val_(val), pointer_(pointer) {
}
void
operator()(const std::string_view* src,
const bool* valid,
size_t size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (size_t i = 0; i < size; ++i) {
if (valid != nullptr && !valid[i]) {
res[i] = valid_res[i] = false;
continue;
}
milvus::BsonView bson(
reinterpret_cast<const uint8_t*>(src[i].data()), src[i].size());
auto array_view = bson.ParseAsArrayAtOffset(0);
if (!array_view.has_value()) {
res[i] = false;
continue;
}
bool equal = CompareTwoJsonArray(array_view.value(), val_);
switch (op_type_) {
case proto::plan::Equal:
res[i] = equal;
break;
case proto::plan::NotEqual:
res[i] = !equal;
break;
default:
ThrowInfo(OpTypeInvalid,
fmt::format("unsupported op_type:{} for ARRAY in "
"ShreddingArrayBsonExecutor",
op_type_));
}
}
}
private:
proto::plan::OpType op_type_;
const proto::plan::Array& val_;
std::string pointer_;
};
class PhyUnaryRangeFilterExpr : public SegmentExpr {
public:
PhyUnaryRangeFilterExpr(
@ -473,7 +820,7 @@ class PhyUnaryRangeFilterExpr : public SegmentExpr {
template <typename ExprValueType>
VectorPtr
ExecRangeVisitorImplJsonForIndex();
ExecRangeVisitorImplJsonByStats();
template <typename T>
VectorPtr

View File

@ -24,7 +24,7 @@
#include "exec/expression/Expr.h"
#include "segcore/SegmentInterface.h"
#include "query/Utils.h"
#include "common/bson_view.h"
namespace milvus {
namespace exec {
@ -128,6 +128,94 @@ CompareTwoJsonArray(T arr1, const proto::plan::Array& arr2) {
return true;
}
template <>
inline bool
CompareTwoJsonArray<bsoncxx::array::view>(bsoncxx::array::view arr1,
const proto::plan::Array& arr2) {
size_t bson_array_length = std::distance(arr1.begin(), arr1.end());
if (arr2.array_size() != bson_array_length) {
return false;
}
auto bson_it = arr1.begin();
for (int i = 0; i < arr2.array_size(); ++i, ++bson_it) {
if (bson_it == arr1.end()) {
return false;
}
const auto& bson_elem = *bson_it;
const auto& proto_elem = arr2.array(i);
switch (proto_elem.val_case()) {
case proto::plan::GenericValue::kBoolVal: {
if (bson_elem.type() != bsoncxx::type::k_bool) {
return false;
}
if (bson_elem.get_bool().value != proto_elem.bool_val()) {
return false;
}
break;
}
case proto::plan::GenericValue::kInt64Val: {
if (bson_elem.type() == bsoncxx::type::k_int32) {
const int32_t val = bson_elem.get_int32().value;
if (val != proto_elem.int64_val()) {
return false;
}
} else if (bson_elem.type() == bsoncxx::type::k_int64) {
const int64_t val = bson_elem.get_int64().value;
if (val != proto_elem.int64_val()) {
return false;
}
} else {
return false;
}
break;
}
case proto::plan::GenericValue::kFloatVal: {
double bson_val;
switch (bson_elem.type()) {
case bsoncxx::type::k_int32:
bson_val = bson_elem.get_int32().value;
break;
case bsoncxx::type::k_int64:
bson_val = bson_elem.get_int64().value;
break;
case bsoncxx::type::k_double:
bson_val = bson_elem.get_double().value;
break;
default:
return false;
}
if (bson_val != proto_elem.float_val()) {
return false;
}
break;
}
case proto::plan::GenericValue::kStringVal: {
if (bson_elem.type() != bsoncxx::type::k_string) {
return false;
}
auto bson_str_view = bson_elem.get_string().value;
if (std::string(bson_str_view.data(), bson_str_view.size()) !=
proto_elem.string_val()) {
return false;
}
break;
}
default:
return false;
}
}
if (bson_it != arr1.end()) {
return false;
}
return true;
}
template <typename T>
T
GetValueFromProtoInternal(const milvus::proto::plan::GenericValue& value_proto,

View File

@ -70,6 +70,11 @@ class IndexStats {
int64_t
GetSerializedSize() const;
const std::vector<SerializedIndexFileInfo>&
GetSerializedIndexFileInfo() const {
return serialized_index_infos_;
}
private:
IndexStats(int64_t mem_size,
std::vector<SerializedIndexFileInfo>&& serialized_index_infos);

View File

@ -320,7 +320,7 @@ class InvertedIndexTantivy : public ScalarIndex<T> {
// Therefore, we should provide a way to allow higher version of milvus to build tantivy index with low version.
uint32_t tantivy_index_version_{0};
// for now, only TextMatchIndex and JsonKeyStatsInvertedIndex can be built for growing segment,
// for now, only TextMatchIndex can be built for growing segment,
// and can read and insert concurrently.
bool is_growing_{false};
};

View File

@ -1,539 +0,0 @@
// 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 <boost/uuid/random_generator.hpp>
#include <boost/uuid/uuid_io.hpp>
#include "index/JsonKeyStatsInvertedIndex.h"
#include "index/InvertedIndexUtil.h"
#include "index/Utils.h"
#include "storage/MmapManager.h"
namespace milvus::index {
constexpr const char* TMP_JSON_INVERTED_LOG_PREFIX =
"/tmp/milvus/json-key-inverted-index-log/";
constexpr size_t MEMORY_THRESHOLD = 128 * 1024 * 1024;
constexpr size_t VECTOR_ELEMENT_SIZE = 8;
constexpr size_t KEY_OVERHEAD = 2;
size_t current_batch_size_ = 0;
size_t
calculateMemoryUsage(const std::map<std::string, std::vector<int64_t>>& mp) {
size_t total_memory = 0;
for (const auto& [key, vec] : mp) {
total_memory += (key.length());
total_memory += (vec.size() * VECTOR_ELEMENT_SIZE);
}
return total_memory * KEY_OVERHEAD;
}
void
JsonKeyStatsInvertedIndex::AddJSONEncodeValue(
const std::vector<std::string>& paths,
uint8_t flag,
uint8_t type,
uint32_t row_id,
uint16_t offset,
uint16_t length,
int32_t value,
std::map<std::string, std::vector<int64_t>>& mp) {
std::string key = milvus::Json::pointer(paths);
LOG_DEBUG(
"insert inverted key: {}, flag: {}, type: {}, row_id: {}, offset: "
"{}, length:{}, value:{}",
key,
flag,
type,
row_id,
offset,
length,
value);
int64_t combine_id = 0;
if (flag) {
combine_id = EncodeValue(flag, type, row_id, value);
} else {
combine_id = EncodeOffset(flag, type, row_id, offset, length);
}
mp[key].push_back(combine_id);
if (calculateMemoryUsage(mp) >= MEMORY_THRESHOLD) {
AddInvertedRecord(mp);
}
}
void
JsonKeyStatsInvertedIndex::AddInvertedRecord(
std::map<std::string, std::vector<int64_t>>& mp) {
std::vector<uintptr_t> json_offsets_lens;
std::vector<const char*> keys;
std::vector<const int64_t*> json_offsets;
if (mp.empty()) {
return;
}
for (auto& iter : mp) {
keys.push_back(iter.first.c_str());
json_offsets.push_back(iter.second.data());
json_offsets_lens.push_back(iter.second.size());
}
wrapper_->add_json_key_stats_data_by_batch(keys.data(),
json_offsets.data(),
json_offsets_lens.data(),
keys.size());
mp.clear();
current_batch_size_ = 0;
}
void
JsonKeyStatsInvertedIndex::TravelJson(
const char* json,
jsmntok* tokens,
int& index,
std::vector<std::string>& path,
int32_t offset,
std::map<std::string, std::vector<int64_t>>& mp) {
jsmntok current = tokens[0];
Assert(current.type != JSMN_UNDEFINED);
if (current.type == JSMN_OBJECT) {
if (!path.empty()) {
AddJSONEncodeValue(path,
0,
0,
offset,
current.start,
current.end - current.start,
0,
mp);
}
int j = 1;
for (int i = 0; i < current.size; i++) {
Assert(tokens[j].type == JSMN_STRING && tokens[j].size != 0);
std::string key(json + tokens[j].start,
tokens[j].end - tokens[j].start);
path.push_back(key);
j++;
int consumed = 0;
TravelJson(json, tokens + j, consumed, path, offset, mp);
path.pop_back();
j += consumed;
}
index = j;
} else if (current.type == JSMN_PRIMITIVE) {
std::string value(json + current.start, current.end - current.start);
auto type = getType(value);
if (type == JSONType::INT32) {
AddJSONEncodeValue(path,
1,
static_cast<uint8_t>(JSONType::INT32),
offset,
current.start,
current.end - current.start,
stoi(value),
mp);
} else if (type == JSONType::INT64) {
AddJSONEncodeValue(path,
0,
static_cast<uint8_t>(JSONType::INT64),
offset,
current.start,
current.end - current.start,
0,
mp);
} else if (type == JSONType::FLOAT) {
auto fvalue = stof(value);
int32_t valueBits = *reinterpret_cast<int32_t*>(&fvalue);
AddJSONEncodeValue(path,
1,
static_cast<uint8_t>(JSONType::FLOAT),
offset,
current.start,
current.end - current.start,
valueBits,
mp);
} else if (type == JSONType::DOUBLE) {
AddJSONEncodeValue(path,
0,
static_cast<uint8_t>(JSONType::DOUBLE),
offset,
current.start,
current.end - current.start,
0,
mp);
} else if (type == JSONType::BOOL) {
AddJSONEncodeValue(path,
1,
static_cast<uint8_t>(JSONType::BOOL),
offset,
current.start,
current.end - current.start,
value == "true" ? 1 : 0,
mp);
}
index++;
} else if (current.type == JSMN_ARRAY) {
AddJSONEncodeValue(path,
0,
static_cast<uint8_t>(JSONType::UNKNOWN),
offset,
current.start,
current.end - current.start,
0,
mp);
// skip array parse
int count = current.size;
int j = 1;
while (count > 0) {
count--;
if (tokens[j].size != 0) {
count += tokens[j].size;
}
j++;
}
index = j;
} else if (current.type == JSMN_STRING) {
Assert(current.size == 0);
std::string value(json + current.start, current.end - current.start);
if (has_escape_sequence(value)) {
AddJSONEncodeValue(path,
0,
static_cast<uint8_t>(JSONType::STRING_ESCAPE),
offset,
current.start - 1,
current.end - current.start + 2,
0,
mp);
} else {
AddJSONEncodeValue(path,
0,
static_cast<uint8_t>(JSONType::STRING),
offset,
current.start,
current.end - current.start,
0,
mp);
}
index++;
}
}
void
JsonKeyStatsInvertedIndex::AddJson(
const char* json,
int64_t offset,
std::map<std::string, std::vector<int64_t>>& mp) {
jsmn_parser parser;
jsmntok_t* tokens = (jsmntok_t*)malloc(16 * sizeof(jsmntok_t));
if (!tokens) {
ThrowInfo(ErrorCode::UnexpectedError, "alloc jsmn token failed");
return;
}
int num_tokens = 0;
int token_capacity = 16;
jsmn_init(&parser);
while (1) {
int r = jsmn_parse(&parser, json, strlen(json), tokens, token_capacity);
if (r < 0) {
if (r == JSMN_ERROR_NOMEM) {
// Reallocate tokens array if not enough space
token_capacity *= 2;
tokens = (jsmntok_t*)realloc(
tokens, token_capacity * sizeof(jsmntok_t));
if (!tokens) {
ThrowInfo(ErrorCode::UnexpectedError, "realloc failed");
}
continue;
} else {
free(tokens);
ThrowInfo(ErrorCode::UnexpectedError,
"Failed to parse Json: {}, error: {}",
json,
int(r));
}
}
num_tokens = r;
break;
}
int index = 0;
std::vector<std::string> paths;
TravelJson(json, tokens, index, paths, offset, mp);
free(tokens);
}
JsonKeyStatsInvertedIndex::JsonKeyStatsInvertedIndex(
const storage::FileManagerContext& ctx,
bool is_load,
int64_t json_stats_tantivy_memory_budget,
uint32_t tantivy_index_version)
: commit_interval_in_ms_(std::numeric_limits<int64_t>::max()),
last_commit_time_(stdclock::now()) {
LOG_INFO("json_stats_tantivy_memory_budget:{}",
json_stats_tantivy_memory_budget);
schema_ = ctx.fieldDataMeta.field_schema;
field_id_ = ctx.fieldDataMeta.field_id;
mem_file_manager_ = std::make_shared<MemFileManager>(ctx);
disk_file_manager_ = std::make_shared<DiskFileManager>(ctx);
if (is_load) {
path_ = disk_file_manager_->GetLocalJsonKeyIndexPrefix();
} else {
path_ = disk_file_manager_->GetLocalTempJsonKeyIndexPrefix();
boost::filesystem::create_directories(path_);
std::string field_name =
std::to_string(disk_file_manager_->GetFieldDataMeta().field_id);
wrapper_ = std::make_shared<TantivyIndexWrapper>(
field_name.c_str(),
path_.c_str(),
tantivy_index_version,
false, /* in_ram */
1,
json_stats_tantivy_memory_budget);
}
}
JsonKeyStatsInvertedIndex::JsonKeyStatsInvertedIndex(
int64_t commit_interval_in_ms, const char* unique_id)
: commit_interval_in_ms_(commit_interval_in_ms),
last_commit_time_(stdclock::now()) {
wrapper_ = std::make_shared<TantivyIndexWrapper>(
unique_id, "", TANTIVY_INDEX_LATEST_VERSION, true /* in_ram */);
set_is_growing(true);
}
JsonKeyStatsInvertedIndex::JsonKeyStatsInvertedIndex(
int64_t commit_interval_in_ms,
const char* unique_id,
const std::string& path)
: commit_interval_in_ms_(commit_interval_in_ms),
last_commit_time_(stdclock::now()) {
boost::filesystem::path prefix = path;
boost::filesystem::path sub_path = unique_id;
path_ = (prefix / sub_path).string();
boost::filesystem::create_directories(path_);
wrapper_ = std::make_shared<TantivyIndexWrapper>(
unique_id, path_.c_str(), TANTIVY_INDEX_LATEST_VERSION);
set_is_growing(true);
}
IndexStatsPtr
JsonKeyStatsInvertedIndex::Upload(const Config& config) {
finish();
index_build_timestamps_.index_build_done_ =
std::chrono::system_clock::now();
LOG_INFO(
"index build done for json key index, field id:{}, json parse "
"duration: {}s, "
"tantivy document add schedule duration : {}s, "
"tantivy total duration : {}s, "
"total duration : {}s",
field_id_,
index_build_timestamps_.getJsonParsingDuration(),
index_build_timestamps_.getTantivyAddSchedulingDuration(),
index_build_timestamps_.getTantivyTotalDuration(),
index_build_timestamps_.getIndexBuildTotalDuration());
boost::filesystem::path p(path_);
boost::filesystem::directory_iterator end_iter;
for (boost::filesystem::directory_iterator iter(p); iter != end_iter;
iter++) {
if (boost::filesystem::is_directory(*iter)) {
LOG_WARN("{} is a directory", iter->path().string());
} else {
LOG_INFO("trying to add json key inverted index log: {}",
iter->path().string());
AssertInfo(
disk_file_manager_->AddJsonKeyIndexLog(iter->path().string()),
"failed to add json key inverted index log: {}",
iter->path().string());
LOG_INFO("json key inverted index log: {} added",
iter->path().string());
}
}
auto remote_paths_to_size = disk_file_manager_->GetRemotePathsToFileSize();
auto binary_set = Serialize(config);
mem_file_manager_->AddFile(binary_set);
auto remote_mem_path_to_size =
mem_file_manager_->GetRemotePathsToFileSize();
std::vector<SerializedIndexFileInfo> index_files;
index_files.reserve(remote_paths_to_size.size() +
remote_mem_path_to_size.size());
for (auto& file : remote_paths_to_size) {
index_files.emplace_back(disk_file_manager_->GetFileName(file.first),
file.second);
}
for (auto& file : remote_mem_path_to_size) {
index_files.emplace_back(file.first, file.second);
}
return IndexStats::New(mem_file_manager_->GetAddedTotalMemSize() +
disk_file_manager_->GetAddedTotalFileSize(),
std::move(index_files));
}
void
JsonKeyStatsInvertedIndex::Load(milvus::tracer::TraceContext ctx,
const Config& config) {
auto index_files =
GetValueFromConfig<std::vector<std::string>>(config, INDEX_FILES);
AssertInfo(index_files.has_value(),
"index file paths is empty when load json key index");
for (auto& index_file : index_files.value()) {
boost::filesystem::path p(index_file);
if (!p.has_parent_path()) {
auto remote_prefix =
disk_file_manager_->GetRemoteJsonKeyLogPrefix();
index_file = remote_prefix + "/" + index_file;
}
}
auto load_priority =
GetValueFromConfig<milvus::proto::common::LoadPriority>(
config, milvus::LOAD_PRIORITY)
.value_or(milvus::proto::common::LoadPriority::HIGH);
disk_file_manager_->CacheJsonKeyIndexToDisk(index_files.value(),
load_priority);
AssertInfo(
tantivy_index_exist(path_.c_str()), "index not exist: {}", path_);
auto load_in_mmap =
GetValueFromConfig<bool>(config, ENABLE_MMAP).value_or(true);
wrapper_ = std::make_shared<TantivyIndexWrapper>(
path_.c_str(), load_in_mmap, milvus::index::SetBitsetSealed);
if (!load_in_mmap) {
// the index is loaded in ram, so we can remove files in advance
disk_file_manager_->RemoveJsonKeyIndexFiles();
}
LOG_INFO(
"load json key index done for field id:{} with dir:{}, load_in_mmap:{}",
field_id_,
path_,
load_in_mmap);
}
void
JsonKeyStatsInvertedIndex::BuildWithFieldData(
const std::vector<FieldDataPtr>& field_datas) {
AssertInfo(schema_.data_type() == proto::schema::DataType::JSON,
"schema data type is {}",
schema_.data_type());
BuildWithFieldData(field_datas, schema_.nullable());
}
void
JsonKeyStatsInvertedIndex::BuildWithFieldData(
const std::vector<FieldDataPtr>& field_datas, bool nullable) {
int64_t offset = 0;
std::map<std::string, std::vector<int64_t>> mp;
index_build_timestamps_.index_build_begin_ =
std::chrono::system_clock::now();
if (nullable) {
for (const auto& data : field_datas) {
auto n = data->get_num_rows();
for (int i = 0; i < n; i++) {
if (!data->is_valid(i)) {
offset++;
continue;
}
AddJson(static_cast<const milvus::Json*>(data->RawValue(i))
->data()
.data(),
offset++,
mp);
}
}
} else {
for (const auto& data : field_datas) {
auto n = data->get_num_rows();
for (int i = 0; i < n; i++) {
AddJson(static_cast<const milvus::Json*>(data->RawValue(i))
->data()
.data(),
offset++,
mp);
}
}
}
index_build_timestamps_.tantivy_build_begin_ =
std::chrono::system_clock::now();
// Schedule all document add operations to tantivy.
AddInvertedRecord(mp);
index_build_timestamps_.tantivy_add_schedule_end_ =
std::chrono::system_clock::now();
}
void
JsonKeyStatsInvertedIndex::AddJSONDatas(size_t n,
const std::string* jsonDatas,
const bool* valids,
int64_t offset_begin) {
std::map<std::string, std::vector<int64_t>> mp;
for (int i = 0; i < n; i++) {
auto offset = i + offset_begin;
if (valids != nullptr && !valids[i]) {
continue;
}
AddJson(jsonDatas[i].c_str(), offset, mp);
}
AddInvertedRecord(mp);
is_data_uncommitted_ = true;
LOG_INFO("build json key index done for AddJSONDatas");
if (shouldTriggerCommit()) {
Commit();
}
}
void
JsonKeyStatsInvertedIndex::Finish() {
finish();
}
bool
JsonKeyStatsInvertedIndex::shouldTriggerCommit() {
auto span = (std::chrono::duration<double, std::milli>(
stdclock::now() - last_commit_time_.load()))
.count();
return span > commit_interval_in_ms_;
}
void
JsonKeyStatsInvertedIndex::Commit() {
std::unique_lock<std::mutex> lck(mtx_, std::defer_lock);
if (lck.try_lock()) {
is_data_uncommitted_ = false;
wrapper_->commit();
last_commit_time_.store(stdclock::now());
}
}
void
JsonKeyStatsInvertedIndex::Reload() {
std::unique_lock<std::mutex> lck(mtx_, std::defer_lock);
if (lck.try_lock()) {
wrapper_->reload();
}
}
void
JsonKeyStatsInvertedIndex::CreateReader(SetBitsetFn set_bitset) {
wrapper_->create_reader(set_bitset);
}
} // namespace milvus::index

View File

@ -1,353 +0,0 @@
// 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 <string>
#include <boost/filesystem.hpp>
#include "index/InvertedIndexTantivy.h"
#include "common/jsmn.h"
namespace milvus::index {
enum class JSONType {
UNKNOWN,
BOOL,
INT32,
INT64,
FLOAT,
DOUBLE,
STRING,
STRING_ESCAPE
};
using stdclock = std::chrono::high_resolution_clock;
class JsonKeyStatsInvertedIndex : public InvertedIndexTantivy<std::string> {
public:
explicit JsonKeyStatsInvertedIndex(
const storage::FileManagerContext& ctx,
bool is_load,
int64_t json_stats_tantivy_memory_budget = 16777216,
uint32_t tantivy_index_version = TANTIVY_INDEX_LATEST_VERSION);
explicit JsonKeyStatsInvertedIndex(int64_t commit_interval_in_ms,
const char* unique_id);
explicit JsonKeyStatsInvertedIndex(int64_t commit_interval_in_ms,
const char* unique_id,
const std::string& path);
~JsonKeyStatsInvertedIndex() override{};
public:
IndexStatsPtr
Upload(const Config& config) override;
void
Load(milvus::tracer::TraceContext ctx, const Config& config) override;
void
BuildWithFieldData(const std::vector<FieldDataPtr>& datas) override;
void
BuildWithFieldData(const std::vector<FieldDataPtr>& datas, bool nullable);
TargetBitmap
FilterByPath(const std::string& path,
int32_t row,
bool is_growing,
bool is_strong_consistency,
std::function<void(const bool*,
const uint8_t*,
const uint32_t*,
const uint16_t*,
const uint16_t*,
const int32_t*,
TargetBitmap&,
const size_t size)> filter) {
auto processArray = [this, &path, row, &filter]() {
TargetBitmap bitset(row);
auto array = wrapper_->term_query_i64(path);
LOG_INFO("json key filter size:{}", array.array_.len);
folly::fbvector<bool> valid_array(array.array_.len);
std::vector<uint8_t> type_array(array.array_.len);
std::vector<uint32_t> row_id_array(array.array_.len);
std::vector<uint16_t> offset_array(array.array_.len);
std::vector<uint16_t> size_array(array.array_.len);
std::vector<int32_t> value_array(array.array_.len);
for (size_t j = 0; j < array.array_.len; j++) {
auto the_offset = array.array_.array[j];
if (DecodeValid(the_offset)) {
auto tuple = DecodeValue(the_offset);
auto row_id = std::get<1>(tuple);
if (row_id >= row) {
continue;
}
valid_array[j] = true;
type_array[j] = std::get<0>(tuple);
row_id_array[j] = std::get<1>(tuple);
offset_array[j] = 0;
size_array[j] = 0;
value_array[j] = std::get<2>(tuple);
} else {
auto tuple = DecodeOffset(the_offset);
auto row_id = std::get<1>(tuple);
if (row_id >= row) {
continue;
}
valid_array[j] = false;
type_array[j] = std::get<0>(tuple);
row_id_array[j] = std::get<1>(tuple);
offset_array[j] = std::get<2>(tuple);
size_array[j] = std::get<3>(tuple);
value_array[j] = 0;
}
}
filter(valid_array.data(),
type_array.data(),
row_id_array.data(),
offset_array.data(),
size_array.data(),
value_array.data(),
bitset,
array.array_.len);
return bitset;
};
if (is_growing && (shouldTriggerCommit() || is_strong_consistency)) {
if (is_data_uncommitted_) {
Commit();
}
Reload();
}
return processArray();
}
void
AddJSONDatas(size_t n,
const std::string* jsonDatas,
const bool* valids,
int64_t offset_begin);
void
Finish();
void
Commit();
void
Reload();
void
CreateReader(SetBitsetFn set_bitset);
bool
has_escape_sequence(const std::string& str) {
for (size_t i = 0; i < str.size(); ++i) {
if (str[i] == '\\' && i + 1 < str.size()) {
char next = str[i + 1];
if (next == 'n' || next == 't' || next == 'r' || next == 'b' ||
next == 'f' || next == 'v' || next == '\\' ||
next == '\"' || next == '\'' || next == '0' ||
next == 'u' || next == '/') {
return true;
}
}
}
return false;
}
private:
void
AddJson(const char* json,
int64_t offset,
std::map<std::string, std::vector<int64_t>>& mp);
void
TravelJson(const char* json,
jsmntok* tokens,
int& index,
std::vector<std::string>& path,
int32_t offset,
std::map<std::string, std::vector<int64_t>>& mp);
void
AddJSONEncodeValue(const std::vector<std::string>& paths,
uint8_t flag,
uint8_t type,
uint32_t row_id,
uint16_t offset,
uint16_t length,
int32_t value,
std::map<std::string, std::vector<int64_t>>& mp);
int64_t
EncodeOffset(uint8_t flag,
uint8_t type,
uint32_t row_id,
uint16_t row_offset,
uint16_t size) {
row_id &= 0x0FFFFFFF;
return static_cast<int64_t>(flag) << 63 |
static_cast<int64_t>(type) << 60 |
static_cast<int64_t>(row_id) << 32 |
static_cast<int64_t>(row_offset) << 16 |
static_cast<int64_t>(size);
}
int64_t
EncodeValue(uint8_t flag, uint8_t type, uint32_t row_id, int32_t value) {
row_id &= 0x0FFFFFFF;
return static_cast<int64_t>(flag) << 63 |
static_cast<int64_t>(type) << 60 |
static_cast<int64_t>(row_id) << 32 |
static_cast<uint32_t>(value);
}
bool
DecodeValid(int64_t encode_offset) {
return (encode_offset >> 63) & 1;
}
std::tuple<uint8_t, uint32_t, int32_t>
DecodeValue(int64_t encode_offset) {
uint8_t type = (encode_offset >> 60) & 0x7;
uint32_t row_id = (encode_offset >> 32) & 0x0FFFFFFF;
int32_t value = static_cast<int32_t>(encode_offset & 0xFFFFFFFF);
return std::make_tuple(type, row_id, value);
}
std::tuple<uint8_t, uint32_t, uint16_t, uint16_t>
DecodeOffset(int64_t encode_offset) {
uint8_t type = (encode_offset >> 60) & 0x7;
uint32_t row_id = (encode_offset >> 32) & 0x0FFFFFFF;
uint16_t row_offset = (encode_offset >> 16) & 0xFFFF;
uint16_t size = encode_offset & 0xFFFF;
return std::make_tuple(type, row_id, row_offset, size);
}
bool
shouldTriggerCommit();
bool
isBoolean(const std::string& str) {
return str == "true" || str == "false";
}
bool
isInt32(const std::string& str) {
std::istringstream iss(str);
int64_t num;
iss >> num;
return !iss.fail() && iss.eof() &&
num >= std::numeric_limits<int32_t>::min() &&
num <= std::numeric_limits<int32_t>::max();
}
bool
isInt64(const std::string& str) {
std::istringstream iss(str);
int64_t num;
iss >> num;
return !iss.fail() && iss.eof();
}
bool
isFloat(const std::string& str) {
try {
float d = std::stof(str);
return true;
} catch (...) {
return false;
}
}
bool
isDouble(const std::string& str) {
try {
double d = std::stod(str);
return true;
} catch (...) {
return false;
}
}
JSONType
getType(const std::string& str) {
if (isBoolean(str)) {
return JSONType::BOOL;
} else if (isInt32(str)) {
return JSONType::INT32;
} else if (isInt64(str)) {
return JSONType::INT64;
} else if (isFloat(str)) {
return JSONType::FLOAT;
} else if (isDouble(str)) {
return JSONType::DOUBLE;
}
return JSONType::UNKNOWN;
}
void
AddInvertedRecord(std::map<std::string, std::vector<int64_t>>& mp);
private:
int64_t field_id_;
mutable std::mutex mtx_;
std::atomic<stdclock::time_point> last_commit_time_;
int64_t commit_interval_in_ms_;
std::atomic<bool> is_data_uncommitted_ = false;
struct IndexBuildTimestamps {
std::chrono::time_point<std::chrono::system_clock> index_build_begin_;
std::chrono::time_point<std::chrono::system_clock> tantivy_build_begin_;
// The time that we have finished push add operations to tantivy, which will be
// executed asynchronously
std::chrono::time_point<std::chrono::system_clock>
tantivy_add_schedule_end_;
std::chrono::time_point<std::chrono::system_clock> index_build_done_;
auto
getJsonParsingDuration() const {
return std::chrono::duration<double>(tantivy_build_begin_ -
index_build_begin_)
.count();
}
auto
getTantivyAddSchedulingDuration() const {
return std::chrono::duration<double>(tantivy_add_schedule_end_ -
tantivy_build_begin_)
.count();
}
auto
getTantivyTotalDuration() const {
return std::chrono::duration<double>(index_build_done_ -
tantivy_build_begin_)
.count();
}
auto
getIndexBuildTotalDuration() const {
return std::chrono::duration<double>(index_build_done_ -
index_build_begin_)
.count();
}
};
IndexBuildTimestamps index_build_timestamps_;
};
} // namespace milvus::index

View File

@ -56,6 +56,7 @@ constexpr const char* NGRAM_INDEX_TYPE = "NGRAM";
constexpr const char* MIN_GRAM = "min_gram";
constexpr const char* MAX_GRAM = "max_gram";
constexpr const char* JSON_KEY_STATS_INDEX_TYPE = "JsonKeyStats";
// index meta
constexpr const char* COLLECTION_ID = "collection_id";
constexpr const char* PARTITION_ID = "partition_id";

View File

@ -36,6 +36,7 @@ enum class ScalarIndexType {
MARISA,
INVERTED,
HYBRID,
JSONSTATS,
};
inline std::string

View File

@ -201,6 +201,11 @@ CheckAndUpdateKnowhereRangeSearchParam(const SearchInfo& search_info,
const MetricType& metric_type,
knowhere::Json& search_config);
// for unused
void inline SetBitsetUnused(void* bitset, const uint32_t* doc_id, uintptr_t n) {
ThrowInfo(ErrorCode::UnexpectedError, "SetBitsetUnused is not supported");
}
// For sealed segment, the doc_id is guaranteed to be less than bitset size which equals to the doc count of tantivy before querying.
void inline SetBitsetSealed(void* bitset, const uint32_t* doc_id, uintptr_t n) {
TargetBitmap* bitmap = static_cast<TargetBitmap*>(bitset);

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,672 @@
// 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 <boost/filesystem.hpp>
#include "index/InvertedIndexTantivy.h"
#include "common/jsmn.h"
#include "mmap/ChunkedColumnInterface.h"
#include "arrow/api.h"
#include "index/json_stats/utils.h"
#include "index/json_stats/bson_inverted.h"
#include "index/json_stats/parquet_writer.h"
#include "index/json_stats/bson_builder.h"
#include "common/bson_view.h"
#include "index/SkipIndex.h"
namespace milvus::index {
class JsonKeyStats : public ScalarIndex<std::string> {
public:
explicit JsonKeyStats(
const storage::FileManagerContext& ctx,
bool is_load,
int64_t json_stats_max_shredding_columns = 1024,
double json_stats_shredding_ratio_threshold = 0.3,
int64_t json_stats_write_batch_size = 81920,
uint32_t tantivy_index_version = TANTIVY_INDEX_LATEST_VERSION);
~JsonKeyStats() override;
public:
void
BuildWithFieldData(const std::vector<FieldDataPtr>& datas, bool nullable);
void
Load(milvus::tracer::TraceContext ctx, const Config& config = {}) override;
void
Load(const BinarySet& binary_set, const Config& config) override {
ThrowInfo(ErrorCode::NotImplemented,
"Load not supported for JsonKeyStats");
}
/*
* deprecated.
* TODO: why not remove this?
*/
void
BuildWithDataset(const DatasetPtr& dataset,
const Config& config = {}) override {
ThrowInfo(ErrorCode::NotImplemented,
"BuildWithDataset should be deprecated");
}
ScalarIndexType
GetIndexType() const override {
return ScalarIndexType::JSONSTATS;
}
void
Build(const Config& config = {}) override;
int64_t
Count() override {
return num_rows_;
}
BinarySet
Serialize(const Config& config) override;
IndexStatsPtr
Upload(const Config& config = {}) override;
const bool
HasRawData() const override {
return false;
}
int64_t
Size() override {
return Count();
}
void
BuildWithRawDataForUT(size_t n,
const void* values,
const Config& config) override {
ThrowInfo(ErrorCode::NotImplemented,
"BuildWithRawDataForUT Not supported for JsonKeyStats");
}
void
Build(size_t n,
const std::string* values,
const bool* valid_data = nullptr) override {
ThrowInfo(ErrorCode::NotImplemented,
"Build not supported for JsonKeyStats");
}
const TargetBitmap
In(size_t n, const std::string* values) override {
ThrowInfo(ErrorCode::NotImplemented,
"In not supported for JsonKeyStats");
}
const TargetBitmap
IsNull() override {
ThrowInfo(ErrorCode::NotImplemented,
"IsNull not supported for JsonKeyStats");
}
TargetBitmap
IsNotNull() override {
ThrowInfo(ErrorCode::NotImplemented,
"IsNotNull not supported for JsonKeyStats");
}
const TargetBitmap
NotIn(size_t n, const std::string* values) override {
ThrowInfo(ErrorCode::NotImplemented,
"NotIn not supported for JsonKeyStats");
}
const TargetBitmap
Range(std::string value, OpType op) override {
ThrowInfo(ErrorCode::NotImplemented,
"Range not supported for JsonKeyStats");
}
const TargetBitmap
Range(std::string lower_bound_value,
bool lb_inclusive,
std::string upper_bound_value,
bool ub_inclusive) override {
ThrowInfo(ErrorCode::NotImplemented,
"Range not supported for JsonKeyStats");
}
std::optional<std::string>
Reverse_Lookup(size_t offset) const override {
ThrowInfo(ErrorCode::NotImplemented,
"Reverse_Lookup not supported for JsonKeyStats");
}
public:
void
ExecuteForSharedData(
const std::string& path,
std::function<void(BsonView bson, uint32_t row_id, uint32_t offset)>
func) {
bson_inverted_index_->TermQuery(
path,
[this, &func](const uint32_t* row_id_array,
const uint32_t* offset_array,
const int64_t array_len) {
shared_column_->BulkRawBsonAt(
func, row_id_array, offset_array, array_len);
});
}
void
ExecuteExistsPathForSharedData(const std::string& path,
TargetBitmapView bitset) {
bson_inverted_index_->TermQueryEach(
path, [&bitset](uint32_t row_id, uint32_t offset) {
bitset[row_id] = true;
});
}
int64_t
ExecutorForGettingValid(const std::string& path,
TargetBitmapView valid_res) {
size_t processed_size = 0;
// if path is not in shredding_columns_, return 0
if (shredding_columns_.find(path) == shredding_columns_.end()) {
return processed_size;
}
auto column = shredding_columns_[path];
auto num_data_chunk = column->num_chunks();
for (size_t i = 0; i < num_data_chunk; i++) {
auto chunk_size = column->chunk_row_nums(i);
const bool* valid_data;
if (GetShreddingJsonType(path) == JSONType::STRING ||
GetShreddingJsonType(path) == JSONType::ARRAY) {
auto pw = column->StringViews(i);
valid_data = pw.get().second.data();
ApplyOnlyValidData(
valid_data, valid_res + processed_size, chunk_size);
} else {
auto pw = column->Span(i);
auto chunk = pw.get();
valid_data = chunk.valid_data();
ApplyOnlyValidData(
valid_data, valid_res + processed_size, chunk_size);
}
processed_size += chunk_size;
}
AssertInfo(processed_size == valid_res.size(),
"Processed size {} is not equal to num_rows {}",
processed_size,
valid_res.size());
return processed_size;
}
template <typename T, typename FUNC, typename... ValTypes>
int64_t
ExecutorForShreddingData(
// path is field_name in shredding_columns_
const std::string& path,
FUNC func,
std::function<bool(const milvus::SkipIndex&, std::string, int)>
skip_func,
TargetBitmapView res,
TargetBitmapView valid_res,
ValTypes... values) {
int64_t processed_size = 0;
// if path is not in shredding_columns_, return 0
if (shredding_columns_.find(path) == shredding_columns_.end()) {
return processed_size;
}
auto column = shredding_columns_[path];
auto num_data_chunk = column->num_chunks();
auto num_rows = column->NumRows();
for (size_t i = 0; i < num_data_chunk; i++) {
auto chunk_size = column->chunk_row_nums(i);
if (!skip_func || !skip_func(skip_index_, path, i)) {
if constexpr (std::is_same_v<T, std::string_view>) {
// first is the raw data, second is valid_data
// use valid_data to see if raw data is null
auto pw = column->StringViews(i);
auto [data_vec, valid_data] = pw.get();
func(data_vec.data(),
valid_data.data(),
chunk_size,
res + processed_size,
valid_res + processed_size,
values...);
} else {
auto pw = column->Span(i);
auto chunk = pw.get();
const T* data = static_cast<const T*>(chunk.data());
const bool* valid_data = chunk.valid_data();
func(data,
valid_data,
chunk_size,
res + processed_size,
valid_res + processed_size,
values...);
}
} else {
const bool* valid_data;
if constexpr (std::is_same_v<T, std::string_view>) {
auto pw = column->StringViews(i);
valid_data = pw.get().second.data();
ApplyValidData(valid_data,
res + processed_size,
valid_res + processed_size,
chunk_size);
} else {
auto pw = column->Span(i);
auto chunk = pw.get();
valid_data = chunk.valid_data();
ApplyValidData(valid_data,
res + processed_size,
valid_res + processed_size,
chunk_size);
}
}
processed_size += chunk_size;
}
AssertInfo(processed_size == num_rows,
"Processed size {} is not equal to num_rows {}",
processed_size,
num_rows);
return processed_size;
}
// Whether shared columns can be skipped for this path (type-agnostic)
bool
CanSkipShared(const std::string& path) {
auto it = key_field_map_.find(path);
if (it == key_field_map_.end()) {
return true;
}
const auto& field_names = it->second;
for (const auto& field_name : field_names) {
if (field_layout_type_map_[field_name] ==
JsonKeyLayoutType::SHARED) {
return false;
}
}
return true;
}
// Whether shared columns can be skipped for this path with type filter
bool
CanSkipShared(const std::string& path,
const std::set<milvus::index::JSONType>& target_types) {
auto it = key_field_map_.find(path);
if (it == key_field_map_.end()) {
return true;
}
const auto& field_names = it->second;
for (const auto& field_name : field_names) {
if (field_layout_type_map_[field_name] !=
JsonKeyLayoutType::SHARED) {
continue;
}
if (!target_types.empty() &&
target_types.find(key_data_type_map_[field_name]) ==
target_types.end()) {
continue;
}
return false;
}
return true;
}
std::set<std::string>
GetShreddingFields(const std::string& pointer) {
std::set<std::string> fields;
if (key_field_map_.find(pointer) != key_field_map_.end()) {
for (const auto& field : key_field_map_[pointer]) {
if (shred_field_data_type_map_.find(field) !=
shred_field_data_type_map_.end()) {
fields.insert(field);
}
}
}
return fields;
}
std::string
GetShreddingField(const std::string& pointer, JSONType type) {
if (key_field_map_.find(pointer) == key_field_map_.end()) {
return "";
}
for (const auto& field : key_field_map_[pointer]) {
if (shred_field_data_type_map_.find(field) !=
shred_field_data_type_map_.end() &&
shred_field_data_type_map_[field] == type) {
return field;
}
}
return "";
}
std::set<std::string>
GetShreddingFields(const std::string& pointer,
std::vector<JSONType> types) {
std::set<std::string> fields;
if (key_field_map_.find(pointer) == key_field_map_.end()) {
return fields;
}
for (const auto& field : key_field_map_[pointer]) {
if (shred_field_data_type_map_.find(field) !=
shred_field_data_type_map_.end() &&
std::find(types.begin(),
types.end(),
shred_field_data_type_map_[field]) != types.end()) {
fields.insert(field);
}
}
return fields;
}
JSONType
GetShreddingJsonType(const std::string& field_name) {
if (shred_field_data_type_map_.find(field_name) !=
shred_field_data_type_map_.end()) {
return shred_field_data_type_map_[field_name];
}
return JSONType::UNKNOWN;
}
private:
void
CollectSingleJsonStatsInfo(const char* json_str,
std::map<JsonKey, KeyStatsInfo>& infos);
std::string
PrintKeyInfo(const std::map<JsonKey, KeyStatsInfo>& infos) {
std::stringstream ss;
for (const auto& [key, info] : infos) {
ss << key.ToString() << " -> " << info.ToString() << "\t";
}
return ss.str();
}
std::map<JsonKey, KeyStatsInfo>
CollectKeyInfo(const std::vector<FieldDataPtr>& field_datas, bool nullable);
void
TraverseJsonForStats(const char* json,
jsmntok* tokens,
int& index,
std::vector<std::string>& path,
std::map<JsonKey, KeyStatsInfo>& infos);
void
AddKeyStatsInfo(const std::vector<std::string>& paths,
JSONType type,
uint8_t* value,
std::map<JsonKey, KeyStatsInfo>& infos);
std::string
PrintJsonKeyLayoutType(const std::map<JsonKey, JsonKeyLayoutType>& infos) {
std::stringstream ss;
std::unordered_map<JsonKeyLayoutType, std::vector<std::string>>
type_to_keys;
for (const auto& [key, type] : infos) {
type_to_keys[type].push_back(key.ToString());
}
for (const auto& [type, keys] : type_to_keys) {
ss << ToString(type) << " -> [" << Join(keys, ", ") << "]\n";
}
return ss.str();
}
std::map<JsonKey, JsonKeyLayoutType>
ClassifyJsonKeyLayoutType(const std::map<JsonKey, KeyStatsInfo>& infos);
void
BuildKeyStats(const std::vector<FieldDataPtr>& field_datas, bool nullable);
void
BuildKeyStatsForRow(const char* json_str, uint32_t row_id);
void
BuildKeyStatsForNullRow();
std::string
GetShreddingDir();
std::string
GetSharedKeyIndexDir();
void
AddKeyStats(const std::vector<std::string>& path,
JSONType type,
const std::string& value,
std::map<JsonKey, std::string>& values);
void
TraverseJsonForBuildStats(const char* json,
jsmntok* tokens,
int& index,
std::vector<std::string>& path,
std::map<JsonKey, std::string>& values);
bool
IsBoolean(const std::string& str) {
return str == "true" || str == "false";
}
bool
IsInt8(const std::string& str) {
std::istringstream iss(str);
int8_t num;
iss >> num;
return !iss.fail() && iss.eof() &&
num >= std::numeric_limits<int8_t>::min() &&
num <= std::numeric_limits<int8_t>::max();
}
bool
IsInt16(const std::string& str) {
std::istringstream iss(str);
int16_t num;
iss >> num;
return !iss.fail() && iss.eof() &&
num >= std::numeric_limits<int16_t>::min() &&
num <= std::numeric_limits<int16_t>::max();
}
bool
IsInt32(const std::string& str) {
std::istringstream iss(str);
int64_t num;
iss >> num;
return !iss.fail() && iss.eof() &&
num >= std::numeric_limits<int32_t>::min() &&
num <= std::numeric_limits<int32_t>::max();
}
bool
IsInt64(const std::string& str) {
std::istringstream iss(str);
int64_t num;
iss >> num;
return !iss.fail() && iss.eof();
}
bool
IsFloat(const std::string& str) {
try {
float d = std::stof(str);
return true;
} catch (...) {
return false;
}
}
bool
IsDouble(const std::string& str) {
try {
double d = std::stod(str);
return true;
} catch (...) {
return false;
}
}
bool
IsNull(const std::string& str) {
return str == "null";
}
JSONType
getType(const std::string& str) {
if (IsBoolean(str)) {
return JSONType::BOOL;
// TODO: add int8, int16, int32 support
// now we only support int64 for build performance
// } else if (IsInt8(str)) {
// return JSONType::INT8;
// } else if (IsInt16(str)) {
// return JSONType::INT16;
// } else if (IsInt32(str)) {
// return JSONType::INT32;
} else if (IsInt64(str)) {
return JSONType::INT64;
} else if (IsFloat(str)) {
return JSONType::FLOAT;
} else if (IsDouble(str)) {
return JSONType::DOUBLE;
} else if (IsNull(str)) {
return JSONType::NONE;
}
LOG_DEBUG("unknown json type for string: {}", str);
return JSONType::UNKNOWN;
}
void
LoadShreddingData(const std::vector<std::string>& index_files);
void
ApplyValidData(const bool* valid_data,
TargetBitmapView res,
TargetBitmapView valid_res,
const int size) {
if (valid_data != nullptr) {
for (int i = 0; i < size; i++) {
if (!valid_data[i]) {
res[i] = valid_res[i] = false;
}
}
}
}
void
ApplyOnlyValidData(const bool* valid_data,
TargetBitmapView valid_res,
const int size) {
if (valid_data != nullptr) {
for (int i = 0; i < size; i++) {
if (!valid_data[i]) {
valid_res[i] = false;
}
}
}
}
void
GetColumnSchemaFromParquet(int64_t column_group_id,
const std::string& file);
void
GetCommonMetaFromParquet(const std::string& file);
void
LoadColumnGroup(int64_t column_group_id,
const std::vector<int64_t>& file_ids);
void
LoadShreddingMeta(
std::vector<std::pair<int64_t, std::vector<int64_t>>> sorted_files);
std::string
AddBucketName(const std::string& remote_prefix);
private:
proto::schema::FieldSchema schema_;
int64_t segment_id_;
int64_t field_id_;
mutable std::mutex mtx_;
int64_t num_rows_{0};
bool is_built_ = false;
std::string path_;
milvus::storage::ChunkManagerPtr rcm_;
std::shared_ptr<milvus::storage::MemFileManagerImpl> mem_file_manager_;
std::shared_ptr<milvus::storage::DiskFileManagerImpl> disk_file_manager_;
int64_t max_shredding_columns_;
double shredding_ratio_threshold_;
int64_t write_batch_size_;
std::map<JsonKey, JsonKeyLayoutType> key_types_;
std::set<JsonKey> shared_keys_;
std::set<JsonKey> column_keys_;
std::shared_ptr<JsonStatsParquetWriter> parquet_writer_;
std::shared_ptr<BsonInvertedIndex> bson_inverted_index_;
milvus::proto::common::LoadPriority load_priority_;
// some meta cache for searching
// json_path -> [json_path_int, json_path_array, json_path_object, ...], only for all keys
std::unordered_map<std::string, std::set<std::string>> key_field_map_;
// field_name -> data_type, such as json_path_int -> JSONType::INT64, only for real shredding columns
std::unordered_map<std::string, JSONType> shred_field_data_type_map_;
// key_name -> data_type, such as json_path_int -> JSONType::INT64, for all keys
std::unordered_map<std::string, JSONType> key_data_type_map_;
// field_name -> key_type, such as json_path_int -> JsonKeyLayoutType::TYPED, for all keys
std::unordered_map<std::string, JsonKeyLayoutType> field_layout_type_map_;
// field_name -> field_id, such as json_path_int -> 1001
std::unordered_map<std::string, int64_t> field_name_to_id_map_;
// field_id -> field_name, such as 1001 -> json_path_int
std::unordered_map<int64_t, std::string> field_id_to_name_map_;
// field_name vector, the sequece is the same as the order of files
std::vector<std::string> field_names_;
// column_group_id -> schema, the sequence of schemas is the same as the order of files
std::map<int64_t, std::shared_ptr<arrow::Schema>> column_group_schemas_;
// field_name -> column
mutable std::unordered_map<std::string,
std::shared_ptr<milvus::ChunkedColumnInterface>>
shredding_columns_;
std::string mmap_filepath_;
std::string shared_column_field_name_;
std::shared_ptr<milvus::ChunkedColumnInterface> shared_column_;
SkipIndex skip_index_;
};
} // namespace milvus::index

View File

@ -0,0 +1,328 @@
// 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 <iostream>
#include <vector>
#include <string>
#include <map>
#include "index/json_stats/bson_builder.h"
namespace milvus::index {
namespace {
using bsoncxx::builder::basic::kvp;
void
AppendDomElementToBsonArray(simdjson::dom::element elem,
bsoncxx::builder::basic::array& out);
void
AppendDomElementToBsonDocument(simdjson::dom::element elem,
std::string_view key,
bsoncxx::builder::basic::document& out) {
using simdjson::dom::element_type;
auto type = elem.type();
if (type == element_type::STRING) {
std::string s(std::string_view(elem.get_string()));
out.append(kvp(std::string(key), std::move(s)));
} else if (type == element_type::INT64) {
out.append(kvp(std::string(key), int64_t(elem.get_int64())));
} else if (type == element_type::UINT64) {
out.append(kvp(std::string(key), int64_t(elem.get_uint64())));
} else if (type == element_type::DOUBLE) {
out.append(kvp(std::string(key), elem.get_double()));
} else if (type == element_type::BOOL) {
out.append(kvp(std::string(key), bool(elem.get_bool())));
} else if (type == element_type::NULL_VALUE) {
out.append(kvp(std::string(key), bsoncxx::types::b_null{}));
} else if (type == element_type::OBJECT) {
bsoncxx::builder::basic::document sub;
for (auto [k, v] : elem.get_object()) {
AppendDomElementToBsonDocument(v, k, sub);
}
out.append(kvp(std::string(key), sub.extract()));
} else if (type == element_type::ARRAY) {
bsoncxx::builder::basic::array subarr;
for (simdjson::dom::element v : elem.get_array()) {
AppendDomElementToBsonArray(v, subarr);
}
out.append(kvp(std::string(key), subarr.extract()));
} else {
out.append(kvp(std::string(key), bsoncxx::types::b_null{}));
}
}
void
AppendDomElementToBsonArray(simdjson::dom::element elem,
bsoncxx::builder::basic::array& out) {
using simdjson::dom::element_type;
auto type = elem.type();
if (type == element_type::STRING) {
out.append(std::string(std::string_view(elem.get_string())));
} else if (type == element_type::INT64) {
out.append(int64_t(elem.get_int64()));
} else if (type == element_type::UINT64) {
out.append(int64_t(elem.get_uint64()));
} else if (type == element_type::DOUBLE) {
out.append(elem.get_double());
} else if (type == element_type::BOOL) {
out.append(bool(elem.get_bool()));
} else if (type == element_type::NULL_VALUE) {
out.append(bsoncxx::types::b_null{});
} else if (type == element_type::OBJECT) {
bsoncxx::builder::basic::document sub;
for (auto [k, v] : elem.get_object()) {
AppendDomElementToBsonDocument(v, k, sub);
}
out.append(sub.extract());
} else if (type == element_type::ARRAY) {
bsoncxx::builder::basic::array subarr;
for (simdjson::dom::element v : elem.get_array()) {
AppendDomElementToBsonArray(v, subarr);
}
out.append(subarr.extract());
} else {
out.append(bsoncxx::types::b_null{});
}
}
} // namespace
// Parse a JSON array string with simdjson and build an owning BSON array value
bsoncxx::array::value
BuildBsonArrayFromJsonString(const std::string& json_array) {
simdjson::dom::parser parser;
simdjson::dom::element root = parser.parse(json_array);
if (root.type() != simdjson::dom::element_type::ARRAY) {
ThrowInfo(ErrorCode::UnexpectedError,
"input is not a JSON array: {}",
json_array);
}
bsoncxx::builder::basic::array out;
for (simdjson::dom::element elem : root.get_array()) {
AppendDomElementToBsonArray(elem, out);
}
return out.extract();
}
std::vector<uint8_t>
BuildBsonArrayBytesFromJsonString(const std::string& json_array) {
auto arr_value = BuildBsonArrayFromJsonString(json_array);
auto view = arr_value.view();
return std::vector<uint8_t>(view.data(), view.data() + view.length());
}
void
BsonBuilder::AppendToDom(DomNode& root,
const std::vector<std::string>& keys,
const std::string& value,
const JSONType& type) {
LOG_TRACE("append to dom: {} with value {} and type {}",
Join(keys, "."),
value,
ToString(type));
DomNode* current = &root;
for (size_t i = 0; i < keys.size(); ++i) {
const std::string& key = keys[i];
if (i == keys.size() - 1) {
current->document_children[key] = CreateValueNode(value, type);
} else {
auto& children = current->document_children;
auto it = children.find(key);
if (it != children.end()) {
if (it->second.type != DomNode::Type::DOCUMENT) {
it->second = DomNode(DomNode::Type::DOCUMENT);
}
current = &it->second;
} else {
children[key] = DomNode(DomNode::Type::DOCUMENT);
current = &children[key];
}
}
}
}
DomNode
BsonBuilder::CreateValueNode(const std::string& value, JSONType type) {
switch (type) {
case JSONType::NONE: {
return DomNode(bsoncxx::types::b_null{});
}
case JSONType::BOOL: {
bool b = (value == "true" || value == "1");
return DomNode(bsoncxx::types::b_bool{b});
}
case JSONType::INT32: {
int32_t i = std::stoi(value);
return DomNode(bsoncxx::types::b_int32{i});
}
case JSONType::INT64: {
int64_t l = std::stoll(value);
return DomNode(bsoncxx::types::b_int64{l});
}
case JSONType::DOUBLE: {
double d = std::stod(value);
return DomNode(bsoncxx::types::b_double{d});
}
case JSONType::STRING: {
return DomNode(bsoncxx::types::b_string{value});
}
case JSONType::ARRAY: {
try {
auto arr_value = BuildBsonArrayFromJsonString(value);
return DomNode(bsoncxx::types::b_array{arr_value.view()});
} catch (const simdjson::simdjson_error& e) {
ThrowInfo(
ErrorCode::UnexpectedError,
"Failed to build bson array (simdjson) from string: {}, {}",
value,
e.what());
} catch (const std::exception& e) {
ThrowInfo(
ErrorCode::UnexpectedError,
"Failed to build bson array (generic) from string: {}, {}",
value,
e.what());
}
}
case JSONType::OBJECT: {
AssertInfo(value == "{}",
"object value should be empty but got {}",
value);
// return an empty json object as a document node
return DomNode(DomNode::Type::DOCUMENT);
}
default:
ThrowInfo(ErrorCode::Unsupported, "Unsupported JSON type {}", type);
}
}
void
BsonBuilder::ConvertDomToBson(const DomNode& node,
bsoncxx::builder::basic::document& builder) {
for (const auto& [key, child] : node.document_children) {
switch (child.type) {
case DomNode::Type::VALUE: {
builder.append(bsoncxx::builder::basic::kvp(
key, child.bson_value.value()));
break;
}
case DomNode::Type::DOCUMENT: {
bsoncxx::builder::basic::document sub_doc;
ConvertDomToBson(child, sub_doc);
builder.append(bsoncxx::builder::basic::kvp(key, sub_doc));
break;
}
default: {
ThrowInfo(ErrorCode::Unsupported,
"Unsupported DOM node type {}",
static_cast<int>(child.type));
}
}
}
}
void
BsonBuilder::ExtractOffsetsRecursive(
const uint8_t* root_base_ptr,
const uint8_t* current_base_ptr,
const std::string& current_path,
std::vector<std::pair<std::string, size_t>>& result) {
uint32_t length;
memcpy(&length, current_base_ptr, 4);
const uint8_t* end_ptr = current_base_ptr + length - 1;
AssertInfo(*(end_ptr) == 0x00, "miss bson document terminator");
const uint8_t* ptr = current_base_ptr + 4;
while (ptr <= end_ptr && *ptr != 0x00) {
// record key offset
size_t key_offset = ptr - root_base_ptr;
// read key type
auto type = static_cast<bsoncxx::type>(*ptr++);
// read key
auto key_name = reinterpret_cast<const char*>(ptr);
ptr += strlen(key_name) + 1;
// construct key path
std::string key_path = AppendJsonPointer(current_path, key_name);
// do not record key offset pair for null value
// because null value is not a valid key
if (type != bsoncxx::type::k_null) {
result.emplace_back(key_path, key_offset);
}
// handle value
switch (type) {
case bsoncxx::type::k_document: {
ExtractOffsetsRecursive(root_base_ptr, ptr, key_path, result);
// skip sub doc
uint32_t child_len;
memcpy(&child_len, ptr, 4);
ptr += child_len;
break;
}
case bsoncxx::type::k_array: {
// not parse array
// skip sub doc
uint32_t child_len;
memcpy(&child_len, ptr, 4);
ptr += child_len;
break;
}
case bsoncxx::type::k_string: {
uint32_t str_len;
memcpy(&str_len, ptr, 4);
ptr += 4 + str_len;
break;
}
case bsoncxx::type::k_int32: {
ptr += 4;
break;
}
case bsoncxx::type::k_int64: {
ptr += 8;
break;
}
case bsoncxx::type::k_double: {
ptr += 8;
break;
}
case bsoncxx::type::k_bool: {
ptr += 1;
break;
}
case bsoncxx::type::k_null: {
break;
}
default: {
ThrowInfo(ErrorCode::Unsupported,
"Unsupported BSON type {}",
static_cast<int>(type));
}
}
}
}
} // namespace milvus::index

View File

@ -0,0 +1,97 @@
// 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 <iostream>
#include <string>
#include <vector>
#include <unordered_map>
#include <tuple>
#include <bsoncxx/builder/basic/document.hpp>
#include <bsoncxx/builder/basic/array.hpp>
#include <bsoncxx/builder/basic/kvp.hpp>
#include <bsoncxx/types/bson_value/value.hpp>
#include <bsoncxx/document/value.hpp>
#include <bsoncxx/json.hpp>
#include <simdjson.h>
#include <bsoncxx/types.hpp>
#include <bson/bson.h>
#include "index/json_stats/utils.h"
#include "common/EasyAssert.h"
namespace milvus::index {
class DomNode {
public:
enum class Type { DOCUMENT, VALUE };
Type type;
std::map<std::string, DomNode> document_children;
std::optional<bsoncxx::types::bson_value::value> bson_value;
DomNode(Type t = Type::DOCUMENT) : type(t) {
}
DomNode(bsoncxx::types::bson_value::value v)
: type(Type::VALUE), bson_value(std::move(v)) {
}
};
// Parse a JSON array string and return a self-owned buffer containing the
// BSON array bytes (length-prefixed, including terminator). The returned
// vector's data() can be used directly with bson_view.
std::vector<uint8_t>
BuildBsonArrayBytesFromJsonString(const std::string& json_array);
class BsonBuilder {
public:
static void
AppendToDom(DomNode& root,
const std::vector<std::string>& keys,
const std::string& value,
const JSONType& type);
static DomNode
CreateValueNode(const std::string& value, JSONType type);
static void
ConvertDomToBson(const DomNode& node,
bsoncxx::builder::basic::document& builder);
// helper function to recursively extract keys with offset
static void
ExtractOffsetsRecursive(
const uint8_t* base_ptr,
const uint8_t* current_base_ptr,
const std::string& current_path,
std::vector<std::pair<std::string, size_t>>& result);
static std::vector<std::pair<std::string, size_t>>
ExtractBsonKeyOffsets(const bsoncxx::document::view& view) {
std::vector<std::pair<std::string, size_t>> result;
const uint8_t* raw_data = view.data();
size_t raw_len = view.length();
ExtractOffsetsRecursive(raw_data, raw_data, "", result);
return result;
}
static std::vector<std::pair<std::string, size_t>>
ExtractBsonKeyOffsets(const uint8_t* data, size_t size) {
std::vector<std::pair<std::string, size_t>> result;
ExtractOffsetsRecursive(data, data, "", result);
return result;
}
};
} // namespace milvus::index

View File

@ -0,0 +1,222 @@
// 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 <iostream>
#include <string>
#include <vector>
#include <unordered_map>
#include <tuple>
#include "common/ScopedTimer.h"
#include "monitor/Monitor.h"
#include "index/json_stats/bson_inverted.h"
#include "storage/LocalChunkManagerSingleton.h"
namespace milvus::index {
BsonInvertedIndex::BsonInvertedIndex(const std::string& path,
int64_t field_id,
bool is_load,
const storage::FileManagerContext& ctx,
int64_t tantivy_index_version)
: is_load_(is_load),
field_id_(field_id),
tantivy_index_version_(tantivy_index_version) {
disk_file_manager_ =
std::make_shared<milvus::storage::DiskFileManagerImpl>(ctx);
if (is_load_) {
auto prefix = disk_file_manager_->GetLocalJsonStatsSharedIndexPrefix();
path_ = prefix;
LOG_INFO("bson inverted index load path:{}", path_);
} else {
path_ = path;
LOG_INFO("bson inverted index build path:{}", path_);
}
}
BsonInvertedIndex::~BsonInvertedIndex() {
if (wrapper_) {
wrapper_->free();
}
if (!is_load_) {
auto local_chunk_manager =
milvus::storage::LocalChunkManagerSingleton::GetInstance()
.GetChunkManager();
auto prefix = path_;
LOG_INFO("bson inverted index remove path:{}", path_);
local_chunk_manager->RemoveDir(prefix);
}
}
void
BsonInvertedIndex::AddRecord(const std::string& key,
uint32_t row_id,
uint32_t offset) {
if (inverted_index_map_.find(key) == inverted_index_map_.end()) {
inverted_index_map_[key] = {EncodeInvertedIndexValue(row_id, offset)};
} else {
inverted_index_map_[key].push_back(
EncodeInvertedIndexValue(row_id, offset));
}
}
void
BsonInvertedIndex::BuildIndex() {
if (wrapper_ == nullptr) {
if (tantivy_index_exist(path_.c_str())) {
ThrowInfo(IndexBuildError,
"build inverted index temp dir:{} not empty",
path_);
}
auto field_name = std::to_string(field_id_) + "_" + "shared";
wrapper_ = std::make_shared<TantivyIndexWrapper>(
field_name.c_str(), path_.c_str(), tantivy_index_version_);
LOG_INFO("build bson inverted index for field id:{} with dir:{}",
field_id_,
path_);
}
std::vector<const char*> keys;
std::vector<const int64_t*> json_offsets;
std::vector<uintptr_t> json_offsets_lens;
for (const auto& [key, offsets] : inverted_index_map_) {
keys.push_back(key.c_str());
json_offsets.push_back(offsets.data());
json_offsets_lens.push_back(offsets.size());
}
wrapper_->add_json_key_stats_data_by_batch(keys.data(),
json_offsets.data(),
json_offsets_lens.data(),
keys.size());
}
void
BsonInvertedIndex::LoadIndex(const std::vector<std::string>& index_files,
milvus::proto::common::LoadPriority priority) {
if (is_load_) {
// convert shared_key_index/... to remote_prefix/shared_key_index/...
std::vector<std::string> remote_files;
for (auto& file : index_files) {
auto remote_prefix =
disk_file_manager_->GetRemoteJsonStatsLogPrefix();
remote_files.emplace_back(remote_prefix + "/" + file);
}
// cache shared_key_index/... to disk
disk_file_manager_->CacheJsonStatsSharedIndexToDisk(remote_files,
priority);
AssertInfo(tantivy_index_exist(path_.c_str()),
"index dir not exist: {}",
path_);
wrapper_ = std::make_shared<TantivyIndexWrapper>(
path_.c_str(), false, milvus::index::SetBitsetUnused);
LOG_INFO("load json shared key index done for field id:{} with dir:{}",
field_id_,
path_);
}
}
IndexStatsPtr
BsonInvertedIndex::UploadIndex() {
AssertInfo(!is_load_, "upload index is not supported for load index");
AssertInfo(wrapper_ != nullptr,
"bson inverted index wrapper is not initialized");
wrapper_->finish();
boost::filesystem::path p(path_);
boost::filesystem::directory_iterator end_iter;
for (boost::filesystem::directory_iterator iter(p); iter != end_iter;
iter++) {
if (boost::filesystem::is_directory(*iter)) {
LOG_WARN("{} is a directory", iter->path().string());
} else {
LOG_INFO("trying to add bson inverted index file: {}",
iter->path().string());
AssertInfo(disk_file_manager_->AddJsonSharedIndexLog(
iter->path().string()),
"failed to add bson inverted index file: {}",
iter->path().string());
LOG_INFO("bson inverted index file: {} added",
iter->path().string());
}
}
auto remote_paths_to_size = disk_file_manager_->GetRemotePathsToFileSize();
std::vector<SerializedIndexFileInfo> index_files;
index_files.reserve(remote_paths_to_size.size());
for (auto& file : remote_paths_to_size) {
index_files.emplace_back(file.first, file.second);
}
return IndexStats::New(disk_file_manager_->GetAddedTotalFileSize(),
std::move(index_files));
}
void
BsonInvertedIndex::TermQuery(
const std::string& path,
const std::function<void(const uint32_t* row_id_array,
const uint32_t* offset_array,
const int64_t array_len)>& visitor) {
AssertInfo(wrapper_ != nullptr,
"bson inverted index wrapper is not initialized");
auto start = std::chrono::steady_clock::now();
auto array = wrapper_->term_query_i64(path);
auto end = std::chrono::steady_clock::now();
LOG_TRACE("term query time:{}",
std::chrono::duration_cast<std::chrono::microseconds>(end - start)
.count());
auto array_len = array.array_.len;
LOG_DEBUG("json stats shared column filter size:{} with path:{}",
array_len,
path);
std::vector<uint32_t> row_id_array(array_len);
std::vector<uint32_t> offset_array(array_len);
for (int64_t i = 0; i < array_len; i++) {
auto value = array.array_.array[i];
auto [row_id, offset] = DecodeInvertedIndexValue(value);
row_id_array[i] = row_id;
offset_array[i] = offset;
}
visitor(row_id_array.data(), offset_array.data(), array_len);
}
void
BsonInvertedIndex::TermQueryEach(
const std::string& path,
const std::function<void(uint32_t row_id, uint32_t offset)>& each) {
AssertInfo(wrapper_ != nullptr,
"bson inverted index wrapper is not initialized");
auto start = std::chrono::steady_clock::now();
auto array = wrapper_->term_query_i64(path);
auto end = std::chrono::steady_clock::now();
LOG_TRACE("term query time:{}",
std::chrono::duration_cast<std::chrono::microseconds>(end - start)
.count());
auto array_len = array.array_.len;
LOG_TRACE("json stats shared column filter size:{} with path:{}",
array_len,
path);
for (int64_t i = 0; i < array_len; i++) {
auto value = array.array_.array[i];
auto [row_id, offset] = DecodeInvertedIndexValue(value);
each(row_id, offset);
}
}
} // namespace milvus::index

View File

@ -0,0 +1,95 @@
// 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 <iostream>
#include <string>
#include <vector>
#include <unordered_map>
#include <tuple>
#include "index/json_stats/utils.h"
#include "common/EasyAssert.h"
#include "index/IndexStats.h"
namespace milvus::index {
inline int64_t
EncodeInvertedIndexValue(uint32_t row_id, uint32_t offset) {
return (static_cast<uint64_t>(row_id) << 32) | offset;
}
inline std::pair<uint32_t, uint32_t>
DecodeInvertedIndexValue(uint64_t value) {
return std::make_pair(static_cast<uint32_t>(value >> 32),
static_cast<uint32_t>(value & 0xFFFFFFFF));
}
class BsonInvertedIndex {
public:
BsonInvertedIndex(const std::string& path,
int64_t field_id,
bool is_load,
const storage::FileManagerContext& ctx,
int64_t tantivy_index_version);
~BsonInvertedIndex();
void
AddRecord(const std::string& key, uint32_t row_id, uint32_t offset);
void
BuildIndex();
void
LoadIndex(const std::vector<std::string>& index_files,
milvus::proto::common::LoadPriority priority);
IndexStatsPtr
UploadIndex();
void
TermQuery(const std::string& path,
const std::function<void(const uint32_t* row_id_array,
const uint32_t* offset_array,
const int64_t array_len)>& visitor);
void
TermQueryEach(
const std::string& path,
const std::function<void(uint32_t row_id, uint32_t offset)>& each);
bool
KeyExists(const std::string& key) {
auto array = wrapper_->term_query_i64(key);
return !array.array_.len == 0;
}
private:
std::string path_;
bool is_load_;
// json field id that this inverted index belongs to
int64_t field_id_;
int64_t tantivy_index_version_;
// key -> encoded([row_id, offset]) map cache for building index
std::map<std::string, std::vector<int64_t>> inverted_index_map_;
// tantivy index wrapper
std::shared_ptr<TantivyIndexWrapper> wrapper_;
std::shared_ptr<milvus::storage::DiskFileManagerImpl> disk_file_manager_;
};
} // namespace milvus::index

View File

@ -0,0 +1,260 @@
// 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 "index/json_stats/parquet_writer.h"
#include <arrow/array/array_binary.h>
#include <arrow/array/array_primitive.h>
#include <arrow/array/builder_binary.h>
#include <arrow/array/builder_primitive.h>
#include <arrow/io/file.h>
#include <parquet/arrow/writer.h>
#include <parquet/exception.h>
namespace milvus::index {
JsonStatsParquetWriter::JsonStatsParquetWriter(
std::shared_ptr<arrow::fs::FileSystem> fs,
milvus_storage::StorageConfig& storage_config,
size_t buffer_size,
size_t batch_size,
std::shared_ptr<parquet::WriterProperties> writer_props)
: fs_(fs),
storage_config_(storage_config),
buffer_size_(buffer_size),
batch_size_(batch_size),
writer_props_(writer_props),
unflushed_row_count_(0),
all_row_count_(0) {
}
JsonStatsParquetWriter::~JsonStatsParquetWriter() {
Close();
}
void
JsonStatsParquetWriter::Close() {
Flush();
packed_writer_->Close();
}
void
JsonStatsParquetWriter::Flush() {
WriteCurrentBatch();
}
void
JsonStatsParquetWriter::UpdatePathSizeMap(
const std::vector<std::shared_ptr<arrow::Array>>& arrays) {
auto index = 0;
for (const auto& group : column_groups_) {
auto path = file_paths_[index];
for (const auto& column_index : group) {
size_t size = GetArrowArrayMemorySize(arrays[column_index]);
path_size_map_[path] += size;
total_size_ += size;
}
index++;
}
}
size_t
JsonStatsParquetWriter::WriteCurrentBatch() {
if (unflushed_row_count_ == 0) {
return 0;
}
std::vector<std::shared_ptr<arrow::Array>> arrays;
for (auto& builder : builders_) {
std::shared_ptr<arrow::Array> array;
auto status = builder->Finish(&array);
AssertInfo(
status.ok(), "failed to finish builder: {}", status.ToString());
arrays.push_back(array);
builder->Reset();
}
UpdatePathSizeMap(arrays);
auto batch =
arrow::RecordBatch::Make(schema_, unflushed_row_count_, arrays);
auto status = packed_writer_->Write(batch);
AssertInfo(
status.ok(), "failed to write batch, error: {}", status.ToString());
auto res = unflushed_row_count_;
unflushed_row_count_ = 0;
return res;
}
void
JsonStatsParquetWriter::Init(const ParquetWriteContext& context) {
schema_ = context.schema;
builders_ = context.builders;
builders_map_ = context.builders_map;
kv_metadata_ = std::move(context.kv_metadata);
column_groups_ = context.column_groups;
file_paths_ = context.file_paths;
packed_writer_ = std::make_unique<milvus_storage::PackedRecordBatchWriter>(
fs_,
file_paths_,
schema_,
storage_config_,
column_groups_,
buffer_size_);
for (const auto& [key, value] : kv_metadata_) {
packed_writer_->AddUserMetadata(key, value);
}
}
size_t
JsonStatsParquetWriter::AddCurrentRow() {
unflushed_row_count_++;
all_row_count_++;
if (unflushed_row_count_ >= batch_size_) {
WriteCurrentBatch();
}
return all_row_count_;
}
void
JsonStatsParquetWriter::AppendSharedRow(const uint8_t* data, size_t length) {
auto builder = builders_.at(builders_.size() - 1);
auto shared_builder =
std::static_pointer_cast<arrow::BinaryBuilder>(builder);
if (length == 0) {
auto status = shared_builder->AppendNull();
AssertInfo(status.ok(), "failed to append null data");
} else {
auto status = shared_builder->Append(data, length);
AssertInfo(status.ok(), "failed to append binary data");
}
}
void
JsonStatsParquetWriter::AppendValue(const std::string& key,
const std::string& value) {
auto it = builders_map_.find(key);
if (it == builders_map_.end()) {
ThrowInfo(
ErrorCode::UnexpectedError, "builder for key {} not found", key);
}
auto builder = it->second;
auto ast = AppendDataToBuilder(value, builder);
AssertInfo(ast.ok(), "failed to append data to builder");
}
void
JsonStatsParquetWriter::AppendRow(
const std::map<std::string, std::string>& row_data) {
for (const auto& [key, value] : row_data) {
auto it = builders_map_.find(key);
if (it == builders_map_.end()) {
ThrowInfo(ErrorCode::UnexpectedError,
"builder for key {} not found",
key);
}
auto builder = it->second;
auto status = AppendDataToBuilder(value, builder);
AssertInfo(status.ok(), "failed to append data to builder");
}
AddCurrentRow();
}
arrow::Status
JsonStatsParquetWriter::AppendDataToBuilder(
const std::string& value, std::shared_ptr<arrow::ArrayBuilder> builder) {
auto type_id = builder->type()->id();
if (value.empty()) {
return builder->AppendNull();
}
arrow::Status ast;
try {
switch (type_id) {
case arrow::Type::INT8: {
auto int8_builder =
std::static_pointer_cast<arrow::Int8Builder>(builder);
ast = int8_builder->Append(ConvertValue<int8_t>(value));
break;
}
case arrow::Type::INT16: {
auto int16_builder =
std::static_pointer_cast<arrow::Int16Builder>(builder);
ast = int16_builder->Append(ConvertValue<int16_t>(value));
break;
}
case arrow::Type::INT32: {
auto int32_builder =
std::static_pointer_cast<arrow::Int32Builder>(builder);
ast = int32_builder->Append(ConvertValue<int32_t>(value));
break;
}
case arrow::Type::INT64: {
auto int64_builder =
std::static_pointer_cast<arrow::Int64Builder>(builder);
ast = int64_builder->Append(ConvertValue<int64_t>(value));
break;
}
case arrow::Type::FLOAT: {
auto float_builder =
std::static_pointer_cast<arrow::FloatBuilder>(builder);
ast = float_builder->Append(ConvertValue<float>(value));
break;
}
case arrow::Type::DOUBLE: {
auto double_builder =
std::static_pointer_cast<arrow::DoubleBuilder>(builder);
ast = double_builder->Append(ConvertValue<double>(value));
break;
}
case arrow::Type::BOOL: {
auto bool_builder =
std::static_pointer_cast<arrow::BooleanBuilder>(builder);
ast = bool_builder->Append(ConvertValue<bool>(value));
break;
}
case arrow::Type::STRING: {
auto string_builder =
std::static_pointer_cast<arrow::StringBuilder>(builder);
ast = string_builder->Append(value);
break;
}
case arrow::Type::BINARY: {
auto binary_builder =
std::static_pointer_cast<arrow::BinaryBuilder>(builder);
ast = binary_builder->Append(value.data(), value.size());
break;
}
default:
ThrowInfo(ErrorCode::Unsupported,
"Unsupported JSON type: {} for value {}",
type_id,
value);
}
} catch (const std::exception& e) {
ThrowInfo(ErrorCode::UnexpectedError,
"failed to append data to builder: {}",
e.what());
}
return ast;
}
} // namespace milvus::index

View File

@ -0,0 +1,244 @@
// 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 <memory>
#include <string>
#include <vector>
#include <map>
#include "arrow/api.h"
#include "arrow/io/api.h"
#include "parquet/arrow/writer.h"
#include "index/json_stats/utils.h"
#include "milvus-storage/packed/writer.h"
#include "milvus-storage/common/config.h"
namespace milvus::index {
template <typename T>
inline T
ConvertValue(const std::string& value);
template <>
inline int8_t
ConvertValue<int8_t>(const std::string& value) {
return static_cast<int8_t>(std::stoi(value));
}
template <>
inline int16_t
ConvertValue<int16_t>(const std::string& value) {
return static_cast<int16_t>(std::stoi(value));
}
template <>
inline int32_t
ConvertValue<int32_t>(const std::string& value) {
return std::stoi(value);
}
template <>
inline int64_t
ConvertValue<int64_t>(const std::string& value) {
return std::stoll(value);
}
template <>
inline float
ConvertValue<float>(const std::string& value) {
return std::stof(value);
}
template <>
inline double
ConvertValue<double>(const std::string& value) {
return std::stod(value);
}
template <>
inline bool
ConvertValue<bool>(const std::string& value) {
return value == "true";
}
template <typename BuilderType,
typename CType = typename BuilderType::value_type>
inline arrow::Status
AppendValueImpl(std::shared_ptr<BuilderType> builder,
const std::string& value) {
return builder->Append(ConvertValue<CType>(value));
}
struct ParquetWriteContext {
std::shared_ptr<arrow::Schema> schema;
std::vector<std::shared_ptr<arrow::ArrayBuilder>> builders;
std::map<std::string, std::shared_ptr<arrow::ArrayBuilder>> builders_map;
std::vector<std::pair<std::string, std::string>> kv_metadata;
std::vector<std::string> file_paths;
std::vector<std::vector<int>> column_groups;
};
struct TableStatsInfo {
std::shared_ptr<arrow::Schema> schema;
std::map<JsonKey, JsonKeyLayoutType> column_map;
};
class ColumnGroupingStrategy {
public:
virtual ~ColumnGroupingStrategy() = default;
virtual std::vector<std::vector<int>>
CreateGroups(const TableStatsInfo& table_info) const = 0;
};
class DefaultColumnGroupingStrategy : public ColumnGroupingStrategy {
public:
std::vector<std::vector<int>>
CreateGroups(const TableStatsInfo& table_info) const override {
// put all columns into one group
std::vector<std::vector<int>> column_groups;
std::vector<int> group;
for (size_t i = 0; i < table_info.schema->num_fields(); ++i) {
group.push_back(i);
}
column_groups.push_back(group);
return column_groups;
}
};
enum class ColumnGroupingStrategyType {
DEFAULT,
};
class ColumnGroupingStrategyFactory {
public:
static std::unique_ptr<ColumnGroupingStrategy>
CreateStrategy(ColumnGroupingStrategyType type) {
switch (type) {
case ColumnGroupingStrategyType::DEFAULT:
return std::make_unique<DefaultColumnGroupingStrategy>();
default:
ThrowInfo(ErrorCode::UnexpectedError,
"Unknown ColumnGroupingStrategyType");
}
}
};
struct ParquetWriterFactory {
static ParquetWriteContext
CreateContext(const std::map<JsonKey, JsonKeyLayoutType>& column_map,
const std::string& path_prefix) {
ParquetWriteContext context;
context.schema = CreateArrowSchema(column_map);
auto builders = CreateArrowBuilders(column_map);
context.builders = std::move(builders.first);
context.builders_map = std::move(builders.second);
context.kv_metadata = CreateParquetKVMetadata(column_map);
context.column_groups =
std::move(ColumnGroupingStrategyFactory::CreateStrategy(
ColumnGroupingStrategyType::DEFAULT)
->CreateGroups(TableStatsInfo{
context.schema,
column_map,
}));
auto column_group_id = 0;
for (const auto& group : context.column_groups) {
auto file_log_id = 0;
context.file_paths.push_back(CreateColumnGroupParquetPath(
path_prefix, column_group_id, file_log_id));
LOG_INFO("create column group parquet path: {} for column group {}",
context.file_paths.back(),
column_group_id);
column_group_id++;
}
return context;
}
};
class JsonStatsParquetWriter {
public:
JsonStatsParquetWriter(
std::shared_ptr<arrow::fs::FileSystem> fs,
milvus_storage::StorageConfig& storage_config,
size_t buffer_size,
size_t batch_size,
std::shared_ptr<parquet::WriterProperties> writer_props =
parquet::default_writer_properties());
~JsonStatsParquetWriter();
void
Init(const ParquetWriteContext& context);
void
AppendValue(const std::string& key, const std::string& value);
void
AppendRow(const std::map<std::string, std::string>& row_data);
void
AppendSharedRow(const uint8_t* data, size_t length);
void
Flush();
void
Close();
size_t
WriteCurrentBatch();
size_t
AddCurrentRow();
std::map<std::string, int64_t>
GetPathsToSize() const {
return path_size_map_;
}
int64_t
GetTotalSize() const {
return total_size_;
}
void
UpdatePathSizeMap(const std::vector<std::shared_ptr<arrow::Array>>& arrays);
private:
arrow::Status
AppendDataToBuilder(const std::string& value,
std::shared_ptr<arrow::ArrayBuilder> builder);
// init info
std::shared_ptr<arrow::Schema> schema_{nullptr};
// for column groups
std::vector<std::string> file_paths_;
std::vector<std::vector<int>> column_groups_;
std::map<std::string, int64_t> path_size_map_;
size_t total_size_{0};
std::shared_ptr<parquet::WriterProperties> writer_props_;
size_t buffer_size_;
size_t batch_size_;
std::shared_ptr<arrow::fs::FileSystem> fs_;
milvus_storage::StorageConfig storage_config_;
std::unique_ptr<milvus_storage::PackedRecordBatchWriter> packed_writer_;
std::vector<std::pair<std::string, std::string>> kv_metadata_;
// cache for builders
std::vector<std::shared_ptr<arrow::ArrayBuilder>> builders_;
std::map<std::string, std::shared_ptr<arrow::ArrayBuilder>> builders_map_;
size_t unflushed_row_count_{0};
size_t all_row_count_{0};
size_t current_row_count_{0};
};
} // namespace milvus::index

View File

@ -0,0 +1,221 @@
// 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 "index/json_stats/utils.h"
#include <boost/filesystem.hpp>
#include <boost/uuid/random_generator.hpp>
#include <boost/uuid/uuid_io.hpp>
#include "milvus-storage/common/constants.h"
namespace milvus::index {
std::shared_ptr<arrow::ArrayBuilder>
CreateSharedArrowBuilder() {
return std::make_shared<arrow::BinaryBuilder>();
}
std::shared_ptr<arrow::ArrayBuilder>
CreateArrowBuilder(JSONType type) {
std::shared_ptr<arrow::ArrayBuilder> builder;
switch (type) {
case JSONType::INT8:
builder = std::make_shared<arrow::Int8Builder>();
break;
case JSONType::INT16:
builder = std::make_shared<arrow::Int16Builder>();
break;
case JSONType::INT32:
builder = std::make_shared<arrow::Int32Builder>();
break;
case JSONType::INT64:
builder = std::make_shared<arrow::Int64Builder>();
break;
case JSONType::FLOAT:
builder = std::make_shared<arrow::FloatBuilder>();
break;
case JSONType::DOUBLE:
builder = std::make_shared<arrow::DoubleBuilder>();
break;
case JSONType::BOOL:
builder = std::make_shared<arrow::BooleanBuilder>();
break;
case JSONType::STRING:
builder = std::make_shared<arrow::StringBuilder>();
break;
case JSONType::ARRAY:
// Store array as bson binary in a dedicated column
builder = std::make_shared<arrow::BinaryBuilder>();
break;
default:
ThrowInfo(ErrorCode::Unsupported,
"Unsupported JSON type:{} ",
ToString(type));
}
return builder;
}
std::shared_ptr<arrow::Field>
CreateSharedArrowField(const std::string& field_name, int64_t field_id) {
auto metadata = std::make_shared<arrow::KeyValueMetadata>();
metadata->Append(milvus_storage::ARROW_FIELD_ID_KEY,
std::to_string(field_id));
metadata->Append(JSON_STATS_META_KEY_LAYOUT_TYPE_MAP,
ToString(JsonKeyLayoutType::SHARED));
return arrow::field(field_name, arrow::binary(), true, metadata);
}
std::shared_ptr<arrow::Field>
CreateArrowField(const JsonKey& key,
const JsonKeyLayoutType& key_type,
int64_t field_id) {
if (key_type == JsonKeyLayoutType::SHARED) {
ThrowInfo(ErrorCode::Unsupported,
"Shared field is not supported in CreateArrowField");
}
std::string field_name = key.ToColumnName();
auto metadata = std::make_shared<arrow::KeyValueMetadata>();
// metadata->Append(JSON_STATS_META_SHREDDING_COLUMN_KEY_MAP, key.key_);
// metadata->Append(JSON_STATS_META_KEY_LAYOUT_TYPE_MAP, ToString(key_type));
metadata->Append(milvus_storage::ARROW_FIELD_ID_KEY,
std::to_string(field_id));
std::shared_ptr<arrow::Field> field;
switch (key.type_) {
case JSONType::INT8:
field = arrow::field(field_name, arrow::int8(), true, metadata);
break;
case JSONType::INT16:
field = arrow::field(field_name, arrow::int16(), true, metadata);
break;
case JSONType::INT32:
field = arrow::field(field_name, arrow::int32(), true, metadata);
break;
case JSONType::INT64:
field = arrow::field(field_name, arrow::int64(), true, metadata);
break;
case JSONType::DOUBLE:
field = arrow::field(field_name, arrow::float64(), true, metadata);
break;
case JSONType::FLOAT:
field = arrow::field(field_name, arrow::float32(), true, metadata);
break;
case JSONType::BOOL:
field = arrow::field(field_name, arrow::boolean(), true, metadata);
break;
case JSONType::STRING:
field = arrow::field(field_name, arrow::utf8(), true, metadata);
break;
case JSONType::ARRAY:
// Store array payload as binary column (bson format)
field = arrow::field(field_name, arrow::binary(), true, metadata);
break;
default:
ThrowInfo(ErrorCode::Unsupported,
"Unsupported JSON type: {} ",
ToString(key.type_));
}
return field;
}
std::pair<std::vector<std::shared_ptr<arrow::ArrayBuilder>>,
std::map<std::string, std::shared_ptr<arrow::ArrayBuilder>>>
CreateArrowBuilders(std::map<JsonKey, JsonKeyLayoutType> column_map) {
std::shared_ptr<arrow::ArrayBuilder> shared_builder =
CreateSharedArrowBuilder();
std::vector<std::shared_ptr<arrow::ArrayBuilder>> builders;
std::map<std::string, std::shared_ptr<arrow::ArrayBuilder>> builders_map;
for (const auto& [key, type] : column_map) {
switch (type) {
case JsonKeyLayoutType::SHARED:
builders_map[key.ToColumnName()] = shared_builder;
break;
case JsonKeyLayoutType::TYPED:
case JsonKeyLayoutType::TYPED_NOT_ALL:
case JsonKeyLayoutType::DYNAMIC:
case JsonKeyLayoutType::DYNAMIC_ONLY: {
auto builder = CreateArrowBuilder(key.type_);
builders.push_back(builder);
builders_map[key.ToColumnName()] = builder;
break;
}
default:
ThrowInfo(ErrorCode::Unsupported,
"Unsupported JSON key type: {}",
ToString(type));
}
}
builders.push_back(shared_builder);
return std::make_pair(builders, builders_map);
}
std::shared_ptr<arrow::Schema>
CreateArrowSchema(std::map<JsonKey, JsonKeyLayoutType> column_map) {
std::vector<std::shared_ptr<arrow::Field>> fields;
std::shared_ptr<arrow::Field> shared_field = nullptr;
bool shared_field_name_conflict = false;
std::vector<std::string> may_conflict_shared_field_names;
auto field_id = START_JSON_STATS_FIELD_ID;
for (const auto& [key, type] : column_map) {
switch (type) {
case JsonKeyLayoutType::TYPED:
case JsonKeyLayoutType::TYPED_NOT_ALL:
case JsonKeyLayoutType::DYNAMIC:
case JsonKeyLayoutType::DYNAMIC_ONLY:
fields.push_back(CreateArrowField(key, type, field_id++));
if (field_id > END_JSON_STATS_FIELD_ID) {
ThrowInfo(ErrorCode::UnexpectedError,
"Field ID exceeds the limit: {}, field_id: {}",
END_JSON_STATS_FIELD_ID,
field_id);
}
if (key.key_ == JSON_KEY_STATS_SHARED_FIELD_NAME) {
shared_field_name_conflict = true;
}
break;
case JsonKeyLayoutType::SHARED:
break;
default:
ThrowInfo(ErrorCode::Unsupported,
"Unsupported JSON key type: {}",
ToString(type));
}
}
std::string field_name = JSON_KEY_STATS_SHARED_FIELD_NAME;
if (shared_field_name_conflict) {
boost::uuids::random_generator generator;
auto uuid = generator();
auto suffix = boost::uuids::to_string(uuid).substr(0, 5);
field_name = suffix + "_" + field_name;
}
fields.push_back(CreateSharedArrowField(field_name, field_id++));
return arrow::schema(fields);
}
std::vector<std::pair<std::string, std::string>>
CreateParquetKVMetadata(std::map<JsonKey, JsonKeyLayoutType> column_map) {
nlohmann::json key_type;
for (const auto& [key, type] : column_map) {
key_type[key.ToColumnName()] = ToString(type);
}
// for shared field, not need to store in metadata
std::vector<std::pair<std::string, std::string>> res;
res.push_back(
std::make_pair(JSON_STATS_META_KEY_LAYOUT_TYPE_MAP, key_type.dump()));
return res;
}
} // namespace milvus::index

View File

@ -0,0 +1,520 @@
// 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 <boost/filesystem.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string.hpp>
#include "index/InvertedIndexTantivy.h"
#include "common/jsmn.h"
#include "arrow/api.h"
#include "common/EasyAssert.h"
namespace milvus::index {
constexpr int64_t DEFAULT_BATCH_SIZE = 8192 * 10;
constexpr int64_t DEFAULT_BUFFER_SIZE = 16 * 1024 * 1024;
constexpr int64_t DEFAULT_PART_UPLOAD_SIZE = 10 * 1024 * 1024;
enum class JSONType {
UNKNOWN,
NONE,
BOOL,
INT8,
INT16,
INT32,
INT64,
FLOAT,
DOUBLE,
STRING,
STRING_ESCAPE,
ARRAY,
OBJECT
};
inline std::string
ToString(JSONType type) {
switch (type) {
case JSONType::NONE:
return "NONE";
case JSONType::BOOL:
return "BOOL";
case JSONType::INT8:
return "INT8";
case JSONType::INT16:
return "INT16";
case JSONType::INT32:
return "INT32";
case JSONType::INT64:
return "INT64";
case JSONType::FLOAT:
return "FLOAT";
case JSONType::DOUBLE:
return "DOUBLE";
case JSONType::STRING:
return "STRING";
case JSONType::STRING_ESCAPE:
return "STRING_ESCAPE";
case JSONType::ARRAY:
return "ARRAY";
case JSONType::OBJECT:
return "OBJECT";
default:
return "UNKNOWN";
}
}
inline milvus::DataType
GetPrimitiveDataType(JSONType type) {
switch (type) {
case JSONType::NONE:
return milvus::DataType::NONE;
case JSONType::BOOL:
return milvus::DataType::BOOL;
case JSONType::INT8:
return milvus::DataType::INT8;
case JSONType::INT16:
return milvus::DataType::INT16;
case JSONType::INT32:
return milvus::DataType::INT32;
case JSONType::INT64:
return milvus::DataType::INT64;
case JSONType::FLOAT:
return milvus::DataType::FLOAT;
case JSONType::DOUBLE:
return milvus::DataType::DOUBLE;
case JSONType::STRING:
return milvus::DataType::STRING;
// for array type(bson format), we use string type instead of real binary type
case JSONType::ARRAY:
return milvus::DataType::STRING;
default:
return milvus::DataType::NONE;
}
}
inline JSONType
GetPrimitiveJsonType(std::shared_ptr<arrow::DataType> type) {
if (type->id() == arrow::Type::BOOL) {
return JSONType::BOOL;
} else if (type->id() == arrow::Type::INT8) {
return JSONType::INT8;
} else if (type->id() == arrow::Type::INT16) {
return JSONType::INT16;
} else if (type->id() == arrow::Type::INT32) {
return JSONType::INT32;
} else if (type->id() == arrow::Type::INT64) {
return JSONType::INT64;
} else if (type->id() == arrow::Type::FLOAT) {
return JSONType::FLOAT;
} else if (type->id() == arrow::Type::DOUBLE) {
return JSONType::DOUBLE;
} else if (type->id() == arrow::Type::STRING) {
return JSONType::STRING;
} else {
return JSONType::UNKNOWN;
}
}
inline bool
IsPrimitiveJsonType(JSONType type) {
return type == JSONType::NONE || type == JSONType::BOOL ||
type == JSONType::INT8 || type == JSONType::INT16 ||
type == JSONType::INT32 || type == JSONType::INT64 ||
type == JSONType::FLOAT || type == JSONType::DOUBLE ||
type == JSONType::STRING || type == JSONType::STRING_ESCAPE;
}
inline bool
IsIntegerJsonType(JSONType type) {
return type == JSONType::INT8 || type == JSONType::INT16 ||
type == JSONType::INT32 || type == JSONType::INT64;
}
inline bool
IsFloatJsonType(JSONType type) {
return type == JSONType::FLOAT || type == JSONType::DOUBLE;
}
inline bool
IsStringJsonType(JSONType type) {
return type == JSONType::STRING || type == JSONType::STRING_ESCAPE;
}
inline bool
IsComplexJsonType(JSONType type) {
return type == JSONType::ARRAY || type == JSONType::OBJECT;
}
inline bool
IsNullJsonType(JSONType type) {
return type == JSONType::NONE;
}
inline bool
IsShreddingJsonType(JSONType type) {
return IsPrimitiveJsonType(type) || type == JSONType::ARRAY;
}
enum class JsonKeyLayoutType {
UNKNOWN = 0,
TYPED = 1,
TYPED_NOT_ALL = 2,
DYNAMIC = 3,
DYNAMIC_ONLY = 4,
SHARED = 5,
};
inline std::string
ToString(JsonKeyLayoutType type) {
switch (type) {
case JsonKeyLayoutType::TYPED:
return "TYPED";
case JsonKeyLayoutType::TYPED_NOT_ALL:
return "TYPED_NOT_ALL";
case JsonKeyLayoutType::DYNAMIC:
return "DYNAMIC";
case JsonKeyLayoutType::DYNAMIC_ONLY:
return "DYNAMIC_ONLY";
case JsonKeyLayoutType::SHARED:
return "SHARED";
default:
return "UNKNOWN";
}
}
inline JsonKeyLayoutType
JsonKeyLayoutTypeFromString(const std::string& str) {
if (str == "TYPED") {
return JsonKeyLayoutType::TYPED;
} else if (str == "TYPED_NOT_ALL") {
return JsonKeyLayoutType::TYPED_NOT_ALL;
} else if (str == "DYNAMIC") {
return JsonKeyLayoutType::DYNAMIC;
} else if (str == "DYNAMIC_ONLY") {
return JsonKeyLayoutType::DYNAMIC_ONLY;
} else if (str == "SHARED") {
return JsonKeyLayoutType::SHARED;
}
return JsonKeyLayoutType::UNKNOWN;
}
inline bool
EndWith(std::string_view str, std::string_view suffix) {
return str.size() >= suffix.size() &&
str.compare(str.size() - suffix.size(), suffix.size(), suffix) == 0;
}
inline JSONType
GetJsonTypeFromKeyName(const std::string& key_name) {
if (EndWith(key_name, "_INT64")) {
return JSONType::INT64;
} else if (EndWith(key_name, "_DOUBLE")) {
return JSONType::DOUBLE;
} else if (EndWith(key_name, "_STRING")) {
return JSONType::STRING;
} else if (EndWith(key_name, "_BOOL")) {
return JSONType::BOOL;
} else if (EndWith(key_name, "_NULL")) {
return JSONType::NONE;
} else if (EndWith(key_name, "_ARRAY")) {
return JSONType::ARRAY;
} else if (EndWith(key_name, "_OBJECT")) {
return JSONType::OBJECT;
} else {
return JSONType::UNKNOWN;
}
}
inline std::string
GetKeyFromColumnName(const std::string& column_name) {
if (EndWith(column_name, "_INT64")) {
return column_name.substr(0, column_name.size() - 6);
} else if (EndWith(column_name, "_DOUBLE")) {
return column_name.substr(0, column_name.size() - 7);
} else if (EndWith(column_name, "_STRING")) {
return column_name.substr(0, column_name.size() - 7);
} else if (EndWith(column_name, "_BOOL")) {
return column_name.substr(0, column_name.size() - 5);
} else if (EndWith(column_name, "_NULL")) {
return column_name.substr(0, column_name.size() - 5);
} else if (EndWith(column_name, "_ARRAY")) {
return column_name.substr(0, column_name.size() - 6);
} else if (EndWith(column_name, "_OBJECT")) {
return column_name.substr(0, column_name.size() - 7);
} else {
return column_name;
}
}
// construct json pointer with nested path vector
inline std::string
JsonPointer(std::vector<std::string> nested_path) {
if (nested_path.empty()) {
return "";
}
std::for_each(nested_path.begin(), nested_path.end(), [](std::string& key) {
boost::replace_all(key, "~", "~0");
boost::replace_all(key, "/", "~1");
});
auto pointer = "/" + boost::algorithm::join(nested_path, "/");
return pointer;
}
inline std::string
AppendJsonPointer(std::string pointer, std::string key) {
boost::replace_all(key, "~", "~0");
boost::replace_all(key, "/", "~1");
return pointer + "/" + key;
}
// parse json pointer to nested path vector
inline std::vector<std::string>
ParseJsonPointerPath(const std::string& pointer) {
if (pointer.empty()) {
return {};
}
if (pointer[0] != '/') {
ThrowInfo(ErrorCode::PathInvalid,
"Invalid JSON pointer: must start with '/'");
}
std::vector<std::string> tokens;
boost::split(tokens, pointer.substr(1), boost::is_any_of("/"));
for (auto& token : tokens) {
if (token.find('~') != std::string::npos) {
boost::replace_all(token, "~1", "/");
boost::replace_all(token, "~0", "~");
}
}
return tokens;
}
struct JsonKey {
std::string key_;
JSONType type_;
JsonKey(const std::string& key, JSONType type) : key_(key), type_(type) {
}
JsonKey() {
}
bool
operator<(const JsonKey& other) const {
return std::tie(key_, type_) < std::tie(other.key_, other.type_);
}
bool
operator==(const JsonKey& other) const {
return key_ == other.key_ && type_ == other.type_;
}
bool
operator!=(const JsonKey& other) const {
return !(*this == other);
}
std::string
ToString() const {
return key_ + ":" + milvus::index::ToString(type_);
}
std::string
ToColumnName() const {
return key_ + "_" + milvus::index::ToString(type_);
}
};
struct KeyStatsInfo {
int32_t hit_row_num_ = 0;
// TODO: add min/max value for stats info
uint8_t min_value_[8] = {0};
uint8_t max_value_[8] = {0};
std::string
ToString() const {
return "row_num: " + std::to_string(hit_row_num_);
}
};
struct PathWriter {
JsonKeyLayoutType type_;
JsonKey key_;
};
std::shared_ptr<arrow::ArrayBuilder>
CreateSharedArrowBuilder();
std::shared_ptr<arrow::ArrayBuilder>
CreateArrowBuilder(JSONType type);
std::shared_ptr<arrow::Field>
CreateSharedArrowField(const std::string& field_name, int64_t field_id);
std::shared_ptr<arrow::Field>
CreateArrowField(const JsonKey& key, const JsonKeyLayoutType& key_type);
std::pair<std::vector<std::shared_ptr<arrow::ArrayBuilder>>,
std::map<std::string, std::shared_ptr<arrow::ArrayBuilder>>>
CreateArrowBuilders(std::map<JsonKey, JsonKeyLayoutType> column_map);
std::shared_ptr<arrow::Schema>
CreateArrowSchema(std::map<JsonKey, JsonKeyLayoutType> column_map);
std::vector<std::pair<std::string, std::string>>
CreateParquetKVMetadata(std::map<JsonKey, JsonKeyLayoutType> column_map);
inline size_t
GetArrowArrayMemorySize(const std::shared_ptr<arrow::Array>& array) {
if (!array || !array->data()) {
return 0;
}
size_t total_size = 0;
for (const auto& buffer : array->data()->buffers) {
if (buffer) {
total_size += buffer->size();
}
}
return total_size;
}
inline std::string
CreateColumnGroupParquetPath(const std::string& path_prefix,
size_t column_group_id,
size_t file_log_id) {
return path_prefix + "/" + std::to_string(column_group_id) + "/" +
std::to_string(file_log_id);
}
// parse <column_group_id, file_id> from parquet path
inline std::pair<size_t, size_t>
ParseParquetPath(const std::string& path) {
const auto last_slash = path.find_last_of('/');
const auto second_last_slash = path.find_last_of('/', last_slash - 1);
const size_t column_group_id = std::stoll(
path.substr(second_last_slash + 1, last_slash - second_last_slash - 1));
const size_t file_id = std::stoll(path.substr(last_slash + 1));
return {column_group_id, file_id};
}
// sort parquet paths by column_group_id and file_id
// return vector of <column_group_id, vector<file_id>>
inline std::vector<std::pair<int64_t, std::vector<int64_t>>>
SortByParquetPath(const std::vector<std::string>& paths) {
// preprocess stage: parse all path info
std::vector<std::pair<int64_t, int64_t>> parsed_info;
parsed_info.reserve(paths.size());
for (const auto& path : paths) {
parsed_info.emplace_back(ParseParquetPath(path));
}
// sort stage: directly compare parsed values
std::sort(parsed_info.begin(),
parsed_info.end(),
[](const auto& a, const auto& b) {
return std::tie(a.first, a.second) <
std::tie(b.first, b.second);
});
// group stage: single traversal to complete grouping
std::vector<std::pair<int64_t, std::vector<int64_t>>> result;
for (const auto& [col, file] : parsed_info) {
if (result.empty() || result.back().first != col) {
result.emplace_back(col, std::vector<int64_t>());
}
result.back().second.push_back(file);
}
return result;
}
} // namespace milvus::index
template <>
struct fmt::formatter<milvus::index::JSONType> : fmt::formatter<std::string> {
template <typename FormatContext>
auto
format(const milvus::index::JSONType& jt, FormatContext& ctx) {
switch (jt) {
case milvus::index::JSONType::UNKNOWN:
return fmt::format_to(ctx.out(), "UNKNOWN");
case milvus::index::JSONType::NONE:
return fmt::format_to(ctx.out(), "NULL");
case milvus::index::JSONType::BOOL:
return fmt::format_to(ctx.out(), "BOOL");
case milvus::index::JSONType::INT8:
return fmt::format_to(ctx.out(), "INT8");
case milvus::index::JSONType::INT16:
return fmt::format_to(ctx.out(), "INT16");
case milvus::index::JSONType::INT32:
return fmt::format_to(ctx.out(), "INT32");
case milvus::index::JSONType::INT64:
return fmt::format_to(ctx.out(), "INT64");
case milvus::index::JSONType::FLOAT:
return fmt::format_to(ctx.out(), "FLOAT");
case milvus::index::JSONType::DOUBLE:
return fmt::format_to(ctx.out(), "DOUBLE");
case milvus::index::JSONType::STRING:
return fmt::format_to(ctx.out(), "STRING");
case milvus::index::JSONType::STRING_ESCAPE:
return fmt::format_to(ctx.out(), "STRING_ESCAPE");
case milvus::index::JSONType::ARRAY:
return fmt::format_to(ctx.out(), "ARRAY");
case milvus::index::JSONType::OBJECT:
return fmt::format_to(ctx.out(), "OBJECT");
default:
return fmt::format_to(ctx.out(), "UNKNOWN");
}
}
};
template <>
struct fmt::formatter<milvus::index::JsonKeyLayoutType>
: fmt::formatter<std::string> {
auto
format(milvus::index::JsonKeyLayoutType type, fmt::format_context& ctx) {
std::string name;
switch (type) {
case milvus::index::JsonKeyLayoutType::TYPED:
name = "TYPED";
break;
case milvus::index::JsonKeyLayoutType::DYNAMIC:
name = "DYNAMIC";
break;
case milvus::index::JsonKeyLayoutType::SHARED:
name = "SHARED";
break;
case milvus::index::JsonKeyLayoutType::TYPED_NOT_ALL:
name = "TYPED_NOT_ALL";
break;
case milvus::index::JsonKeyLayoutType::DYNAMIC_ONLY:
name = "DYNAMIC_ONLY";
break;
default:
name = "UNKNOWN";
break;
}
return fmt::formatter<std::string>::format(name, ctx);
}
};

View File

@ -36,7 +36,7 @@
#include "pb/index_cgo_msg.pb.h"
#include "storage/Util.h"
#include "index/Meta.h"
#include "index/JsonKeyStatsInvertedIndex.h"
#include "index/json_stats/JsonKeyStats.h"
#include "milvus-storage/filesystem/fs.h"
#include "monitor/scope_metric.h"
@ -298,8 +298,8 @@ BuildJsonKeyIndex(ProtoLayoutInterface result,
build_index_info->ParseFromArray(serialized_build_index_info, len);
AssertInfo(res, "Unmarshall build index info failed");
auto field_type =
static_cast<DataType>(build_index_info->field_schema().data_type());
auto field_type = static_cast<milvus::DataType>(
build_index_info->field_schema().data_type());
auto storage_config =
get_storage_config(build_index_info->storage_config());
@ -353,10 +353,12 @@ BuildJsonKeyIndex(ProtoLayoutInterface result,
auto field_schema =
FieldMeta::ParseFrom(build_index_info->field_schema());
auto index = std::make_unique<index::JsonKeyStatsInvertedIndex>(
auto index = std::make_unique<index::JsonKeyStats>(
fileManagerContext,
false,
build_index_info->json_key_stats_tantivy_memory(),
build_index_info->json_stats_max_shredding_columns(),
build_index_info->json_stats_shredding_ratio_threshold(),
build_index_info->json_stats_write_batch_size(),
tantivy_index_version);
index->Build(config);
auto create_index_result = index->Upload(config);
@ -392,8 +394,8 @@ BuildTextIndex(ProtoLayoutInterface result,
build_index_info->ParseFromArray(serialized_build_index_info, len);
AssertInfo(res, "Unmarshal build index info failed");
auto field_type =
static_cast<DataType>(build_index_info->field_schema().data_type());
auto field_type = static_cast<milvus::DataType>(
build_index_info->field_schema().data_type());
auto storage_config =
get_storage_config(build_index_info->storage_config());

View File

@ -554,6 +554,30 @@ class ChunkedVariableColumn : public ChunkedColumnBase {
fn(Json(str_view.data(), str_view.size()), i, valid);
}
}
void
BulkRawBsonAt(std::function<void(BsonView, uint32_t, uint32_t)> fn,
const uint32_t* row_offsets,
const uint32_t* value_offsets,
int64_t count) const override {
if (count == 0) {
return;
}
AssertInfo(row_offsets != nullptr && value_offsets != nullptr,
"row_offsets and value_offsets must be provided");
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(row_offsets, count);
auto ca = SemiInlineGet(slot_->PinCells(cids));
for (int64_t i = 0; i < count; i++) {
auto chunk = ca->get_cell_of(cids[i]);
auto str_view = static_cast<StringChunk*>(chunk)->operator[](
offsets_in_chunk[i]);
fn(BsonView(reinterpret_cast<const uint8_t*>(str_view.data()),
str_view.size()),
row_offsets[i],
value_offsets[i]);
}
}
};
class ChunkedArrayColumn : public ChunkedColumnBase {

View File

@ -556,6 +556,37 @@ class ProxyChunkColumn : public ChunkedColumnInterface {
}
}
void
BulkRawBsonAt(std::function<void(BsonView, uint32_t, uint32_t)> fn,
const uint32_t* row_offsets,
const uint32_t* value_offsets,
int64_t count) const override {
if (data_type_ != DataType::STRING) {
ThrowInfo(ErrorCode::Unsupported,
"BulkRawBsonAt only supported for ProxyChunkColumn of "
"Bson type");
}
if (count == 0) {
return;
}
AssertInfo(row_offsets != nullptr, "row_offsets is nullptr");
auto [cids, offsets_in_chunk] = ToChunkIdAndOffset(row_offsets, count);
auto ca = group_->GetGroupChunks(cids);
for (int64_t i = 0; i < count; i++) {
auto* group_chunk = ca->get_cell_of(cids[i]);
auto chunk = group_chunk->GetChunk(field_id_);
auto str_view = static_cast<StringChunk*>(chunk.get())
->
operator[](offsets_in_chunk[i]);
fn(BsonView(reinterpret_cast<const uint8_t*>(str_view.data()),
str_view.size()),
row_offsets[i],
value_offsets[i]);
}
}
void
BulkArrayAt(std::function<void(ScalarFieldProto&&, size_t)> fn,
const int64_t* offsets,

View File

@ -17,7 +17,7 @@
#include "cachinglayer/CacheSlot.h"
#include "common/Chunk.h"
#include "common/bson_view.h"
namespace milvus {
using namespace milvus::cachinglayer;
@ -157,6 +157,16 @@ class ChunkedColumnInterface {
"RawJsonAt only supported for ChunkColumnInterface of Json type");
}
virtual void
BulkRawBsonAt(std::function<void(BsonView, uint32_t, uint32_t)> fn,
const uint32_t* row_offsets,
const uint32_t* value_offsets,
int64_t count) const {
ThrowInfo(ErrorCode::Unsupported,
"BulkRawBsonAt only supported for ChunkColumnInterface of "
"Bson type");
}
virtual void
BulkArrayAt(std::function<void(ScalarFieldProto&&, size_t)> fn,
const int64_t* offsets,
@ -221,6 +231,22 @@ class ChunkedColumnInterface {
}
return GetChunkIDsByOffsets(offsets, count);
}
std::pair<std::vector<milvus::cachinglayer::cid_t>, std::vector<uint32_t>>
ToChunkIdAndOffset(const uint32_t* offsets, int64_t count) const {
AssertInfo(offsets != nullptr, "Offsets cannot be nullptr");
std::vector<milvus::cachinglayer::cid_t> cids;
cids.reserve(count);
std::vector<uint32_t> offsets_in_chunk;
offsets_in_chunk.reserve(count);
for (int64_t i = 0; i < count; i++) {
auto [chunk_id, offset_in_chunk] = GetChunkIDByOffset(offsets[i]);
cids.push_back(chunk_id);
offsets_in_chunk.push_back(offset_in_chunk);
}
return std::make_pair(std::move(cids), std::move(offsets_in_chunk));
}
};
} // namespace milvus

View File

@ -130,6 +130,30 @@ DEFINE_PROMETHEUS_HISTOGRAM(internal_storage_deserialize_duration,
internal_storage_load_duration,
deserializeDurationLabels)
// json stats metrics
std::map<std::string, std::string> invertedIndexLatencyLabels{
{"type", "inverted_index_latency"}};
std::map<std::string, std::string> shreddingLatencyLabels{
{"type", "shredding_latency"}};
std::map<std::string, std::string> sharedLatencyLabels{
{"type", "shared_latency"}};
std::map<std::string, std::string> loadLatencyLabels{{"type", "load_latency"}};
DEFINE_PROMETHEUS_HISTOGRAM_FAMILY(internal_json_stats_latency,
"[cpp]durations of json stats")
DEFINE_PROMETHEUS_HISTOGRAM(internal_json_stats_latency_term_query,
internal_json_stats_latency,
invertedIndexLatencyLabels)
DEFINE_PROMETHEUS_HISTOGRAM(internal_json_stats_latency_shredding,
internal_json_stats_latency,
shreddingLatencyLabels)
DEFINE_PROMETHEUS_HISTOGRAM(internal_json_stats_latency_shared,
internal_json_stats_latency,
sharedLatencyLabels)
DEFINE_PROMETHEUS_HISTOGRAM(internal_json_stats_latency_load,
internal_json_stats_latency,
loadLatencyLabels)
// search latency metrics
std::map<std::string, std::string> scalarLatencyLabels{
{"type", "scalar_latency"}};

View File

@ -94,6 +94,13 @@ DECLARE_PROMETHEUS_GAUGE(internal_cgo_inflight_task_total_all);
DECLARE_PROMETHEUS_GAUGE_FAMILY(internal_cgo_executing_task_total);
DECLARE_PROMETHEUS_GAUGE(internal_cgo_executing_task_total_all);
// json stats metrics
DECLARE_PROMETHEUS_HISTOGRAM_FAMILY(internal_json_stats_latency);
DECLARE_PROMETHEUS_HISTOGRAM(internal_json_stats_latency_term_query);
DECLARE_PROMETHEUS_HISTOGRAM(internal_json_stats_latency_shredding);
DECLARE_PROMETHEUS_HISTOGRAM(internal_json_stats_latency_shared);
DECLARE_PROMETHEUS_HISTOGRAM(internal_json_stats_latency_load);
// --- file writer metrics ---
DECLARE_PROMETHEUS_COUNTER_FAMILY(disk_write_total_bytes);

View File

@ -431,8 +431,7 @@ class CountNode : public PlanNode {
std::string
ToString() const override {
return fmt::format("VectorSearchNode:\n\t[source node:{}]",
SourceToString());
return fmt::format("CountNode:\n\t[source node:{}]", SourceToString());
}
private:

View File

@ -138,7 +138,9 @@ ExecPlanNodeVisitor::VectorVisitorImpl(VectorPlanNode& node) {
active_count,
timestamp_,
collection_ttl_timestamp_,
consystency_level_);
consystency_level_,
node.plan_options_);
query_context->set_search_info(node.search_info_);
query_context->set_placeholder_group(placeholder_group_);
@ -194,7 +196,8 @@ ExecPlanNodeVisitor::visit(RetrievePlanNode& node) {
active_count,
timestamp_,
collection_ttl_timestamp_,
consystency_level_);
consystency_level_,
node.plan_options_);
// Do task execution
auto bitset_holder = ExecuteTask(plan, query_context);

View File

@ -25,12 +25,19 @@ class PlanNode;
namespace milvus::query {
class PlanNodeVisitor;
struct PlanOptions {
bool expr_use_json_stats = true;
};
// Base of all Nodes
struct PlanNode {
public:
virtual ~PlanNode() = default;
virtual void
accept(PlanNodeVisitor&) = 0;
PlanOptions plan_options_;
};
using PlanNodePtr = std::unique_ptr<PlanNode>;

View File

@ -29,6 +29,15 @@
namespace milvus::query {
namespace planpb = milvus::proto::plan;
void
ProtoParser::PlanOptionsFromProto(
const proto::plan::PlanOption& plan_option_proto,
PlanOptions& plan_options) {
plan_options.expr_use_json_stats = plan_option_proto.expr_use_json_stats();
LOG_INFO("plan_options.expr_use_json_stats: {}",
plan_options.expr_use_json_stats);
}
std::unique_ptr<VectorPlanNode>
ProtoParser::PlanNodeFromProto(const planpb::PlanNode& plan_node_proto) {
// TODO: add more buffs
@ -224,6 +233,9 @@ ProtoParser::PlanNodeFromProto(const planpb::PlanNode& plan_node_proto) {
plan_node->plannodes_ = plannode;
PlanOptionsFromProto(plan_node_proto.plan_options(),
plan_node->plan_options_);
return plan_node;
}
@ -306,6 +318,9 @@ ProtoParser::RetrievePlanNodeFromProto(
return node;
}();
PlanOptionsFromProto(plan_node_proto.plan_options(),
plan_node->plan_options_);
return plan_node;
}

View File

@ -101,6 +101,10 @@ class ProtoParser {
expr::TypedExprPtr
ParseValueExprs(const proto::plan::ValueExpr& expr_pb);
void
PlanOptionsFromProto(const proto::plan::PlanOption& plan_option_proto,
PlanOptions& plan_options);
private:
const SchemaPtr schema;
};

View File

@ -102,20 +102,25 @@ class ChunkedSegmentSealedImpl : public SegmentSealed {
void
LoadTextIndex(FieldId field_id,
std::unique_ptr<index::TextMatchIndex> index) override;
void
LoadJsonKeyIndex(
FieldId field_id,
std::unique_ptr<index::JsonKeyStatsInvertedIndex> index) override {
RemoveJsonStats(FieldId field_id) override {
std::unique_lock lck(mutex_);
const auto& field_meta = schema_->operator[](field_id);
json_key_indexes_[field_id] = std::move(index);
json_stats_.erase(field_id);
}
index::JsonKeyStatsInvertedIndex*
GetJsonKeyIndex(FieldId field_id) const override {
void
LoadJsonStats(FieldId field_id,
std::shared_ptr<index::JsonKeyStats> stats) override {
std::unique_lock lck(mutex_);
json_stats_[field_id] = stats;
}
index::JsonKeyStats*
GetJsonStats(FieldId field_id) const override {
std::shared_lock lck(mutex_);
auto iter = json_key_indexes_.find(field_id);
if (iter == json_key_indexes_.end()) {
auto iter = json_stats_.find(field_id);
if (iter == json_stats_.end()) {
return nullptr;
}
return iter->second.get();
@ -536,10 +541,6 @@ 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;
// used for json expr optimization
std::unordered_map<FieldId,
std::unique_ptr<index::JsonKeyStatsInvertedIndex>>
json_key_indexes_;
};
inline SegmentSealedUPtr

View File

@ -215,33 +215,6 @@ SegmentGrowingImpl::Insert(int64_t reserved_offset,
reserved_offset);
}
// index json.
if (field_meta.enable_growing_jsonStats()) {
std::vector<std::string> jsonDatas(
insert_record_proto->fields_data(data_offset)
.scalars()
.json_data()
.data()
.begin(),
insert_record_proto->fields_data(data_offset)
.scalars()
.json_data()
.data()
.end());
FixedVector<bool> jsonDatas_valid_data(
insert_record_proto->fields_data(data_offset)
.valid_data()
.begin(),
insert_record_proto->fields_data(data_offset)
.valid_data()
.end());
AddJSONDatas(field_id,
jsonDatas.data(),
jsonDatas_valid_data.data(),
num_rows,
reserved_offset);
}
// update average row data size
auto field_data_size = GetRawDataSizeOfDataArray(
&insert_record_proto->fields_data(data_offset),
@ -410,15 +383,6 @@ SegmentGrowingImpl::load_field_data_common(
index->Reload();
}
// build json match index
if (field_meta.enable_growing_jsonStats()) {
auto index = GetJsonKeyIndex(field_id);
index->BuildWithFieldData(field_data, field_meta.is_nullable());
index->Commit();
// Reload reader so that the index can be read immediately
index->Reload();
}
// update the mem size
stats_.mem_size += storage::GetByteSizeOfFieldDatas(field_data);
@ -1241,44 +1205,6 @@ SegmentGrowingImpl::AddTexts(milvus::FieldId field_id,
iter->second->AddTextsGrowing(n, texts, texts_valid_data, offset_begin);
}
void
SegmentGrowingImpl::AddJSONDatas(FieldId field_id,
const std::string* jsondatas,
const bool* jsondatas_valid_data,
size_t n,
int64_t offset_begin) {
std::unique_lock lock(mutex_);
auto iter = json_indexes_.find(field_id);
AssertInfo(iter != json_indexes_.end(), "json index not found");
iter->second->AddJSONDatas(
n, jsondatas, jsondatas_valid_data, offset_begin);
}
void
SegmentGrowingImpl::CreateJSONIndexes() {
for (auto [field_id, field_meta] : schema_->get_fields()) {
if (field_meta.enable_growing_jsonStats()) {
CreateJSONIndex(FieldId(field_id));
}
}
}
void
SegmentGrowingImpl::CreateJSONIndex(FieldId field_id) {
std::unique_lock lock(mutex_);
const auto& field_meta = schema_->operator[](field_id);
AssertInfo(IsJsonDataType(field_meta.get_data_type()),
"cannot create json index on non-json type");
std::string unique_id = GetUniqueFieldId(field_meta.get_id().get());
auto index = std::make_unique<index::JsonKeyStatsInvertedIndex>(
JSON_KEY_STATS_COMMIT_INTERVAL, unique_id.c_str());
index->Commit();
index->CreateReader(milvus::index::SetBitsetGrowing);
json_indexes_[field_id] = std::move(index);
}
void
SegmentGrowingImpl::BulkGetJsonData(
FieldId field_id,

View File

@ -307,7 +307,6 @@ class SegmentGrowingImpl : public SegmentGrowing {
},
segment_id) {
this->CreateTextIndexes();
this->CreateJSONIndexes();
}
~SegmentGrowingImpl() {
@ -479,19 +478,6 @@ class SegmentGrowingImpl : public SegmentGrowing {
void
CreateTextIndexes();
void
AddJSONDatas(FieldId field_id,
const std::string* jsondatas,
const bool* jsondatas_valid_data,
size_t n,
int64_t offset_begin);
void
CreateJSONIndexes();
void
CreateJSONIndex(FieldId field_id);
private:
storage::MmapChunkDescriptorPtr mmap_descriptor_ = nullptr;
SegcoreConfig segcore_config_;

View File

@ -538,16 +538,6 @@ SegmentInternalInterface::bulk_subscript_not_exist_field(
return result;
}
index::JsonKeyStatsInvertedIndex*
SegmentInternalInterface::GetJsonKeyIndex(FieldId field_id) const {
std::shared_lock lock(mutex_);
auto iter = json_indexes_.find(field_id);
if (iter == json_indexes_.end()) {
return nullptr;
}
return iter->second.get();
}
// Only sealed segment has ngram index
PinWrapper<index::NgramInvertedIndex*>
SegmentInternalInterface::GetNgramIndex(FieldId field_id) const {
@ -571,4 +561,14 @@ SegmentInternalInterface::HasNgramIndexForJson(
return false;
}
index::JsonKeyStats*
SegmentInternalInterface::GetJsonStats(FieldId field_id) const {
std::shared_lock lock(mutex_);
auto iter = json_stats_.find(field_id);
if (iter == json_stats_.end()) {
return nullptr;
}
return iter->second.get();
}
} // namespace milvus::segcore

View File

@ -40,10 +40,10 @@
#include "pb/segcore.pb.h"
#include "index/SkipIndex.h"
#include "index/TextMatchIndex.h"
#include "index/JsonKeyStatsInvertedIndex.h"
#include "segcore/ConcurrentVector.h"
#include "segcore/InsertRecord.h"
#include "index/NgramInvertedIndex.h"
#include "index/json_stats/JsonKeyStats.h"
namespace milvus::segcore {
@ -145,8 +145,6 @@ class SegmentInterface {
GetJsonIndex(FieldId field_id, std::string path) const {
return nullptr;
}
virtual index::JsonKeyStatsInvertedIndex*
GetJsonKeyIndex(FieldId field_id) const = 0;
virtual void
BulkGetJsonData(FieldId field_id,
@ -168,6 +166,9 @@ class SegmentInterface {
HasNgramIndexForJson(FieldId field_id,
const std::string& nested_path) const = 0;
virtual index::JsonKeyStats*
GetJsonStats(FieldId field_id) const = 0;
virtual void
LazyCheckSchema(SchemaPtr sch) = 0;
@ -379,9 +380,6 @@ class SegmentInternalInterface : public SegmentInterface {
index::TextMatchIndex*
GetTextIndex(FieldId field_id) const override;
virtual index::JsonKeyStatsInvertedIndex*
GetJsonKeyIndex(FieldId field_id) const override;
virtual PinWrapper<index::NgramInvertedIndex*>
GetNgramIndex(FieldId field_id) const override;
@ -396,6 +394,9 @@ class SegmentInternalInterface : public SegmentInterface {
HasNgramIndexForJson(FieldId field_id,
const std::string& nested_path) const override;
virtual index::JsonKeyStats*
GetJsonStats(FieldId field_id) const override;
public:
// `query_lims` is not null only for vector array (embedding list) search
// where it denotes the number of vectors in each embedding list. The length
@ -618,9 +619,8 @@ class SegmentInternalInterface : public SegmentInterface {
std::unordered_map<FieldId, std::unique_ptr<index::TextMatchIndex>>
text_indexes_;
std::unordered_map<FieldId,
std::unique_ptr<index::JsonKeyStatsInvertedIndex>>
json_indexes_;
std::unordered_map<FieldId, std::shared_ptr<index::JsonKeyStats>>
json_stats_;
};
} // namespace milvus::segcore

View File

@ -99,11 +99,6 @@ class SegmentSealed : public SegmentInternalInterface {
return PinWrapper<index::IndexBase*>(ca, index);
}
virtual void
LoadJsonKeyIndex(
FieldId field_id,
std::unique_ptr<index::JsonKeyStatsInvertedIndex> index) = 0;
virtual bool
HasNgramIndex(FieldId field_id) const = 0;
@ -117,6 +112,12 @@ class SegmentSealed : public SegmentInternalInterface {
virtual PinWrapper<index::NgramInvertedIndex*>
GetNgramIndexForJson(FieldId field_id,
const std::string& nested_path) const override = 0;
virtual void
LoadJsonStats(FieldId field_id,
std::shared_ptr<index::JsonKeyStats> stats) = 0;
virtual void
RemoveJsonStats(FieldId field_id) = 0;
SegmentType
type() const override {

View File

@ -988,6 +988,60 @@ ReverseDataFromIndex(const index::IndexBase* index,
return data_array;
}
void
LoadArrowReaderForJsonStatsFromRemote(
const std::vector<std::string>& remote_files,
std::shared_ptr<ArrowReaderChannel> channel) {
try {
auto rcm = storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
auto& pool = ThreadPools::GetThreadPool(ThreadPoolPriority::HIGH);
std::vector<std::future<std::shared_ptr<milvus::ArrowDataWrapper>>>
futures;
futures.reserve(remote_files.size());
for (const auto& file : remote_files) {
auto future = pool.Submit([rcm, file]() {
auto fileSize = rcm->Size(file);
auto buf = std::shared_ptr<uint8_t[]>(new uint8_t[fileSize]);
rcm->Read(file, buf.get(), fileSize);
auto arrow_buf =
std::make_shared<arrow::Buffer>(buf.get(), fileSize);
auto buffer_reader =
std::make_shared<arrow::io::BufferReader>(arrow_buf);
std::unique_ptr<parquet::arrow::FileReader> arrow_reader;
auto status = parquet::arrow::OpenFile(
buffer_reader, arrow::default_memory_pool(), &arrow_reader);
AssertInfo(status.ok(),
"failed to open parquet file: {}",
status.message());
std::shared_ptr<arrow::RecordBatchReader> batch_reader;
status = arrow_reader->GetRecordBatchReader(&batch_reader);
AssertInfo(status.ok(),
"failed to get record batch reader: {}",
status.message());
return std::make_shared<ArrowDataWrapper>(
std::move(batch_reader), std::move(arrow_reader), buf);
});
futures.emplace_back(std::move(future));
}
for (auto& future : futures) {
auto field_data = future.get();
channel->push(field_data);
}
channel->close();
} catch (std::exception& e) {
LOG_INFO("failed to load data from remote: {}", e.what());
channel->close(std::current_exception());
}
}
// init segcore storage config first, and create default remote chunk manager
// segcore use default remote chunk manager to load data from minio/s3
void

View File

@ -106,6 +106,10 @@ ReverseDataFromIndex(const index::IndexBase* index,
const int64_t* seg_offsets,
int64_t count,
const FieldMeta& field_meta);
void
LoadArrowReaderForJsonStatsFromRemote(
const std::vector<std::string>& remote_files,
std::shared_ptr<ArrowReaderChannel> channel);
void
LoadArrowReaderFromRemote(const std::vector<std::string>& remote_files,

View File

@ -22,6 +22,7 @@
#include "common/Types.h"
#include "common/Tracer.h"
#include "common/type_c.h"
#include "common/ScopedTimer.h"
#include "google/protobuf/text_format.h"
#include "log/Log.h"
#include "mmap/Types.h"
@ -39,6 +40,7 @@
#include "mmap/Types.h"
#include "storage/RemoteChunkManagerSingleton.h"
#include "exec/expression/ExprCache.h"
#include "monitor/Monitor.h"
////////////////////////////// common interfaces //////////////////////////////
CStatus
@ -529,15 +531,32 @@ LoadJsonKeyIndex(CTraceContext c_trace,
config[milvus::index::INDEX_FILES] = files;
config[milvus::LOAD_PRIORITY] = info_proto->load_priority();
config[milvus::index::ENABLE_MMAP] = info_proto->enable_mmap();
if (info_proto->enable_mmap()) {
config[milvus::index::MMAP_FILE_PATH] = info_proto->mmap_dir_path();
}
milvus::storage::FileManagerContext file_ctx(
field_meta, index_meta, remote_chunk_manager);
auto index = std::make_unique<milvus::index::JsonKeyStatsInvertedIndex>(
file_ctx, true);
index->Load(ctx, config);
auto index =
std::make_shared<milvus::index::JsonKeyStats>(file_ctx, true);
{
milvus::ScopedTimer timer(
"json_stats_load",
[](double ms) {
milvus::monitor::internal_json_stats_latency_load.Observe(
ms);
},
milvus::ScopedTimer::LogLevel::Info);
index->Load(ctx, config);
}
segment->LoadJsonKeyIndex(milvus::FieldId(info_proto->fieldid()),
std::move(index));
segment->LoadJsonStats(milvus::FieldId(info_proto->fieldid()),
std::move(index));
LOG_INFO("load json stats success for field:{} of segment:{}",
info_proto->fieldid(),
segment->get_segment_id());
return milvus::SuccessCStatus();
} catch (std::exception& e) {

View File

@ -271,6 +271,7 @@ std::unique_ptr<milvus::GroupChunk>
GroupChunkTranslator::load_group_chunk(
const std::shared_ptr<arrow::Table>& table,
const milvus::cachinglayer::cid_t cid) {
AssertInfo(table != nullptr, "arrow table is nullptr");
// Create chunks for each field in this batch
std::unordered_map<FieldId, std::shared_ptr<Chunk>> chunks;
// Iterate through field_id_list to get field_id and create chunk

View File

@ -63,7 +63,7 @@ DiskFileManagerImpl::DiskFileManagerImpl(
DiskFileManagerImpl::~DiskFileManagerImpl() {
RemoveIndexFiles();
RemoveTextLogFiles();
RemoveJsonKeyIndexFiles();
RemoveJsonStatsFiles();
RemoveNgramIndexFiles();
}
@ -94,10 +94,21 @@ DiskFileManagerImpl::GetRemoteTextLogPath(const std::string& file_name,
}
std::string
DiskFileManagerImpl::GetRemoteJsonKeyIndexPath(const std::string& file_name,
int64_t slice_num) {
auto remote_prefix = GetRemoteJsonKeyLogPrefix();
return remote_prefix + "/" + file_name + "_" + std::to_string(slice_num);
DiskFileManagerImpl::GetRemoteJsonStatsSharedIndexPath(
const std::string& file_name, int64_t slice_num) {
namespace fs = std::filesystem;
fs::path prefix = GetRemoteJsonStatsLogPrefix();
fs::path suffix = JSON_STATS_SHARED_INDEX_PATH;
fs::path file = file_name + "_" + std::to_string(slice_num);
return (prefix / suffix / file).string();
}
std::string
DiskFileManagerImpl::GetRemoteJsonStatsShreddingPrefix() {
namespace fs = std::filesystem;
fs::path prefix = GetRemoteJsonStatsLogPrefix();
fs::path suffix = JSON_STATS_SHREDDING_DATA_PATH;
return (prefix / suffix).string();
}
bool
@ -205,10 +216,10 @@ DiskFileManagerImpl::AddFileMeta(const FileMeta& file_meta) {
}
bool
DiskFileManagerImpl::AddJsonKeyIndexLog(const std::string& file) noexcept {
DiskFileManagerImpl::AddJsonSharedIndexLog(const std::string& file) noexcept {
return AddFileInternal(
file, [this](const std::string& file_name, int slice_num) {
return GetRemoteJsonKeyIndexPath(file_name, slice_num);
return GetRemoteJsonStatsSharedIndexPath(file_name, slice_num);
});
}
@ -364,14 +375,6 @@ DiskFileManagerImpl::CacheTextLogToDisk(
remote_files, GetLocalTextIndexPrefix(), priority);
}
void
DiskFileManagerImpl::CacheJsonKeyIndexToDisk(
const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority) {
return CacheIndexToDiskInternal(
remote_files, GetLocalJsonKeyIndexPrefix(), priority);
}
void
DiskFileManagerImpl::CacheNgramIndexToDisk(
const std::vector<std::string>& remote_files,
@ -380,6 +383,14 @@ DiskFileManagerImpl::CacheNgramIndexToDisk(
remote_files, GetLocalNgramIndexPrefix(), priority);
}
void
DiskFileManagerImpl::CacheJsonStatsSharedIndexToDisk(
const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority) {
return CacheIndexToDiskInternal(
remote_files, GetLocalJsonStatsSharedIndexPrefix(), priority);
}
template <typename DataType>
std::string
DiskFileManagerImpl::CacheRawDataToDisk(const Config& config) {
@ -587,10 +598,10 @@ DiskFileManagerImpl::RemoveTextLogFiles() {
}
void
DiskFileManagerImpl::RemoveJsonKeyIndexFiles() {
DiskFileManagerImpl::RemoveJsonStatsFiles() {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
local_chunk_manager->RemoveDir(GetLocalJsonKeyIndexPrefix());
local_chunk_manager->RemoveDir(GetLocalJsonStatsPrefix());
}
void
@ -872,41 +883,58 @@ DiskFileManagerImpl::GetLocalTempTextIndexPrefix() {
true);
}
// path to store pre-built index contents downloaded from remote storage
std::string
DiskFileManagerImpl::GetLocalJsonKeyIndexPrefix() {
DiskFileManagerImpl::GetLocalJsonStatsPrefix() {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
return GenJsonKeyIndexPathPrefix(local_chunk_manager,
index_meta_.build_id,
index_meta_.index_version,
field_meta_.segment_id,
field_meta_.field_id,
false);
}
// temporary path used during index building
std::string
DiskFileManagerImpl::GetLocalTempJsonKeyIndexPrefix() {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
return GenJsonKeyIndexPathPrefix(local_chunk_manager,
index_meta_.build_id,
index_meta_.index_version,
field_meta_.segment_id,
field_meta_.field_id,
true);
return GenJsonStatsPathPrefix(local_chunk_manager,
index_meta_.build_id,
index_meta_.index_version,
field_meta_.segment_id,
field_meta_.field_id,
false);
}
std::string
DiskFileManagerImpl::GetRemoteJsonKeyLogPrefix() {
return GenRemoteJsonKeyIndexPathPrefix(rcm_,
index_meta_.build_id,
index_meta_.index_version,
field_meta_.collection_id,
field_meta_.partition_id,
field_meta_.segment_id,
field_meta_.field_id);
DiskFileManagerImpl::GetLocalTempJsonStatsPrefix() {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
return GenJsonStatsPathPrefix(local_chunk_manager,
index_meta_.build_id,
index_meta_.index_version,
field_meta_.segment_id,
field_meta_.field_id,
true);
}
std::string
DiskFileManagerImpl::GetLocalJsonStatsShreddingPrefix() {
namespace fs = std::filesystem;
fs::path prefix = GetLocalJsonStatsPrefix();
fs::path suffix = JSON_STATS_SHREDDING_DATA_PATH;
return (prefix / suffix).string();
}
std::string
DiskFileManagerImpl::GetLocalJsonStatsSharedIndexPrefix() {
// make sure the path end with '/'
namespace fs = std::filesystem;
fs::path prefix = GetLocalJsonStatsPrefix();
fs::path suffix = JSON_STATS_SHARED_INDEX_PATH;
auto result = (prefix / suffix).string();
if (!result.empty() && result.back() != fs::path::preferred_separator) {
result += fs::path::preferred_separator;
}
return result;
}
std::string
DiskFileManagerImpl::GetLocalJsonStatsShreddingPath(
const std::string& file_name) {
namespace fs = std::filesystem;
fs::path prefix = GetLocalJsonStatsShreddingPrefix();
fs::path file = file_name;
return (prefix / file).string();
}
std::string
@ -933,6 +961,17 @@ DiskFileManagerImpl::GetLocalTempNgramIndexPrefix() {
true);
}
std::string
DiskFileManagerImpl::GetRemoteJsonStatsLogPrefix() {
return GenRemoteJsonStatsPathPrefix(rcm_,
index_meta_.build_id,
index_meta_.index_version,
field_meta_.collection_id,
field_meta_.partition_id,
field_meta_.segment_id,
field_meta_.field_id);
}
std::string
DiskFileManagerImpl::GetLocalRawDataObjectPrefix() {
auto local_chunk_manager =

View File

@ -63,7 +63,7 @@ class DiskFileManagerImpl : public FileManagerImpl {
AddTextLog(const std::string& filename) noexcept;
bool
AddJsonKeyIndexLog(const std::string& filename) noexcept;
AddJsonSharedIndexLog(const std::string& filename) noexcept;
public:
std::string
@ -87,16 +87,20 @@ class DiskFileManagerImpl : public FileManagerImpl {
std::string
GetLocalTempTextIndexPrefix();
// Used for loading index, using this index prefix dir to store index.
std::string
GetLocalJsonKeyIndexPrefix();
GetLocalJsonStatsPrefix();
std::string
GetLocalTempJsonKeyIndexPrefix();
GetLocalTempJsonStatsPrefix();
// Used for upload index to remote storage, using this index prefix dir as remote storage directory
std::string
GetRemoteJsonKeyLogPrefix();
GetLocalJsonStatsShreddingPrefix();
std::string
GetLocalJsonStatsSharedIndexPrefix();
std::string
GetLocalJsonStatsShreddingPath(const std::string& file_name);
// Used for upload index to remote storage, using this index prefix dir as remote storage directory
std::string
@ -106,6 +110,16 @@ class DiskFileManagerImpl : public FileManagerImpl {
std::string
GetLocalTempNgramIndexPrefix();
std::string
GetRemoteJsonStatsLogPrefix();
std::string
GetRemoteJsonStatsShreddingPrefix();
std::string
GetRemoteJsonStatsSharedIndexPath(const std::string& file_name,
int64_t slice_num);
std::string
GetLocalRawDataObjectPrefix();
@ -127,10 +141,6 @@ class DiskFileManagerImpl : public FileManagerImpl {
CacheTextLogToDisk(const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority);
void
CacheJsonKeyIndexToDisk(const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority);
void
CacheNgramIndexToDisk(const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority);
@ -142,7 +152,12 @@ class DiskFileManagerImpl : public FileManagerImpl {
RemoveTextLogFiles();
void
RemoveJsonKeyIndexFiles();
RemoveJsonStatsFiles();
void
CacheJsonStatsSharedIndexToDisk(
const std::vector<std::string>& remote_files,
milvus::proto::common::LoadPriority priority);
void
RemoveNgramIndexFiles();
@ -196,9 +211,6 @@ class DiskFileManagerImpl : public FileManagerImpl {
std::string
GetRemoteTextLogPath(const std::string& file_name, int64_t slice_num) const;
std::string
GetRemoteJsonKeyIndexPath(const std::string& file_name, int64_t slice_num);
bool
AddFileInternal(const std::string& file_name,
const std::function<std::string(const std::string&, int)>&

View File

@ -106,6 +106,11 @@ class LocalChunkManager : public ChunkManager {
return path_prefix_;
}
virtual std::string
GetBucketName() const {
return "";
}
bool
CreateFile(const std::string& filepath);
@ -133,11 +138,6 @@ class LocalChunkManager : public ChunkManager {
int64_t
GetSizeOfDir(const std::string& dir);
virtual std::string
GetBucketName() const {
return "";
}
private:
std::string path_prefix_;
};

View File

@ -200,6 +200,7 @@ class MinioChunkManager : public ChunkManager {
const std::string& object_name,
void* buf,
uint64_t size);
uint64_t
GetObjectBuffer(const std::string& bucket_name,
const std::string& object_name,

View File

@ -644,50 +644,58 @@ GenTextIndexPathPrefix(ChunkManagerPtr cm,
}
std::string
GenJsonKeyIndexPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t segment_id,
int64_t field_id,
bool is_temp) {
return GenIndexPathPrefixByType(cm,
build_id,
GenJsonStatsPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t segment_id,
int64_t field_id,
bool is_temp) {
boost::filesystem::path prefix = cm->GetRootPath();
if (is_temp) {
prefix = prefix / TEMP;
}
boost::filesystem::path path = std::string(JSON_STATS_ROOT_PATH);
boost::filesystem::path path1 =
GenIndexPathIdentifier(build_id, index_version, segment_id, field_id);
return (prefix / path / path1).string();
}
std::string
GenJsonStatsPathIdentifier(int64_t build_id,
int64_t index_version,
int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id) {
boost::filesystem::path p =
boost::filesystem::path(std::to_string(build_id)) /
std::to_string(index_version) / std::to_string(collection_id) /
std::to_string(partition_id) / std::to_string(segment_id) /
std::to_string(field_id);
return p.string() + "/";
}
std::string
GenRemoteJsonStatsPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id) {
boost::filesystem::path p = cm->GetRootPath();
p /= std::string(JSON_STATS_ROOT_PATH);
p /= std::string(JSON_STATS_DATA_FORMAT_VERSION);
p /= GenJsonStatsPathIdentifier(build_id,
index_version,
collection_id,
partition_id,
segment_id,
field_id,
JSON_KEY_INDEX_LOG_ROOT_PATH,
is_temp);
}
std::string
GenJsonKeyIndexPathIdentifier(int64_t build_id,
int64_t index_version,
int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id) {
return std::to_string(build_id) + "/" + std::to_string(index_version) +
"/" + std::to_string(collection_id) + "/" +
std::to_string(partition_id) + "/" + std::to_string(segment_id) +
"/" + std::to_string(field_id) + "/";
}
std::string
GenRemoteJsonKeyIndexPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id) {
return cm->GetRootPath() + "/" + std::string(JSON_KEY_INDEX_LOG_ROOT_PATH) +
"/" +
GenJsonKeyIndexPathIdentifier(build_id,
index_version,
collection_id,
partition_id,
segment_id,
field_id);
field_id);
return p.string();
}
std::string

View File

@ -125,24 +125,31 @@ GenTextIndexPathPrefix(ChunkManagerPtr cm,
int64_t field_id,
bool is_temp);
// is_temp: true for temporary path used during index building,
// false for path to store pre-built index contents downloaded from remote storage
std::string
GenJsonKeyIndexPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t segment_id,
int64_t field_id,
bool is_temp);
GenJsonStatsPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t segment_id,
int64_t field_id,
bool is_temp);
std::string
GenRemoteJsonKeyIndexPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id);
GenJsonStatsPathIdentifier(int64_t build_id,
int64_t index_version,
int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id);
std::string
GenRemoteJsonStatsPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id);
std::string
GenNgramIndexPrefix(ChunkManagerPtr cm,
int64_t build_id,

View File

@ -107,12 +107,12 @@ class GcpNativeChunkManager : public ChunkManager {
return path_prefix_;
}
public:
virtual std::string
GetBucketName() const {
return default_bucket_name_;
}
public:
inline void
SetBucketName(const std::string& bucket_name) {
default_bucket_name_ = bucket_name;

View File

@ -79,8 +79,8 @@ class OpenDALChunkManager : public ChunkManager {
return remote_root_path_;
}
virtual std::string
GetBucketName() const {
std::string
GetBucketName() const override {
return default_bucket_name_;
}

View File

@ -97,7 +97,6 @@ set(MILVUS_TEST_FILES
test_cached_search_iterator.cpp
test_random_sample.cpp
test_json_index.cpp
test_json_key_stats_index.cpp
test_types.cpp
test_growing_storage_v2.cpp
test_memory_planner.cpp
@ -202,6 +201,7 @@ target_link_libraries(all_tests
install(TARGETS all_tests DESTINATION unittest)
add_subdirectory(bench)
add_subdirectory(test_json_stats)
# if (USE_DYNAMIC_SIMD)
# add_executable(dynamic_simd_test

View File

@ -51,6 +51,7 @@
#include "test_utils/DataGen.h"
#include "segcore/vector_index_c.h"
#include "common/jsmn.h"
#include "exec/expression/Element.h"
using namespace milvus;
using namespace milvus::test;
@ -1050,6 +1051,40 @@ TEST(CApiTest, InsertSamePkAfterDeleteOnGrowingSegment) {
DeleteSegment(segment);
}
TEST(CApiTest, TestMultiElement) {
std::vector<std::string> params;
for (int i = 0; i < 100; i++) {
params.push_back(std::to_string(i));
}
auto multi_element =
std::make_shared<milvus::exec::SortVectorElement<std::string>>(params);
std::string target = "50";
auto res = multi_element->In(target);
ASSERT_EQ(res, true);
target = "30";
res = multi_element->In(target);
ASSERT_EQ(res, true);
target = "40";
res = multi_element->In(target);
ASSERT_EQ(res, true);
target = "100";
res = multi_element->In(target);
ASSERT_EQ(res, false);
target = "1000";
res = multi_element->In(target);
ASSERT_EQ(res, false);
std::string_view target_view = "30";
res = multi_element->In(target_view);
ASSERT_EQ(res, true);
target_view = "40";
res = multi_element->In(target_view);
ASSERT_EQ(res, true);
target_view = "50";
res = multi_element->In(target_view);
ASSERT_EQ(res, true);
}
TEST(CApiTest, InsertSamePkAfterDeleteOnSealedSegment) {
auto collection = NewCollection(get_default_schema_config().c_str());
CSegmentInterface segment;

View File

@ -514,7 +514,7 @@ TEST_F(DiskAnnFileManagerTest, CacheOptFieldToDiskOnlyOneCategory) {
TEST_F(DiskAnnFileManagerTest, FileCleanup) {
std::string local_index_file_path;
std::string local_text_index_file_path;
std::string local_json_key_index_file_path;
std::string local_json_stats_file_path;
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
@ -525,25 +525,25 @@ TEST_F(DiskAnnFileManagerTest, FileCleanup) {
auto random_file_suffix = std::to_string(rand());
local_text_index_file_path =
file_manager->GetLocalTextIndexPrefix() + random_file_suffix;
local_json_key_index_file_path =
file_manager->GetLocalJsonKeyIndexPrefix() + random_file_suffix;
local_index_file_path =
file_manager->GetLocalIndexObjectPrefix() + random_file_suffix;
local_json_stats_file_path =
file_manager->GetLocalJsonStatsPrefix() + random_file_suffix;
local_chunk_manager->CreateFile(local_text_index_file_path);
local_chunk_manager->CreateFile(local_json_key_index_file_path);
local_chunk_manager->CreateFile(local_index_file_path);
local_chunk_manager->CreateFile(local_json_stats_file_path);
// verify these files exist
EXPECT_TRUE(
file_manager->IsExisted(local_text_index_file_path).value());
EXPECT_TRUE(
file_manager->IsExisted(local_json_key_index_file_path).value());
EXPECT_TRUE(file_manager->IsExisted(local_index_file_path).value());
EXPECT_TRUE(
file_manager->IsExisted(local_json_stats_file_path).value());
}
// verify these files not exist
EXPECT_FALSE(local_chunk_manager->Exist(local_text_index_file_path));
EXPECT_FALSE(local_chunk_manager->Exist(local_json_key_index_file_path));
EXPECT_FALSE(local_chunk_manager->Exist(local_index_file_path));
EXPECT_FALSE(local_chunk_manager->Exist(local_json_stats_file_path));
}

View File

@ -145,6 +145,7 @@ TEST_P(TaskTest, CallExprEmpty) {
MAX_TIMESTAMP,
0,
0,
query::PlanOptions{false},
std::make_shared<milvus::exec::QueryConfig>(
std::unordered_map<std::string, std::string>{}));
@ -184,6 +185,7 @@ TEST_P(TaskTest, UnaryExpr) {
MAX_TIMESTAMP,
0,
0,
query::PlanOptions{false},
std::make_shared<milvus::exec::QueryConfig>(
std::unordered_map<std::string, std::string>{}));
@ -232,6 +234,7 @@ TEST_P(TaskTest, LogicalExpr) {
MAX_TIMESTAMP,
0,
0,
query::PlanOptions{false},
std::make_shared<milvus::exec::QueryConfig>(
std::unordered_map<std::string, std::string>{}));

View File

@ -1,640 +0,0 @@
// 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 <functional>
#include <boost/filesystem.hpp>
#include <unordered_set>
#include <memory>
#include "common/Tracer.h"
#include "index/BitmapIndex.h"
#include "storage/Util.h"
#include "storage/InsertData.h"
#include "indexbuilder/IndexFactory.h"
#include "index/IndexFactory.h"
#include "test_utils/indexbuilder_test_utils.h"
#include "index/Meta.h"
#include "index/JsonKeyStatsInvertedIndex.h"
#include "common/Json.h"
#include "common/Types.h"
using namespace milvus::index;
using namespace milvus::indexbuilder;
using namespace milvus;
using namespace milvus::index;
static std::vector<milvus::Json>
GenerateJsons(int size) {
std::vector<Json> jsons;
std::default_random_engine random(42);
std::normal_distribution<> distr(0, 1);
for (int i = 0; i < size; i++) {
auto str = R"({"int":)" + std::to_string(random()) + R"(,"double":)" +
std::to_string(static_cast<double>(random())) +
R"(,"string":")" + std::to_string(random()) +
R"(","bool": true)" + R"(, "array": [1,2,3])" + "}";
jsons.push_back(milvus::Json(simdjson::padded_string(str)));
}
return jsons;
}
class JsonKeyStatsIndexTest : public ::testing::TestWithParam<bool> {
protected:
void
Init(int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id,
int64_t index_build_id,
int64_t index_version,
int64_t size) {
proto::schema::FieldSchema field_schema;
field_schema.set_data_type(proto::schema::DataType::JSON);
field_schema.set_nullable(nullable_);
auto field_meta = storage::FieldDataMeta{
collection_id, partition_id, segment_id, field_id, field_schema};
auto index_meta = storage::IndexMeta{
segment_id, field_id, index_build_id, index_version};
data_ = std::move(GenerateJsons(size));
auto field_data = storage::CreateFieldData(DataType::JSON, nullable_);
if (nullable_) {
valid_data.reserve(size_);
for (size_t i = 0; i < size_; i++) {
valid_data.push_back(false);
}
}
if (nullable_) {
int byteSize = (size_ + 7) / 8;
uint8_t* valid_data_ = new uint8_t[byteSize];
for (int i = 0; i < size_; i++) {
bool value = valid_data[i];
int byteIndex = i / 8;
int bitIndex = i % 8;
if (value) {
valid_data_[byteIndex] |= (1 << bitIndex);
} else {
valid_data_[byteIndex] &= ~(1 << bitIndex);
}
}
field_data->FillFieldData(
data_.data(), valid_data_, data_.size(), 0);
delete[] valid_data_;
} else {
field_data->FillFieldData(data_.data(), data_.size());
}
auto payload_reader =
std::make_shared<milvus::storage::PayloadReader>(field_data);
storage::InsertData insert_data(payload_reader);
insert_data.SetFieldDataMeta(field_meta);
insert_data.SetTimestamps(0, 100);
auto serialized_bytes = insert_data.Serialize(storage::Remote);
auto log_path = fmt::format("/{}/{}/{}/{}/{}/{}",
"/tmp/test-jsonkey-index/",
collection_id,
partition_id,
segment_id,
field_id,
0);
chunk_manager_->Write(
log_path, serialized_bytes.data(), serialized_bytes.size());
storage::FileManagerContext ctx(field_meta, index_meta, chunk_manager_);
std::vector<std::string> index_files;
Config config;
config[INSERT_FILES_KEY] = std::vector<std::string>{log_path};
auto build_index =
std::make_shared<JsonKeyStatsInvertedIndex>(ctx, false);
build_index->Build(config);
auto create_index_result = build_index->Upload(config);
auto memSize = create_index_result->GetMemSize();
auto serializedSize = create_index_result->GetSerializedSize();
ASSERT_GT(memSize, 0);
ASSERT_GT(serializedSize, 0);
index_files = create_index_result->GetIndexFiles();
index::CreateIndexInfo index_info{};
config["index_files"] = index_files;
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
index_ = std::make_shared<JsonKeyStatsInvertedIndex>(ctx, true);
index_->Load(milvus::tracer::TraceContext{}, config);
}
void
SetUp() override {
nullable_ = GetParam();
type_ = DataType::JSON;
int64_t collection_id = 1;
int64_t partition_id = 2;
int64_t segment_id = 3;
int64_t field_id = 101;
int64_t index_build_id = 1000;
int64_t index_version = 10000;
size_ = 1;
std::string root_path = "/tmp/test-jsonkey-index/";
storage::StorageConfig storage_config;
storage_config.storage_type = "local";
storage_config.root_path = root_path;
chunk_manager_ = storage::CreateChunkManager(storage_config);
Init(collection_id,
partition_id,
segment_id,
field_id,
index_build_id,
index_version,
size_);
}
virtual ~JsonKeyStatsIndexTest() override {
boost::filesystem::remove_all(chunk_manager_->GetRootPath());
}
public:
std::shared_ptr<JsonKeyStatsInvertedIndex> index_;
DataType type_;
bool nullable_;
size_t size_;
FixedVector<bool> valid_data;
std::vector<milvus::Json> data_;
std::vector<std::string> json_col;
std::shared_ptr<storage::ChunkManager> chunk_manager_;
};
INSTANTIATE_TEST_SUITE_P(JsonKeyStatsIndexTestSuite,
JsonKeyStatsIndexTest,
::testing::Values(true, false));
TEST_P(JsonKeyStatsIndexTest, HasEscapeSequence) {
EXPECT_TRUE(index_->has_escape_sequence("Hello\\nWorld"));
EXPECT_TRUE(index_->has_escape_sequence("Tab\\tCharacter"));
EXPECT_TRUE(index_->has_escape_sequence("Carriage\\rReturn"));
EXPECT_TRUE(index_->has_escape_sequence("Backspace\\bTest"));
EXPECT_TRUE(index_->has_escape_sequence("FormFeed\\fTest"));
EXPECT_TRUE(index_->has_escape_sequence("Vertical\\vTab"));
EXPECT_TRUE(index_->has_escape_sequence("Backslash\\\\Test"));
EXPECT_TRUE(index_->has_escape_sequence("Quote\\\"Test"));
EXPECT_TRUE(index_->has_escape_sequence("SingleQuote\\'Test"));
EXPECT_FALSE(index_->has_escape_sequence("No escape sequence here"));
EXPECT_FALSE(index_->has_escape_sequence("Just a backslash \\"));
EXPECT_FALSE(index_->has_escape_sequence(""));
}
TEST_P(JsonKeyStatsIndexTest, TestTermInFunc) {
struct Testcase {
std::vector<int64_t> term;
std::vector<std::string> nested_path;
};
std::vector<Testcase> testcases{
{{1, 2, 3, 4}, {"int"}},
{{10, 100, 1000, 10000}, {"int"}},
{{100, 10000, 9999, 444}, {"int"}},
{{23, 42, 66, 17, 25}, {"int"}},
};
for (auto testcase : testcases) {
auto check = [&](int64_t value) {
std::unordered_set<int64_t> term_set(testcase.term.begin(),
testcase.term.end());
return term_set.find(value) != term_set.end();
};
std::unordered_set<int64_t> term_set(testcase.term.begin(),
testcase.term.end());
auto filter_func = [&term_set, this](const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
for (size_t i = 0; i < n; i++) {
auto value = value_array[i];
bitset[row_id_array[i]] =
term_set.find(int64_t(value)) != term_set.end();
}
};
auto pointer = milvus::Json::pointer(testcase.nested_path);
auto bitset =
index_->FilterByPath(pointer, size_, false, true, filter_func);
ASSERT_EQ(bitset.size(), size_);
for (int i = 0; i < bitset.size(); ++i) {
if (nullable_ && !valid_data[i]) {
ASSERT_EQ(bitset[i], false);
} else {
auto val = data_[i].template at<int64_t>(pointer).value();
auto ans = bitset[i];
auto ref = check(val);
ASSERT_EQ(ans, ref);
}
}
}
}
TEST_P(JsonKeyStatsIndexTest, TestUnaryRangeInFunc) {
struct Testcase {
int64_t val;
std::vector<std::string> nested_path;
};
std::vector<Testcase> testcases{
{10, {"int"}},
{20, {"int"}},
{30, {"int"}},
{40, {"int"}},
};
std::vector<OpType> ops{
OpType::Equal,
OpType::NotEqual,
OpType::GreaterThan,
OpType::GreaterEqual,
OpType::LessThan,
OpType::LessEqual,
};
for (const auto& testcase : testcases) {
auto check = [&](int64_t value) { return value == testcase.val; };
std::function<bool(int64_t)> f = check;
for (auto& op : ops) {
switch (op) {
case OpType::Equal: {
f = [&](int64_t value) { return value == testcase.val; };
break;
}
case OpType::NotEqual: {
f = [&](int64_t value) { return value != testcase.val; };
break;
}
case OpType::GreaterEqual: {
f = [&](int64_t value) { return value >= testcase.val; };
break;
}
case OpType::GreaterThan: {
f = [&](int64_t value) { return value > testcase.val; };
break;
}
case OpType::LessEqual: {
f = [&](int64_t value) { return value <= testcase.val; };
break;
}
case OpType::LessThan: {
f = [&](int64_t value) { return value < testcase.val; };
break;
}
default: {
ThrowInfo(Unsupported, "unsupported range node");
}
}
auto filter_func = [&op, &testcase, this](
const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
for (size_t i = 0; i < n; i++) {
auto value = value_array[i];
auto row_id = row_id_array[i];
switch (op) {
case OpType::GreaterThan:
bitset[row_id] = value > testcase.val;
break;
case OpType::GreaterEqual:
bitset[row_id] = value >= testcase.val;
break;
case OpType::LessThan:
bitset[row_id] = value < testcase.val;
break;
case OpType::LessEqual:
bitset[row_id] = value <= testcase.val;
break;
case OpType::Equal:
bitset[row_id] = value == testcase.val;
break;
case OpType::NotEqual:
bitset[row_id] = value != testcase.val;
break;
default:
bitset[row_id] = false;
break;
}
}
};
auto pointer = milvus::Json::pointer(testcase.nested_path);
auto bitset =
index_->FilterByPath(pointer, size_, false, true, filter_func);
ASSERT_EQ(bitset.size(), size_);
for (int i = 0; i < bitset.size(); ++i) {
if (nullable_ && !valid_data[i]) {
ASSERT_EQ(bitset[i], false);
} else {
auto ans = bitset[i];
if (testcase.nested_path[0] == "int") {
auto val =
data_[i].template at<int64_t>(pointer).value();
auto ref = f(val);
ASSERT_EQ(ans, ref);
} else {
auto val =
data_[i].template at<double>(pointer).value();
auto ref = f(val);
ASSERT_EQ(ans, ref);
}
}
}
}
}
}
TEST_P(JsonKeyStatsIndexTest, TestBinaryRangeInFunc) {
struct Testcase {
bool lower_inclusive;
bool upper_inclusive;
int64_t lower;
int64_t upper;
std::vector<std::string> nested_path;
};
std::vector<Testcase> testcases{
{true, false, 10, 20, {"int"}},
{true, true, 20, 30, {"int"}},
{false, true, 30, 40, {"int"}},
{false, false, 40, 50, {"int"}},
{true, false, 10, 20, {"double"}},
{true, true, 20, 30, {"double"}},
{false, true, 30, 40, {"double"}},
{false, false, 40, 50, {"double"}},
};
for (const auto& testcase : testcases) {
auto check = [&](int64_t value) {
if (testcase.lower_inclusive && testcase.upper_inclusive) {
return testcase.lower <= value && value <= testcase.upper;
} else if (testcase.lower_inclusive && !testcase.upper_inclusive) {
return testcase.lower <= value && value < testcase.upper;
} else if (!testcase.lower_inclusive && testcase.upper_inclusive) {
return testcase.lower < value && value <= testcase.upper;
} else {
return testcase.lower < value && value < testcase.upper;
}
};
auto filter_func = [&testcase, this](const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
for (size_t i = 0; i < n; i++) {
auto valid = valid_array[i];
if (valid) {
auto value = value_array[i];
auto row_id = row_id_array[i];
if (testcase.lower_inclusive && testcase.upper_inclusive) {
bitset[row_id] =
testcase.lower <= value && value <= testcase.upper;
} else if (testcase.lower_inclusive &&
!testcase.upper_inclusive) {
bitset[row_id] =
testcase.lower <= value && value < testcase.upper;
} else if (!testcase.lower_inclusive &&
testcase.upper_inclusive) {
bitset[row_id] =
testcase.lower < value && value <= testcase.upper;
} else {
bitset[row_id] =
testcase.lower < value && value < testcase.upper;
}
} else {
auto offset = offset_array[i];
auto size = size_array[i];
auto row_id = row_id_array[i];
auto val =
this->data_[row_id].template at<int64_t>(offset, size);
if (val.error()) {
bitset[row_id] = false;
}
if (testcase.lower_inclusive && testcase.upper_inclusive) {
bitset[row_id] =
testcase.lower <= int64_t(val.value()) &&
int64_t(val.value()) <= testcase.upper;
} else if (testcase.lower_inclusive &&
!testcase.upper_inclusive) {
bitset[row_id] =
testcase.lower <= int64_t(val.value()) &&
int64_t(val.value()) < testcase.upper;
} else if (!testcase.lower_inclusive &&
testcase.upper_inclusive) {
bitset[row_id] =
testcase.lower < int64_t(val.value()) &&
int64_t(val.value()) <= testcase.upper;
} else {
bitset[row_id] =
testcase.lower < int64_t(val.value()) &&
int64_t(val.value()) < testcase.upper;
}
}
}
};
auto pointer = milvus::Json::pointer(testcase.nested_path);
auto bitset =
index_->FilterByPath(pointer, size_, false, true, filter_func);
ASSERT_EQ(bitset.size(), size_);
for (int i = 0; i < bitset.size(); ++i) {
if (nullable_ && !valid_data[i]) {
ASSERT_EQ(bitset[i], false);
} else {
auto ans = bitset[i];
if (testcase.nested_path[0] == "int") {
auto val = data_[i].template at<int64_t>(pointer).value();
auto ref = check(val);
ASSERT_EQ(ans, ref);
} else {
auto val = data_[i].template at<double>(pointer).value();
auto ref = check(val);
ASSERT_EQ(ans, ref);
}
}
}
}
}
TEST_P(JsonKeyStatsIndexTest, TestExistInFunc) {
struct Testcase {
std::vector<std::string> nested_path;
};
std::vector<Testcase> testcases{
{{"A"}},
{{"int"}},
{{"double"}},
{{"B"}},
};
for (const auto& testcase : testcases) {
auto pointer = milvus::Json::pointer(testcase.nested_path);
auto filter_func = [&pointer, this](const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
for (size_t i = 0; i < n; i++) {
auto row_id = row_id_array[i];
bitset[row_id] = this->data_[row_id].exist(pointer);
}
};
auto bitset =
index_->FilterByPath(pointer, size_, false, true, filter_func);
ASSERT_EQ(bitset.size(), size_);
for (int i = 0; i < bitset.size(); ++i) {
if (nullable_ && !valid_data[i]) {
ASSERT_EQ(bitset[i], false);
} else {
auto ans = bitset[i];
auto val = data_[i].exist(pointer);
ASSERT_EQ(ans, val);
}
}
}
}
TEST_P(JsonKeyStatsIndexTest, TestJsonContainsAllFunc) {
struct Testcase {
std::vector<int64_t> term;
std::vector<std::string> nested_path;
};
{
std::vector<Testcase> testcases{
{{1, 2, 3}, {"array"}},
{{10, 100}, {"array"}},
{{100, 1000}, {"array"}},
};
for (const auto& testcase : testcases) {
auto check = [&](const std::vector<int64_t>& values) {
for (auto const& e : testcase.term) {
if (std::find(values.begin(), values.end(), e) ==
values.end()) {
return false;
}
}
return true;
};
auto pointer = milvus::Json::pointer(testcase.nested_path);
std::unordered_set<int64_t> elements;
for (auto const& element : testcase.term) {
elements.insert(element);
}
auto filter_func = [&elements, this](const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
for (size_t i = 0; i < n; i++) {
auto row_id = row_id_array[i];
auto offset = offset_array[i];
auto size = size_array[i];
auto array = this->data_[row_id].array_at(offset, size);
std::unordered_set<int64_t> tmp_elements(elements);
for (auto&& it : array) {
auto val = it.template get<int64_t>();
if (val.error()) {
continue;
}
tmp_elements.erase(val.value());
}
if (tmp_elements.size() == 0) {
bitset[row_id] = true;
}
bitset[row_id] = tmp_elements.empty();
}
};
auto bitset =
index_->FilterByPath(pointer, size_, false, true, filter_func);
ASSERT_EQ(bitset.size(), size_);
for (int i = 0; i < bitset.size(); ++i) {
if (nullable_ && !valid_data[i]) {
ASSERT_EQ(bitset[i], false);
} else {
auto ans = bitset[i];
auto array = data_[i].array_at(pointer);
std::vector<int64_t> res;
for (const auto& element : array) {
res.push_back(element.template get<int64_t>());
}
ASSERT_EQ(ans, check(res));
}
}
}
}
}
TEST(GrowingJsonKeyStatsIndexTest, GrowingIndex) {
using Index = index::JsonKeyStatsInvertedIndex;
auto index = std::make_unique<Index>(std::numeric_limits<int64_t>::max(),
"json",
"/tmp/test-jsonkey-index/");
auto str = R"({"int":)" + std::to_string(1) + R"(,"double":)" +
std::to_string(static_cast<double>(1)) + R"(,"string":")" +
std::to_string(1) + R"(","bool": true)" +
R"(, "array": [1,2,3])" + "}";
auto str1 = R"({"int":)" + std::to_string(2) + "}";
auto str2 = R"({"int":)" + std::to_string(3) + "}";
std::vector<std::string> jsonDatas;
jsonDatas.push_back(str);
jsonDatas.push_back(str1);
jsonDatas.push_back(str2);
std::vector<milvus::Json> jsons;
for (const auto& jsonData : jsonDatas) {
jsons.push_back(milvus::Json(simdjson::padded_string(jsonData)));
}
index->CreateReader(milvus::index::SetBitsetSealed);
index->AddJSONDatas(jsonDatas.size(), jsonDatas.data(), nullptr, 0);
index->Commit();
index->Reload();
int64_t checkVal = 1;
auto filter_func = [jsons, checkVal](const bool* valid_array,
const uint8_t* type_array,
const uint32_t* row_id_array,
const uint16_t* offset_array,
const uint16_t* size_array,
const int32_t* value_array,
TargetBitmap& bitset,
const size_t n) {
for (size_t i = 0; i < n; i++) {
auto value = value_array[i];
bitset[row_id_array[i]] = value == checkVal;
}
};
auto pointer = milvus::Json::pointer({"int"});
auto bitset =
index->FilterByPath(pointer, jsonDatas.size(), true, true, filter_func);
ASSERT_EQ(bitset.size(), jsonDatas.size());
for (int i = 0; i < bitset.size(); ++i) {
auto val = jsons[i].template at<int64_t>(pointer).value();
auto ans = bitset[i];
auto ref = val == checkVal;
ASSERT_EQ(ans, ref);
}
}

View File

@ -0,0 +1,32 @@
# 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
set(JSON_STATS_TEST_FILES
../init_gtest.cpp
test_bson_builder.cpp
test_bson_view.cpp
test_parquet_writer.cpp
test_utils.cpp
test_json_key_stats.cpp
)
add_executable(json_stats_test
${JSON_STATS_TEST_FILES}
)
target_link_libraries(json_stats_test
gtest
gmock
milvus_core
milvus-storage
)
install(TARGETS json_stats_test DESTINATION unittest)

View File

@ -0,0 +1,476 @@
#include <gtest/gtest.h>
#include "common/bson_view.h"
#include <bsoncxx/builder/basic/document.hpp>
#include <bsoncxx/builder/basic/array.hpp>
#include <bsoncxx/json.hpp>
#include <bsoncxx/types.hpp>
#include <bsoncxx/types/value.hpp>
#include <bsoncxx/types/bson_value/view.hpp>
#include <bsoncxx/types/bson_value/value.hpp>
#include "index/json_stats/bson_builder.h"
namespace milvus::index {
class DomNodeTest : public ::testing::Test {
protected:
void
SetUp() override {
}
};
TEST_F(DomNodeTest, DefaultConstructorTest) {
DomNode node;
EXPECT_EQ(node.type, DomNode::Type::DOCUMENT);
EXPECT_TRUE(node.document_children.empty());
EXPECT_FALSE(node.bson_value.has_value());
}
TEST_F(DomNodeTest, TypeConstructorTest) {
// Test DOCUMENT type
DomNode doc_node(DomNode::Type::DOCUMENT);
EXPECT_EQ(doc_node.type, DomNode::Type::DOCUMENT);
EXPECT_TRUE(doc_node.document_children.empty());
EXPECT_FALSE(doc_node.bson_value.has_value());
// Test VALUE type
DomNode value_node(DomNode::Type::VALUE);
EXPECT_EQ(value_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(value_node.document_children.empty());
EXPECT_FALSE(value_node.bson_value.has_value());
}
TEST_F(DomNodeTest, ValueConstructorTest) {
// Test with boolean value
bsoncxx::types::b_bool bool_val{true};
DomNode bool_node{bsoncxx::types::bson_value::value(bool_val)};
EXPECT_EQ(bool_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(bool_node.bson_value.has_value());
EXPECT_TRUE(bool_node.bson_value.value().view().get_bool());
// Test with int32 value
bsoncxx::types::b_int32 int_val{42};
DomNode int_node{bsoncxx::types::bson_value::value(int_val)};
EXPECT_EQ(int_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(int_node.bson_value.has_value());
EXPECT_EQ(int_node.bson_value.value().view().get_int32(), 42);
// Test with string value
bsoncxx::types::b_string str_val{"test"};
DomNode str_node{bsoncxx::types::bson_value::value(str_val)};
EXPECT_EQ(str_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(str_node.bson_value.has_value());
EXPECT_STREQ(str_node.bson_value.value().view().get_string().value.data(),
"test");
}
TEST_F(DomNodeTest, DocumentChildrenTest) {
DomNode root;
// Add child nodes
root.document_children["child1"] = DomNode(DomNode::Type::VALUE);
root.document_children["child2"] = DomNode(DomNode::Type::DOCUMENT);
EXPECT_EQ(root.document_children.size(), 2);
EXPECT_EQ(root.document_children["child1"].type, DomNode::Type::VALUE);
EXPECT_EQ(root.document_children["child2"].type, DomNode::Type::DOCUMENT);
// Test nested document
root.document_children["child2"].document_children["nested"] = DomNode(
bsoncxx::types::bson_value::value(bsoncxx::types::b_int32{123}));
EXPECT_EQ(root.document_children["child2"].document_children["nested"].type,
DomNode::Type::VALUE);
EXPECT_EQ(root.document_children["child2"]
.document_children["nested"]
.bson_value.value()
.view()
.get_int32(),
123);
}
class BsonBuilderTest : public ::testing::Test {
protected:
void
SetUp() override {
builder_ = std::make_unique<BsonBuilder>();
}
std::unique_ptr<BsonBuilder> builder_;
};
TEST_F(BsonBuilderTest, CreateValueNodeTest) {
BsonBuilder builder;
// Test NONE type
auto none_node = builder.CreateValueNode("", JSONType::NONE);
EXPECT_EQ(none_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(none_node.bson_value.value().view().type() ==
bsoncxx::type::k_null);
// Test BOOL type
auto true_node = builder.CreateValueNode("true", JSONType::BOOL);
EXPECT_EQ(true_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(true_node.bson_value.value().view().type() ==
bsoncxx::type::k_bool);
EXPECT_TRUE(true_node.bson_value.value().view().get_bool().value);
auto false_node = builder.CreateValueNode("false", JSONType::BOOL);
EXPECT_EQ(false_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(false_node.bson_value.value().view().type() ==
bsoncxx::type::k_bool);
EXPECT_FALSE(false_node.bson_value.value().view().get_bool().value);
// Test INT64 type
auto int64_node =
builder.CreateValueNode("9223372036854775807", JSONType::INT64);
EXPECT_EQ(int64_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(int64_node.bson_value.value().view().type() ==
bsoncxx::type::k_int64);
EXPECT_EQ(int64_node.bson_value.value().view().get_int64().value,
9223372036854775807LL);
// Test DOUBLE type
auto double_node = builder.CreateValueNode("3.14159", JSONType::DOUBLE);
EXPECT_EQ(double_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(double_node.bson_value.value().view().type() ==
bsoncxx::type::k_double);
EXPECT_DOUBLE_EQ(double_node.bson_value.value().view().get_double().value,
3.14159);
// Test STRING type
auto string_node = builder.CreateValueNode("hello world", JSONType::STRING);
EXPECT_EQ(string_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(string_node.bson_value.value().view().type() ==
bsoncxx::type::k_string);
EXPECT_EQ(string_node.bson_value.value().view().get_string().value,
"hello world");
// Test ARRAY type
auto array_node = builder.CreateValueNode("[1, 2, 3]", JSONType::ARRAY);
EXPECT_EQ(array_node.type, DomNode::Type::VALUE);
EXPECT_TRUE(array_node.bson_value.value().view().type() ==
bsoncxx::type::k_array);
auto array_view = array_node.bson_value.value().view().get_array().value;
EXPECT_EQ(std::distance(array_view.begin(), array_view.end()), 3);
EXPECT_EQ(array_view[0].get_int64().value, 1);
EXPECT_EQ(array_view[1].get_int64().value, 2);
EXPECT_EQ(array_view[2].get_int64().value, 3);
// Test invalid type
EXPECT_THROW(builder.CreateValueNode("value", static_cast<JSONType>(999)),
std::runtime_error);
}
TEST_F(BsonBuilderTest, AppendToDomTest) {
BsonBuilder builder;
DomNode root(DomNode::Type::DOCUMENT);
// Test single level append
builder.AppendToDom(root, {"key1"}, "value1", JSONType::STRING);
EXPECT_TRUE(root.document_children.find("key1") !=
root.document_children.end());
EXPECT_EQ(root.document_children["key1"].type, DomNode::Type::VALUE);
EXPECT_EQ(root.document_children["key1"]
.bson_value.value()
.view()
.get_string()
.value,
"value1");
// Test nested document append
builder.AppendToDom(
root, {"level1", "level2", "key2"}, "42", JSONType::INT64);
EXPECT_TRUE(root.document_children.find("level1") !=
root.document_children.end());
EXPECT_EQ(root.document_children["level1"].type, DomNode::Type::DOCUMENT);
EXPECT_TRUE(
root.document_children["level1"].document_children.find("level2") !=
root.document_children["level1"].document_children.end());
EXPECT_EQ(root.document_children["level1"].document_children["level2"].type,
DomNode::Type::DOCUMENT);
EXPECT_EQ(root.document_children["level1"]
.document_children["level2"]
.document_children["key2"]
.type,
DomNode::Type::VALUE);
EXPECT_EQ(root.document_children["level1"]
.document_children["level2"]
.document_children["key2"]
.bson_value.value()
.view()
.get_int64()
.value,
42);
// Test overwriting existing value with document
builder.AppendToDom(root, {"key1", "nested"}, "value3", JSONType::STRING);
EXPECT_EQ(root.document_children["key1"].type, DomNode::Type::DOCUMENT);
EXPECT_EQ(root.document_children["key1"].document_children["nested"].type,
DomNode::Type::VALUE);
EXPECT_EQ(root.document_children["key1"]
.document_children["nested"]
.bson_value.value()
.view()
.get_string()
.value,
"value3");
}
TEST_F(BsonBuilderTest, ConvertDomToBsonTest) {
BsonBuilder builder;
DomNode root(DomNode::Type::DOCUMENT);
// Build a complex document structure
builder.AppendToDom(root, {"string_field"}, "hello", JSONType::STRING);
builder.AppendToDom(root, {"int_field"}, "42", JSONType::INT64);
builder.AppendToDom(root, {"double_field"}, "3.14", JSONType::DOUBLE);
builder.AppendToDom(root, {"bool_field"}, "true", JSONType::BOOL);
builder.AppendToDom(
root, {"nested", "field"}, "nested_value", JSONType::STRING);
builder.AppendToDom(root, {"array_field"}, "[1, 2, 3]", JSONType::ARRAY);
// Convert to BSON
bsoncxx::builder::basic::document bson_doc;
builder.ConvertDomToBson(root, bson_doc);
auto bson_view = bson_doc.view();
// Verify the converted BSON document
EXPECT_EQ(bson_view["string_field"].get_string().value, "hello");
EXPECT_EQ(bson_view["int_field"].get_int64().value, 42);
EXPECT_DOUBLE_EQ(bson_view["double_field"].get_double().value, 3.14);
EXPECT_TRUE(bson_view["bool_field"].get_bool().value);
EXPECT_EQ(bson_view["nested"]["field"].get_string().value, "nested_value");
auto array_view = bson_view["array_field"].get_array().value;
EXPECT_EQ(std::distance(array_view.begin(), array_view.end()), 3);
EXPECT_EQ(array_view[0].get_int64().value, 1);
EXPECT_EQ(array_view[1].get_int64().value, 2);
EXPECT_EQ(array_view[2].get_int64().value, 3);
}
TEST_F(BsonBuilderTest, ComplexDocumentTest) {
BsonBuilder builder;
DomNode root(DomNode::Type::DOCUMENT);
// Create a complex document with multiple levels and types
builder.AppendToDom(root, {"user", "name"}, "John", JSONType::STRING);
builder.AppendToDom(root, {"user", "age"}, "30", JSONType::INT64);
builder.AppendToDom(
root, {"user", "scores"}, "[85, 90, 95]", JSONType::ARRAY);
builder.AppendToDom(
root, {"user", "address", "city"}, "New York", JSONType::STRING);
builder.AppendToDom(
root, {"user", "address", "zip"}, "10001", JSONType::STRING);
builder.AppendToDom(root, {"user", "active"}, "true", JSONType::BOOL);
builder.AppendToDom(root, {"metadata", "version"}, "1.0", JSONType::STRING);
builder.AppendToDom(root, {"metadata", "count"}, "1000", JSONType::INT64);
// Convert to BSON
bsoncxx::builder::basic::document bson_doc;
builder.ConvertDomToBson(root, bson_doc);
auto bson_view = bson_doc.view();
// Verify the complex document structure
EXPECT_EQ(bson_view["user"]["name"].get_string().value, "John");
EXPECT_EQ(bson_view["user"]["age"].get_int64().value, 30);
EXPECT_TRUE(bson_view["user"]["active"].get_bool().value);
EXPECT_EQ(bson_view["user"]["address"]["city"].get_string().value,
"New York");
EXPECT_EQ(bson_view["user"]["address"]["zip"].get_string().value, "10001");
EXPECT_EQ(bson_view["metadata"]["version"].get_string().value, "1.0");
EXPECT_EQ(bson_view["metadata"]["count"].get_int64().value, 1000);
// Verify array
auto scores_array = bson_view["user"]["scores"].get_array().value;
EXPECT_EQ(std::distance(scores_array.begin(), scores_array.end()), 3);
EXPECT_EQ(scores_array[0].get_int64().value, 85);
EXPECT_EQ(scores_array[1].get_int64().value, 90);
EXPECT_EQ(scores_array[2].get_int64().value, 95);
}
TEST_F(BsonBuilderTest, ExtractBsonKeyOffsetsTest) {
{
auto doc = bsoncxx::from_json(R"({ "age": 30 })");
auto offsets = BsonBuilder::ExtractBsonKeyOffsets(doc.view());
EXPECT_EQ(offsets.size(), 1);
EXPECT_EQ(offsets[0].first, "/age");
EXPECT_EQ(offsets[0].second, 4);
BsonView view(doc.view().data(), doc.view().length());
auto res = view.ParseAsValueAtOffset<int64_t>(4);
EXPECT_EQ(res.value(), 30);
}
{
auto doc = bsoncxx::from_json(R"({ "age": "30"})");
auto offsets = BsonBuilder::ExtractBsonKeyOffsets(doc.view());
EXPECT_EQ(offsets.size(), 1);
EXPECT_EQ(offsets[0].first, "/age");
EXPECT_EQ(offsets[0].second, 4);
auto bson_view = BsonView(doc.view().data(), doc.view().length());
auto res = bson_view.ParseAsValueAtOffset<std::string_view>(4);
EXPECT_STREQ(res.value().data(), "30");
}
{
auto doc = bsoncxx::from_json(
R"({ "age": 30, "name": "Alice", "active": true })");
auto offsets = BsonBuilder::ExtractBsonKeyOffsets(doc.view());
EXPECT_EQ(offsets.size(), 3);
EXPECT_EQ(offsets[0].first, "/age");
EXPECT_EQ(offsets[1].first, "/name");
EXPECT_EQ(offsets[2].first, "/active");
auto view = BsonView(doc.view().data(), doc.view().length());
auto res1 = view.ParseAsValueAtOffset<int64_t>(offsets[0].second);
EXPECT_EQ(res1.value(), 30);
auto res2 =
view.ParseAsValueAtOffset<std::string_view>(offsets[1].second);
EXPECT_STREQ(res2.value().data(), "Alice");
auto res3 = view.ParseAsValueAtOffset<bool>(offsets[2].second);
EXPECT_EQ(res3.value(), true);
}
}
TEST_F(BsonBuilderTest, ExtractBsonKeyOffsetsTestAllTypes) {
// Create a complex BSON document with nested structures
bsoncxx::builder::basic::document doc;
// Add simple fields
doc.append(bsoncxx::builder::basic::kvp("string_field", "value1"));
doc.append(bsoncxx::builder::basic::kvp("int_field", 42));
doc.append(bsoncxx::builder::basic::kvp("double_field", 3.14));
doc.append(bsoncxx::builder::basic::kvp("bool_field", true));
// Add nested document
bsoncxx::builder::basic::document nested_doc;
nested_doc.append(
bsoncxx::builder::basic::kvp("nested_string", "nested_value"));
nested_doc.append(bsoncxx::builder::basic::kvp("nested_int", 123));
doc.append(bsoncxx::builder::basic::kvp("nested_doc", nested_doc));
// Add array
bsoncxx::builder::basic::array arr;
arr.append("array_item1");
arr.append(456);
arr.append(false);
doc.append(bsoncxx::builder::basic::kvp("array_field", arr));
// Add nested array with document
bsoncxx::builder::basic::array nested_arr;
bsoncxx::builder::basic::document arr_doc;
arr_doc.append(
bsoncxx::builder::basic::kvp("arr_doc_field", "arr_doc_value"));
nested_arr.append(arr_doc);
doc.append(bsoncxx::builder::basic::kvp("nested_array", nested_arr));
// Extract offsets
auto offsets = BsonBuilder::ExtractBsonKeyOffsets(doc.view());
// Verify results
EXPECT_FALSE(offsets.empty());
// Create a map for easier lookup
std::map<std::string, size_t> offset_map;
for (const auto& [key, offset] : offsets) {
offset_map[key] = offset;
}
// Verify all expected fields are present
EXPECT_TRUE(offset_map.find("/string_field") != offset_map.end());
EXPECT_TRUE(offset_map.find("/int_field") != offset_map.end());
EXPECT_TRUE(offset_map.find("/double_field") != offset_map.end());
EXPECT_TRUE(offset_map.find("/bool_field") != offset_map.end());
EXPECT_TRUE(offset_map.find("/nested_doc") != offset_map.end());
EXPECT_TRUE(offset_map.find("/nested_doc/nested_string") !=
offset_map.end());
EXPECT_TRUE(offset_map.find("/nested_doc/nested_int") != offset_map.end());
EXPECT_TRUE(offset_map.find("/array_field") != offset_map.end());
EXPECT_TRUE(offset_map.find("/nested_array") != offset_map.end());
// Verify offsets are within document bounds
size_t doc_size = doc.view().length();
for (const auto& [key, offset] : offsets) {
EXPECT_LT(offset, doc_size)
<< "Offset for key " << key << " is out of bounds";
}
// Verify offsets are unique
std::set<size_t> unique_offsets;
for (const auto& [key, offset] : offsets) {
EXPECT_TRUE(unique_offsets.insert(offset).second)
<< "Duplicate offset found for key " << key;
}
// Verify the total number of fields
// This number should match the total number of fields we added
// 1. string_field
// 2. int_field
// 3. double_field
// 4. bool_field
// 5. nested_doc
// 6. nested_doc/nested_string
// 7. nested_doc/nested_int
// 8. array_field
// 9. nested_array
// 10. nested_array/0/arr_doc_field
EXPECT_EQ(offsets.size(), 9);
}
TEST_F(BsonBuilderTest, ExtractOffsetsRecursiveTest) {
// Test empty document
bsoncxx::builder::basic::document empty_doc;
std::vector<std::pair<std::string, size_t>> empty_result;
builder_->ExtractOffsetsRecursive(
empty_doc.view().data(), empty_doc.view().data(), "", empty_result);
EXPECT_TRUE(empty_result.empty());
// Test simple document with one field
bsoncxx::builder::basic::document simple_doc;
simple_doc.append(bsoncxx::builder::basic::kvp("field", "value"));
std::vector<std::pair<std::string, size_t>> simple_result;
builder_->ExtractOffsetsRecursive(
simple_doc.view().data(), simple_doc.view().data(), "", simple_result);
EXPECT_EQ(simple_result.size(), 1);
EXPECT_EQ(simple_result[0].first, "/field");
}
TEST_F(BsonBuilderTest, ExtractOffsetsRecursiveWithNestedPathTest) {
bsoncxx::builder::basic::document doc;
bsoncxx::builder::basic::document nested_doc;
nested_doc.append(bsoncxx::builder::basic::kvp("nested_field", "value"));
doc.append(bsoncxx::builder::basic::kvp("parent", nested_doc));
std::vector<std::pair<std::string, size_t>> result;
builder_->ExtractOffsetsRecursive(
doc.view().data(), doc.view().data(), "prefix", result);
EXPECT_EQ(result.size(), 2);
EXPECT_EQ(result[0].first, "prefix/parent");
EXPECT_EQ(result[1].first, "prefix/parent/nested_field");
}
TEST_F(BsonBuilderTest, ExtractOffsetsRecursiveWithArrayElementsTest) {
bsoncxx::builder::basic::document doc;
bsoncxx::builder::basic::array arr;
// Add array with mixed types
arr.append("string_item");
arr.append(42);
arr.append(true);
// Add nested document in array
bsoncxx::builder::basic::document nested_doc;
nested_doc.append(bsoncxx::builder::basic::kvp("nested_field", "value"));
arr.append(nested_doc);
doc.append(bsoncxx::builder::basic::kvp("mixed_array", arr));
std::vector<std::pair<std::string, size_t>> result;
builder_->ExtractOffsetsRecursive(
doc.view().data(), doc.view().data(), "", result);
EXPECT_EQ(result.size(), 1); // mixed_array
EXPECT_EQ(result[0].first, "/mixed_array");
}
} // namespace milvus::index

View File

@ -0,0 +1,313 @@
#include <gtest/gtest.h>
#include "common/bson_view.h"
#include <bsoncxx/builder/basic/document.hpp>
#include <bsoncxx/builder/basic/array.hpp>
#include <bsoncxx/json.hpp>
#include <bsoncxx/types.hpp>
#include <bsoncxx/types/value.hpp>
#include <bsoncxx/types/bson_value/view.hpp>
#include <bsoncxx/types/bson_value/value.hpp>
#include "index/json_stats/bson_builder.h"
namespace milvus::index {
class BsonViewTest : public ::testing::Test {
protected:
void
SetUp() override {
}
};
TEST_F(BsonViewTest, ConstructorTest) {
// Test vector constructor
std::vector<uint8_t> data = {
0x05, 0x00, 0x00, 0x00, 0x00}; // Empty BSON document
BsonView view1(data);
EXPECT_EQ(view1.ToString(), "{ }");
}
TEST_F(BsonViewTest, ParseAsValueAtOffsetTest) {
// Create a BSON document with various types
bsoncxx::builder::basic::document doc;
doc.append(bsoncxx::builder::basic::kvp("int32_field", 42));
doc.append(bsoncxx::builder::basic::kvp("double_field", 3.14159));
doc.append(bsoncxx::builder::basic::kvp("bool_field", true));
doc.append(bsoncxx::builder::basic::kvp("string_field", "test string"));
BsonView view(doc.view().data(), doc.view().length());
auto offsets = BsonBuilder::ExtractBsonKeyOffsets(doc.view());
for (const auto& offset : offsets) {
std::cout << offset.first << " " << offset.second << std::endl;
}
// Test int32 parsing
auto int32_val = view.ParseAsValueAtOffset<int32_t>(offsets[0].second);
EXPECT_TRUE(int32_val.has_value());
EXPECT_EQ(int32_val.value(), 42);
// Test double parsing
auto double_val = view.ParseAsValueAtOffset<double>(offsets[1].second);
EXPECT_TRUE(double_val.has_value());
EXPECT_EQ(double_val.value(), 3.14159);
// Test bool parsing
auto bool_val = view.ParseAsValueAtOffset<bool>(offsets[2].second);
EXPECT_TRUE(bool_val.has_value());
EXPECT_TRUE(bool_val.value());
// Test string parsing
auto string_val = view.ParseAsValueAtOffset<std::string>(offsets[3].second);
EXPECT_TRUE(string_val.has_value());
EXPECT_EQ(string_val.value(), "test string");
}
TEST_F(BsonViewTest, ParseAsArrayAtOffsetTest) {
// Test case 1: offset = 0 (whole array)
{
bsoncxx::builder::basic::array arr;
arr.append("value1");
arr.append(42);
arr.append(true);
BsonView view(arr.view().data(), arr.view().length());
auto array_view = view.ParseAsArrayAtOffset(0);
// Verify array content
EXPECT_TRUE(array_view.has_value());
EXPECT_EQ(
std::distance(array_view.value().begin(), array_view.value().end()),
3);
EXPECT_STREQ(array_view.value()[0].get_string().value.data(), "value1");
EXPECT_EQ(array_view.value()[1].get_int32(), 42);
EXPECT_TRUE(array_view.value()[2].get_bool());
}
// Test case 2: offset != 0 (array in document)
{
bsoncxx::builder::basic::array arr;
arr.append("array_value1");
arr.append(123);
arr.append(3.14159);
bsoncxx::builder::basic::document doc;
doc.append(bsoncxx::builder::basic::kvp("top_field", "top_value"));
doc.append(bsoncxx::builder::basic::kvp("array_field", arr));
BsonView view(doc.view().data(), doc.view().length());
// Find the offset of the array
auto offsets = BsonBuilder::ExtractBsonKeyOffsets(doc.view());
size_t array_offset = 0;
for (const auto& offset : offsets) {
if (offset.first == "/array_field") {
array_offset = offset.second;
break;
}
}
auto array_view = view.ParseAsArrayAtOffset(array_offset);
// Verify array content
EXPECT_TRUE(array_view.has_value());
EXPECT_EQ(
std::distance(array_view.value().begin(), array_view.value().end()),
3);
EXPECT_STREQ(array_view.value()[0].get_string().value.data(),
"array_value1");
EXPECT_EQ(array_view.value()[1].get_int32(), 123);
EXPECT_DOUBLE_EQ(array_view.value()[2].get_double(), 3.14159);
}
// Test case 4: Error cases
{
bsoncxx::builder::basic::document doc;
doc.append(bsoncxx::builder::basic::kvp("field", "value"));
BsonView view(doc.view().data(), doc.view().length());
// Test invalid offset (out of range)
EXPECT_THROW(view.ParseAsArrayAtOffset(1000), std::runtime_error);
}
}
TEST_F(BsonViewTest, FindByPathTest) {
// Create a complex nested BSON document
bsoncxx::builder::basic::document nested_doc;
nested_doc.append(bsoncxx::builder::basic::kvp("nested_field", "value"));
bsoncxx::builder::basic::document doc;
doc.append(bsoncxx::builder::basic::kvp("level1", nested_doc));
doc.append(bsoncxx::builder::basic::kvp("simple_field", "simple_value"));
BsonView view(doc.view().data(), doc.view().length());
// Test finding nested field
std::vector<std::string> path = {"level1", "nested_field"};
auto value = view.FindByPath(doc.view(), path);
EXPECT_TRUE(value.has_value());
EXPECT_STREQ(value.value().get_string().value.data(), "value");
// Test finding simple field
path = {"simple_field"};
value = view.FindByPath(doc.view(), path);
EXPECT_TRUE(value.has_value());
EXPECT_STREQ(value.value().get_string().value.data(), "simple_value");
// Test non-existent path
path = {"non_existent"};
value = view.FindByPath(doc.view(), path);
EXPECT_FALSE(value.has_value());
// Test invalid nested path
path = {"level1", "non_existent"};
value = view.FindByPath(doc.view(), path);
EXPECT_FALSE(value.has_value());
}
TEST_F(BsonViewTest, GetNthElementInArrayTest) {
// Create a BSON array with mixed types
bsoncxx::builder::basic::array arr;
arr.append("string_item");
arr.append(42);
arr.append(true);
arr.append(3.14159);
// Test string element
auto string_val =
BsonView::GetNthElementInArray<std::string>(arr.view().data(), 0);
EXPECT_TRUE(string_val.has_value());
EXPECT_EQ(string_val.value(), "string_item");
// Test int32 element
auto int_val =
BsonView::GetNthElementInArray<int32_t>(arr.view().data(), 1);
EXPECT_TRUE(int_val.has_value());
EXPECT_EQ(int_val.value(), 42);
// Test bool element
auto bool_val = BsonView::GetNthElementInArray<bool>(arr.view().data(), 2);
EXPECT_TRUE(bool_val.has_value());
EXPECT_TRUE(bool_val.value());
// Test double element
auto double_val =
BsonView::GetNthElementInArray<double>(arr.view().data(), 3);
EXPECT_TRUE(double_val.has_value());
EXPECT_DOUBLE_EQ(double_val.value(), 3.14159);
// Test out of bounds
auto out_of_bounds =
BsonView::GetNthElementInArray<std::string>(arr.view().data(), 10);
EXPECT_FALSE(out_of_bounds.has_value());
// Test invalid type conversion
auto invalid_type =
BsonView::GetNthElementInArray<std::string>(arr.view().data(), 1);
EXPECT_FALSE(invalid_type.has_value());
}
TEST_F(BsonViewTest, ParseBsonFieldTest) {
// Create a simple BSON document
bsoncxx::builder::basic::document doc;
doc.append(bsoncxx::builder::basic::kvp("test_field", "test_value"));
BsonView view(doc.view().data(), doc.view().length());
// Test field parsing
auto field = view.ParseBsonField(doc.view().data(), 4);
EXPECT_EQ(field.type, bsoncxx::type::k_string);
EXPECT_EQ(field.key, "test_field");
EXPECT_NE(field.value_ptr, nullptr);
}
TEST_F(BsonViewTest, GetValueFromElementTest) {
// Create a BSON document with various types
bsoncxx::builder::basic::document doc;
doc.append(bsoncxx::builder::basic::kvp("int32_field", 42));
doc.append(bsoncxx::builder::basic::kvp("double_field", 3.14159));
doc.append(bsoncxx::builder::basic::kvp("bool_field", true));
doc.append(bsoncxx::builder::basic::kvp("string_field", "test string"));
auto view = doc.view();
// Test int32
auto int32_val =
BsonView::GetValueFromElement<int32_t>(view["int32_field"]);
EXPECT_TRUE(int32_val.has_value());
EXPECT_EQ(int32_val.value(), 42);
// Test double
auto double_val =
BsonView::GetValueFromElement<double>(view["double_field"]);
EXPECT_TRUE(double_val.has_value());
EXPECT_DOUBLE_EQ(double_val.value(), 3.14159);
// Test bool
auto bool_val = BsonView::GetValueFromElement<bool>(view["bool_field"]);
EXPECT_TRUE(bool_val.has_value());
EXPECT_TRUE(bool_val.value());
// Test string
auto string_val =
BsonView::GetValueFromElement<std::string>(view["string_field"]);
EXPECT_TRUE(string_val.has_value());
EXPECT_EQ(string_val.value(), "test string");
// Test string_view
auto string_view_val =
BsonView::GetValueFromElement<std::string_view>(view["string_field"]);
EXPECT_TRUE(string_view_val.has_value());
EXPECT_EQ(string_view_val.value(), "test string");
// Test invalid type conversion
auto invalid_val =
BsonView::GetValueFromElement<std::string>(view["int32_field"]);
EXPECT_FALSE(invalid_val.has_value());
}
TEST_F(BsonViewTest, GetValueFromBsonViewTest) {
// Create a BSON document with various types
bsoncxx::builder::basic::document doc;
doc.append(bsoncxx::builder::basic::kvp("int32_field", 42));
doc.append(bsoncxx::builder::basic::kvp("double_field", 3.14159));
doc.append(bsoncxx::builder::basic::kvp("bool_field", true));
doc.append(bsoncxx::builder::basic::kvp("string_field", "test string"));
auto view = doc.view();
// Test int32
auto int32_val = BsonView::GetValueFromBsonView<int32_t>(
view["int32_field"].get_value());
EXPECT_TRUE(int32_val.has_value());
EXPECT_EQ(int32_val.value(), 42);
// Test double
auto double_val = BsonView::GetValueFromBsonView<double>(
view["double_field"].get_value());
EXPECT_TRUE(double_val.has_value());
EXPECT_DOUBLE_EQ(double_val.value(), 3.14159);
// Test bool
auto bool_val =
BsonView::GetValueFromBsonView<bool>(view["bool_field"].get_value());
EXPECT_TRUE(bool_val.has_value());
EXPECT_TRUE(bool_val.value());
// Test string
auto string_val = BsonView::GetValueFromBsonView<std::string>(
view["string_field"].get_value());
EXPECT_TRUE(string_val.has_value());
EXPECT_EQ(string_val.value(), "test string");
// Test string_view
auto string_view_val = BsonView::GetValueFromBsonView<std::string_view>(
view["string_field"].get_value());
EXPECT_TRUE(string_view_val.has_value());
EXPECT_EQ(string_view_val.value(), "test string");
// Test invalid type conversion
auto invalid_val = BsonView::GetValueFromBsonView<std::string>(
view["int32_field"].get_value());
EXPECT_FALSE(invalid_val.has_value());
}
} // namespace milvus::index

View File

@ -0,0 +1,562 @@
// 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 <functional>
#include <boost/filesystem.hpp>
#include <unordered_set>
#include <memory>
#include <random>
#include "common/Tracer.h"
#include "index/BitmapIndex.h"
#include "storage/Util.h"
#include "storage/InsertData.h"
#include "indexbuilder/IndexFactory.h"
#include "index/IndexFactory.h"
#include "index/Meta.h"
#include "index/json_stats/JsonKeyStats.h"
#include "common/Json.h"
#include "common/Types.h"
using namespace milvus::index;
using namespace milvus::indexbuilder;
using namespace milvus;
using namespace milvus::index;
int64_t
GenerateRandomInt64(int64_t min, int64_t max) {
static std::random_device rd;
static std::mt19937_64 gen(rd());
std::uniform_int_distribution<int64_t> dist(min, max);
return dist(gen);
}
static std::vector<milvus::Json>
GenerateJsons(int size) {
std::vector<Json> jsons;
std::default_random_engine random(42);
std::normal_distribution<> distr(0, 1);
for (int i = 0; i < size; i++) {
std::string str;
if (i % 10 < 2) {
str = R"({"int_shared":)" + std::to_string(random()) +
R"(,"double_shared":)" +
std::to_string(static_cast<double>(random())) +
R"(,"string_shared":")" + std::to_string(random()) +
R"(","bool_shared": true)" + R"(, "array_shared": [1,2,3])" +
"}";
} else {
str = R"({"int":)" + std::to_string(random()) + R"(,"double":)" +
std::to_string(static_cast<double>(random())) +
R"(,"string":")" + std::to_string(random()) +
R"(","bool": true)" + R"(, "array": [1,2,3])" + "}";
}
jsons.push_back(milvus::Json(simdjson::padded_string(str)));
}
return jsons;
}
class JsonKeyStatsTest : public ::testing::TestWithParam<bool> {
protected:
void
Init(int64_t collection_id,
int64_t partition_id,
int64_t segment_id,
int64_t field_id,
int64_t index_build_id,
int64_t index_version,
int64_t size) {
proto::schema::FieldSchema field_schema;
field_schema.set_data_type(proto::schema::DataType::JSON);
field_schema.set_nullable(nullable_);
auto field_meta = storage::FieldDataMeta{
collection_id, partition_id, segment_id, field_id, field_schema};
auto index_meta = storage::IndexMeta{
segment_id, field_id, index_build_id, index_version};
data_ = std::move(GenerateJsons(size));
auto field_data = storage::CreateFieldData(DataType::JSON, nullable_);
if (nullable_) {
valid_data.reserve(size_);
for (size_t i = 0; i < size_; i++) {
if (i % 2 == 0) {
valid_data.push_back(true);
} else {
valid_data.push_back(false);
}
}
int byteSize = (size_ + 7) / 8;
uint8_t* valid_data_ = new uint8_t[byteSize];
for (int i = 0; i < size_; i++) {
bool value = valid_data[i];
int byteIndex = i / 8;
int bitIndex = i % 8;
if (value) {
valid_data_[byteIndex] |= (1 << bitIndex);
} else {
valid_data_[byteIndex] &= ~(1 << bitIndex);
}
}
field_data->FillFieldData(
data_.data(), valid_data_, data_.size(), 0);
delete[] valid_data_;
} else {
field_data->FillFieldData(data_.data(), data_.size());
}
auto payload_reader =
std::make_shared<milvus::storage::PayloadReader>(field_data);
storage::InsertData insert_data(payload_reader);
insert_data.SetFieldDataMeta(field_meta);
insert_data.SetTimestamps(0, 100);
auto serialized_bytes = insert_data.Serialize(storage::Remote);
auto log_path = fmt::format("/{}/{}/{}/{}/{}/{}",
"/tmp/test-jsonkey-stats/",
collection_id,
partition_id,
segment_id,
field_id,
0);
chunk_manager_->Write(
log_path, serialized_bytes.data(), serialized_bytes.size());
storage::FileManagerContext ctx(field_meta, index_meta, chunk_manager_);
std::vector<std::string> index_files;
Config config;
config[INSERT_FILES_KEY] = std::vector<std::string>{log_path};
auto build_index = std::make_shared<JsonKeyStats>(ctx, false);
build_index->Build(config);
auto create_index_result = build_index->Upload(config);
auto memSize = create_index_result->GetMemSize();
auto serializedSize = create_index_result->GetSerializedSize();
ASSERT_GT(memSize, 0);
ASSERT_GT(serializedSize, 0);
index_files = create_index_result->GetIndexFiles();
index::CreateIndexInfo index_info{};
config["index_files"] = index_files;
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
config[milvus::index::ENABLE_MMAP] = true;
config[milvus::index::MMAP_FILE_PATH] =
"/tmp/test-jsonkey-stats/mmap-file";
index_ = std::make_shared<JsonKeyStats>(ctx, true);
index_->Load(milvus::tracer::TraceContext{}, config);
}
void
SetUp() override {
nullable_ = GetParam();
type_ = DataType::JSON;
int64_t collection_id = 1;
int64_t partition_id = 2;
int64_t segment_id = 3;
int64_t field_id = 101;
int64_t index_build_id = GenerateRandomInt64(1, 100000);
int64_t index_version = 1;
size_ = 1000; // Use a larger size for better testing
std::string root_path = "/tmp/test-jsonkey-stats/";
storage::StorageConfig storage_config;
storage_config.storage_type = "local";
storage_config.root_path = root_path;
chunk_manager_ = storage::CreateChunkManager(storage_config);
auto conf = milvus_storage::ArrowFileSystemConfig();
conf.storage_type = "local";
conf.root_path = "/tmp/test-jsonkey-stats/arrow-fs";
milvus_storage::ArrowFileSystemSingleton::GetInstance().Init(conf);
Init(collection_id,
partition_id,
segment_id,
field_id,
index_build_id,
index_version,
size_);
}
virtual ~JsonKeyStatsTest() override {
boost::filesystem::remove_all(chunk_manager_->GetRootPath());
}
public:
std::shared_ptr<JsonKeyStats> index_;
DataType type_;
bool nullable_;
size_t size_;
FixedVector<bool> valid_data;
std::vector<milvus::Json> data_;
std::vector<std::string> json_col;
std::shared_ptr<storage::ChunkManager> chunk_manager_;
};
INSTANTIATE_TEST_SUITE_P(JsonKeyStatsTestSuite,
JsonKeyStatsTest,
::testing::Values(true, false));
TEST_P(JsonKeyStatsTest, TestBasicOperations) {
// Test Count
EXPECT_EQ(index_->Count(), size_);
// Test Size
EXPECT_EQ(index_->Size(), size_);
// Test HasRawData
EXPECT_FALSE(index_->HasRawData());
}
TEST_P(JsonKeyStatsTest, TestExecuteForSharedData) {
std::string path = "/int_shared";
int count = 0;
index_->ExecuteForSharedData(
path,
[&](BsonView bson, uint32_t row_id, uint32_t offset) { count++; });
std::cout << "count: " << count << std::endl;
if (nullable_) {
EXPECT_EQ(count, 100);
} else {
EXPECT_EQ(count, 200);
}
}
TEST_P(JsonKeyStatsTest, TestExecuteExistsPathForSharedData) {
std::string path = "/int_shared";
TargetBitmap bitset(size_);
TargetBitmapView bitset_view(bitset);
index_->ExecuteExistsPathForSharedData(path, bitset_view);
std::cout << "bitset.count(): " << bitset.count() << std::endl;
auto count = bitset.count();
if (nullable_) {
EXPECT_EQ(count, 100);
} else {
EXPECT_EQ(count, 200);
}
}
TEST_P(JsonKeyStatsTest, TestExecutorForGettingValid) {
std::string path = "/int";
TargetBitmap valid_res(size_, true);
TargetBitmapView valid_res_view(valid_res);
auto shredding_fields = index_->GetShreddingFields(path);
for (const auto& field : shredding_fields) {
auto processed_size =
index_->ExecutorForGettingValid(field, valid_res_view);
EXPECT_EQ(processed_size, size_);
}
if (!index_->CanSkipShared(path)) {
std::cout << "can not skip shared" << std::endl;
index_->ExecuteExistsPathForSharedData(path, valid_res_view);
}
std::cout << "valid_res.count(): " << valid_res.count() << std::endl;
if (nullable_) {
EXPECT_EQ(valid_res.count(), 400);
} else {
EXPECT_EQ(valid_res.count(), 800);
}
}
TEST_P(JsonKeyStatsTest, TestExecutorForShreddingData) {
std::string path = "/int";
TargetBitmap res(size_);
TargetBitmap valid_res(size_, true);
TargetBitmapView res_view(res);
TargetBitmapView valid_res_view(valid_res);
auto func = [](const int64_t* data,
const bool* valid_data,
const int size,
TargetBitmapView res,
TargetBitmapView valid_res) {
for (int i = 0; i < size; i++) {
if (valid_data[i]) {
res[i] = true;
valid_res[i] = true;
}
}
};
auto field_name = *(index_->GetShreddingFields(path).begin());
std::cout << "field_name: " << field_name << std::endl;
int processed_size = index_->ExecutorForShreddingData<int64_t>(
field_name, func, nullptr, res_view, valid_res_view);
std::cout << "processed_size: " << processed_size << std::endl;
EXPECT_EQ(processed_size, size_);
if (nullable_) {
EXPECT_EQ(res.count(), 400);
} else {
EXPECT_EQ(res.count(), 800);
}
}
TEST_P(JsonKeyStatsTest, TestGetShreddingFields) {
std::string pointer = "/int";
auto fields = index_->GetShreddingFields(pointer);
EXPECT_FALSE(fields.empty());
std::vector<JSONType> types = {JSONType::INT64};
auto typed_fields = index_->GetShreddingFields(pointer, types);
EXPECT_FALSE(typed_fields.empty());
}
TEST_P(JsonKeyStatsTest, TestCanSkipShared) {
std::string path = "/int";
std::set<JSONType> target_types = {JSONType::INT64};
EXPECT_TRUE(index_->CanSkipShared(path, target_types));
target_types = {JSONType::STRING};
EXPECT_TRUE(index_->CanSkipShared(path, target_types));
}
class JsonKeyStatsUploadLoadTest : public ::testing::Test {
protected:
void
SetUp() override {
collection_id_ = 1;
partition_id_ = 2;
segment_id_ = 3;
field_id_ = 101;
index_build_id_ = GenerateRandomInt64(1, 100000);
index_version_ = 10000;
root_path_ = "/tmp/test-jsonkey-stats-upload-load/";
storage::StorageConfig storage_config;
storage_config.storage_type = "local";
storage_config.root_path = root_path_;
chunk_manager_ = storage::CreateChunkManager(storage_config);
auto conf = milvus_storage::ArrowFileSystemConfig();
conf.storage_type = "local";
conf.root_path = "/tmp/test-jsonkey-stats-upload-load/arrow-fs";
milvus_storage::ArrowFileSystemSingleton::GetInstance().Init(conf);
}
void
TearDown() override {
boost::filesystem::remove_all(chunk_manager_->GetRootPath());
}
void
InitContext() {
proto::schema::FieldSchema field_schema;
field_schema.set_data_type(proto::schema::DataType::JSON);
field_schema.set_nullable(false);
field_meta_ = storage::FieldDataMeta{collection_id_,
partition_id_,
segment_id_,
field_id_,
field_schema};
index_meta_ = storage::IndexMeta{
segment_id_, field_id_, index_build_id_, index_version_};
}
void
PrepareData(const std::vector<std::string>& json_strings) {
data_.clear();
for (const auto& str : json_strings) {
data_.push_back(milvus::Json(simdjson::padded_string(str)));
}
}
void
BuildAndUpload() {
auto field_data = storage::CreateFieldData(DataType::JSON, false);
field_data->FillFieldData(data_.data(), data_.size());
auto payload_reader =
std::make_shared<milvus::storage::PayloadReader>(field_data);
storage::InsertData insert_data(payload_reader);
insert_data.SetFieldDataMeta(field_meta_);
insert_data.SetTimestamps(0, 100);
auto serialized_bytes = insert_data.Serialize(storage::Remote);
auto log_path = fmt::format("/{}/{}/{}/{}/{}/{}",
root_path_,
collection_id_,
partition_id_,
segment_id_,
field_id_,
0);
chunk_manager_->Write(
log_path, serialized_bytes.data(), serialized_bytes.size());
storage::FileManagerContext ctx(
field_meta_, index_meta_, chunk_manager_);
Config config;
config[INSERT_FILES_KEY] = std::vector<std::string>{log_path};
build_index_ = std::make_shared<JsonKeyStats>(ctx, false);
build_index_->Build(config);
auto create_index_result = build_index_->Upload(config);
auto memSize = create_index_result->GetMemSize();
auto serializedSize = create_index_result->GetSerializedSize();
ASSERT_GT(memSize, 0);
ASSERT_GT(serializedSize, 0);
index_files_ = create_index_result->GetIndexFiles();
}
void
Load() {
storage::FileManagerContext ctx(
field_meta_, index_meta_, chunk_manager_);
Config config;
config["index_files"] = index_files_;
config[milvus::index::ENABLE_MMAP] = true;
config[milvus::index::MMAP_FILE_PATH] =
"/tmp/test-jsonkey-stats-upload-load/mmap-file";
config[milvus::LOAD_PRIORITY] =
milvus::proto::common::LoadPriority::HIGH;
load_index_ = std::make_shared<JsonKeyStats>(ctx, true);
load_index_->Load(milvus::tracer::TraceContext{}, config);
}
void
VerifyBasicOperations() {
EXPECT_EQ(load_index_->Count(), data_.size());
EXPECT_EQ(load_index_->Size(), data_.size());
EXPECT_FALSE(load_index_->HasRawData());
}
void
VerifyPathInShared(const std::string& path) {
TargetBitmap bitset(data_.size());
TargetBitmapView bitset_view(bitset);
load_index_->ExecuteExistsPathForSharedData(path, bitset_view);
EXPECT_GT(bitset.size(), 0);
}
void
VerifyPathInShredding(const std::string& path) {
auto fields = load_index_->GetShreddingFields(path);
EXPECT_GT(fields.size(), 0);
}
void
VerifyJsonType(const std::string& path, JSONType expected_type) {
auto type = load_index_->GetShreddingJsonType(path);
EXPECT_EQ(int(type), int(expected_type));
}
public:
int64_t collection_id_;
int64_t partition_id_;
int64_t segment_id_;
int64_t field_id_;
int64_t index_build_id_;
int64_t index_version_;
std::string root_path_;
storage::FieldDataMeta field_meta_;
storage::IndexMeta index_meta_;
std::vector<milvus::Json> data_;
std::shared_ptr<storage::ChunkManager> chunk_manager_;
std::shared_ptr<JsonKeyStats> build_index_;
std::shared_ptr<JsonKeyStats> load_index_;
std::vector<std::string> index_files_;
};
TEST_F(JsonKeyStatsUploadLoadTest, TestSimpleJson) {
std::vector<std::string> json_strings = {
R"({"int": 1, "double": 1.5, "string": "test", "bool": true})",
R"({"int": 2, "double": 2.5, "string": "test2", "bool": false})",
R"({"int": 3, "double": 3.5, "string": "test3", "bool": true})"};
InitContext();
PrepareData(json_strings);
BuildAndUpload();
Load();
VerifyBasicOperations();
VerifyPathInShredding("/int");
VerifyPathInShredding("/double");
VerifyPathInShredding("/string");
VerifyPathInShredding("/bool");
VerifyJsonType("/int_INT64", JSONType::INT64);
VerifyJsonType("/double_DOUBLE", JSONType::DOUBLE);
VerifyJsonType("/string_STRING", JSONType::STRING);
VerifyJsonType("/bool_BOOL", JSONType::BOOL);
}
TEST_F(JsonKeyStatsUploadLoadTest, TestNestedJson) {
std::vector<std::string> json_strings = {
R"({"nested": {"int": 1, "double": 1.5}, "array": [1, 2, 3]})",
R"({"nested": {"int": 2, "double": 2.5}, "array": [4, 5, 6]})",
R"({"nested": {"int": 3, "double": 3.5}, "array": [7, 8, 9]})"};
InitContext();
PrepareData(json_strings);
BuildAndUpload();
Load();
VerifyBasicOperations();
VerifyPathInShredding("/nested/int");
VerifyPathInShredding("/nested/double");
VerifyPathInShared("/array");
VerifyJsonType("/nested/int_INT64", JSONType::INT64);
VerifyJsonType("/nested/double_DOUBLE", JSONType::DOUBLE);
}
TEST_F(JsonKeyStatsUploadLoadTest, TestComplexJson) {
std::vector<std::string> json_strings = {
R"({
"user": {
"id": 1,
"name": "John",
"scores": [85, 90, 95],
"address": {
"city": "New York",
"zip": 10001
}
},
"timestamp": 1234567890
})",
R"({
"user": {
"id": 2,
"name": "Jane",
"scores": [88, 92, 98],
"address": {
"city": "Los Angeles",
"zip": 90001
}
},
"timestamp": 1234567891
})"};
InitContext();
PrepareData(json_strings);
BuildAndUpload();
Load();
VerifyBasicOperations();
VerifyPathInShredding("/user/id");
VerifyPathInShredding("/user/name");
VerifyPathInShared("/user/scores");
VerifyPathInShredding("/user/address/city");
VerifyPathInShredding("/user/address/zip");
VerifyPathInShredding("/timestamp");
VerifyJsonType("/user/id_INT64", JSONType::INT64);
VerifyJsonType("/user/name_STRING", JSONType::STRING);
VerifyJsonType("/user/address/city_STRING", JSONType::STRING);
VerifyJsonType("/user/address/zip_INT64", JSONType::INT64);
VerifyJsonType("/timestamp_INT64", JSONType::INT64);
}

View File

@ -0,0 +1,108 @@
#include <gtest/gtest.h>
#include "index/json_stats/parquet_writer.h"
#include <arrow/io/memory.h>
#include <arrow/io/file.h>
#include <parquet/arrow/writer.h>
#include <memory>
#include <string>
#include <vector>
#include <map>
namespace milvus::index {
class ParquetWriterFactoryTest : public ::testing::Test {
protected:
void
SetUp() override {
// Setup test column map
column_map_ = {
{JsonKey("int_key", JSONType::INT64), JsonKeyLayoutType::DYNAMIC},
{JsonKey("string_key", JSONType::STRING),
JsonKeyLayoutType::DYNAMIC},
{JsonKey("double_key", JSONType::DOUBLE), JsonKeyLayoutType::TYPED},
{JsonKey("bool_key", JSONType::BOOL), JsonKeyLayoutType::TYPED},
{JsonKey("shared_key", JSONType::STRING),
JsonKeyLayoutType::SHARED}};
path_prefix_ = "test_prefix";
}
std::map<JsonKey, JsonKeyLayoutType> column_map_;
std::string path_prefix_;
};
TEST_F(ParquetWriterFactoryTest, ColumnGroupingStrategyFactoryTest) {
// Test creating default strategy
auto default_strategy = ColumnGroupingStrategyFactory::CreateStrategy(
ColumnGroupingStrategyType::DEFAULT);
EXPECT_NE(default_strategy, nullptr);
// Test creating with invalid type
EXPECT_THROW(ColumnGroupingStrategyFactory::CreateStrategy(
static_cast<ColumnGroupingStrategyType>(999)),
std::runtime_error);
}
TEST_F(ParquetWriterFactoryTest, CreateContextBasicTest) {
// Test creating context with basic column map
auto context =
ParquetWriterFactory::CreateContext(column_map_, path_prefix_);
// Verify schema
EXPECT_NE(context.schema, nullptr);
EXPECT_EQ(context.schema->num_fields(), column_map_.size());
// Verify builders
EXPECT_FALSE(context.builders.empty());
EXPECT_FALSE(context.builders_map.empty());
EXPECT_EQ(context.builders.size(), column_map_.size());
EXPECT_EQ(context.builders_map.size(), column_map_.size());
// Verify metadata
EXPECT_FALSE(context.kv_metadata.empty());
// Verify column groups
EXPECT_FALSE(context.column_groups.empty());
// Verify file paths
EXPECT_FALSE(context.file_paths.empty());
EXPECT_EQ(context.file_paths.size(), context.column_groups.size());
}
TEST_F(ParquetWriterFactoryTest, CreateContextWithSharedFields) {
// Test creating context with shared fields
std::map<JsonKey, JsonKeyLayoutType> shared_map = {
{JsonKey("shared_key1", JSONType::STRING), JsonKeyLayoutType::SHARED},
{JsonKey("shared_key2", JSONType::STRING), JsonKeyLayoutType::SHARED},
{JsonKey("normal_key", JSONType::INT64), JsonKeyLayoutType::TYPED}};
auto context =
ParquetWriterFactory::CreateContext(shared_map, path_prefix_);
// Verify schema includes shared fields
EXPECT_NE(context.schema, nullptr);
EXPECT_EQ(context.schema->num_fields(), 2);
EXPECT_EQ(context.builders_map.size(), 3);
}
TEST_F(ParquetWriterFactoryTest, CreateContextWithColumnGroups) {
// Test creating context and verify column grouping
auto context =
ParquetWriterFactory::CreateContext(column_map_, path_prefix_);
// Verify column groups are created
EXPECT_FALSE(context.column_groups.empty());
// Verify each column is assigned to a group
std::set<int> group_ids;
for (const auto& group : context.column_groups) {
for (const auto& col_idx : group) {
group_ids.insert(col_idx);
}
}
// All columns should be assigned to a group
EXPECT_EQ(group_ids.size(), column_map_.size());
}
} // namespace milvus::index

View File

@ -0,0 +1,70 @@
#include <gtest/gtest.h>
#include "index/json_stats/utils.h"
#include <arrow/array/builder_binary.h>
#include <arrow/array/builder_primitive.h>
#include <arrow/type.h>
#include <nlohmann/json.hpp>
namespace milvus::index {
class UtilsTest : public ::testing::Test {
protected:
void
SetUp() override {
}
};
TEST_F(UtilsTest, CreateSharedArrowBuilderTest) {
auto builder = CreateSharedArrowBuilder();
EXPECT_NE(builder, nullptr);
EXPECT_EQ(builder->type()->id(), arrow::Type::BINARY);
}
TEST_F(UtilsTest, CreateSharedArrowFieldTest) {
auto field = CreateSharedArrowField("test_field", 1);
EXPECT_NE(field, nullptr);
EXPECT_EQ(field->name(), "test_field");
EXPECT_EQ(field->type()->id(), arrow::Type::BINARY);
EXPECT_TRUE(field->nullable());
EXPECT_TRUE(field->HasMetadata());
}
TEST_F(UtilsTest, CreateArrowBuildersTest) {
std::map<JsonKey, JsonKeyLayoutType> column_map = {
{JsonKey("int_key", JSONType::INT64), JsonKeyLayoutType::TYPED},
{JsonKey("string_key", JSONType::STRING), JsonKeyLayoutType::DYNAMIC},
{JsonKey("double_key", JSONType::DOUBLE), JsonKeyLayoutType::TYPED},
{JsonKey("bool_key", JSONType::BOOL), JsonKeyLayoutType::TYPED},
{JsonKey("shared_key", JSONType::STRING), JsonKeyLayoutType::SHARED}};
auto [builders, builders_map] = CreateArrowBuilders(column_map);
EXPECT_EQ(builders.size(), column_map.size());
EXPECT_EQ(builders_map.size(), column_map.size());
auto schema = CreateArrowSchema(column_map);
EXPECT_NE(schema, nullptr);
EXPECT_EQ(schema->num_fields(), column_map.size());
}
TEST_F(UtilsTest, CreateParquetKVMetadataTest) {
std::map<JsonKey, JsonKeyLayoutType> column_map = {
{JsonKey("int_key", JSONType::INT64), JsonKeyLayoutType::TYPED},
{JsonKey("string_key", JSONType::STRING), JsonKeyLayoutType::DYNAMIC},
{JsonKey("double_key", JSONType::DOUBLE), JsonKeyLayoutType::TYPED},
{JsonKey("bool_key", JSONType::BOOL), JsonKeyLayoutType::TYPED},
{JsonKey("shared_key", JSONType::STRING), JsonKeyLayoutType::SHARED}};
auto metadata = CreateParquetKVMetadata(column_map);
EXPECT_FALSE(metadata.empty());
EXPECT_EQ(metadata.size(), 1); // layout_type
// Parse and verify layout_type
auto layout_type_json = nlohmann::json::parse(metadata[0].second);
for (const auto& [key, type] : column_map) {
std::string key_with_type = key.key_ + "_" + ToString(key.type_);
EXPECT_TRUE(layout_type_json.contains(key_with_type));
EXPECT_EQ(layout_type_json[key_with_type], ToString(type));
}
}
} // namespace milvus::index

View File

@ -0,0 +1,125 @@
// Copyright (C) 2019-2025 Zilliz. All rights reserved.
// Licensed under the Apache License, Version 2.0
#include <gtest/gtest.h>
#include <optional>
#include <random>
#include <string>
#include <string_view>
#include <vector>
#include "arrow/api.h"
#include "arrow/array/array_binary.h"
#include "common/Chunk.h"
#include "common/ChunkWriter.h"
using milvus::StringChunk;
using milvus::StringChunkWriter;
namespace {
std::shared_ptr<arrow::BinaryArray>
BuildBinaryArray(const std::vector<std::optional<std::string>>& values) {
arrow::BinaryBuilder builder;
for (const auto& v : values) {
if (v.has_value()) {
auto st =
builder.Append(v->data(), static_cast<int32_t>(v->size()));
ASSERT_TRUE(st.ok());
} else {
auto st = builder.AppendNull();
ASSERT_TRUE(st.ok());
}
}
std::shared_ptr<arrow::Array> arr;
auto st = builder.Finish(&arr);
EXPECT_TRUE(st.ok());
return std::static_pointer_cast<arrow::BinaryArray>(arr);
}
} // namespace
TEST(StringChunkWriterTest, NoNullsMultiBatches) {
// Prepare two batches without nulls
std::vector<std::optional<std::string>> b1 = {std::string("a"),
std::string(""),
std::string("hello"),
std::string("world"),
std::string("foobar")};
std::vector<std::optional<std::string>> b2;
for (int i = 0; i < 100; ++i) {
b2.emplace_back(std::string(1 + (i % 7), 'x' + (i % 3)));
}
auto a1 = BuildBinaryArray(b1);
auto a2 = BuildBinaryArray(b2);
arrow::ArrayVector vec{a1, a2};
StringChunkWriter writer(/*nullable=*/false);
writer.write(vec);
auto chunk_up = writer.finish();
auto* chunk = dynamic_cast<StringChunk*>(chunk_up.get());
ASSERT_NE(chunk, nullptr);
// Verify rows and content
int64_t total_rows = static_cast<int64_t>(b1.size() + b2.size());
EXPECT_EQ(chunk->RowNums(), total_rows);
for (int64_t i = 0; i < static_cast<int64_t>(b1.size()); ++i) {
auto sv = (*chunk)[static_cast<int>(i)];
ASSERT_TRUE(chunk->isValid(static_cast<int>(i)));
EXPECT_EQ(sv, b1[static_cast<size_t>(i)].value());
}
for (int64_t i = 0; i < static_cast<int64_t>(b2.size()); ++i) {
auto idx = static_cast<int>(b1.size() + i);
auto sv = (*chunk)[idx];
ASSERT_TRUE(chunk->isValid(idx));
EXPECT_EQ(sv, b2[static_cast<size_t>(i)].value());
}
}
TEST(StringChunkWriterTest, WithNullsMergedBitmap) {
// Prepare batches with nulls; only verify validity for nulls
std::vector<std::optional<std::string>> b1 = {
std::nullopt,
std::string("alpha"),
std::nullopt,
std::string("beta"),
std::string("gamma"),
};
std::vector<std::optional<std::string>> b2 = {
std::string("one"),
std::nullopt,
std::string("two"),
std::string(""),
};
auto a1 = BuildBinaryArray(b1);
auto a2 = BuildBinaryArray(b2);
arrow::ArrayVector vec{a1, a2};
StringChunkWriter writer(/*nullable=*/true);
writer.write(vec);
auto chunk_up = writer.finish();
auto* chunk = dynamic_cast<StringChunk*>(chunk_up.get());
ASSERT_NE(chunk, nullptr);
// Verify validity and content for non-null entries only
std::vector<std::optional<std::string>> all;
all.reserve(b1.size() + b2.size());
all.insert(all.end(), b1.begin(), b1.end());
all.insert(all.end(), b2.begin(), b2.end());
for (int i = 0; i < static_cast<int>(all.size()); ++i) {
bool expect_valid = all[static_cast<size_t>(i)].has_value();
EXPECT_EQ(chunk->isValid(i), expect_valid);
if (expect_valid) {
EXPECT_EQ((*chunk)[i], all[static_cast<size_t>(i)].value());
}
}
}

View File

@ -196,6 +196,7 @@ func (gc *garbageCollector) work(ctx context.Context) {
gc.recycleUnusedAnalyzeFiles(ctx)
gc.recycleUnusedTextIndexFiles(ctx)
gc.recycleUnusedJSONIndexFiles(ctx)
gc.recycleUnusedJSONStatsFiles(ctx)
})
}()
go func() {
@ -989,8 +990,106 @@ func (gc *garbageCollector) recycleUnusedTextIndexFiles(ctx context.Context) {
metrics.GarbageCollectorRunCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Add(1)
}
// recycleUnusedJSONIndexFiles load meta file info and compares OSS keys
// recycleUnusedJSONStatsFiles load meta file info and compares OSS keys
// if missing found, performs gc cleanup
func (gc *garbageCollector) recycleUnusedJSONStatsFiles(ctx context.Context) {
start := time.Now()
log := log.Ctx(ctx).With(zap.String("gcName", "recycleUnusedJSONStatsFiles"), zap.Time("startAt", start))
log.Info("start recycleUnusedJSONStatsFiles...")
defer func() { log.Info("recycleUnusedJSONStatsFiles done", zap.Duration("timeCost", time.Since(start))) }()
hasJSONStatsSegments := gc.meta.SelectSegments(ctx, SegmentFilterFunc(func(info *SegmentInfo) bool {
return len(info.GetJsonKeyStats()) != 0
}))
fileNum := 0
deletedFilesNum := atomic.NewInt32(0)
for _, seg := range hasJSONStatsSegments {
for _, fieldStats := range seg.GetJsonKeyStats() {
log := log.With(zap.Int64("segmentID", seg.GetID()), zap.Int64("fieldID", fieldStats.GetFieldID()))
// clear low version task
for i := int64(1); i < fieldStats.GetVersion(); i++ {
prefix := fmt.Sprintf("%s/%s/%d/%d/%d/%d/%d/%d/%d", gc.option.cli.RootPath(), common.JSONStatsPath, fieldStats.GetJsonKeyStatsDataFormat(),
fieldStats.GetBuildID(), i, seg.GetCollectionID(), seg.GetPartitionID(), seg.GetID(), fieldStats.GetFieldID())
futures := make([]*conc.Future[struct{}], 0)
err := gc.option.cli.WalkWithPrefix(ctx, prefix, true, func(files *storage.ChunkObjectInfo) bool {
file := files.FilePath
future := gc.option.removeObjectPool.Submit(func() (struct{}, error) {
log := log.With(zap.String("file", file))
log.Info("garbageCollector recycleUnusedJSONStatsFiles remove file...")
if err := gc.option.cli.Remove(ctx, file); err != nil {
log.Warn("garbageCollector recycleUnusedJSONStatsFiles remove file failed", zap.Error(err))
return struct{}{}, err
}
deletedFilesNum.Inc()
log.Info("garbageCollector recycleUnusedJSONStatsFiles remove file success")
return struct{}{}, nil
})
futures = append(futures, future)
return true
})
// Wait for all remove tasks done.
if err := conc.BlockOnAll(futures...); err != nil {
// error is logged, and can be ignored here.
log.Warn("some task failure in remove object pool", zap.Error(err))
}
log = log.With(zap.Int("deleteJSONStatsNum", int(deletedFilesNum.Load())), zap.Int("walkFileNum", fileNum))
if err != nil {
log.Warn("json stats files recycle failed when walk with prefix", zap.Error(err))
return
}
}
// clear low data format version stats file
// for upgrade from old version to new version, we need to clear the old data format version stats file
for i := int64(1); i < fieldStats.GetJsonKeyStatsDataFormat(); i++ {
prefix := fmt.Sprintf("%s/%s/%d", gc.option.cli.RootPath(), common.JSONStatsPath, i)
futures := make([]*conc.Future[struct{}], 0)
err := gc.option.cli.WalkWithPrefix(ctx, prefix, true, func(files *storage.ChunkObjectInfo) bool {
file := files.FilePath
future := gc.option.removeObjectPool.Submit(func() (struct{}, error) {
log := log.With(zap.String("file", file))
log.Info("garbageCollector recycleUnusedJSONStatsFiles remove file...")
if err := gc.option.cli.Remove(ctx, file); err != nil {
log.Warn("garbageCollector recycleUnusedJSONStatsFiles remove file failed", zap.Error(err))
return struct{}{}, err
}
deletedFilesNum.Inc()
log.Info("garbageCollector recycleUnusedJSONStatsFiles remove file success")
return struct{}{}, nil
})
futures = append(futures, future)
return true
})
// Wait for all remove tasks done.
if err := conc.BlockOnAll(futures...); err != nil {
// error is logged, and can be ignored here.
log.Warn("some task failure in remove object pool", zap.Error(err))
}
log = log.With(zap.Int("deleteJSONStatsLowerDataFormatNum", int(deletedFilesNum.Load())), zap.Int("walkFileNum", fileNum))
if err != nil {
log.Warn("json stats lower data format files recycle failed when walk with prefix", zap.Error(err))
return
}
}
}
}
log.Info("json stats files recycle done")
metrics.GarbageCollectorRunCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Add(1)
}
// recycleUnusedJSONIndexFiles load meta file info and compares OSS keys
func (gc *garbageCollector) recycleUnusedJSONIndexFiles(ctx context.Context) {
start := time.Now()
log := log.Ctx(ctx).With(zap.String("gcName", "recycleUnusedJSONIndexFiles"), zap.Time("startAt", start))

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/task"
"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/indexpb"
@ -173,6 +174,11 @@ func needDoJsonKeyIndex(segment *SegmentInfo, fieldIDs []UniqueID) bool {
if segment.GetJsonKeyStats()[fieldID] == nil {
return true
}
// if the data format version is less than the current version, we need to do the stats task again
// because the data format is updated, the old data format need to be converted to the new data format
if segment.GetJsonKeyStats()[fieldID].GetJsonKeyStatsDataFormat() < common.JSONStatsDataFormatVersion {
return true
}
}
return false
}

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/session"
globalTask "github.com/milvus-io/milvus/internal/datacoord/task"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/proto/workerpb"
@ -330,13 +331,15 @@ func (st *statsTask) prepareJobRequest(ctx context.Context, segment *SegmentInfo
EndLogID: end,
NumRows: segment.GetNumOfRows(),
// update version after check
TaskVersion: st.GetVersion(),
EnableJsonKeyStats: Params.CommonCfg.EnabledJSONKeyStats.GetAsBool(),
JsonKeyStatsTantivyMemory: Params.DataCoordCfg.JSONKeyStatsMemoryBudgetInTantivy.GetAsInt64(),
JsonKeyStatsDataFormat: 1,
TaskSlot: st.taskSlot,
StorageVersion: segment.StorageVersion,
CurrentScalarIndexVersion: st.ievm.GetCurrentScalarIndexEngineVersion(),
TaskVersion: st.GetVersion(),
EnableJsonKeyStats: Params.CommonCfg.EnabledJSONKeyStats.GetAsBool(),
JsonKeyStatsDataFormat: common.JSONStatsDataFormatVersion,
TaskSlot: st.taskSlot,
StorageVersion: segment.StorageVersion,
CurrentScalarIndexVersion: st.ievm.GetCurrentScalarIndexEngineVersion(),
JsonStatsMaxShreddingColumns: Params.DataCoordCfg.JSONStatsMaxShreddingColumns.GetAsInt64(),
JsonStatsShreddingRatioThreshold: Params.DataCoordCfg.JSONStatsShreddingRatioThreshold.GetAsFloat(),
JsonStatsWriteBatchSize: Params.DataCoordCfg.JSONStatsWriteBatchSize.GetAsInt64(),
}
WrapPluginContext(segment.GetCollectionID(), collInfo.Schema.GetProperties(), req)

View File

@ -22,6 +22,7 @@ import (
"github.com/stretchr/testify/suite"
"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/indexpb"
)
@ -102,7 +103,7 @@ func (s *statsTaskInfoSuite) Test_Methods() {
Files: []string{"file1"},
LogSize: 1024,
MemorySize: 1024,
JsonKeyStatsDataFormat: 1,
JsonKeyStatsDataFormat: common.JSONStatsDataFormatVersion,
},
})
})

View File

@ -73,6 +73,13 @@ type statsTask struct {
currentTime time.Time
}
type BuildIndexOptions struct {
TantivyMemory int64
JsonStatsMaxShreddingColumns int64
JsonStatsShreddingRatio float64
JsonStatsWriteBatchSize int64
}
func NewStatsTask(ctx context.Context,
cancel context.CancelFunc,
req *workerpb.CreateStatsRequest,
@ -337,9 +344,11 @@ func (st *statsTask) Execute(ctx context.Context) error {
st.req.GetTargetSegmentID(),
st.req.GetTaskVersion(),
st.req.GetTaskID(),
st.req.GetJsonKeyStatsTantivyMemory(),
st.req.GetJsonKeyStatsDataFormat(),
insertLogs)
insertLogs,
st.req.GetJsonStatsMaxShreddingColumns(),
st.req.GetJsonStatsShreddingRatioThreshold(),
st.req.GetJsonStatsWriteBatchSize())
if err != nil {
log.Warn("stats wrong, failed to create json index", zap.Error(err))
return err
@ -467,7 +476,7 @@ func (st *statsTask) createTextIndex(ctx context.Context,
req := proto.Clone(st.req).(*workerpb.CreateStatsRequest)
req.InsertLogs = insertBinlogs
buildIndexParams := buildIndexParams(req, files, field, newStorageConfig, 0)
buildIndexParams := buildIndexParams(req, files, field, newStorageConfig, nil)
uploaded, err := indexcgowrapper.CreateTextIndex(ctx, buildIndexParams)
if err != nil {
@ -505,20 +514,27 @@ func (st *statsTask) createJSONKeyStats(ctx context.Context,
segmentID int64,
version int64,
taskID int64,
tantivyMemory int64,
jsonKeyStatsDataFormat int64,
insertBinlogs []*datapb.FieldBinlog,
jsonStatsMaxShreddingColumns int64,
jsonStatsShreddingRatioThreshold float64,
jsonStatsWriteBatchSize int64,
) error {
log := log.Ctx(ctx).With(
zap.String("clusterID", st.req.GetClusterID()),
zap.Int64("taskID", st.req.GetTaskID()),
zap.Int64("version", version),
zap.Int64("collectionID", st.req.GetCollectionID()),
zap.Int64("partitionID", st.req.GetPartitionID()),
zap.Int64("segmentID", st.req.GetSegmentID()),
zap.Any("statsJobType", st.req.GetSubJobType()),
zap.Int64("jsonKeyStatsDataFormat", jsonKeyStatsDataFormat),
zap.Int64("jsonStatsMaxShreddingColumns", jsonStatsMaxShreddingColumns),
zap.Float64("jsonStatsShreddingRatioThreshold", jsonStatsShreddingRatioThreshold),
zap.Int64("jsonStatsWriteBatchSize", jsonStatsWriteBatchSize),
)
if jsonKeyStatsDataFormat != 1 {
if jsonKeyStatsDataFormat != common.JSONStatsDataFormatVersion {
log.Info("create json key index failed dataformat invalid")
return nil
}
@ -562,18 +578,29 @@ func (st *statsTask) createJSONKeyStats(ctx context.Context,
req := proto.Clone(st.req).(*workerpb.CreateStatsRequest)
req.InsertLogs = insertBinlogs
buildIndexParams := buildIndexParams(req, files, field, newStorageConfig, tantivyMemory)
options := &BuildIndexOptions{
JsonStatsMaxShreddingColumns: jsonStatsMaxShreddingColumns,
JsonStatsShreddingRatio: jsonStatsShreddingRatioThreshold,
JsonStatsWriteBatchSize: jsonStatsWriteBatchSize,
}
buildIndexParams := buildIndexParams(req, files, field, newStorageConfig, options)
uploaded, err := indexcgowrapper.CreateJSONKeyStats(ctx, buildIndexParams)
if err != nil {
return err
}
memorySize := int64(0)
for _, file := range uploaded {
memorySize += file
}
jsonKeyIndexStats[field.GetFieldID()] = &datapb.JsonKeyStats{
FieldID: field.GetFieldID(),
Version: version,
BuildID: taskID,
Files: lo.Keys(uploaded),
JsonKeyStatsDataFormat: jsonKeyStatsDataFormat,
MemorySize: memorySize,
}
log.Info("field enable json key index, create json key index done",
zap.Int64("field id", field.GetFieldID()),
@ -603,20 +630,26 @@ func buildIndexParams(
files []string,
field *schemapb.FieldSchema,
storageConfig *indexcgopb.StorageConfig,
tantivyMemory int64,
options *BuildIndexOptions,
) *indexcgopb.BuildIndexInfo {
if options == nil {
options = &BuildIndexOptions{}
}
params := &indexcgopb.BuildIndexInfo{
BuildID: req.GetTaskID(),
CollectionID: req.GetCollectionID(),
PartitionID: req.GetPartitionID(),
SegmentID: req.GetTargetSegmentID(),
IndexVersion: req.GetTaskVersion(),
InsertFiles: files,
FieldSchema: field,
StorageConfig: storageConfig,
CurrentScalarIndexVersion: req.GetCurrentScalarIndexVersion(),
StorageVersion: req.GetStorageVersion(),
JsonKeyStatsTantivyMemory: tantivyMemory,
BuildID: req.GetTaskID(),
CollectionID: req.GetCollectionID(),
PartitionID: req.GetPartitionID(),
SegmentID: req.GetTargetSegmentID(),
IndexVersion: req.GetTaskVersion(),
InsertFiles: files,
FieldSchema: field,
StorageConfig: storageConfig,
CurrentScalarIndexVersion: req.GetCurrentScalarIndexVersion(),
StorageVersion: req.GetStorageVersion(),
JsonStatsMaxShreddingColumns: options.JsonStatsMaxShreddingColumns,
JsonStatsShreddingRatioThreshold: options.JsonStatsShreddingRatio,
JsonStatsWriteBatchSize: options.JsonStatsWriteBatchSize,
}
if req.GetStorageVersion() == storage.StorageV2 {

View File

@ -191,7 +191,13 @@ func (s *TaskStatsSuite) TestBuildIndexParams() {
StorageConfig: &indexpb.StorageConfig{RootPath: "/test/path"},
}
params := buildIndexParams(req, []string{"file1", "file2"}, nil, &indexcgopb.StorageConfig{}, 0)
options := &BuildIndexOptions{
TantivyMemory: 0,
JsonStatsMaxShreddingColumns: 256,
JsonStatsShreddingRatio: 0.3,
JsonStatsWriteBatchSize: 81920,
}
params := buildIndexParams(req, []string{"file1", "file2"}, nil, &indexcgopb.StorageConfig{}, options)
s.Equal(storage.StorageV2, params.StorageVersion)
s.NotNil(params.SegmentInsertFiles)

View File

@ -13,9 +13,11 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
planparserv2 "github.com/milvus-io/milvus/internal/parser/planparserv2/generated"
"github.com/milvus-io/milvus/internal/util/function/rerank"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/planpb"
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
@ -27,6 +29,23 @@ var (
}
)
type ExprParams struct {
UseJSONStats bool
}
func ParseExprParams(vals map[string]*schemapb.TemplateValue) *ExprParams {
ep := &ExprParams{
UseJSONStats: paramtable.Get().CommonCfg.UsingJSONStatsForQuery.GetAsBool(),
}
if vals != nil {
if v, ok := vals[common.ExprUseJSONStatsKey]; ok && v != nil {
ep.UseJSONStats = v.GetBoolVal()
}
}
return ep
}
func handleInternal(exprStr string) (ast planparserv2.IExprContext, err error) {
val, ok := exprCache.Get(exprStr)
if ok {
@ -149,12 +168,17 @@ func CreateRetrievePlan(schema *typeutil.SchemaHelper, exprStr string, exprTempl
return nil, err
}
exprParams := ParseExprParams(exprTemplateValues)
planNode := &planpb.PlanNode{
Node: &planpb.PlanNode_Query{
Query: &planpb.QueryPlanNode{
Predicates: expr,
},
},
PlanOptions: &planpb.PlanOption{
ExprUseJsonStats: exprParams.UseJSONStats,
},
}
return planNode, nil
}
@ -222,6 +246,8 @@ func CreateSearchPlan(schema *typeutil.SchemaHelper, exprStr string, vectorField
return nil, fmt.Errorf("don't support use segment scorer with group_by or search_iterator")
}
exprParams := ParseExprParams(exprTemplateValues)
planNode := &planpb.PlanNode{
Node: &planpb.PlanNode_VectorAnns{
VectorAnns: &planpb.VectorANNS{
@ -233,6 +259,9 @@ func CreateSearchPlan(schema *typeutil.SchemaHelper, exprStr string, vectorField
},
},
Scorers: scorers,
PlanOptions: &planpb.PlanOption{
ExprUseJsonStats: exprParams.UseJSONStats,
},
}
return planNode, nil
}

View File

@ -29,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/querycoordv2/session"
"github.com/milvus-io/milvus/internal/querycoordv2/task"
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
@ -247,17 +248,15 @@ func (c *IndexChecker) checkSegmentStats(segment *meta.Segment, schema *schemapb
for _, v := range loadField {
loadFieldMap[v] = struct{}{}
}
jsonStatsFieldMap := make(map[int64]struct{})
for _, v := range segment.JSONIndexField {
jsonStatsFieldMap[v] = struct{}{}
}
for _, field := range schema.GetFields() {
// Check if the field exists in both loadFieldMap and jsonStatsFieldMap
h := typeutil.CreateFieldSchemaHelper(field)
fieldID := field.GetFieldID()
if h.EnableJSONKeyStatsIndex() {
if _, ok := loadFieldMap[field.FieldID]; ok {
if _, ok := jsonStatsFieldMap[field.FieldID]; !ok {
result = append(result, field.FieldID)
if _, ok := loadFieldMap[fieldID]; ok {
if info, ok := segment.JSONStatsField[fieldID]; !ok || info.GetDataFormatVersion() < common.JSONStatsDataFormatVersion {
result = append(result, fieldID)
}
}
}

View File

@ -205,7 +205,7 @@ func (dh *distHandler) updateSegmentsDistribution(ctx context.Context, resp *que
Version: s.GetVersion(),
LastDeltaTimestamp: s.GetLastDeltaTimestamp(),
IndexInfo: s.GetIndexInfo(),
JSONIndexField: s.GetFieldJsonIndexStats(),
JSONStatsField: s.GetJsonStatsInfo(),
})
}

View File

@ -125,7 +125,7 @@ type Segment struct {
Version int64 // Version is the timestamp of loading segment
LastDeltaTimestamp uint64 // The timestamp of the last delta record
IndexInfo map[int64]*querypb.FieldIndexInfo // index info of loaded segment, indexID -> FieldIndexInfo
JSONIndexField []int64 // json index info of loaded segment
JSONStatsField map[int64]*querypb.JsonStatsInfo // json index info of loaded segment
}
func SegmentFromInfo(info *datapb.SegmentInfo) *Segment {

View File

@ -124,26 +124,6 @@ func (li *LoadIndexInfo) appendStorageInfo(uri string, version int64) {
}).Await()
}
// appendIndexData appends index path to cLoadIndexInfo and create index
func (li *LoadIndexInfo) appendIndexData(ctx context.Context, indexKeys []string) error {
for _, indexPath := range indexKeys {
err := li.appendIndexFile(ctx, indexPath)
if err != nil {
return err
}
}
var status C.CStatus
GetLoadPool().Submit(func() (any, error) {
traceCtx := ParseCTraceContext(ctx)
status = C.AppendIndexV2(traceCtx.ctx, li.cLoadIndexInfo)
runtime.KeepAlive(traceCtx)
return nil, nil
}).Await()
return HandleCStatus(ctx, &status, "AppendIndex failed")
}
func (li *LoadIndexInfo) appendIndexEngineVersion(ctx context.Context, indexEngineVersion int32) error {
cIndexEngineVersion := C.int32_t(indexEngineVersion)

View File

@ -409,19 +409,19 @@ func (_c *MockSegment_GetBM25Stats_Call) RunAndReturn(run func() map[int64]*stor
}
// GetFieldJSONIndexStats provides a mock function with no fields
func (_m *MockSegment) GetFieldJSONIndexStats() []int64 {
func (_m *MockSegment) GetFieldJSONIndexStats() map[int64]*querypb.JsonStatsInfo {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for GetFieldJSONIndexStats")
}
var r0 []int64
if rf, ok := ret.Get(0).(func() []int64); ok {
var r0 map[int64]*querypb.JsonStatsInfo
if rf, ok := ret.Get(0).(func() map[int64]*querypb.JsonStatsInfo); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).([]int64)
r0 = ret.Get(0).(map[int64]*querypb.JsonStatsInfo)
}
}

View File

@ -295,7 +295,7 @@ type LocalSegment struct {
lastDeltaTimestamp *atomic.Uint64
fields *typeutil.ConcurrentMap[int64, *FieldInfo]
fieldIndexes *typeutil.ConcurrentMap[int64, *IndexedFieldInfo] // indexID -> IndexedFieldInfo
fieldJSONStats []int64
fieldJSONStats map[int64]*querypb.JsonStatsInfo
}
func NewSegment(ctx context.Context,
@ -362,6 +362,7 @@ func NewSegment(ctx context.Context,
lastDeltaTimestamp: atomic.NewUint64(0),
fields: typeutil.NewConcurrentMap[int64, *FieldInfo](),
fieldIndexes: typeutil.NewConcurrentMap[int64, *IndexedFieldInfo](),
fieldJSONStats: make(map[int64]*querypb.JsonStatsInfo),
memSize: atomic.NewInt64(-1),
rowNum: atomic.NewInt64(-1),
@ -1132,29 +1133,27 @@ func (s *LocalSegment) LoadJSONKeyIndex(ctx context.Context, jsonKeyStats *datap
}
defer s.ptrLock.Unpin()
if jsonKeyStats.GetJsonKeyStatsDataFormat() == 0 {
if !paramtable.Get().CommonCfg.EnabledJSONKeyStats.GetAsBool() {
log.Ctx(ctx).Warn("load json key index failed, json key stats is not enabled")
return nil
}
if jsonKeyStats.GetJsonKeyStatsDataFormat() != common.JSONStatsDataFormatVersion {
log.Ctx(ctx).Info("load json key index failed dataformat invalid", zap.Int64("dataformat", jsonKeyStats.GetJsonKeyStatsDataFormat()), zap.Int64("field id", jsonKeyStats.GetFieldID()), zap.Any("json key logs", jsonKeyStats))
return nil
}
log.Ctx(ctx).Info("load json key index", zap.Int64("field id", jsonKeyStats.GetFieldID()), zap.Any("json key logs", jsonKeyStats))
exists := false
for _, field := range s.fieldJSONStats {
if field == jsonKeyStats.GetFieldID() {
exists = true
break
}
}
if exists {
log.Warn("JsonKeyIndexStats already loaded")
if info, ok := s.fieldJSONStats[jsonKeyStats.GetFieldID()]; ok && info.GetDataFormatVersion() >= common.JSONStatsDataFormatVersion {
log.Warn("JsonKeyIndexStats already loaded", zap.Int64("field id", jsonKeyStats.GetFieldID()), zap.Any("json key logs", jsonKeyStats))
return nil
}
f, err := schemaHelper.GetFieldFromID(jsonKeyStats.GetFieldID())
if err != nil {
return err
}
// Json key stats index mmap config is based on the raw data mmap.
enableMmap := isDataMmapEnable(f)
cgoProto := &indexcgopb.LoadJsonKeyIndexInfo{
FieldID: jsonKeyStats.GetFieldID(),
Version: jsonKeyStats.GetVersion(),
@ -1164,7 +1163,8 @@ func (s *LocalSegment) LoadJSONKeyIndex(ctx context.Context, jsonKeyStats *datap
CollectionID: s.Collection(),
PartitionID: s.Partition(),
LoadPriority: s.loadInfo.Load().GetPriority(),
EnableMmap: enableMmap,
EnableMmap: paramtable.Get().QueryNodeCfg.MmapJSONStats.GetAsBool(),
MmapDirPath: paramtable.Get().QueryNodeCfg.MmapDirPath.GetValue(),
}
marshaled, err := proto.Marshal(cgoProto)
@ -1178,7 +1178,13 @@ func (s *LocalSegment) LoadJSONKeyIndex(ctx context.Context, jsonKeyStats *datap
status = C.LoadJsonKeyIndex(traceCtx.ctx, s.ptr, (*C.uint8_t)(unsafe.Pointer(&marshaled[0])), (C.uint64_t)(len(marshaled)))
return nil, nil
}).Await()
s.fieldJSONStats = append(s.fieldJSONStats, jsonKeyStats.GetFieldID())
s.fieldJSONStats[jsonKeyStats.GetFieldID()] = &querypb.JsonStatsInfo{
FieldID: jsonKeyStats.GetFieldID(),
DataFormatVersion: jsonKeyStats.GetJsonKeyStatsDataFormat(),
BuildID: jsonKeyStats.GetBuildID(),
VersionID: jsonKeyStats.GetVersion(),
}
return HandleCStatus(ctx, &status, "Load JsonKeyStats failed")
}
@ -1395,6 +1401,6 @@ func (s *LocalSegment) indexNeedLoadRawData(schema *schemapb.CollectionSchema, i
return !typeutil.IsVectorType(fieldSchema.DataType) && s.HasRawData(indexInfo.IndexInfo.FieldID), nil
}
func (s *LocalSegment) GetFieldJSONIndexStats() []int64 {
func (s *LocalSegment) GetFieldJSONIndexStats() map[int64]*querypb.JsonStatsInfo {
return s.fieldJSONStats
}

View File

@ -106,5 +106,5 @@ type Segment interface {
NeedUpdatedVersion() int64
RemoveUnusedFieldFiles() error
GetFieldJSONIndexStats() []int64
GetFieldJSONIndexStats() map[int64]*querypb.JsonStatsInfo
}

View File

@ -197,6 +197,6 @@ func (s *L0Segment) RemoveUnusedFieldFiles() error {
panic("not implemented")
}
func (s *L0Segment) GetFieldJSONIndexStats() []int64 {
func (s *L0Segment) GetFieldJSONIndexStats() map[int64]*querypb.JsonStatsInfo {
return nil
}

View File

@ -303,11 +303,9 @@ func (node *QueryNode) InitSegcore() error {
cOptimizeExprEnabled := C.bool(paramtable.Get().CommonCfg.EnabledOptimizeExpr.GetAsBool())
C.SetDefaultOptimizeExprEnable(cOptimizeExprEnabled)
cJSONKeyStatsCommitInterval := C.int64_t(paramtable.Get().QueryNodeCfg.JSONKeyStatsCommitInterval.GetAsInt64())
C.SetDefaultJSONKeyStatsCommitInterval(cJSONKeyStatsCommitInterval)
cGrowingJSONKeyStatsEnabled := C.bool(paramtable.Get().CommonCfg.EnabledGrowingSegmentJSONKeyStats.GetAsBool())
C.SetDefaultGrowingJSONKeyStatsEnable(cGrowingJSONKeyStatsEnabled)
cGpuMemoryPoolInitSize := C.uint32_t(paramtable.Get().GpuConfig.InitSize.GetAsUint32())
cGpuMemoryPoolMaxSize := C.uint32_t(paramtable.Get().GpuConfig.MaxSize.GetAsUint32())
C.SegcoreSetKnowhereGpuMemoryPoolSize(cGpuMemoryPoolInitSize, cGpuMemoryPoolMaxSize)

View File

@ -1264,7 +1264,7 @@ func (node *QueryNode) GetDataDistribution(ctx context.Context, req *querypb.Get
IndexInfo: lo.SliceToMap(s.Indexes(), func(info *segments.IndexedFieldInfo) (int64, *querypb.FieldIndexInfo) {
return info.IndexInfo.IndexID, info.IndexInfo
}),
FieldJsonIndexStats: s.GetFieldJSONIndexStats(),
JsonStatsInfo: s.GetFieldJSONIndexStats(),
})
}

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