enhance: full-support for wildcard pattern matching (#30288)

issue: #29988 
This pr adds full-support for wildcard pattern matching from end to end.
Before this pr, the users can only use prefix match in their expression,
for example, "like 'prefix%'". With this pr, more flexible syntax can be
combined.

To do so, this pr makes these changes:
- 1. support regex query both on index and raw data;
- 2. translate the pattern matching to regex query, so that it can be
handled by the regex query logic;
- 3. loose the limit of the expression parsing, which allows general
pattern matching syntax;

With the support of regex query in segcore backend, we can also add
mysql-like `REGEXP` syntax later easily.

---------

Signed-off-by: longjiquan <jiquan.long@zilliz.com>
This commit is contained in:
Jiquan Long 2024-02-01 12:37:04 +08:00 committed by GitHub
parent 46d9ac5931
commit e549148a19
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
28 changed files with 903 additions and 118 deletions

View File

@ -23,7 +23,8 @@ set(COMMON_SRC
IndexMeta.cpp
EasyAssert.cpp
FieldData.cpp
)
RegexQuery.cpp
)
add_library(milvus_common SHARED ${COMMON_SRC})
@ -36,6 +37,7 @@ target_link_libraries(milvus_common
simdjson
opendal
${CONAN_LIBS}
re2
)
install(TARGETS milvus_common DESTINATION "${CMAKE_INSTALL_LIBDIR}")

View File

@ -0,0 +1,58 @@
// 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 <re2/re2.h>
#include <regex>
#include "common/RegexQuery.h"
namespace milvus {
std::string
ReplaceUnescapedChars(const std::string& input,
char src,
const std::string& replacement) {
std::string result;
bool escapeMode = false;
for (char c : input) {
if (escapeMode) {
result += '\\';
result += c;
escapeMode = false;
} else {
if (c == '\\') {
escapeMode = true;
} else if (c == src) {
result += replacement;
} else {
result += c;
}
}
}
return result;
}
std::string
TranslatePatternMatchToRegex(const std::string& pattern) {
std::string regex_pattern;
#if 0
regex_pattern = R"([\.\*\+\?\|\(\)\[\]\{\}\\])";
#else
regex_pattern = R"([\.\*\+\?\|\(\)\[\]\{\}])";
#endif
std::string regex =
std::regex_replace(pattern, std::regex(regex_pattern), R"(\$&)");
regex = ReplaceUnescapedChars(regex, '%', ".*");
regex = ReplaceUnescapedChars(regex, '_', ".");
return regex;
}
} // namespace milvus

View File

@ -0,0 +1,22 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <string>
namespace milvus {
std::string
ReplaceUnescapedChars(const std::string& input,
char src,
const std::string& replacement);
std::string
TranslatePatternMatchToRegex(const std::string& pattern);
} // namespace milvus

View File

@ -279,6 +279,34 @@ class SegmentExpr : public Expr {
return result;
}
template <typename T>
bool
CanUseIndex(OpType op) const {
typedef std::
conditional_t<std::is_same_v<T, std::string_view>, std::string, T>
IndexInnerType;
if constexpr (!std::is_same_v<IndexInnerType, std::string>) {
return true;
}
using Index = index::ScalarIndex<IndexInnerType>;
if (op == OpType::Match) {
for (size_t i = current_index_chunk_; i < num_index_chunk_; i++) {
const Index& index =
segment_->chunk_scalar_index<IndexInnerType>(field_id_, i);
// 1, index support regex query, then index handles the query;
// 2, index has raw data, then call index.Reverse_Lookup to handle the query;
if (!index.SupportRegexQuery() && !index.HasRawData()) {
return false;
}
// all chunks have same index.
return true;
}
}
return true;
}
protected:
const segcore::SegmentInternalInterface* segment_;
const FieldId field_id_;

View File

@ -353,7 +353,7 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplJson() {
template <typename T>
VectorPtr
PhyUnaryRangeFilterExpr::ExecRangeVisitorImpl() {
if (is_index_mode_) {
if (CanUseIndex<T>()) {
return ExecRangeVisitorImplForIndex<T>();
} else {
return ExecRangeVisitorImplForData<T>();
@ -414,6 +414,11 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplForIndex() {
res = std::move(func(index_ptr, val));
break;
}
case proto::plan::Match: {
UnaryIndexFunc<T, proto::plan::Match> func;
res = std::move(func(index_ptr, val));
break;
}
default:
PanicInfo(
OpTypeInvalid,
@ -567,6 +572,11 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplForData() {
func(data, size, val, res);
break;
}
case proto::plan::Match: {
UnaryElementFunc<T, proto::plan::Match> func;
func(data, size, val, res);
break;
}
default:
PanicInfo(
OpTypeInvalid,
@ -595,5 +605,14 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplForData() {
return res_vec;
}
template <typename T>
bool
PhyUnaryRangeFilterExpr::CanUseIndex() const {
if (!is_index_mode_) {
return false;
}
return SegmentExpr::CanUseIndex<T>(expr_->op_type_);
}
} // namespace exec
} // namespace milvus

View File

@ -18,6 +18,8 @@
#include <fmt/core.h>
#include <utility>
#include "common/EasyAssert.h"
#include "common/Types.h"
#include "common/Vector.h"
@ -25,10 +27,37 @@
#include "index/Meta.h"
#include "segcore/SegmentInterface.h"
#include "query/Utils.h"
#include "common/RegexQuery.h"
namespace milvus {
namespace exec {
template <typename T>
struct UnaryElementFuncForMatch {
typedef std::
conditional_t<std::is_same_v<T, std::string_view>, std::string, T>
IndexInnerType;
void
operator()(const T* src, size_t size, IndexInnerType val, bool* res) {
if constexpr (std::is_same_v<T, std::string_view>) {
// translate the pattern match in advance, which avoid computing it every loop.
std::regex reg(TranslatePatternMatchToRegex(val));
for (int i = 0; i < size; ++i) {
res[i] = std::regex_match(src[i].data(), reg);
}
} else if constexpr (std::is_same_v<T, std::string>) {
// translate the pattern match in advance, which avoid computing it every loop.
std::regex reg(TranslatePatternMatchToRegex(val));
for (int i = 0; i < size; ++i) {
res[i] = std::regex_match(src[i], reg);
}
} else {
PanicInfo(Unsupported, "regex query is only supported on string");
}
}
};
template <typename T, proto::plan::OpType op>
struct UnaryElementFunc {
typedef std::
@ -36,6 +65,12 @@ struct UnaryElementFunc {
IndexInnerType;
void
operator()(const T* src, size_t size, IndexInnerType val, bool* res) {
if constexpr (op == proto::plan::OpType::Match) {
UnaryElementFuncForMatch<T> func;
func(src, size, val, res);
return;
}
for (int i = 0; i < size; ++i) {
if constexpr (op == proto::plan::OpType::Equal) {
res[i] = src[i] == val;
@ -130,6 +165,41 @@ struct UnaryElementFuncForArray {
}
};
template <typename T>
struct UnaryIndexFuncForMatch {
typedef std::
conditional_t<std::is_same_v<T, std::string_view>, std::string, T>
IndexInnerType;
using Index = index::ScalarIndex<IndexInnerType>;
FixedVector<bool>
operator()(Index* index, IndexInnerType val) {
if constexpr (!std::is_same_v<T, std::string_view> &&
!std::is_same_v<T, std::string>) {
PanicInfo(Unsupported, "regex query is only supported on string");
} else {
auto reg = TranslatePatternMatchToRegex(val);
if (index->SupportRegexQuery()) {
return index->RegexQuery(reg);
}
if (!index->HasRawData()) {
PanicInfo(Unsupported,
"index don't support regex query and don't have "
"raw data");
}
// retrieve raw data to do brute force query, may be very slow.
auto cnt = index->Count();
std::regex r(reg);
TargetBitmap res(cnt);
for (int64_t i = 0; i < cnt; i++) {
auto raw = index->Reverse_Lookup(i);
res[i] = std::regex_match(raw, r);
}
return res;
}
}
};
template <typename T, proto::plan::OpType op>
struct UnaryIndexFunc {
typedef std::
@ -156,6 +226,9 @@ struct UnaryIndexFunc {
proto::plan::OpType::PrefixMatch);
dataset->Set(milvus::index::PREFIX_VALUE, val);
return index->Query(std::move(dataset));
} else if constexpr (op == proto::plan::OpType::Match) {
UnaryIndexFuncForMatch<T> func;
return func(index, val);
} else {
PanicInfo(
OpTypeInvalid,
@ -211,6 +284,10 @@ class PhyUnaryRangeFilterExpr : public SegmentExpr {
ColumnVectorPtr
PreCheckOverflow();
template <typename T>
bool
CanUseIndex() const;
private:
std::shared_ptr<const milvus::expr::UnaryRangeFilterExpr> expr_;
ColumnVectorPtr cached_overflow_res_{nullptr};

View File

@ -18,6 +18,9 @@
#include "storage/Util.h"
#include <boost/filesystem.hpp>
#include <boost/uuid/random_generator.hpp>
#include <boost/uuid/uuid_io.hpp>
#include "InvertedIndexTantivy.h"
namespace milvus::index {
template <typename T>
@ -412,6 +415,42 @@ InvertedIndexTantivy<std::string>::Query(const DatasetPtr& dataset) {
return ScalarIndex<std::string>::Query(dataset);
}
template <typename T>
const TargetBitmap
InvertedIndexTantivy<T>::RegexQuery(const std::string& pattern) {
TargetBitmap bitset(Count());
auto array = wrapper_->regex_query(pattern);
apply_hits(bitset, array, true);
return bitset;
}
template <typename T>
void
InvertedIndexTantivy<T>::BuildWithRawData(size_t n,
const void* values,
const Config& config) {
if constexpr (!std::is_same_v<T, std::string>) {
PanicInfo(Unsupported,
"InvertedIndex.BuildWithRawData only support string");
} else {
boost::uuids::random_generator generator;
auto uuid = generator();
auto prefix = boost::uuids::to_string(uuid);
path_ = fmt::format("/tmp/{}", prefix);
boost::filesystem::create_directories(path_);
cfg_ = TantivyConfig{
.data_type_ = DataType::VARCHAR,
};
d_type_ = cfg_.to_tantivy_data_type();
std::string field = "test_inverted_index";
wrapper_ = std::make_shared<TantivyIndexWrapper>(
field.c_str(), d_type_, path_.c_str());
wrapper_->add_data<std::string>(static_cast<const std::string*>(values),
n);
finish();
}
}
template class InvertedIndexTantivy<bool>;
template class InvertedIndexTantivy<int8_t>;
template class InvertedIndexTantivy<int16_t>;

View File

@ -34,6 +34,8 @@ class InvertedIndexTantivy : public ScalarIndex<T> {
using DiskFileManager = storage::DiskFileManagerImpl;
using DiskFileManagerPtr = std::shared_ptr<DiskFileManager>;
InvertedIndexTantivy() = default;
explicit InvertedIndexTantivy(const TantivyConfig& cfg,
const storage::FileManagerContext& ctx)
: InvertedIndexTantivy(cfg, ctx, nullptr) {
@ -82,17 +84,11 @@ class InvertedIndexTantivy : public ScalarIndex<T> {
return wrapper_->count();
}
/*
* deprecated.
* TODO: why not remove this?
*/
// BuildWithRawData should be only used in ut. Only string is supported.
void
BuildWithRawData(size_t n,
const void* values,
const Config& config = {}) override {
PanicInfo(ErrorCode::NotImplemented,
"BuildWithRawData should be deprecated");
}
const Config& config = {}) override;
/*
* deprecated.
@ -152,6 +148,14 @@ class InvertedIndexTantivy : public ScalarIndex<T> {
const TargetBitmap
Query(const DatasetPtr& dataset) override;
bool
SupportRegexQuery() const override {
return true;
}
const TargetBitmap
RegexQuery(const std::string& pattern) override;
private:
void
finish();

View File

@ -70,6 +70,16 @@ class ScalarIndex : public IndexBase {
virtual int64_t
Size() = 0;
virtual bool
SupportRegexQuery() const {
return false;
}
virtual const TargetBitmap
RegexQuery(const std::string& pattern) {
PanicInfo(Unsupported, "regex query is not supported");
}
};
template <typename T>

View File

@ -73,6 +73,8 @@ RustArray tantivy_range_query_keyword(void *ptr,
RustArray tantivy_prefix_query_keyword(void *ptr, const char *prefix);
RustArray tantivy_regex_query(void *ptr, const char *pattern);
void *tantivy_create_index(const char *field_name, TantivyDataType data_type, const char *path);
void tantivy_free_index_writer(void *ptr);

View File

@ -187,6 +187,10 @@ impl IndexReaderWrapper {
pub fn prefix_query_keyword(&self, prefix: &str) -> Vec<u32> {
let pattern = format!("{}(.|\n)*", prefix);
self.regex_query(&pattern)
}
pub fn regex_query(&self, pattern: &str) -> Vec<u32> {
let q = RegexQuery::from_pattern(&pattern, self.field).unwrap();
self.search(&q)
}

View File

@ -210,3 +210,16 @@ pub extern "C" fn tantivy_prefix_query_keyword(
RustArray::from_vec(hits)
}
}
#[no_mangle]
pub extern "C" fn tantivy_regex_query(
ptr: *mut c_void,
pattern: *const c_char,
) -> RustArray {
let real = ptr as *mut IndexReaderWrapper;
unsafe {
let c_str = CStr::from_ptr(pattern);
let hits = (*real).regex_query(c_str.to_str().unwrap());
RustArray::from_vec(hits)
}
}

View File

@ -336,6 +336,12 @@ struct TantivyIndexWrapper {
return RustArrayWrapper(array);
}
RustArrayWrapper
regex_query(const std::string& pattern) {
auto array = tantivy_regex_query(reader_, pattern.c_str());
return RustArrayWrapper(array);
}
public:
inline IndexWriter
get_writer() {

View File

@ -145,6 +145,11 @@ run<std::string>() {
auto hits = w.prefix_query("a");
hits.debug();
}
{
auto hits = w.regex_query("a(.|\n)*");
hits.debug();
}
}
int

View File

@ -62,6 +62,8 @@ set(MILVUS_TEST_FILES
test_exec.cpp
test_inverted_index.cpp
test_group_by.cpp
test_regex_query_util.cpp
test_regex_query.cpp
)
if ( BUILD_DISK_ANN STREQUAL "ON" )

View File

@ -130,8 +130,12 @@ class TestFloatSearchBruteForce : public ::testing::Test {
// ASSERT_ANY_THROW(BruteForceSearch(dataset, base.data(), nb, bitset_view));
return;
}
auto result = BruteForceSearch(
dataset, base.data(), nb, knowhere::Json(), bitset_view, DataType::VECTOR_FLOAT);
auto result = BruteForceSearch(dataset,
base.data(),
nb,
knowhere::Json(),
bitset_view,
DataType::VECTOR_FLOAT);
for (int i = 0; i < nq; i++) {
auto ref = Ref(base.data(),
query.data() + i * dim,

View File

@ -398,7 +398,8 @@ TEST_F(DiskAnnFileManagerTest, CacheOptFieldToDiskOptFieldMoreThanOne) {
opt_fields[kOptFieldId + 1] = {
kOptFieldName + "second", kOptFiledType, {insert_file_space_path}};
EXPECT_THROW(file_manager->CacheOptFieldToDisk(opt_fields), SegcoreError);
EXPECT_THROW(file_manager->CacheOptFieldToDisk(space, opt_fields), SegcoreError);
EXPECT_THROW(file_manager->CacheOptFieldToDisk(space, opt_fields),
SegcoreError);
}
TEST_F(DiskAnnFileManagerTest, CacheOptFieldToDiskCorrect) {

View File

@ -93,7 +93,8 @@ TEST(Float16VecIndex, All) {
ok = google::protobuf::TextFormat::PrintToString(index_params,
&index_params_str);
assert(ok);
auto dataset = GenDatasetWithDataType(NB, metric_type, milvus::DataType::VECTOR_FLOAT16);
auto dataset = GenDatasetWithDataType(
NB, metric_type, milvus::DataType::VECTOR_FLOAT16);
auto xb_data = dataset.get_col<uint8_t>(milvus::FieldId(100));
CDataType dtype = Float16Vector;
@ -151,7 +152,8 @@ TEST(BFloat16VecIndex, All) {
ok = google::protobuf::TextFormat::PrintToString(index_params,
&index_params_str);
assert(ok);
auto dataset = GenDatasetWithDataType(NB, metric_type, milvus::DataType::VECTOR_BFLOAT16);
auto dataset = GenDatasetWithDataType(
NB, metric_type, milvus::DataType::VECTOR_BFLOAT16);
auto xb_data = dataset.get_col<uint8_t>(milvus::FieldId(100));
CDataType dtype = BFloat16Vector;

View File

@ -494,6 +494,16 @@ test_string() {
ASSERT_EQ(bitset[i], boost::starts_with(data[i], prefix));
}
}
{
ASSERT_TRUE(real_index->SupportRegexQuery());
auto prefix = data[0];
auto bitset = real_index->RegexQuery(prefix + "(.|\n)*");
ASSERT_EQ(cnt, bitset.size());
for (size_t i = 0; i < bitset.size(); i++) {
ASSERT_EQ(bitset[i], boost::starts_with(data[i], prefix));
}
}
}
}

View File

@ -0,0 +1,416 @@
// 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 <boost/format.hpp>
#include <regex>
#include "pb/plan.pb.h"
#include "segcore/segcore_init_c.h"
#include "segcore/SegmentSealed.h"
#include "segcore/SegmentSealedImpl.h"
#include "segcore/SegmentGrowing.h"
#include "segcore/SegmentGrowingImpl.h"
#include "pb/schema.pb.h"
#include "test_utils/DataGen.h"
#include "index/IndexFactory.h"
#include "query/Plan.h"
#include "knowhere/comp/brute_force.h"
#include "test_utils/GenExprProto.h"
#include "query/PlanProto.h"
#include "query/generated/ExecPlanNodeVisitor.h"
#include "index/InvertedIndexTantivy.h"
using namespace milvus;
using namespace milvus::query;
using namespace milvus::segcore;
SchemaPtr
GenTestSchema() {
auto schema = std::make_shared<Schema>();
schema->AddDebugField("str", DataType::VARCHAR);
schema->AddDebugField("another_str", DataType::VARCHAR);
schema->AddDebugField(
"fvec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto pk = schema->AddDebugField("int64", DataType::INT64);
schema->set_primary_field_id(pk);
schema->AddDebugField("another_int64", DataType::INT64);
return schema;
}
class GrowingSegmentRegexQueryTest : public ::testing::Test {
public:
void
SetUp() override {
schema = GenTestSchema();
seg = CreateGrowingSegment(schema, empty_index_meta);
raw_str = {
"b",
"a",
"aaa",
"abbb",
"abcabcabc",
};
N = 5;
uint64_t seed = 19190504;
auto raw_data = DataGen(schema, N, seed);
auto str_col = raw_data.raw_->mutable_fields_data()
->at(0)
.mutable_scalars()
->mutable_string_data()
->mutable_data();
for (int64_t i = 0; i < N; i++) {
str_col->at(i) = raw_str[i];
}
seg->PreInsert(N);
seg->Insert(0,
N,
raw_data.row_ids_.data(),
raw_data.timestamps_.data(),
raw_data.raw_);
}
void
TearDown() override {
}
public:
SchemaPtr schema;
SegmentGrowingPtr seg;
int64_t N;
std::vector<std::string> raw_str;
};
TEST_F(GrowingSegmentRegexQueryTest, RegexQueryOnNonStringField) {
int64_t operand = 120;
const auto& int_meta = schema->operator[](FieldName("int64"));
auto column_info = test::GenColumnInfo(
int_meta.get_id().get(), proto::schema::DataType::Int64, false, false);
auto unary_range_expr = test::GenUnaryRangeExpr(OpType::Match, operand);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = test::GenExpr();
expr->set_allocated_unary_range_expr(unary_range_expr);
auto parser = ProtoParser(*schema);
auto typed_expr = parser.ParseExprs(*expr);
auto parsed =
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, typed_expr);
auto segpromote = dynamic_cast<SegmentGrowingImpl*>(seg.get());
query::ExecPlanNodeVisitor visitor(*segpromote, MAX_TIMESTAMP);
BitsetType final;
ASSERT_ANY_THROW(
visitor.ExecuteExprNode(parsed, segpromote, N, final));
}
TEST_F(GrowingSegmentRegexQueryTest, RegexQueryOnStringField) {
std::string operand = "a%";
const auto& str_meta = schema->operator[](FieldName("str"));
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto unary_range_expr = test::GenUnaryRangeExpr(OpType::Match, operand);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = test::GenExpr();
expr->set_allocated_unary_range_expr(unary_range_expr);
auto parser = ProtoParser(*schema);
auto typed_expr = parser.ParseExprs(*expr);
auto parsed =
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, typed_expr);
auto segpromote = dynamic_cast<SegmentGrowingImpl*>(seg.get());
query::ExecPlanNodeVisitor visitor(*segpromote, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(parsed, segpromote, N, final);
ASSERT_FALSE(final[0]);
ASSERT_TRUE(final[1]);
ASSERT_TRUE(final[2]);
ASSERT_TRUE(final[3]);
ASSERT_TRUE(final[4]);
}
struct MockStringIndex : index::StringIndexSort {
const bool
HasRawData() const override {
return false;
}
bool
SupportRegexQuery() const override {
return false;
}
};
class SealedSegmentRegexQueryTest : public ::testing::Test {
public:
void
SetUp() override {
schema = GenTestSchema();
seg = CreateSealedSegment(schema);
raw_str = {
"b",
"a",
"aaa",
"abbb",
"abcabcabc",
};
N = 5;
uint64_t seed = 19190504;
auto raw_data = DataGen(schema, N, seed);
auto str_col = raw_data.raw_->mutable_fields_data()
->at(0)
.mutable_scalars()
->mutable_string_data()
->mutable_data();
auto int_col = raw_data.get_col<int64_t>(
schema->get_field_id(FieldName("another_int64")));
raw_int.assign(int_col.begin(), int_col.end());
for (int64_t i = 0; i < N; i++) {
str_col->at(i) = raw_str[i];
}
SealedLoadFieldData(raw_data, *seg);
}
void
TearDown() override {
}
void
LoadStlSortIndex() {
{
proto::schema::StringArray arr;
for (int64_t i = 0; i < N; i++) {
*(arr.mutable_data()->Add()) = raw_str[i];
}
auto index = index::CreateStringIndexSort();
std::vector<uint8_t> buffer(arr.ByteSize());
ASSERT_TRUE(arr.SerializeToArray(buffer.data(), arr.ByteSize()));
index->BuildWithRawData(arr.ByteSize(), buffer.data());
LoadIndexInfo info{
.field_id = schema->get_field_id(FieldName("str")).get(),
.index = std::move(index),
};
seg->LoadIndex(info);
}
{
auto index = index::CreateScalarIndexSort<int64_t>();
index->BuildWithRawData(N, raw_int.data());
LoadIndexInfo info{
.field_id =
schema->get_field_id(FieldName("another_int64")).get(),
.index = std::move(index),
};
seg->LoadIndex(info);
}
}
void
LoadInvertedIndex() {
auto index =
std::make_unique<index::InvertedIndexTantivy<std::string>>();
index->BuildWithRawData(N, raw_str.data());
LoadIndexInfo info{
.field_id = schema->get_field_id(FieldName("str")).get(),
.index = std::move(index),
};
seg->LoadIndex(info);
}
void
LoadMockIndex() {
proto::schema::StringArray arr;
for (int64_t i = 0; i < N; i++) {
*(arr.mutable_data()->Add()) = raw_str[i];
}
auto index = std::make_unique<MockStringIndex>();
std::vector<uint8_t> buffer(arr.ByteSize());
ASSERT_TRUE(arr.SerializeToArray(buffer.data(), arr.ByteSize()));
index->BuildWithRawData(arr.ByteSize(), buffer.data());
LoadIndexInfo info{
.field_id = schema->get_field_id(FieldName("str")).get(),
.index = std::move(index),
};
seg->LoadIndex(info);
}
public:
SchemaPtr schema;
SegmentSealedUPtr seg;
int64_t N;
std::vector<std::string> raw_str;
std::vector<int64_t> raw_int;
};
TEST_F(SealedSegmentRegexQueryTest, BFRegexQueryOnNonStringField) {
int64_t operand = 120;
const auto& int_meta = schema->operator[](FieldName("another_int64"));
auto column_info = test::GenColumnInfo(
int_meta.get_id().get(), proto::schema::DataType::Int64, false, false);
auto unary_range_expr = test::GenUnaryRangeExpr(OpType::Match, operand);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = test::GenExpr();
expr->set_allocated_unary_range_expr(unary_range_expr);
auto parser = ProtoParser(*schema);
auto typed_expr = parser.ParseExprs(*expr);
auto parsed =
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, typed_expr);
auto segpromote = dynamic_cast<SegmentSealedImpl*>(seg.get());
query::ExecPlanNodeVisitor visitor(*segpromote, MAX_TIMESTAMP);
BitsetType final;
ASSERT_ANY_THROW(
visitor.ExecuteExprNode(parsed, segpromote, N, final));
}
TEST_F(SealedSegmentRegexQueryTest, BFRegexQueryOnStringField) {
std::string operand = "a%";
const auto& str_meta = schema->operator[](FieldName("str"));
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto unary_range_expr = test::GenUnaryRangeExpr(OpType::Match, operand);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = test::GenExpr();
expr->set_allocated_unary_range_expr(unary_range_expr);
auto parser = ProtoParser(*schema);
auto typed_expr = parser.ParseExprs(*expr);
auto parsed =
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, typed_expr);
auto segpromote = dynamic_cast<SegmentSealedImpl*>(seg.get());
query::ExecPlanNodeVisitor visitor(*segpromote, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(parsed, segpromote, N, final);
ASSERT_FALSE(final[0]);
ASSERT_TRUE(final[1]);
ASSERT_TRUE(final[2]);
ASSERT_TRUE(final[3]);
ASSERT_TRUE(final[4]);
}
TEST_F(SealedSegmentRegexQueryTest, RegexQueryOnIndexedNonStringField) {
int64_t operand = 120;
const auto& int_meta = schema->operator[](FieldName("another_int64"));
auto column_info = test::GenColumnInfo(
int_meta.get_id().get(), proto::schema::DataType::Int64, false, false);
auto unary_range_expr = test::GenUnaryRangeExpr(OpType::Match, operand);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = test::GenExpr();
expr->set_allocated_unary_range_expr(unary_range_expr);
auto parser = ProtoParser(*schema);
auto typed_expr = parser.ParseExprs(*expr);
auto parsed =
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, typed_expr);
LoadStlSortIndex();
auto segpromote = dynamic_cast<SegmentSealedImpl*>(seg.get());
query::ExecPlanNodeVisitor visitor(*segpromote, MAX_TIMESTAMP);
BitsetType final;
ASSERT_ANY_THROW(
visitor.ExecuteExprNode(parsed, segpromote, N, final));
}
TEST_F(SealedSegmentRegexQueryTest, RegexQueryOnStlSortStringField) {
std::string operand = "a%";
const auto& str_meta = schema->operator[](FieldName("str"));
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto unary_range_expr = test::GenUnaryRangeExpr(OpType::Match, operand);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = test::GenExpr();
expr->set_allocated_unary_range_expr(unary_range_expr);
auto parser = ProtoParser(*schema);
auto typed_expr = parser.ParseExprs(*expr);
auto parsed =
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, typed_expr);
LoadStlSortIndex();
auto segpromote = dynamic_cast<SegmentSealedImpl*>(seg.get());
query::ExecPlanNodeVisitor visitor(*segpromote, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(parsed, segpromote, N, final);
ASSERT_FALSE(final[0]);
ASSERT_TRUE(final[1]);
ASSERT_TRUE(final[2]);
ASSERT_TRUE(final[3]);
ASSERT_TRUE(final[4]);
}
TEST_F(SealedSegmentRegexQueryTest, RegexQueryOnInvertedIndexStringField) {
std::string operand = "a%";
const auto& str_meta = schema->operator[](FieldName("str"));
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto unary_range_expr = test::GenUnaryRangeExpr(OpType::Match, operand);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = test::GenExpr();
expr->set_allocated_unary_range_expr(unary_range_expr);
auto parser = ProtoParser(*schema);
auto typed_expr = parser.ParseExprs(*expr);
auto parsed =
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, typed_expr);
LoadInvertedIndex();
auto segpromote = dynamic_cast<SegmentSealedImpl*>(seg.get());
query::ExecPlanNodeVisitor visitor(*segpromote, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(parsed, segpromote, N, final);
ASSERT_FALSE(final[0]);
ASSERT_TRUE(final[1]);
ASSERT_TRUE(final[2]);
ASSERT_TRUE(final[3]);
ASSERT_TRUE(final[4]);
}
TEST_F(SealedSegmentRegexQueryTest, RegexQueryOnUnsupportedIndex) {
std::string operand = "a%";
const auto& str_meta = schema->operator[](FieldName("str"));
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto unary_range_expr = test::GenUnaryRangeExpr(OpType::Match, operand);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = test::GenExpr();
expr->set_allocated_unary_range_expr(unary_range_expr);
auto parser = ProtoParser(*schema);
auto typed_expr = parser.ParseExprs(*expr);
auto parsed =
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, typed_expr);
LoadMockIndex();
auto segpromote = dynamic_cast<SegmentSealedImpl*>(seg.get());
query::ExecPlanNodeVisitor visitor(*segpromote, MAX_TIMESTAMP);
BitsetType final;
ASSERT_ANY_THROW(visitor.ExecuteExprNode(parsed, segpromote, N, final));
}

View File

@ -0,0 +1,45 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <gtest/gtest.h>
#include "common/RegexQuery.h"
TEST(TranslatePatternMatchToRegexTest, SimplePatternWithPercent) {
std::string pattern = "abc%";
std::string result = milvus::TranslatePatternMatchToRegex(pattern);
EXPECT_EQ(result, "abc.*");
}
TEST(TranslatePatternMatchToRegexTest, PatternWithUnderscore) {
std::string pattern = "a_c";
std::string result = milvus::TranslatePatternMatchToRegex(pattern);
EXPECT_EQ(result, "a.c");
}
TEST(TranslatePatternMatchToRegexTest, PatternWithSpecialCharacters) {
std::string pattern = "a\\%b\\_c";
std::string result = milvus::TranslatePatternMatchToRegex(pattern);
EXPECT_EQ(result, "a\\%b\\_c");
}
TEST(TranslatePatternMatchToRegexTest,
PatternWithMultiplePercentAndUnderscore) {
std::string pattern = "%a_b%";
std::string result = milvus::TranslatePatternMatchToRegex(pattern);
EXPECT_EQ(result, ".*a.b.*");
}
TEST(TranslatePatternMatchToRegexTest, PatternWithRegexChar) {
std::string pattern = "abc*def.ghi+";
std::string result = milvus::TranslatePatternMatchToRegex(pattern);
EXPECT_EQ(result, "abc\\*def\\.ghi\\+");
}

View File

@ -20,6 +20,7 @@
#include "query/generated/ExecExprVisitor.h"
#include "segcore/SegmentGrowingImpl.h"
#include "test_utils/DataGen.h"
#include "test_utils/GenExprProto.h"
#include "query/PlanProto.h"
#include "query/Utils.h"
#include "query/SearchBruteForce.h"
@ -27,37 +28,6 @@
using namespace milvus;
namespace {
template <typename T>
auto
GenGenericValue(T value) {
auto generic = new proto::plan::GenericValue();
if constexpr (std::is_same_v<T, bool>) {
generic->set_bool_val(static_cast<bool>(value));
} else if constexpr (std::is_integral_v<T>) {
generic->set_int64_val(static_cast<int64_t>(value));
} else if constexpr (std::is_floating_point_v<T>) {
generic->set_float_val(static_cast<float>(value));
} else if constexpr (std::is_same_v<T, std::string>) {
generic->set_string_val(static_cast<std::string>(value));
} else {
static_assert(always_false<T>);
}
return generic;
}
auto
GenColumnInfo(int64_t field_id,
proto::schema::DataType field_type,
bool auto_id,
bool is_pk) {
auto column_info = new proto::plan::ColumnInfo();
column_info->set_field_id(field_id);
column_info->set_data_type(field_type);
column_info->set_is_autoid(auto_id);
column_info->set_is_primary_key(is_pk);
return column_info;
}
auto
GenQueryInfo(int64_t topk,
std::string metric_type,
@ -114,24 +84,14 @@ GenCompareExpr(proto::plan::OpType op) {
return compare_expr;
}
template <typename T>
auto
GenUnaryRangeExpr(proto::plan::OpType op, T& value) {
auto unary_range_expr = new proto::plan::UnaryRangeExpr();
unary_range_expr->set_op(op);
auto generic = GenGenericValue(value);
unary_range_expr->set_allocated_value(generic);
return unary_range_expr;
}
template <typename T>
auto
GenBinaryRangeExpr(bool lb_inclusive, bool ub_inclusive, T lb, T ub) {
auto binary_range_expr = new proto::plan::BinaryRangeExpr();
binary_range_expr->set_lower_inclusive(lb_inclusive);
binary_range_expr->set_upper_inclusive(ub_inclusive);
auto lb_generic = GenGenericValue(lb);
auto ub_generic = GenGenericValue(ub);
auto lb_generic = test::GenGenericValue(lb);
auto ub_generic = test::GenGenericValue(ub);
binary_range_expr->set_allocated_lower_value(lb_generic);
binary_range_expr->set_allocated_upper_value(ub_generic);
return binary_range_expr;
@ -144,11 +104,6 @@ GenNotExpr() {
return not_expr;
}
auto
GenExpr() {
return std::make_unique<proto::plan::Expr>();
}
auto
GenPlanNode() {
return std::make_unique<proto::plan::PlanNode>();
@ -167,14 +122,14 @@ GenTermPlan(const FieldMeta& fvec_meta,
const FieldMeta& str_meta,
const std::vector<std::string>& strs)
-> std::unique_ptr<proto::plan::PlanNode> {
auto column_info = GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto term_expr = GenTermExpr<std::string>(strs);
term_expr->set_allocated_column_info(column_info);
auto expr = GenExpr().release();
auto expr = test::GenExpr().release();
expr->set_allocated_term_expr(term_expr);
proto::plan::VectorType vector_type;
@ -195,15 +150,15 @@ GenTermPlan(const FieldMeta& fvec_meta,
auto
GenAlwaysFalseExpr(const FieldMeta& fvec_meta, const FieldMeta& str_meta) {
auto column_info = GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto term_expr =
GenTermExpr<std::string>({}); // in empty set, always false.
term_expr->set_allocated_column_info(column_info);
auto expr = GenExpr().release();
auto expr = test::GenExpr().release();
expr->set_allocated_term_expr(term_expr);
return expr;
}
@ -213,7 +168,7 @@ GenAlwaysTrueExpr(const FieldMeta& fvec_meta, const FieldMeta& str_meta) {
auto always_false_expr = GenAlwaysFalseExpr(fvec_meta, str_meta);
auto not_expr = GenNotExpr();
not_expr->set_allocated_child(always_false_expr);
auto expr = GenExpr().release();
auto expr = test::GenExpr().release();
expr->set_allocated_unary_expr(not_expr);
return expr;
}
@ -357,21 +312,22 @@ TEST(StringExpr, Compare) {
auto gen_compare_plan =
[&, fvec_meta, str_meta, another_str_meta](
proto::plan::OpType op) -> std::unique_ptr<proto::plan::PlanNode> {
auto str_col_info = GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto str_col_info =
test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto another_str_col_info =
GenColumnInfo(another_str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
test::GenColumnInfo(another_str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto compare_expr = GenCompareExpr(op);
compare_expr->set_allocated_left_column_info(str_col_info);
compare_expr->set_allocated_right_column_info(another_str_col_info);
auto expr = GenExpr().release();
auto expr = test::GenExpr().release();
expr->set_allocated_compare_expr(compare_expr);
proto::plan::VectorType vector_type;
@ -474,14 +430,14 @@ TEST(StringExpr, UnaryRange) {
[&, fvec_meta, str_meta](
proto::plan::OpType op,
std::string value) -> std::unique_ptr<proto::plan::PlanNode> {
auto column_info = GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto unary_range_expr = GenUnaryRangeExpr(op, value);
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto unary_range_expr = test::GenUnaryRangeExpr(op, value);
unary_range_expr->set_allocated_column_info(column_info);
auto expr = GenExpr().release();
auto expr = test::GenExpr().release();
expr->set_allocated_unary_range_expr(unary_range_expr);
proto::plan::VectorType vector_type;
@ -575,15 +531,15 @@ TEST(StringExpr, BinaryRange) {
bool ub_inclusive,
std::string lb,
std::string ub) -> std::unique_ptr<proto::plan::PlanNode> {
auto column_info = GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
proto::schema::DataType::VarChar,
false,
false);
auto binary_range_expr =
GenBinaryRangeExpr(lb_inclusive, ub_inclusive, lb, ub);
binary_range_expr->set_allocated_column_info(column_info);
auto expr = GenExpr().release();
auto expr = test::GenExpr().release();
expr->set_allocated_binary_range_expr(binary_range_expr);
proto::plan::VectorType vector_type;
@ -723,8 +679,12 @@ TEST(AlwaysTrueStringPlan, SearchWithOutputFields) {
dim, //
query_ptr //
};
auto sub_result = BruteForceSearch(
search_dataset, vec_col.data(), N, knowhere::Json(), nullptr, DataType::VECTOR_FLOAT);
auto sub_result = BruteForceSearch(search_dataset,
vec_col.data(),
N,
knowhere::Json(),
nullptr,
DataType::VECTOR_FLOAT);
auto sr = segment->Search(plan.get(), ph_group.get(), MAX_TIMESTAMP);
segment->FillPrimaryKeys(plan.get(), *sr);

View File

@ -206,4 +206,4 @@ TEST(Util, get_common_prefix) {
str2 = "";
common_prefix = milvus::GetCommonPrefix(str1, str2);
EXPECT_STREQ(common_prefix.c_str(), "");
}
}

View File

@ -0,0 +1,62 @@
// 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 "pb/plan.pb.h"
namespace milvus::test {
inline auto
GenColumnInfo(int64_t field_id,
proto::schema::DataType field_type,
bool auto_id,
bool is_pk) {
auto column_info = new proto::plan::ColumnInfo();
column_info->set_field_id(field_id);
column_info->set_data_type(field_type);
column_info->set_is_autoid(auto_id);
column_info->set_is_primary_key(is_pk);
return column_info;
}
template <typename T>
auto
GenGenericValue(T value) {
auto generic = new proto::plan::GenericValue();
if constexpr (std::is_same_v<T, bool>) {
generic->set_bool_val(static_cast<bool>(value));
} else if constexpr (std::is_integral_v<T>) {
generic->set_int64_val(static_cast<int64_t>(value));
} else if constexpr (std::is_floating_point_v<T>) {
generic->set_float_val(static_cast<float>(value));
} else if constexpr (std::is_same_v<T, std::string>) {
generic->set_string_val(static_cast<std::string>(value));
} else {
static_assert(always_false<T>);
}
return generic;
}
template <typename T>
auto
GenUnaryRangeExpr(proto::plan::OpType op, T& value) {
auto unary_range_expr = new proto::plan::UnaryRangeExpr();
unary_range_expr->set_op(op);
auto generic = GenGenericValue(value);
unary_range_expr->set_allocated_value(generic);
return unary_range_expr;
}
inline auto
GenExpr() {
return std::make_unique<proto::plan::Expr>();
}
} // namespace milvus::test

View File

@ -219,9 +219,9 @@ GenDataset(int64_t N,
auto
GenDatasetWithDataType(int64_t N,
const knowhere::MetricType& metric_type,
milvus::DataType data_type,
int64_t dim = DIM) {
const knowhere::MetricType& metric_type,
milvus::DataType data_type,
int64_t dim = DIM) {
auto schema = std::make_shared<milvus::Schema>();
if (data_type == milvus::DataType::VECTOR_FLOAT16) {
schema->AddDebugField(

View File

@ -1,8 +1,6 @@
package planparserv2
import (
"fmt"
"github.com/milvus-io/milvus/internal/proto/planpb"
)
@ -67,9 +65,5 @@ func translatePatternMatch(pattern string) (op planpb.OpType, operand string, er
return planpb.OpType_PrefixMatch, pattern[:loc+1], nil
}
return planpb.OpType_Invalid, "", fmt.Errorf(
"unsupported pattern: %s, "+
"only prefix pattern match like %s "+
"and equal match like %s(no wildcards) are supported",
pattern, "ab%", "ab")
return planpb.OpType_Match, pattern, nil
}

View File

@ -117,9 +117,9 @@ func Test_translatePatternMatch(t *testing.T) {
},
{
args: args{pattern: "prefix%suffix"},
wantOp: planpb.OpType_Invalid,
wantOperand: "",
wantErr: true,
wantOp: planpb.OpType_Match,
wantOperand: "prefix%suffix",
wantErr: false,
},
}
for _, tt := range tests {

View File

@ -155,14 +155,14 @@ func TestExpr_Like(t *testing.T) {
}
// TODO: enable these after regex-match is supported.
unsupported := []string{
`VarCharField like "not_%_supported"`,
`JSONField["A"] like "not_%_supported"`,
`$meta["A"] like "not_%_supported"`,
}
for _, exprStr := range unsupported {
assertInvalidExpr(t, helper, exprStr)
}
//unsupported := []string{
// `VarCharField like "not_%_supported"`,
// `JSONField["A"] like "not_%_supported"`,
// `$meta["A"] like "not_%_supported"`,
//}
//for _, exprStr := range unsupported {
// assertInvalidExpr(t, helper, exprStr)
//}
}
func TestExpr_BinaryRange(t *testing.T) {