feat: support keyword text match (#35923)

fix: #35922

---------

Signed-off-by: longjiquan <jiquan.long@zilliz.com>
This commit is contained in:
Jiquan Long 2024-09-10 15:11:08 +08:00 committed by GitHub
parent b0939fd20e
commit 89bf226f0b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
88 changed files with 3362 additions and 939 deletions

View File

@ -84,6 +84,7 @@ type component interface {
const (
TmpInvertedIndexPrefix = "/tmp/milvus/inverted-index/"
TmpTextLogPrefix = "/tmp/milvus/text-log/"
)
func cleanLocalDir(path string) {
@ -209,6 +210,7 @@ func (mr *MilvusRoles) runQueryNode(ctx context.Context, localMsg bool, wg *sync
cleanLocalDir(mmapDir)
}
cleanLocalDir(TmpInvertedIndexPrefix)
cleanLocalDir(TmpTextLogPrefix)
return runComponent(ctx, localMsg, wg, components.NewQueryNode, metrics.RegisterQueryNode)
}
@ -239,6 +241,7 @@ func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool, wg *sync
indexDataLocalPath := filepath.Join(rootPath, typeutil.IndexNodeRole)
cleanLocalDir(indexDataLocalPath)
cleanLocalDir(TmpInvertedIndexPrefix)
cleanLocalDir(TmpTextLogPrefix)
return runComponent(ctx, localMsg, wg, components.NewIndexNode, metrics.RegisterIndexNode)
}

View File

@ -46,6 +46,7 @@ const char NUM_CLUSTERS[] = "num_clusters";
const char KMEANS_CLUSTER[] = "KMEANS";
const char VEC_OPT_FIELDS[] = "opt_fields";
const char PAGE_RETAIN_ORDER[] = "page_retain_order";
const char TEXT_LOG_ROOT_PATH[] = "text_log";
const char DEFAULT_PLANNODE_ID[] = "0";
const char DEAFULT_QUERY_ID[] = "0";

View File

@ -0,0 +1,131 @@
// 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 "common/FieldMeta.h"
#include "common/SystemProperty.h"
#include "common/protobuf_utils.h"
#include <boost/lexical_cast.hpp>
#include "Consts.h"
namespace milvus {
TokenizerParams
ParseTokenizerParams(const TypeParams& params) {
auto iter = params.find("analyzer_params");
if (iter == params.end()) {
return {};
}
nlohmann::json j = nlohmann::json::parse(iter->second);
std::map<std::string, std::string> ret;
for (const auto& [k, v] : j.items()) {
try {
ret[k] = v.get<std::string>();
} catch (std::exception& e) {
ret[k] = v.dump();
}
}
return ret;
}
bool
FieldMeta::enable_match() const {
if (!IsStringDataType(type_)) {
return false;
}
if (!string_info_.has_value()) {
return false;
}
return string_info_->enable_match;
}
TokenizerParams
FieldMeta::get_tokenizer_params() const {
Assert(enable_match());
auto params = string_info_->params;
return ParseTokenizerParams(params);
}
FieldMeta
FieldMeta::ParseFrom(const milvus::proto::schema::FieldSchema& schema_proto) {
auto field_id = FieldId(schema_proto.fieldid());
auto name = FieldName(schema_proto.name());
auto nullable = schema_proto.nullable();
if (field_id.get() < 100) {
// system field id
auto is_system =
SystemProperty::Instance().SystemFieldVerify(name, field_id);
AssertInfo(is_system,
"invalid system type: name(" + name.get() + "), id(" +
std::to_string(field_id.get()) + ")");
}
auto data_type = DataType(schema_proto.data_type());
if (IsVectorDataType(data_type)) {
auto type_map = RepeatedKeyValToMap(schema_proto.type_params());
auto index_map = RepeatedKeyValToMap(schema_proto.index_params());
int64_t dim = 0;
if (!IsSparseFloatVectorDataType(data_type)) {
AssertInfo(type_map.count("dim"), "dim not found");
dim = boost::lexical_cast<int64_t>(type_map.at("dim"));
}
if (!index_map.count("metric_type")) {
return FieldMeta{
name, field_id, data_type, dim, std::nullopt, false};
}
auto metric_type = index_map.at("metric_type");
return FieldMeta{name, field_id, data_type, dim, metric_type, false};
}
if (IsStringDataType(data_type)) {
auto type_map = RepeatedKeyValToMap(schema_proto.type_params());
AssertInfo(type_map.count(MAX_LENGTH), "max_length not found");
auto max_len = boost::lexical_cast<int64_t>(type_map.at(MAX_LENGTH));
bool enable_match = false;
if (type_map.count("enable_match")) {
auto param_str = type_map.at("enable_match");
std::transform(param_str.begin(),
param_str.end(),
param_str.begin(),
::tolower);
auto bool_cast = [](const std::string& arg) -> bool {
std::istringstream ss(arg);
bool b;
ss >> std::boolalpha >> b;
return b;
};
enable_match = bool_cast(param_str);
}
return FieldMeta{name,
field_id,
data_type,
max_len,
nullable,
enable_match,
type_map};
}
if (IsArrayDataType(data_type)) {
return FieldMeta{name,
field_id,
data_type,
DataType(schema_proto.element_type()),
nullable};
}
return FieldMeta{name, field_id, data_type, nullable};
}
} // namespace milvus

View File

@ -24,6 +24,11 @@
#include "common/Types.h"
namespace milvus {
using TypeParams = std::map<std::string, std::string>;
using TokenizerParams = std::map<std::string, std::string>;
TokenizerParams
ParseTokenizerParams(const TypeParams& params);
class FieldMeta {
public:
@ -53,6 +58,21 @@ class FieldMeta {
Assert(IsStringDataType(type_));
}
FieldMeta(const FieldName& name,
FieldId id,
DataType type,
int64_t max_length,
bool nullable,
bool enable_match,
std::map<std::string, std::string>& params)
: name_(name),
id_(id),
type_(type),
string_info_(StringInfo{max_length, enable_match, std::move(params)}),
nullable_(nullable) {
Assert(IsStringDataType(type_));
}
FieldMeta(const FieldName& name,
FieldId id,
DataType type,
@ -99,6 +119,12 @@ class FieldMeta {
return string_info_->max_length;
}
bool
enable_match() const;
TokenizerParams
get_tokenizer_params() const;
std::optional<knowhere::MetricType>
get_metric_type() const {
Assert(IsVectorDataType(type_));
@ -160,6 +186,10 @@ class FieldMeta {
}
}
public:
static FieldMeta
ParseFrom(const milvus::proto::schema::FieldSchema& schema_proto);
private:
struct VectorInfo {
int64_t dim_;
@ -167,6 +197,8 @@ class FieldMeta {
};
struct StringInfo {
int64_t max_length;
bool enable_match;
std::map<std::string, std::string> params;
};
FieldName name_;
FieldId id_;

View File

@ -37,51 +37,9 @@ Schema::ParseFrom(const milvus::proto::schema::CollectionSchema& schema_proto) {
for (const milvus::proto::schema::FieldSchema& child :
schema_proto.fields()) {
auto field_id = FieldId(child.fieldid());
auto name = FieldName(child.name());
auto nullable = child.nullable();
if (field_id.get() < 100) {
// system field id
auto is_system =
SystemProperty::Instance().SystemFieldVerify(name, field_id);
AssertInfo(is_system,
"invalid system type: name(" + name.get() + "), id(" +
std::to_string(field_id.get()) + ")");
}
auto data_type = DataType(child.data_type());
if (IsVectorDataType(data_type)) {
auto type_map = RepeatedKeyValToMap(child.type_params());
auto index_map = RepeatedKeyValToMap(child.index_params());
int64_t dim = 0;
if (!IsSparseFloatVectorDataType(data_type)) {
AssertInfo(type_map.count("dim"), "dim not found");
dim = boost::lexical_cast<int64_t>(type_map.at("dim"));
}
if (!index_map.count("metric_type")) {
schema->AddField(
name, field_id, data_type, dim, std::nullopt, false);
} else {
auto metric_type = index_map.at("metric_type");
schema->AddField(
name, field_id, data_type, dim, metric_type, false);
}
} else if (IsStringDataType(data_type)) {
auto type_map = RepeatedKeyValToMap(child.type_params());
AssertInfo(type_map.count(MAX_LENGTH), "max_length not found");
auto max_len =
boost::lexical_cast<int64_t>(type_map.at(MAX_LENGTH));
schema->AddField(name, field_id, data_type, max_len, nullable);
} else if (IsArrayDataType(data_type)) {
schema->AddField(name,
field_id,
data_type,
DataType(child.element_type()),
nullable);
} else {
schema->AddField(name, field_id, data_type, nullable);
}
auto f = FieldMeta::ParseFrom(child);
schema->AddField(std::move(f));
if (child.is_primary_key()) {
AssertInfo(!schema->get_primary_field_id().has_value(),

View File

@ -113,6 +113,20 @@ class Schema {
this->AddField(std::move(field_meta));
}
// string type
void
AddField(const FieldName& name,
const FieldId id,
DataType data_type,
int64_t max_length,
bool nullable,
bool enable_match,
std::map<std::string, std::string>& params) {
auto field_meta = FieldMeta(
name, id, data_type, max_length, nullable, enable_match, params);
this->AddField(std::move(field_meta));
}
// vector type
void
AddField(const FieldName& name,

View File

@ -100,6 +100,7 @@ typedef struct CMmapConfig {
uint64_t disk_limit;
uint64_t fix_file_size;
bool growing_enable_mmap;
bool enable_mmap;
} CMmapConfig;
typedef struct CTraceConfig {

View File

@ -10,4 +10,4 @@
# or implied. See the License for the specific language governing permissions and limitations under the License
add_source_at_current_directory_recursively()
add_library(milvus_exec OBJECT ${SOURCE_FILES})
add_library(milvus_exec OBJECT ${SOURCE_FILES})

View File

@ -323,6 +323,35 @@ class SegmentExpr : public Expr {
return result;
}
template <typename FUNC, typename... ValTypes>
TargetBitmap
ProcessTextMatchIndex(FUNC func, ValTypes... values) {
TargetBitmap result;
if (cached_match_res_ == nullptr) {
auto index = segment_->GetTextIndex(field_id_);
auto res = std::move(func(index, values...));
cached_match_res_ = std::make_shared<TargetBitmap>(std::move(res));
if (cached_match_res_->size() < active_count_) {
// some entities are not visible in inverted index.
// only happend on growing segment.
TargetBitmap tail(active_count_ - cached_match_res_->size());
cached_match_res_->append(tail);
}
}
// return batch size, not sure if we should use the data position.
auto real_batch_size =
current_data_chunk_pos_ + batch_size_ > active_count_
? active_count_ - current_data_chunk_pos_
: batch_size_;
result.append(
*cached_match_res_, current_data_chunk_pos_, real_batch_size);
current_data_chunk_pos_ += real_batch_size;
return result;
}
template <typename T, typename FUNC, typename... ValTypes>
void
ProcessIndexChunksV2(FUNC func, ValTypes... values) {
@ -418,6 +447,9 @@ class SegmentExpr : public Expr {
// Cache for index scan to avoid search index every batch
int64_t cached_index_chunk_id_{-1};
TargetBitmap cached_index_chunk_res_{};
// Cache for text match.
std::shared_ptr<TargetBitmap> cached_match_res_{nullptr};
};
void

View File

@ -601,6 +601,10 @@ PhyUnaryRangeFilterExpr::ExecRangeVisitorImplJson() {
template <typename T>
VectorPtr
PhyUnaryRangeFilterExpr::ExecRangeVisitorImpl() {
if (expr_->op_type_ == proto::plan::OpType::TextMatch) {
return ExecTextMatch();
}
if (CanUseIndex<T>()) {
return ExecRangeVisitorImplForIndex<T>();
} else {
@ -857,5 +861,16 @@ PhyUnaryRangeFilterExpr::CanUseIndex() {
return res;
}
VectorPtr
PhyUnaryRangeFilterExpr::ExecTextMatch() {
using Index = index::TextMatchIndex;
auto query = GetValueFromProto<std::string>(expr_->val_);
auto func = [](Index* index, const std::string& query) -> TargetBitmap {
return index->MatchQuery(query);
};
auto res = ProcessTextMatchIndex(func, query);
return std::make_shared<ColumnVector>(std::move(res));
};
} // namespace exec
} // namespace milvus

View File

@ -331,6 +331,9 @@ class PhyUnaryRangeFilterExpr : public SegmentExpr {
bool
CanUseIndexForArray();
VectorPtr
ExecTextMatch();
private:
std::shared_ptr<const milvus::expr::UnaryRangeFilterExpr> expr_;
ColumnVectorPtr cached_overflow_res_{nullptr};

View File

@ -10,4 +10,4 @@
# or implied. See the License for the specific language governing permissions and limitations under the License
add_source_at_current_directory_recursively()
add_library(milvus_index OBJECT ${SOURCE_FILES})
add_library(milvus_index OBJECT ${SOURCE_FILES})

View File

@ -14,6 +14,7 @@
#include "common/RegexQuery.h"
#include "storage/LocalChunkManagerSingleton.h"
#include "index/InvertedIndexTantivy.h"
#include "index/InvertedIndexUtil.h"
#include "log/Log.h"
#include "index/Utils.h"
#include "storage/Util.h"
@ -216,32 +217,6 @@ InvertedIndexTantivy<T>::Load(milvus::tracer::TraceContext ctx,
wrapper_ = std::make_shared<TantivyIndexWrapper>(prefix.c_str());
}
inline void
apply_hits(TargetBitmap& bitset, const RustArrayWrapper& w, bool v) {
for (size_t j = 0; j < w.array_.len; j++) {
bitset[w.array_.array[j]] = v;
}
}
inline void
apply_hits_with_filter(TargetBitmap& bitset,
const RustArrayWrapper& w,
const std::function<bool(size_t /* offset */)>& filter) {
for (size_t j = 0; j < w.array_.len; j++) {
auto the_offset = w.array_.array[j];
bitset[the_offset] = filter(the_offset);
}
}
inline void
apply_hits_with_callback(
const RustArrayWrapper& w,
const std::function<void(size_t /* offset */)>& callback) {
for (size_t j = 0; j < w.array_.len; j++) {
callback(w.array_.array[j]);
}
}
template <typename T>
const TargetBitmap
InvertedIndexTantivy<T>::In(size_t n, const T* values) {
@ -436,7 +411,9 @@ InvertedIndexTantivy<T>::BuildWithRawData(size_t n,
} else {
wrapper_->add_data<T>(static_cast<const T*>(values), n, 0);
}
wrapper_->create_reader();
finish();
wrapper_->reload();
}
template <typename T>

View File

@ -173,10 +173,10 @@ class InvertedIndexTantivy : public ScalarIndex<T> {
const TargetBitmap
RegexQuery(const std::string& regex_pattern) override;
protected:
void
BuildWithFieldData(const std::vector<FieldDataPtr>& datas) override;
private:
void
finish();
@ -184,7 +184,7 @@ class InvertedIndexTantivy : public ScalarIndex<T> {
build_index_for_array(
const std::vector<std::shared_ptr<FieldDataBase>>& field_datas);
private:
protected:
std::shared_ptr<TantivyIndexWrapper> wrapper_;
TantivyDataType d_type_;
std::string path_;

View File

@ -0,0 +1,42 @@
// 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
namespace milvus::index {
inline void
apply_hits(milvus::TargetBitmap& bitset,
const milvus::index::RustArrayWrapper& w,
bool v) {
for (size_t j = 0; j < w.array_.len; j++) {
bitset[w.array_.array[j]] = v;
}
}
inline void
apply_hits_with_filter(milvus::TargetBitmap& bitset,
const milvus::index::RustArrayWrapper& w,
const std::function<bool(size_t /* offset */)>& filter) {
for (size_t j = 0; j < w.array_.len; j++) {
auto the_offset = w.array_.array[j];
bitset[the_offset] = filter(the_offset);
}
}
inline void
apply_hits_with_callback(
const milvus::index::RustArrayWrapper& w,
const std::function<void(size_t /* offset */)>& callback) {
for (size_t j = 0; j < w.array_.len; j++) {
callback(w.array_.array[j]);
}
}
} // namespace milvus::index

View File

@ -0,0 +1,199 @@
// 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/TextMatchIndex.h"
#include "index/InvertedIndexUtil.h"
#include "index/Utils.h"
namespace milvus::index {
constexpr const char* TMP_TEXT_LOG_PREFIX = "/tmp/milvus/text-log/";
TextMatchIndex::TextMatchIndex(
int64_t commit_interval_in_ms,
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params)
: commit_interval_in_ms_(commit_interval_in_ms),
last_commit_time_(stdclock::now()) {
d_type_ = TantivyDataType::Text;
std::string field_name = "tmp_text_index";
wrapper_ = std::make_shared<TantivyIndexWrapper>(
field_name.c_str(), true, "", tokenizer_name, tokenizer_params);
}
TextMatchIndex::TextMatchIndex(
const std::string& path,
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params)
: commit_interval_in_ms_(std::numeric_limits<int64_t>::max()),
last_commit_time_(stdclock::now()) {
path_ = path;
d_type_ = TantivyDataType::Text;
std::string field_name = "tmp_text_index";
wrapper_ = std::make_shared<TantivyIndexWrapper>(field_name.c_str(),
false,
path_.c_str(),
tokenizer_name,
tokenizer_params);
}
TextMatchIndex::TextMatchIndex(
const storage::FileManagerContext& ctx,
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params)
: commit_interval_in_ms_(std::numeric_limits<int64_t>::max()),
last_commit_time_(stdclock::now()) {
schema_ = ctx.fieldDataMeta.field_schema;
mem_file_manager_ = std::make_shared<MemFileManager>(ctx);
disk_file_manager_ = std::make_shared<DiskFileManager>(ctx);
auto prefix = disk_file_manager_->GetTextIndexIdentifier();
path_ = std::string(TMP_TEXT_LOG_PREFIX) + prefix;
boost::filesystem::create_directories(path_);
d_type_ = TantivyDataType::Text;
std::string field_name =
std::to_string(disk_file_manager_->GetFieldDataMeta().field_id);
wrapper_ = std::make_shared<TantivyIndexWrapper>(field_name.c_str(),
false,
path_.c_str(),
tokenizer_name,
tokenizer_params);
}
TextMatchIndex::TextMatchIndex(const storage::FileManagerContext& ctx)
: commit_interval_in_ms_(std::numeric_limits<int64_t>::max()),
last_commit_time_(stdclock::now()) {
schema_ = ctx.fieldDataMeta.field_schema;
mem_file_manager_ = std::make_shared<MemFileManager>(ctx);
disk_file_manager_ = std::make_shared<DiskFileManager>(ctx);
d_type_ = TantivyDataType::Text;
}
BinarySet
TextMatchIndex::Upload(const Config& config) {
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 text log: {}", iter->path().string());
AssertInfo(disk_file_manager_->AddTextLog(iter->path().string()),
"failed to add text log: {}",
iter->path().string());
LOG_INFO("text log: {} added", iter->path().string());
}
}
BinarySet ret;
auto remote_paths_to_size = disk_file_manager_->GetRemotePathsToFileSize();
for (auto& file : remote_paths_to_size) {
ret.Append(file.first, nullptr, file.second);
}
return ret;
}
void
TextMatchIndex::Load(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 text log index");
auto prefix = disk_file_manager_->GetLocalTextIndexPrefix();
disk_file_manager_->CacheTextLogToDisk(index_files.value());
AssertInfo(
tantivy_index_exist(prefix.c_str()), "index not exist: {}", prefix);
wrapper_ = std::make_shared<TantivyIndexWrapper>(prefix.c_str());
}
void
TextMatchIndex::AddText(const std::string& text, int64_t offset) {
AddTexts(1, &text, offset);
}
void
TextMatchIndex::AddTexts(size_t n,
const std::string* texts,
int64_t offset_begin) {
wrapper_->add_data(texts, n, offset_begin);
if (shouldTriggerCommit()) {
Commit();
}
}
void
TextMatchIndex::Finish() {
finish();
}
bool
TextMatchIndex::shouldTriggerCommit() {
auto span = (std::chrono::duration<double, std::milli>(
stdclock::now() - last_commit_time_.load()))
.count();
return span > commit_interval_in_ms_;
}
void
TextMatchIndex::Commit() {
std::unique_lock<std::mutex> lck(mtx_, std::defer_lock);
if (lck.try_lock()) {
wrapper_->commit();
last_commit_time_.store(stdclock::now());
}
}
void
TextMatchIndex::Reload() {
std::unique_lock<std::mutex> lck(mtx_, std::defer_lock);
if (lck.try_lock()) {
wrapper_->reload();
}
}
void
TextMatchIndex::CreateReader() {
wrapper_->create_reader();
}
void
TextMatchIndex::RegisterTokenizer(
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params) {
wrapper_->register_tokenizer(tokenizer_name, tokenizer_params);
}
TargetBitmap
TextMatchIndex::MatchQuery(const std::string& query) {
if (shouldTriggerCommit()) {
Commit();
Reload();
}
auto cnt = wrapper_->count();
TargetBitmap bitset(cnt);
if (bitset.empty()) {
return bitset;
}
auto hits = wrapper_->match_query(query);
apply_hits(bitset, hits, true);
return bitset;
}
} // namespace milvus::index

View File

@ -0,0 +1,86 @@
// 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"
namespace milvus::index {
using stdclock = std::chrono::high_resolution_clock;
class TextMatchIndex : public InvertedIndexTantivy<std::string> {
public:
// for growing segment.
explicit TextMatchIndex(
int64_t commit_interval_in_ms,
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params);
// for sealed segment.
explicit TextMatchIndex(
const std::string& path,
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params);
// for building index.
explicit TextMatchIndex(
const storage::FileManagerContext& ctx,
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params);
// for loading index
explicit TextMatchIndex(const storage::FileManagerContext& ctx);
public:
BinarySet
Upload(const Config& config) override;
void
Load(const Config& config);
public:
void
AddText(const std::string& text, int64_t offset);
void
AddTexts(size_t n, const std::string* texts, int64_t offset_begin);
void
Finish();
void
Commit();
void
Reload();
public:
void
CreateReader();
void
RegisterTokenizer(
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params);
TargetBitmap
MatchQuery(const std::string& query);
private:
bool
shouldTriggerCommit();
private:
mutable std::mutex mtx_;
std::atomic<stdclock::time_point> last_commit_time_;
int64_t commit_interval_in_ms_;
};
} // namespace milvus::index

View File

@ -23,6 +23,9 @@
#include "common/EasyAssert.h"
#include "indexbuilder/VecIndexCreator.h"
#include "indexbuilder/index_c.h"
#include "index/TextMatchIndex.h"
#include "indexbuilder/IndexFactory.h"
#include "common/type_c.h"
#include "storage/Types.h"
@ -232,6 +235,75 @@ CreateIndex(CIndex* res_index,
}
}
CStatus
BuildTextIndex(CBinarySet* c_binary_set,
const uint8_t* serialized_build_index_info,
const uint64_t len) {
try {
auto build_index_info =
std::make_unique<milvus::proto::indexcgo::BuildIndexInfo>();
auto res =
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 storage_config =
get_storage_config(build_index_info->storage_config());
auto config = get_config(build_index_info);
// init file manager
milvus::storage::FieldDataMeta field_meta{
build_index_info->collectionid(),
build_index_info->partitionid(),
build_index_info->segmentid(),
build_index_info->field_schema().fieldid(),
build_index_info->field_schema()};
milvus::storage::IndexMeta index_meta{
build_index_info->segmentid(),
build_index_info->field_schema().fieldid(),
build_index_info->buildid(),
build_index_info->index_version(),
"",
build_index_info->field_schema().name(),
field_type,
build_index_info->dim(),
};
auto chunk_manager =
milvus::storage::CreateChunkManager(storage_config);
milvus::storage::FileManagerContext fileManagerContext(
field_meta, index_meta, chunk_manager);
auto field_schema =
FieldMeta::ParseFrom(build_index_info->field_schema());
auto index = std::make_unique<index::TextMatchIndex>(
fileManagerContext,
"milvus_tokenizer",
field_schema.get_tokenizer_params());
index->Build(config);
auto binary =
std::make_unique<knowhere::BinarySet>(index->Upload(config));
*c_binary_set = binary.release();
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (SegcoreError& e) {
auto status = CStatus();
status.error_code = e.get_error_code();
status.error_msg = strdup(e.what());
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedError;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus
DeleteIndex(CIndex index) {
auto status = CStatus();

View File

@ -35,6 +35,11 @@ CreateIndex(CIndex* res_index,
CStatus
DeleteIndex(CIndex index);
CStatus
BuildTextIndex(CBinarySet* c_binary_set,
const uint8_t* serialized_build_index_info,
const uint64_t len);
CStatus
BuildFloatVecIndex(CIndex index, int64_t float_value_num, const float* vectors);

View File

@ -141,6 +141,23 @@ SegmentGrowingImpl::Insert(int64_t reserved_offset,
insert_record_);
}
// index text.
if (field_meta.enable_match()) {
// TODO: iterate texts and call `AddText` instead of `AddTexts`. This may cost much more memory.
std::vector<std::string> texts(
insert_record_proto->fields_data(data_offset)
.scalars()
.string_data()
.data()
.begin(),
insert_record_proto->fields_data(data_offset)
.scalars()
.string_data()
.data()
.end());
AddTexts(field_id, texts.data(), num_rows, reserved_offset);
}
// update average row data size
auto field_data_size = GetRawDataSizeOfDataArray(
&insert_record_proto->fields_data(data_offset),
@ -804,4 +821,41 @@ SegmentGrowingImpl::mask_with_timestamps(BitsetType& bitset_chunk,
// DO NOTHING
}
void
SegmentGrowingImpl::CreateTextIndex(FieldId field_id) {
std::unique_lock lock(mutex_);
const auto& field_meta = schema_->operator[](field_id);
AssertInfo(IsStringDataType(field_meta.get_data_type()),
"cannot create text index on non-string type");
// todo: make this(200) configurable.
auto index = std::make_unique<index::TextMatchIndex>(
200, "milvus_tokenizer", field_meta.get_tokenizer_params());
index->Commit();
index->CreateReader();
index->RegisterTokenizer("milvus_tokenizer",
field_meta.get_tokenizer_params());
text_indexes_[field_id] = std::move(index);
}
void
SegmentGrowingImpl::CreateTextIndexes() {
for (auto [field_id, field_meta] : schema_->get_fields()) {
if (IsStringDataType(field_meta.get_data_type()) &&
field_meta.enable_match()) {
CreateTextIndex(FieldId(field_id));
}
}
}
void
SegmentGrowingImpl::AddTexts(milvus::FieldId field_id,
const std::string* texts,
size_t n,
int64_t offset_begin) {
std::unique_lock lock(mutex_);
auto iter = text_indexes_.find(field_id);
AssertInfo(iter != text_indexes_.end(), "text index not found");
iter->second->AddTexts(n, texts, offset_begin);
}
} // namespace milvus::segcore

View File

@ -81,6 +81,9 @@ class SegmentGrowingImpl : public SegmentGrowing {
return insert_record_.is_valid_data_exist(field_id);
};
void
CreateTextIndex(FieldId field_id) override;
public:
const InsertRecord<>&
get_insert_record() const {
@ -243,6 +246,7 @@ class SegmentGrowingImpl : public SegmentGrowing {
storage::MmapManager::GetInstance().GetMmapChunkManager();
mcm->Register(mmap_descriptor_);
}
this->CreateTextIndexes();
}
~SegmentGrowingImpl() {
@ -344,6 +348,16 @@ class SegmentGrowingImpl : public SegmentGrowing {
return insert_record_.timestamps_;
}
private:
void
AddTexts(FieldId field_id,
const std::string* texts,
size_t n,
int64_t offset_begin);
void
CreateTextIndexes();
private:
storage::MmapChunkDescriptorPtr mmap_descriptor_ = nullptr;
SegcoreConfig segcore_config_;

View File

@ -392,4 +392,13 @@ SegmentInternalInterface::LoadStringSkipIndex(
skip_index_.LoadString(field_id, chunk_id, var_column);
}
index::TextMatchIndex*
SegmentInternalInterface::GetTextIndex(FieldId field_id) const {
std::shared_lock lock(mutex_);
auto iter = text_indexes_.find(field_id);
AssertInfo(iter != text_indexes_.end(),
"failed to get text index, text index not found");
return iter->second.get();
}
} // namespace milvus::segcore

View File

@ -36,6 +36,7 @@
#include "index/IndexInfo.h"
#include "index/SkipIndex.h"
#include "mmap/Column.h"
#include "index/TextMatchIndex.h"
namespace milvus::segcore {
@ -126,6 +127,12 @@ class SegmentInterface {
virtual bool
is_nullable(FieldId field_id) const = 0;
virtual void
CreateTextIndex(FieldId field_id) = 0;
virtual index::TextMatchIndex*
GetTextIndex(FieldId field_id) const = 0;
};
// internal API for DSL calculation
@ -259,6 +266,9 @@ class SegmentInternalInterface : public SegmentInterface {
virtual DataType
GetFieldDataType(FieldId fieldId) const = 0;
index::TextMatchIndex*
GetTextIndex(FieldId field_id) const override;
public:
virtual void
vector_search(SearchInfo& search_info,
@ -407,6 +417,10 @@ class SegmentInternalInterface : public SegmentInterface {
std::unordered_map<FieldId, std::pair<int64_t, int64_t>>
variable_fields_avg_size_; // bytes;
SkipIndex skip_index_;
// text-indexes used to do match.
std::unordered_map<FieldId, std::unique_ptr<index::TextMatchIndex>>
text_indexes_;
};
} // namespace milvus::segcore

View File

@ -43,6 +43,10 @@ class SegmentSealed : public SegmentInternalInterface {
virtual void
WarmupChunkCache(const FieldId field_id, bool mmap_enabled) = 0;
virtual void
LoadTextIndex(FieldId field_id,
std::unique_ptr<index::TextMatchIndex> index) = 0;
SegmentType
type() const override {
return SegmentType::Sealed;

View File

@ -23,6 +23,7 @@
#include <string_view>
#include <unordered_map>
#include <vector>
#include <boost/pointer_cast.hpp>
#include "Utils.h"
#include "Types.h"
@ -1992,4 +1993,83 @@ SegmentSealedImpl::RemoveFieldFile(const FieldId field_id) {
}
}
void
SegmentSealedImpl::CreateTextIndex(FieldId field_id) {
std::unique_lock lck(mutex_);
const auto& field_meta = schema_->operator[](field_id);
auto& cfg = storage::MmapManager::GetInstance().GetMmapConfig();
std::unique_ptr<index::TextMatchIndex> index;
if (!cfg.GetEnableMmap()) {
// build text index in ram.
index = std::make_unique<index::TextMatchIndex>(
std::numeric_limits<int64_t>::max(),
"milvus_tokenizer",
field_meta.get_tokenizer_params());
} else {
// build text index using mmap.
index = std::make_unique<index::TextMatchIndex>(
cfg.GetMmapPath(),
"milvus_tokenizer",
field_meta.get_tokenizer_params());
}
{
// build
auto iter = fields_.find(field_id);
if (iter != fields_.end()) {
auto column =
std::dynamic_pointer_cast<VariableColumn<std::string>>(
iter->second);
AssertInfo(
column != nullptr,
"failed to create text index, field is not of text type: {}",
field_id.get());
auto n = column->NumRows();
for (size_t i = 0; i < n; i++) {
index->AddText(std::string(column->RawAt(i)), i);
}
} else { // fetch raw data from index.
auto field_index_iter = scalar_indexings_.find(field_id);
AssertInfo(field_index_iter != scalar_indexings_.end(),
"failed to create text index, neither raw data nor "
"index are found");
auto ptr = field_index_iter->second.get();
AssertInfo(ptr->HasRawData(),
"text raw data not found, trying to create text index "
"from index, but this index don't contain raw data");
auto impl = dynamic_cast<index::ScalarIndex<std::string>*>(ptr);
AssertInfo(impl != nullptr,
"failed to create text index, field index cannot be "
"converted to string index");
auto n = impl->Size();
for (size_t i = 0; i < n; i++) {
index->AddText(impl->Reverse_Lookup(i), i);
}
}
}
// create index reader.
index->CreateReader();
// release index writer.
index->Finish();
index->Reload();
index->RegisterTokenizer("milvus_tokenizer",
field_meta.get_tokenizer_params());
text_indexes_[field_id] = std::move(index);
}
void
SegmentSealedImpl::LoadTextIndex(FieldId field_id,
std::unique_ptr<index::TextMatchIndex> index) {
std::unique_lock lck(mutex_);
const auto& field_meta = schema_->operator[](field_id);
index->RegisterTokenizer("milvus_tokenizer",
field_meta.get_tokenizer_params());
text_indexes_[field_id] = std::move(index);
}
} // namespace milvus::segcore

View File

@ -34,6 +34,7 @@
#include "sys/mman.h"
#include "common/Types.h"
#include "common/IndexMeta.h"
#include "index/TextMatchIndex.h"
namespace milvus::segcore {
@ -91,6 +92,13 @@ class SegmentSealedImpl : public SegmentSealed {
void
RemoveFieldFile(const FieldId field_id);
void
CreateTextIndex(FieldId field_id) override;
void
LoadTextIndex(FieldId field_id,
std::unique_ptr<index::TextMatchIndex> index) override;
public:
size_t
GetMemoryUsageInBytes() const override {

View File

@ -14,6 +14,9 @@
#include <memory>
#include <limits>
#include "pb/cgo_msg.pb.h"
#include "pb/index_cgo_msg.pb.h"
#include "common/FieldData.h"
#include "common/LoadInfo.h"
#include "common/Types.h"
@ -412,6 +415,56 @@ UpdateSealedSegmentIndex(CSegmentInterface c_segment,
}
}
CStatus
LoadTextIndex(CSegmentInterface c_segment,
const uint8_t* serialized_load_text_index_info,
const uint64_t len) {
try {
auto segment_interface =
reinterpret_cast<milvus::segcore::SegmentInterface*>(c_segment);
auto segment =
dynamic_cast<milvus::segcore::SegmentSealed*>(segment_interface);
AssertInfo(segment != nullptr, "segment conversion failed");
auto info_proto =
std::make_unique<milvus::proto::indexcgo::LoadTextIndexInfo>();
info_proto->ParseFromArray(serialized_load_text_index_info, len);
milvus::storage::FieldDataMeta field_meta{info_proto->collectionid(),
info_proto->partitionid(),
segment->get_segment_id(),
info_proto->fieldid(),
info_proto->schema()};
milvus::storage::IndexMeta index_meta{segment->get_segment_id(),
info_proto->fieldid(),
info_proto->buildid(),
info_proto->version()};
auto remote_chunk_manager =
milvus::storage::RemoteChunkManagerSingleton::GetInstance()
.GetRemoteChunkManager();
milvus::Config config;
std::vector<std::string> files;
for (const auto& f : info_proto->files()) {
files.push_back(f);
}
config["index_files"] = files;
milvus::storage::FileManagerContext ctx(
field_meta, index_meta, remote_chunk_manager);
auto index = std::make_unique<milvus::index::TextMatchIndex>(ctx);
index->Load(config);
segment->LoadTextIndex(milvus::FieldId(info_proto->fieldid()),
std::move(index));
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
UpdateFieldRawDataSize(CSegmentInterface c_segment,
int64_t field_id,
@ -500,3 +553,15 @@ RemoveFieldFile(CSegmentInterface c_segment, int64_t field_id) {
reinterpret_cast<milvus::segcore::SegmentSealedImpl*>(c_segment);
segment->RemoveFieldFile(milvus::FieldId(field_id));
}
CStatus
CreateTextIndex(CSegmentInterface c_segment, int64_t field_id) {
try {
auto segment_interface =
reinterpret_cast<milvus::segcore::SegmentInterface*>(c_segment);
segment_interface->CreateTextIndex(milvus::FieldId(field_id));
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(milvus::UnexpectedError, e.what());
}
}

View File

@ -117,6 +117,11 @@ CStatus
UpdateSealedSegmentIndex(CSegmentInterface c_segment,
CLoadIndexInfo c_load_index_info);
CStatus
LoadTextIndex(CSegmentInterface c_segment,
const uint8_t* serialized_load_text_index_info,
const uint64_t len);
CStatus
UpdateFieldRawDataSize(CSegmentInterface c_segment,
int64_t field_id,
@ -156,6 +161,9 @@ Delete(CSegmentInterface c_segment,
void
RemoveFieldFile(CSegmentInterface c_segment, int64_t field_id);
CStatus
CreateTextIndex(CSegmentInterface c_segment, int64_t field_id);
#ifdef __cplusplus
}
#endif

View File

@ -10,8 +10,10 @@
// or implied. See the License for the specific language governing permissions and limitations under the License
#include "segcore/tokenizer_c.h"
#include "common/FieldMeta.h"
#include "common/protobuf_utils.h"
#include "pb/schema.pb.h"
#include "common/EasyAssert.h"
#include "tokenizer.h"
using Map = std::map<std::string, std::string>;
@ -39,3 +41,19 @@ create_token_stream(CTokenizer tokenizer, const char* text, uint32_t text_len) {
auto impl = reinterpret_cast<milvus::tantivy::Tokenizer*>(tokenizer);
return impl->CreateTokenStream(std::string(text, text_len)).release();
}
CStatus
validate_text_schema(const uint8_t* field_schema, uint64_t length) {
try {
auto schema = std::make_unique<milvus::proto::schema::FieldSchema>();
AssertInfo(schema->ParseFromArray(field_schema, length),
"failed to create field schema");
auto type_params = milvus::RepeatedKeyValToMap(schema->type_params());
milvus::tantivy::Tokenizer _(milvus::ParseTokenizerParams(type_params));
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}

View File

@ -32,6 +32,9 @@ free_tokenizer(CTokenizer tokenizer);
CTokenStream
create_token_stream(CTokenizer tokenizer, const char* text, uint32_t text_len);
CStatus
validate_text_schema(const uint8_t* field_schema, uint64_t length);
#ifdef __cplusplus
}
#endif

View File

@ -72,6 +72,13 @@ DiskFileManagerImpl::GetRemoteIndexPath(const std::string& file_name,
return remote_prefix + "/" + file_name + "_" + std::to_string(slice_num);
}
std::string
DiskFileManagerImpl::GetRemoteTextLogPath(const std::string& file_name,
int64_t slice_num) const {
auto remote_prefix = GetRemoteTextLogPrefix();
return remote_prefix + "/" + file_name + "_" + std::to_string(slice_num);
}
bool
DiskFileManagerImpl::AddFile(const std::string& file) noexcept {
auto local_chunk_manager =
@ -124,6 +131,58 @@ DiskFileManagerImpl::AddFile(const std::string& file) noexcept {
return true;
} // namespace knowhere
bool
DiskFileManagerImpl::AddTextLog(const std::string& file) noexcept {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
FILEMANAGER_TRY
if (!local_chunk_manager->Exist(file)) {
LOG_ERROR("local file {} not exists", file);
return false;
}
// record local file path
local_paths_.emplace_back(file);
auto fileName = GetFileName(file);
auto fileSize = local_chunk_manager->Size(file);
std::vector<std::string> batch_remote_files;
std::vector<int64_t> remote_file_sizes;
std::vector<int64_t> local_file_offsets;
int slice_num = 0;
auto parallel_degree =
uint64_t(DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
for (int64_t offset = 0; offset < fileSize; slice_num++) {
if (batch_remote_files.size() >= parallel_degree) {
AddBatchIndexFiles(file,
local_file_offsets,
batch_remote_files,
remote_file_sizes);
batch_remote_files.clear();
remote_file_sizes.clear();
local_file_offsets.clear();
}
auto batch_size = std::min(FILE_SLICE_SIZE, int64_t(fileSize) - offset);
batch_remote_files.emplace_back(
GetRemoteTextLogPath(fileName, slice_num));
remote_file_sizes.emplace_back(batch_size);
local_file_offsets.emplace_back(offset);
offset += batch_size;
}
if (batch_remote_files.size() > 0) {
AddBatchIndexFiles(
file, local_file_offsets, batch_remote_files, remote_file_sizes);
}
FILEMANAGER_CATCH
FILEMANAGER_END
return true;
} // namespace knowhere
void
DiskFileManagerImpl::AddBatchIndexFiles(
const std::string& local_file_name,
@ -236,6 +295,52 @@ DiskFileManagerImpl::CacheIndexToDisk(
}
}
void
DiskFileManagerImpl::CacheTextLogToDisk(
const std::vector<std::string>& remote_files) {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
std::map<std::string, std::vector<int>> index_slices;
for (auto& file_path : remote_files) {
auto pos = file_path.find_last_of('_');
index_slices[file_path.substr(0, pos)].emplace_back(
std::stoi(file_path.substr(pos + 1)));
}
for (auto& slices : index_slices) {
std::sort(slices.second.begin(), slices.second.end());
}
for (auto& slices : index_slices) {
auto prefix = slices.first;
auto local_index_file_name =
GetLocalTextIndexPrefix() + "/" +
prefix.substr(prefix.find_last_of('/') + 1);
local_chunk_manager->CreateFile(local_index_file_name);
auto file =
File::Open(local_index_file_name, O_CREAT | O_RDWR | O_TRUNC);
// Get the remote files
std::vector<std::string> batch_remote_files;
batch_remote_files.reserve(slices.second.size());
for (int& iter : slices.second) {
auto origin_file = prefix + "_" + std::to_string(iter);
batch_remote_files.push_back(origin_file);
}
auto index_chunks = GetObjectData(rcm_.get(), batch_remote_files);
for (auto& chunk : index_chunks) {
auto index_data = chunk.get()->GetFieldData();
auto index_size = index_data->Size();
auto chunk_data = reinterpret_cast<uint8_t*>(
const_cast<void*>(index_data->Data()));
file.Write(chunk_data, index_size);
}
local_paths_.emplace_back(local_index_file_name);
}
}
void
SortByPath(std::vector<std::string>& paths) {
std::sort(paths.begin(),
@ -580,12 +685,31 @@ DiskFileManagerImpl::GetLocalIndexObjectPrefix() {
local_chunk_manager, index_meta_.build_id, index_meta_.index_version);
}
std::string
DiskFileManagerImpl::GetLocalTextIndexPrefix() {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
return GenTextIndexPathPrefix(local_chunk_manager,
index_meta_.build_id,
index_meta_.index_version,
field_meta_.segment_id,
field_meta_.field_id);
}
std::string
DiskFileManagerImpl::GetIndexIdentifier() {
return GenIndexPathIdentifier(index_meta_.build_id,
index_meta_.index_version);
}
std::string
DiskFileManagerImpl::GetTextIndexIdentifier() {
return std::to_string(index_meta_.build_id) + "/" +
std::to_string(index_meta_.index_version) + "/" +
std::to_string(field_meta_.segment_id) +
std::to_string(field_meta_.field_id);
}
std::string
DiskFileManagerImpl::GetLocalRawDataObjectPrefix() {
auto local_chunk_manager =

View File

@ -33,32 +33,46 @@ class DiskFileManagerImpl : public FileManagerImpl {
public:
explicit DiskFileManagerImpl(const FileManagerContext& fileManagerContext);
virtual ~DiskFileManagerImpl();
~DiskFileManagerImpl() override;
virtual bool
LoadFile(const std::string& filename) noexcept;
bool
LoadFile(const std::string& filename) noexcept override;
virtual bool
AddFile(const std::string& filename) noexcept;
bool
AddFile(const std::string& filename) noexcept override;
virtual std::optional<bool>
IsExisted(const std::string& filename) noexcept;
std::optional<bool>
IsExisted(const std::string& filename) noexcept override;
virtual bool
RemoveFile(const std::string& filename) noexcept;
bool
RemoveFile(const std::string& filename) noexcept override;
public:
virtual std::string
GetName() const {
bool
AddTextLog(const std::string& filename) noexcept;
public:
std::string
GetName() const override {
return "DiskFileManagerImpl";
}
std::string
GetLocalIndexObjectPrefix();
// Similar to GetTextIndexIdentifier, segment_id and field_id is also required.
std::string
GetLocalTextIndexPrefix();
std::string
GetIndexIdentifier();
// Different from user index, a text index task may have multiple text fields sharing same build_id/task_id. So
// segment_id and field_id are required to identify a unique text index, in case that we support multiple index task
// in the same indexnode at the same time later.
std::string
GetTextIndexIdentifier();
std::string
GetLocalRawDataObjectPrefix();
@ -75,6 +89,9 @@ class DiskFileManagerImpl : public FileManagerImpl {
void
CacheIndexToDisk(const std::vector<std::string>& remote_files);
void
CacheTextLogToDisk(const std::vector<std::string>& remote_files);
void
AddBatchIndexFiles(const std::string& local_file_name,
const std::vector<int64_t>& local_file_offsets,
@ -105,6 +122,9 @@ class DiskFileManagerImpl : public FileManagerImpl {
std::string
GetRemoteIndexPath(const std::string& file_name, int64_t slice_num) const;
std::string
GetRemoteTextLogPath(const std::string& file_name, int64_t slice_num) const;
private:
// local file path (abs path)
std::vector<std::string> local_paths_;

View File

@ -145,6 +145,17 @@ class FileManagerImpl : public knowhere::FileManager {
std::to_string(field_meta_.segment_id);
}
virtual std::string
GetRemoteTextLogPrefix() const {
return rcm_->GetRootPath() + "/" + std::string(TEXT_LOG_ROOT_PATH) +
"/" + std::to_string(index_meta_.build_id) + "/" +
std::to_string(index_meta_.index_version) + "/" +
std::to_string(field_meta_.collection_id) + "/" +
std::to_string(field_meta_.partition_id) + "/" +
std::to_string(field_meta_.segment_id) + "/" +
std::to_string(field_meta_.field_id);
}
protected:
// collection meta
FieldDataMeta field_meta_;

View File

@ -125,6 +125,7 @@ struct MmapConfig {
uint64_t disk_limit;
uint64_t fix_file_size;
bool growing_enable_mmap;
bool enable_mmap;
bool
GetEnableGrowingMmap() const {
return growing_enable_mmap;
@ -133,6 +134,18 @@ struct MmapConfig {
SetEnableGrowingMmap(bool flag) {
this->growing_enable_mmap = flag;
}
bool
GetEnableMmap() const {
return enable_mmap;
}
void
SetEnableMmap(bool flag) {
this->enable_mmap = flag;
}
std::string
GetMmapPath() {
return mmap_path;
}
std::string
ToString() const {
std::stringstream ss;
@ -141,7 +154,7 @@ struct MmapConfig {
<< ", disk_limit=" << disk_limit / (1024 * 1024) << "MB"
<< ", fix_file_size=" << fix_file_size / (1024 * 1024) << "MB"
<< ", growing_enable_mmap=" << std::boolalpha << growing_enable_mmap
<< "]";
<< ", enable_mmap=" << std::boolalpha << enable_mmap << "]";
return ss.str();
}
};

View File

@ -483,6 +483,16 @@ GenIndexPathIdentifier(int64_t build_id, int64_t index_version) {
return std::to_string(build_id) + "/" + std::to_string(index_version) + "/";
}
std::string
GenTextIndexPathIdentifier(int64_t build_id,
int64_t index_version,
int64_t segment_id,
int64_t field_id) {
return std::to_string(build_id) + "/" + std::to_string(index_version) +
"/" + std::to_string(segment_id) + "/" + std::to_string(field_id) +
"/";
}
std::string
GenIndexPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
@ -491,6 +501,17 @@ GenIndexPathPrefix(ChunkManagerPtr cm,
GenIndexPathIdentifier(build_id, index_version);
}
std::string
GenTextIndexPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t segment_id,
int64_t field_id) {
return cm->GetRootPath() + "/" + std::string(TEXT_LOG_ROOT_PATH) + "/" +
GenTextIndexPathIdentifier(
build_id, index_version, segment_id, field_id);
}
std::string
GetIndexPathPrefixWithBuildID(ChunkManagerPtr cm, int64_t build_id) {
return cm->GetRootPath() + "/" + std::string(INDEX_ROOT_PATH) + "/" +

View File

@ -76,9 +76,22 @@ GetIndexPathPrefixWithBuildID(ChunkManagerPtr cm, int64_t build_id);
std::string
GenIndexPathIdentifier(int64_t build_id, int64_t index_version);
std::string
GenTextIndexPathIdentifier(int64_t build_id,
int64_t index_version,
int64_t segment_id,
int64_t field_id);
std::string
GenIndexPathPrefix(ChunkManagerPtr cm, int64_t build_id, int64_t index_version);
std::string
GenTextIndexPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version,
int64_t segment_id,
int64_t field_id);
std::string
GenFieldRawDataPathPrefix(ChunkManagerPtr cm,
int64_t segment_id,

View File

@ -95,6 +95,7 @@ InitMmapManager(CMmapConfig c_mmap_config) {
mmap_config.disk_limit = c_mmap_config.disk_limit;
mmap_config.fix_file_size = c_mmap_config.fix_file_size;
mmap_config.growing_enable_mmap = c_mmap_config.growing_enable_mmap;
mmap_config.enable_mmap = c_mmap_config.enable_mmap;
milvus::storage::MmapManager::GetInstance().Init(mmap_config);
return milvus::SuccessCStatus();
} catch (std::exception& e) {

View File

@ -90,6 +90,18 @@ if (BUILD_TANTIVY_WITH_UT)
tantivy_binding
dl
)
add_executable(text_demo text_demo.cpp)
target_link_libraries(text_demo
tantivy_binding
dl
)
add_executable(jieba_demo jieba_demo.cpp)
target_link_libraries(jieba_demo
tantivy_binding
dl
)
else ()
endif ()

View File

@ -0,0 +1,63 @@
#include <string>
#include <vector>
#include <boost/filesystem/operations.hpp>
#include "tantivy-binding.h"
#include "tantivy-wrapper.h"
using namespace milvus::tantivy;
std::set<uint32_t>
to_set(const RustArrayWrapper& w) {
std::set<uint32_t> s(w.array_.array, w.array_.array + w.array_.len);
return s;
}
int
main(int argc, char* argv[]) {
std::string tokenizer_name = "jieba";
std::map<std::string, std::string> tokenizer_params;
tokenizer_params["tokenizer"] = tokenizer_name;
auto text_index = TantivyIndexWrapper(
"text_demo", true, "", tokenizer_name.c_str(), tokenizer_params);
auto write_single_text = [&text_index](const std::string& s,
int64_t offset) {
text_index.add_data(&s, 1, offset);
};
{
write_single_text(
"张华考上了北京大学;李萍进了中等技术学校;我在百货公司当售货员:我"
"们都有光明的前途",
0);
write_single_text("测试中文分词器的效果", 1);
write_single_text("黄金时代", 2);
write_single_text("青铜时代", 3);
text_index.commit();
}
text_index.create_reader();
text_index.register_tokenizer(tokenizer_name.c_str(), tokenizer_params);
{
auto result = to_set(text_index.match_query("北京"));
assert(result.size() == 1);
assert(result.find(0) != result.end());
}
{
auto result = to_set(text_index.match_query("效果"));
assert(result.size() == 1);
assert(result.find(1) != result.end());
}
{
auto result = to_set(text_index.match_query("时代"));
assert(result.size() == 2);
assert(result.find(2) != result.end());
assert(result.find(3) != result.end());
}
return 0;
}

View File

@ -7,6 +7,7 @@
#include <new>
enum class TantivyDataType : uint8_t {
Text,
Keyword,
I64,
F64,
@ -35,6 +36,8 @@ void *tantivy_load_index(const char *path);
void tantivy_free_index_reader(void *ptr);
void tantivy_reload_index(void *ptr);
uint32_t tantivy_index_count(void *ptr);
RustArray tantivy_term_query_i64(void *ptr, int64_t term);
@ -83,6 +86,10 @@ RustArray tantivy_prefix_query_keyword(void *ptr, const char *prefix);
RustArray tantivy_regex_query(void *ptr, const char *pattern);
RustArray tantivy_match_query(void *ptr, const char *query);
void tantivy_register_tokenizer(void *ptr, const char *tokenizer_name, void *tokenizer_params);
void *tantivy_create_index(const char *field_name,
TantivyDataType data_type,
const char *path,
@ -93,6 +100,10 @@ void tantivy_free_index_writer(void *ptr);
void tantivy_finish_index(void *ptr);
void tantivy_commit_index(void *ptr);
void *tantivy_create_reader_from_writer(void *ptr);
void tantivy_index_add_int8s(void *ptr, const int8_t *array, uintptr_t len, int64_t offset_begin);
void tantivy_index_add_int16s(void *ptr, const int16_t *array, uintptr_t len, int64_t offset_begin);
@ -128,6 +139,14 @@ void tantivy_index_add_multi_keywords(void *ptr,
uintptr_t len,
int64_t offset);
void *tantivy_create_text_writer(const char *field_name,
const char *path,
const char *tokenizer_name,
void *tokenizer_params,
uintptr_t num_threads,
uintptr_t overall_memory_budget_in_bytes,
bool in_ram);
void free_rust_string(const char *ptr);
void *tantivy_create_token_stream(void *tokenizer, const char *text);

View File

@ -1,6 +1,6 @@
#[repr(u8)]
pub enum TantivyDataType {
// Text,
Text,
Keyword,
// U64,
I64,

View File

@ -1,59 +0,0 @@
use std::collections::HashSet;
use tantivy::{
collector::{Collector, SegmentCollector},
DocId,
};
pub struct HashSetCollector;
impl Collector for HashSetCollector {
type Fruit = HashSet<DocId>;
type Child = HashSetChildCollector;
fn for_segment(
&self,
_segment_local_id: tantivy::SegmentOrdinal,
_segment: &tantivy::SegmentReader,
) -> tantivy::Result<Self::Child> {
Ok(HashSetChildCollector {
docs: HashSet::new(),
})
}
fn requires_scoring(&self) -> bool {
false
}
fn merge_fruits(&self, segment_fruits: Vec<HashSet<DocId>>) -> tantivy::Result<HashSet<DocId>> {
if segment_fruits.len() == 1 {
Ok(segment_fruits.into_iter().next().unwrap())
} else {
let len: usize = segment_fruits.iter().map(|docset| docset.len()).sum();
let mut result = HashSet::with_capacity(len);
for docs in segment_fruits {
for doc in docs {
result.insert(doc);
}
}
Ok(result)
}
}
}
pub struct HashSetChildCollector {
docs: HashSet<DocId>,
}
impl SegmentCollector for HashSetChildCollector {
type Fruit = HashSet<DocId>;
fn collect(&mut self, doc: DocId, _score: tantivy::Score) {
self.docs.insert(doc);
}
fn harvest(self) -> Self::Fruit {
self.docs
}
}

View File

@ -21,6 +21,14 @@ pub extern "C" fn tantivy_free_index_reader(ptr: *mut c_void) {
}
// -------------------------query--------------------
#[no_mangle]
pub extern "C" fn tantivy_reload_index(ptr: *mut c_void) {
let real = ptr as *mut IndexReaderWrapper;
unsafe {
(*real).reload();
}
}
#[no_mangle]
pub extern "C" fn tantivy_index_count(ptr: *mut c_void) -> u32 {
let real = ptr as *mut IndexReaderWrapper;

View File

@ -0,0 +1,32 @@
use tantivy::{
query::BooleanQuery,
tokenizer::{TextAnalyzer, TokenStream},
Term,
};
use crate::{index_reader::IndexReaderWrapper, tokenizer::default_tokenizer};
impl IndexReaderWrapper {
// split the query string into multiple tokens using index's default tokenizer,
// and then execute the disconjunction of term query.
pub(crate) fn match_query(&self, q: &str) -> Vec<u32> {
// clone the tokenizer to make `match_query` thread-safe.
let mut tokenizer = self
.index
.tokenizer_for_field(self.field)
.unwrap_or(default_tokenizer())
.clone();
let mut token_stream = tokenizer.token_stream(q);
let mut terms: Vec<Term> = Vec::new();
while token_stream.advance() {
let token = token_stream.token();
terms.push(Term::from_field_text(self.field, &token.text));
}
let query = BooleanQuery::new_multiterms_query(terms);
self.search(&query)
}
pub(crate) fn register_tokenizer(&self, tokenizer_name: String, tokenizer: TextAnalyzer) {
self.index.tokenizers().register(&tokenizer_name, tokenizer)
}
}

View File

@ -0,0 +1,40 @@
use std::{collections::HashMap, ffi::CStr};
use libc::{c_char, c_void};
use crate::{array::RustArray, index_reader::IndexReaderWrapper, tokenizer::create_tokenizer};
#[no_mangle]
pub extern "C" fn tantivy_match_query(ptr: *mut c_void, query: *const c_char) -> RustArray {
let real = ptr as *mut IndexReaderWrapper;
unsafe {
let c_str = CStr::from_ptr(query);
let hits = (*real).match_query(c_str.to_str().unwrap());
RustArray::from_vec(hits)
}
}
#[no_mangle]
pub extern "C" fn tantivy_register_tokenizer(
ptr: *mut c_void,
tokenizer_name: *const c_char,
tokenizer_params: *mut c_void,
) {
let real = ptr as *mut IndexReaderWrapper;
let tokenizer_name_str = unsafe { CStr::from_ptr(tokenizer_name) };
let analyzer = unsafe {
let m = tokenizer_params as *const HashMap<String, String>;
create_tokenizer(&(*m))
};
match analyzer {
Some(text_analyzer) => unsafe {
(*real).register_tokenizer(
String::from(tokenizer_name_str.to_str().unwrap()),
text_analyzer,
);
},
None => {
panic!("unsupported tokenizer");
}
}
}

View File

@ -10,6 +10,7 @@ use tantivy::{doc, tokenizer, Document, Index, IndexWriter};
use crate::data_type::TantivyDataType;
use crate::index_reader::IndexReaderWrapper;
use crate::log::init_log;
pub(crate) struct IndexWriterWrapper {
@ -50,6 +51,9 @@ impl IndexWriterWrapper {
field = schema_builder.add_text_field(&field_name, text_options);
use_raw_tokenizer = true;
}
TantivyDataType::Text => {
panic!("text should be indexed with analyzer");
}
}
let id_field = schema_builder.add_i64_field("doc_id", FAST);
let schema = schema_builder.build();
@ -70,6 +74,10 @@ impl IndexWriterWrapper {
}
}
pub fn create_reader(&self) -> IndexReaderWrapper {
IndexReaderWrapper::from_index(self.index.clone())
}
pub fn add_i8(&mut self, data: i8, offset: i64) {
self.add_i64(data.into(), offset)
}

View File

@ -40,6 +40,21 @@ pub extern "C" fn tantivy_finish_index(ptr: *mut c_void) {
unsafe { Box::from_raw(real).finish() }
}
#[no_mangle]
pub extern "C" fn tantivy_commit_index(ptr: *mut c_void) {
let real = ptr as *mut IndexWriterWrapper;
unsafe {
(*real).commit();
}
}
#[no_mangle]
pub extern "C" fn tantivy_create_reader_from_writer(ptr: *mut c_void) -> *mut c_void {
let writer = ptr as *mut IndexWriterWrapper;
let reader = unsafe { (*writer).create_reader() };
create_binding(reader)
}
// -------------------------build--------------------
#[no_mangle]
pub extern "C" fn tantivy_index_add_int8s(

View File

@ -0,0 +1,52 @@
use std::sync::Arc;
use tantivy::schema::{Field, IndexRecordOption, Schema, TextFieldIndexing, TextOptions, FAST};
use tantivy::tokenizer::TextAnalyzer;
use tantivy::Index;
use crate::{index_writer::IndexWriterWrapper, log::init_log};
fn build_text_schema(field_name: &String, tokenizer_name: &String) -> (Schema, Field, Field) {
let mut schema_builder = Schema::builder();
// positions is required for matching phase.
let indexing = TextFieldIndexing::default()
.set_tokenizer(&tokenizer_name)
.set_index_option(IndexRecordOption::WithFreqsAndPositions);
let option = TextOptions::default().set_indexing_options(indexing);
let field = schema_builder.add_text_field(&field_name, option);
let id_field = schema_builder.add_i64_field("doc_id", FAST);
(schema_builder.build(), field, id_field)
}
impl IndexWriterWrapper {
pub(crate) fn create_text_writer(
field_name: String,
path: String,
tokenizer_name: String,
tokenizer: TextAnalyzer,
num_threads: usize,
overall_memory_budget_in_bytes: usize,
in_ram: bool,
) -> IndexWriterWrapper {
init_log();
let (schema, field, id_field) = build_text_schema(&field_name, &tokenizer_name);
let index: Index;
if in_ram {
index = Index::create_in_ram(schema);
} else {
index = Index::create_in_dir(path.clone(), schema).unwrap();
}
index.tokenizers().register(&tokenizer_name, tokenizer);
let index_writer = index
.writer_with_num_threads(num_threads, overall_memory_budget_in_bytes)
.unwrap();
IndexWriterWrapper {
field,
index_writer,
id_field,
index: Arc::new(index),
}
}
}

View File

@ -0,0 +1,44 @@
use std::collections::HashMap;
use std::ffi::c_char;
use std::ffi::c_void;
use std::ffi::CStr;
use crate::index_writer::IndexWriterWrapper;
use crate::tokenizer::create_tokenizer;
use crate::util::create_binding;
#[no_mangle]
pub extern "C" fn tantivy_create_text_writer(
field_name: *const c_char,
path: *const c_char,
tokenizer_name: *const c_char,
tokenizer_params: *mut c_void,
num_threads: usize,
overall_memory_budget_in_bytes: usize,
in_ram: bool,
) -> *mut c_void {
let field_name_str = unsafe { CStr::from_ptr(field_name).to_str().unwrap() };
let path_str = unsafe { CStr::from_ptr(path).to_str().unwrap() };
let tokenizer_name_str = unsafe { CStr::from_ptr(tokenizer_name).to_str().unwrap() };
let analyzer = unsafe {
let m = tokenizer_params as *const HashMap<String, String>;
create_tokenizer(&(*m))
};
match analyzer {
Some(text_analyzer) => {
let wrapper = IndexWriterWrapper::create_text_writer(
String::from(field_name_str),
String::from(path_str),
String::from(tokenizer_name_str),
text_analyzer,
num_threads,
overall_memory_budget_in_bytes,
in_ram,
);
create_binding(wrapper)
}
None => {
std::ptr::null_mut()
}
}
}

View File

@ -3,12 +3,14 @@ mod data_type;
mod demo_c;
mod docid_collector;
mod hashmap_c;
mod hashset_collector;
mod index_reader;
mod index_reader_c;
mod index_reader_text;
mod index_reader_text_c;
mod index_writer;
mod index_writer_c;
mod linkedlist_collector;
mod index_writer_text;
mod index_writer_text_c;
mod log;
mod string_c;
mod token_stream_c;

View File

@ -1,61 +0,0 @@
use std::collections::LinkedList;
use tantivy::{
collector::{Collector, SegmentCollector},
DocId,
};
pub struct LinkedListCollector;
impl Collector for LinkedListCollector {
type Fruit = LinkedList<DocId>;
type Child = LinkedListChildCollector;
fn for_segment(
&self,
_segment_local_id: tantivy::SegmentOrdinal,
_segment: &tantivy::SegmentReader,
) -> tantivy::Result<Self::Child> {
Ok(LinkedListChildCollector {
docs: LinkedList::new(),
})
}
fn requires_scoring(&self) -> bool {
false
}
fn merge_fruits(
&self,
segment_fruits: Vec<LinkedList<DocId>>,
) -> tantivy::Result<LinkedList<DocId>> {
if segment_fruits.len() == 1 {
Ok(segment_fruits.into_iter().next().unwrap())
} else {
let mut result = LinkedList::new();
for docs in segment_fruits {
for doc in docs {
result.push_front(doc);
}
}
Ok(result)
}
}
}
pub struct LinkedListChildCollector {
docs: LinkedList<DocId>,
}
impl SegmentCollector for LinkedListChildCollector {
type Fruit = LinkedList<DocId>;
fn collect(&mut self, doc: DocId, _score: tantivy::Score) {
self.docs.push_front(doc);
}
fn harvest(self) -> Self::Fruit {
self.docs
}
}

View File

@ -1,4 +1,4 @@
use std::ffi::{c_char};
use std::ffi::c_char;
use libc::c_void;
use tantivy::tokenizer::{BoxTokenStream, TextAnalyzer};

View File

@ -1,7 +1,8 @@
use lazy_static::lazy_static;
use log::info;
use log::{info, warn};
use std::collections::HashMap;
use tantivy::tokenizer::{TextAnalyzer, TokenizerManager};
use crate::log::init_log;
lazy_static! {
static ref DEFAULT_TOKENIZER_MANAGER: TokenizerManager = TokenizerManager::default();
@ -16,19 +17,39 @@ fn jieba_tokenizer() -> TextAnalyzer {
}
pub(crate) fn create_tokenizer(params: &HashMap<String, String>) -> Option<TextAnalyzer> {
init_log();
match params.get("tokenizer") {
Some(tokenizer_name) => match tokenizer_name.as_str() {
"default" => {
return Some(default_tokenizer());
Some(default_tokenizer())
}
"jieba" => return Some(jieba_tokenizer()),
_ => {
return None;
"jieba" => {
Some(jieba_tokenizer())
}
s => {
warn!("unsupported tokenizer: {}", s);
None
}
},
None => {
info!("no tokenizer is specific, use default tokenizer");
return Some(default_tokenizer());
Some(default_tokenizer())
}
}
}
#[cfg(test)]
mod tests {
use std::collections::HashMap;
use crate::tokenizer::create_tokenizer;
#[test]
fn test_create_tokenizer() {
let mut params : HashMap<String, String> = HashMap::new();
params.insert("tokenizer".parse().unwrap(), "jieba".parse().unwrap());
let tokenizer = create_tokenizer(&params);
assert!(tokenizer.is_some());
}
}

View File

@ -8,8 +8,13 @@
#include "tantivy-binding.h"
#include "rust-binding.h"
#include "rust-array.h"
#include "rust-hashmap.h"
namespace milvus::tantivy {
using Map = std::map<std::string, std::string>;
static constexpr const char* DEFAULT_TOKENIZER_NAME = "milvus_tokenizer";
static Map DEFAULT_TOKENIZER_PARAMS = {};
static constexpr uintptr_t DEFAULT_NUM_THREADS = 4;
static constexpr uintptr_t DEFAULT_OVERALL_MEMORY_BUDGET_IN_BYTES =
DEFAULT_NUM_THREADS * 15 * 1024 * 1024;
@ -69,6 +74,7 @@ struct TantivyIndexWrapper {
return *this;
}
// create index writer for non-text type.
TantivyIndexWrapper(const char* field_name,
TantivyDataType data_type,
const char* path,
@ -90,10 +96,55 @@ struct TantivyIndexWrapper {
path_ = std::string(path);
}
// create index writer for text type with tokenizer.
TantivyIndexWrapper(const char* field_name,
bool in_ram,
const char* path,
const char* tokenizer_name = DEFAULT_TOKENIZER_NAME,
const std::map<std::string, std::string>&
tokenizer_params = DEFAULT_TOKENIZER_PARAMS,
uintptr_t num_threads = DEFAULT_NUM_THREADS,
uintptr_t overall_memory_budget_in_bytes =
DEFAULT_OVERALL_MEMORY_BUDGET_IN_BYTES) {
RustHashMap m;
m.from(tokenizer_params);
writer_ = tantivy_create_text_writer(field_name,
path,
tokenizer_name,
m.get_pointer(),
num_threads,
overall_memory_budget_in_bytes,
in_ram);
path_ = std::string(path);
}
// create reader.
void
create_reader() {
if (writer_ != nullptr) {
reader_ = tantivy_create_reader_from_writer(writer_);
} else if (!path_.empty()) {
assert(tantivy_index_exist(path_.c_str()));
reader_ = tantivy_load_index(path_.c_str());
}
}
~TantivyIndexWrapper() {
free();
}
void
register_tokenizer(
const char* tokenizer_name,
const std::map<std::string, std::string>& tokenizer_params) {
RustHashMap m;
m.from(tokenizer_params);
if (reader_ != nullptr) {
tantivy_register_tokenizer(
reader_, tokenizer_name, m.get_pointer());
}
}
template <typename T>
void
add_data(const T* array, uintptr_t len, int64_t offset_begin) {
@ -212,10 +263,23 @@ struct TantivyIndexWrapper {
tantivy_finish_index(writer_);
writer_ = nullptr;
reader_ = tantivy_load_index(path_.c_str());
finished_ = true;
}
inline void
commit() {
if (writer_ != nullptr) {
tantivy_commit_index(writer_);
}
}
inline void
reload() {
if (reader_ != nullptr) {
tantivy_reload_index(reader_);
}
}
inline uint32_t
count() {
return tantivy_index_count(reader_);
@ -362,6 +426,12 @@ struct TantivyIndexWrapper {
return RustArrayWrapper(array);
}
RustArrayWrapper
match_query(const std::string& query) {
auto array = tantivy_match_query(reader_, query.c_str());
return RustArrayWrapper(array);
}
public:
inline IndexWriter
get_writer() {

View File

@ -0,0 +1,68 @@
#include <string>
#include <vector>
#include <boost/filesystem/operations.hpp>
#include "tantivy-binding.h"
#include "tantivy-wrapper.h"
using namespace milvus::tantivy;
std::set<uint32_t>
to_set(const RustArrayWrapper& w) {
std::set<uint32_t> s(w.array_.array, w.array_.array + w.array_.len);
return s;
}
int
main(int argc, char* argv[]) {
auto text_index = TantivyIndexWrapper("text_demo", true, "");
auto write_single_text = [&text_index](const std::string& s,
int64_t offset) {
text_index.add_data(&s, 1, offset);
};
{
write_single_text("football, basketball, pingpang", 0);
write_single_text("swimming, football", 1);
write_single_text("Avatar", 2);
write_single_text("Action, Adventure, Fantasy, Science Fiction", 3);
write_single_text("Ingenious Film Partners, Twentiesth Century Fox", 4);
write_single_text("Sam Worthington as Jack Sully", 5);
text_index.commit();
}
text_index.create_reader();
{
auto result = to_set(text_index.match_query("football"));
assert(result.size() == 2);
assert(result.find(0) != result.end());
assert(result.find(1) != result.end());
}
{
auto result = to_set(text_index.match_query("basketball"));
assert(result.size() == 1);
assert(result.find(0) != result.end());
}
{
auto result = to_set(text_index.match_query("swimming"));
assert(result.size() == 1);
assert(result.find(1) != result.end());
}
{
auto result = to_set(text_index.match_query("basketball, swimming"));
assert(result.size() == 2);
assert(result.find(0) != result.end());
assert(result.find(1) != result.end());
}
{
auto result = to_set(text_index.match_query("avatar"));
assert(result.size() == 1);
assert(result.find(2) != result.end());
}
return 0;
}

View File

@ -16,7 +16,7 @@ struct Tokenizer {
m.from(params);
ptr_ = tantivy_create_tokenizer(m.get_pointer());
if (ptr_ == nullptr) {
throw "invalid tokenizer parameters";
throw std::invalid_argument("invalid tokenizer parameters");
}
}

View File

@ -23,6 +23,10 @@ main(int argc, char* argv[]) {
{
Map m;
test_tokenizer(m, "football, basketball, pingpang");
test_tokenizer(m, "Avatar");
test_tokenizer(m, "Action, Adventure, Fantasy, Science Fiction");
test_tokenizer(m, "Ingenious Film Partners, Twentiesth Century Fox");
test_tokenizer(m, "Sam Worthington as Jack Sully");
}
// jieba tokenizer
@ -33,6 +37,10 @@ main(int argc, char* argv[]) {
test_tokenizer(m,
"张华考上了北京大学;李萍进了中等技术学校;我在百货公司"
"当售货员:我们都有光明的前途");
test_tokenizer(m, "青铜时代");
test_tokenizer(m, "黄金时代");
test_tokenizer(m, "时代");
test_tokenizer(m, "测试中文分词器的效果");
}
return 0;

View File

@ -23,29 +23,53 @@ add_definitions(-DMILVUS_TEST_SEGCORE_YAML_PATH="${CMAKE_SOURCE_DIR}/unittest/te
# TODO: better to use ls/find pattern
set(MILVUS_TEST_FILES
init_gtest.cpp
test_always_true_expr.cpp
test_array_bitmap_index.cpp
test_array_inverted_index.cpp
test_bf.cpp
test_bf_sparse.cpp
test_binary.cpp
test_binlog_index.cpp
test_bitmap_index.cpp
test_bool_index.cpp
test_c_api.cpp
test_chunk_cache.cpp
test_chunk.cpp
test_chunk_vector.cpp
test_common.cpp
test_concurrent_vector.cpp
test_c_api.cpp
test_expr_materialized_view.cpp
test_c_stream_reduce.cpp
test_c_tokenizer.cpp
test_data_codec.cpp
test_disk_file_manager_test.cpp
test_exec.cpp
test_expr.cpp
test_expr_materialized_view.cpp
test_float16.cpp
test_futures.cpp
test_group_by.cpp
test_growing.cpp
test_growing_index.cpp
test_indexing.cpp
test_bitmap_index.cpp
test_hybrid_index.cpp
test_array_bitmap_index.cpp
test_index_c_api.cpp
test_indexing.cpp
test_index_wrapper.cpp
test_init.cpp
test_integer_overflow.cpp
test_inverted_index.cpp
test_local_chunk_manager.cpp
test_mmap_chunk_manager.cpp
test_monitor.cpp
test_offset_ordered_array.cpp
test_offset_ordered_map.cpp
test_plan_proto.cpp
test_query.cpp
test_reduce.cpp
test_range_search_sort.cpp
test_reduce_c.cpp
test_reduce.cpp
test_regex_query.cpp
test_regex_query_util.cpp
test_relational.cpp
test_retrieve.cpp
test_scalar_index.cpp
@ -53,33 +77,12 @@ set(MILVUS_TEST_FILES
test_segcore.cpp
test_similarity_corelation.cpp
test_span.cpp
test_string_expr.cpp
test_timestamp_index.cpp
test_utils.cpp
test_data_codec.cpp
test_range_search_sort.cpp
test_tracer.cpp
test_local_chunk_manager.cpp
test_disk_file_manager_test.cpp
test_integer_overflow.cpp
test_offset_ordered_map.cpp
test_offset_ordered_array.cpp
test_always_true_expr.cpp
test_plan_proto.cpp
test_chunk_cache.cpp
test_binlog_index.cpp
test_storage.cpp
test_exec.cpp
test_inverted_index.cpp
test_group_by.cpp
test_regex_query_util.cpp
test_regex_query.cpp
test_futures.cpp
test_array_inverted_index.cpp
test_chunk_vector.cpp
test_mmap_chunk_manager.cpp
test_monitor.cpp
test_chunk.cpp
test_string_expr.cpp
test_text_match.cpp
test_timestamp_index.cpp
test_tracer.cpp
test_utils.cpp
)
if ( INDEX_ENGINE STREQUAL "cardinal" )

View File

@ -0,0 +1,75 @@
// 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/EasyAssert.h"
#include "pb/schema.pb.h"
#include "segcore/token_stream_c.h"
#include "segcore/tokenizer_c.h"
#include "segcore/map_c.h"
using Map = std::map<std::string, std::string>;
TEST(ValidateTextSchema, Default) {
milvus::proto::schema::FieldSchema schema;
std::vector<uint8_t> buffer(schema.ByteSizeLong());
schema.SerializeToArray(buffer.data(), buffer.size());
auto status = validate_text_schema(buffer.data(), buffer.size());
ASSERT_EQ(milvus::ErrorCode::Success, status.error_code);
}
TEST(ValidateTextSchema, JieBa) {
milvus::proto::schema::FieldSchema schema;
{
auto kv = schema.add_type_params();
kv->set_key("analyzer_params");
kv->set_value(R"({"tokenizer": "jieba"})");
}
std::vector<uint8_t> buffer(schema.ByteSizeLong());
schema.SerializeToArray(buffer.data(), buffer.size());
auto status = validate_text_schema(buffer.data(), buffer.size());
ASSERT_EQ(milvus::ErrorCode::Success, status.error_code);
}
void
set_cmap(CMap m, const std::string& key, const std::string& value) {
cmap_set(m, key.c_str(), key.length(), value.c_str(), value.length());
}
TEST(CTokenizer, Default) {
auto m = create_cmap();
set_cmap(m, "tokenizer", "default");
CTokenizer tokenizer;
{
auto status = create_tokenizer(m, &tokenizer);
ASSERT_EQ(milvus::ErrorCode::Success, status.error_code);
}
std::string text("football, basketball, swimming");
auto token_stream =
create_token_stream(tokenizer, text.c_str(), text.length());
std::vector<std::string> refs{"football", "basketball", "swimming"};
for (int i = 0; i < 3; i++) {
ASSERT_TRUE(token_stream_advance(token_stream));
auto token = token_stream_get_token(token_stream);
ASSERT_EQ(refs[i], std::string(token));
free_token(const_cast<char*>(token));
}
ASSERT_FALSE(token_stream_advance(token_stream));
free_token_stream(token_stream);
free_tokenizer(tokenizer);
free_cmap(m);
}

View File

@ -0,0 +1,407 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <gtest/gtest.h>
#include "common/Schema.h"
#include "segcore/segment_c.h"
#include "segcore/SegmentGrowing.h"
#include "segcore/SegmentGrowingImpl.h"
#include "test_utils/DataGen.h"
#include "test_utils/GenExprProto.h"
#include "query/PlanProto.h"
#include "query/generated/ExecPlanNodeVisitor.h"
using namespace milvus;
using namespace milvus::query;
using namespace milvus::segcore;
namespace {
SchemaPtr
GenTestSchema(std::map<std::string, std::string> params = {}) {
auto schema = std::make_shared<Schema>();
{
FieldMeta f(FieldName("pk"), FieldId(100), DataType::INT64, false);
schema->AddField(std::move(f));
schema->set_primary_field_id(FieldId(100));
}
{
FieldMeta f(FieldName("str"),
FieldId(101),
DataType::VARCHAR,
65536,
false,
true,
params);
schema->AddField(std::move(f));
}
{
FieldMeta f(FieldName("fvec"),
FieldId(102),
DataType::VECTOR_FLOAT,
16,
knowhere::metric::L2,
false);
schema->AddField(std::move(f));
}
return schema;
}
} // namespace
TEST(ParseJson, Naive) {
{
std::string s(R"({"tokenizer": "jieba"})");
nlohmann::json j = nlohmann::json::parse(s);
auto m = j.get<std::map<std::string, std::string>>();
for (const auto& [k, v] : m) {
std::cout << k << ": " << v << std::endl;
}
}
{
std::string s(
R"({"analyzer":"stop","stop_words":["an","the"],"case_insensitive":false})");
nlohmann::json j = nlohmann::json::parse(s);
for (const auto& [key, value] : j.items()) {
std::cout << key << ": " << value.dump() << std::endl;
}
}
}
TEST(ParseTokenizerParams, NoAnalyzerParams) {
TypeParams params{{"k", "v"}};
auto p = ParseTokenizerParams(params);
ASSERT_EQ(0, p.size());
}
TEST(ParseTokenizerParams, Default) {
TypeParams params{{"analyzer_params", R"({"tokenizer": "default"})"}};
auto p = ParseTokenizerParams(params);
ASSERT_EQ(1, p.size());
auto iter = p.find("tokenizer");
ASSERT_NE(p.end(), iter);
ASSERT_EQ("default", iter->second);
}
TEST(TextMatch, Index) {
using Index = index::TextMatchIndex;
auto index = std::make_unique<Index>(std::numeric_limits<int64_t>::max(),
"milvus_tokenizer",
std::map<std::string, std::string>{});
index->CreateReader();
index->AddText("football, basketball, pingpang", 0);
index->AddText("swimming, football", 1);
index->Commit();
index->Reload();
auto res = index->MatchQuery("football");
ASSERT_TRUE(res[0]);
ASSERT_TRUE(res[1]);
}
TEST(TextMatch, GrowingNaive) {
auto schema = GenTestSchema();
auto seg = CreateGrowingSegment(schema, empty_index_meta);
std::vector<std::string> raw_str = {"football, basketball, pingpang",
"swimming, football"};
int64_t N = 2;
uint64_t seed = 19190504;
auto raw_data = DataGen(schema, N, seed);
auto str_col = raw_data.raw_->mutable_fields_data()
->at(1)
.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_);
std::this_thread::sleep_for(std::chrono::milliseconds(200) * 2);
auto get_text_match_expr = [&schema](const std::string& query) -> auto {
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::TextMatch, query);
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);
return parsed;
};
{
auto expr = get_text_match_expr("football");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_TRUE(final[0]);
ASSERT_TRUE(final[1]);
}
{
auto expr = get_text_match_expr("swimming");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_FALSE(final[0]);
ASSERT_TRUE(final[1]);
}
{
auto expr = get_text_match_expr("basketball, swimming");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_TRUE(final[0]);
ASSERT_TRUE(final[1]);
}
}
TEST(TextMatch, SealedNaive) {
auto schema = GenTestSchema();
auto seg = CreateSealedSegment(schema, empty_index_meta);
std::vector<std::string> raw_str = {"football, basketball, pingpang",
"swimming, football"};
int64_t N = 2;
uint64_t seed = 19190504;
auto raw_data = DataGen(schema, N, seed);
auto str_col = raw_data.raw_->mutable_fields_data()
->at(1)
.mutable_scalars()
->mutable_string_data()
->mutable_data();
for (int64_t i = 0; i < N; i++) {
str_col->at(i) = raw_str[i];
}
SealedLoadFieldData(raw_data, *seg);
seg->CreateTextIndex(FieldId(101));
auto get_text_match_expr = [&schema](const std::string& query) -> auto {
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::TextMatch, query);
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);
return parsed;
};
{
auto expr = get_text_match_expr("football");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_TRUE(final[0]);
ASSERT_TRUE(final[1]);
}
{
auto expr = get_text_match_expr("swimming");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_FALSE(final[0]);
ASSERT_TRUE(final[1]);
}
{
auto expr = get_text_match_expr("basketball, swimming");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_TRUE(final[0]);
ASSERT_TRUE(final[1]);
}
}
TEST(TextMatch, GrowingJieBa) {
auto schema = GenTestSchema({
{"enable_match", "true"},
{"analyzer_params", R"({"tokenizer": "jieba"})"},
});
auto seg = CreateGrowingSegment(schema, empty_index_meta);
std::vector<std::string> raw_str = {"青铜时代", "黄金时代"};
int64_t N = 2;
uint64_t seed = 19190504;
auto raw_data = DataGen(schema, N, seed);
auto str_col = raw_data.raw_->mutable_fields_data()
->at(1)
.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_);
std::this_thread::sleep_for(std::chrono::milliseconds(200) * 2);
auto get_text_match_expr = [&schema](const std::string& query) -> auto {
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::TextMatch, query);
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);
return parsed;
};
{
auto expr = get_text_match_expr("青铜");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_TRUE(final[0]);
ASSERT_FALSE(final[1]);
}
{
auto expr = get_text_match_expr("黄金");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_FALSE(final[0]);
ASSERT_TRUE(final[1]);
}
{
auto expr = get_text_match_expr("时代");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_TRUE(final[0]);
ASSERT_TRUE(final[1]);
}
}
TEST(TextMatch, SealedJieBa) {
auto schema = GenTestSchema({
{"enable_match", "true"},
{"analyzer_params", R"({"tokenizer": "jieba"})"},
});
auto seg = CreateSealedSegment(schema, empty_index_meta);
std::vector<std::string> raw_str = {"青铜时代", "黄金时代"};
int64_t N = 2;
uint64_t seed = 19190504;
auto raw_data = DataGen(schema, N, seed);
auto str_col = raw_data.raw_->mutable_fields_data()
->at(1)
.mutable_scalars()
->mutable_string_data()
->mutable_data();
for (int64_t i = 0; i < N; i++) {
str_col->at(i) = raw_str[i];
}
SealedLoadFieldData(raw_data, *seg);
seg->CreateTextIndex(FieldId(101));
auto get_text_match_expr = [&schema](const std::string& query) -> auto {
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::TextMatch, query);
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);
return parsed;
};
{
auto expr = get_text_match_expr("青铜");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_TRUE(final[0]);
ASSERT_FALSE(final[1]);
}
{
auto expr = get_text_match_expr("黄金");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_FALSE(final[0]);
ASSERT_TRUE(final[1]);
}
{
auto expr = get_text_match_expr("时代");
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
BitsetType final;
visitor.ExecuteExprNode(expr, seg.get(), N, final);
ASSERT_EQ(final.size(), N);
ASSERT_TRUE(final[0]);
ASSERT_TRUE(final[1]);
}
}

View File

@ -27,19 +27,22 @@ import (
"github.com/samber/lo"
"go.opentelemetry.io/otel"
"go.uber.org/zap"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/datanode/compaction"
iter "github.com/milvus-io/milvus/internal/datanode/iterators"
"github.com/milvus-io/milvus/internal/flushcommon/io"
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexcgopb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/workerpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
_ "github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/metautil"
"github.com/milvus-io/milvus/pkg/util/timerecord"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
@ -290,6 +293,7 @@ func (st *statsTask) Execute(ctx context.Context) error {
st.req.GetPartitionID(),
st.req.GetTargetSegmentID(),
st.req.GetTaskVersion(),
st.req.GetTaskID(),
lo.Values(allBinlogs))
if err != nil {
log.Warn("stats wrong, failed to create text index", zap.Error(err))
@ -548,12 +552,23 @@ func buildTextLogPrefix(rootPath string, collID, partID, segID, fieldID, version
return fmt.Sprintf("%s/%s/%d/%d/%d/%d/%d", rootPath, common.TextIndexPath, collID, partID, segID, fieldID, version)
}
func ParseStorageConfig(s *indexpb.StorageConfig) (*indexcgopb.StorageConfig, error) {
bs, err := proto.Marshal(s)
if err != nil {
return nil, err
}
res := &indexcgopb.StorageConfig{}
err = proto.Unmarshal(bs, res)
return res, err
}
func (st *statsTask) createTextIndex(ctx context.Context,
storageConfig *indexpb.StorageConfig,
collectionID int64,
partitionID int64,
segmentID int64,
version int64,
buildID int64,
insertBinlogs []*datapb.FieldBinlog,
) (map[int64]*datapb.TextIndexStats, error) {
log := log.Ctx(ctx).With(
@ -564,22 +579,67 @@ func (st *statsTask) createTextIndex(ctx context.Context,
zap.Int64("segmentID", st.req.GetSegmentID()),
)
fieldStatsLogs := make(map[int64]*datapb.TextIndexStats)
for _, field := range st.req.GetSchema().GetFields() {
if field.GetDataType() == schemapb.DataType_VarChar {
for _, binlog := range insertBinlogs {
if binlog.GetFieldID() == field.GetFieldID() {
// do text index
_ = buildTextLogPrefix(storageConfig.GetRootPath(), collectionID, partitionID, segmentID, field.GetFieldID(), version)
fieldStatsLogs[field.GetFieldID()] = &datapb.TextIndexStats{
Version: version,
Files: nil,
}
log.Info("TODO: call CGO CreateTextIndex", zap.Int64("fieldID", field.GetFieldID()))
break
}
fieldBinlogs := lo.GroupBy(insertBinlogs, func(binlog *datapb.FieldBinlog) int64 {
return binlog.GetFieldID()
})
getInsertFiles := func(fieldID int64) ([]string, error) {
binlogs, ok := fieldBinlogs[fieldID]
if !ok {
return nil, fmt.Errorf("field binlog not found for field %d", fieldID)
}
result := make([]string, 0, len(binlogs))
for _, binlog := range binlogs {
for _, file := range binlog.GetBinlogs() {
result = append(result, metautil.BuildInsertLogPath(storageConfig.GetRootPath(), collectionID, partitionID, segmentID, fieldID, file.GetLogID()))
}
}
return result, nil
}
newStorageConfig, err := ParseStorageConfig(storageConfig)
if err != nil {
return nil, err
}
fieldStatsLogs := make(map[int64]*datapb.TextIndexStats)
for _, field := range st.req.GetSchema().GetFields() {
h := typeutil.CreateFieldSchemaHelper(field)
if !h.EnableMatch() {
continue
}
log.Info("field enable match, ready to create text index", zap.Int64("field id", field.GetFieldID()))
// create text index and upload the text index files.
files, err := getInsertFiles(field.GetFieldID())
if err != nil {
return nil, err
}
buildIndexParams := &indexcgopb.BuildIndexInfo{
BuildID: buildID,
CollectionID: collectionID,
PartitionID: partitionID,
SegmentID: segmentID,
IndexVersion: version,
InsertFiles: files,
FieldSchema: field,
StorageConfig: newStorageConfig,
}
uploaded, err := indexcgowrapper.CreateTextIndex(ctx, buildIndexParams)
if err != nil {
return nil, err
}
fieldStatsLogs[field.GetFieldID()] = &datapb.TextIndexStats{
FieldID: field.GetFieldID(),
Version: version,
BuildID: buildID,
Files: lo.Keys(uploaded),
}
log.Info("field enable match, create text index done",
zap.Int64("field id", field.GetFieldID()),
zap.Strings("files", lo.Keys(uploaded)),
)
}
totalElapse := st.tr.RecordSpan()

View File

@ -10,6 +10,7 @@ expr:
| '(' expr ')' # Parens
| '[' expr (',' expr)* ','? ']' # Array
| expr LIKE StringLiteral # Like
| TEXTMATCH'('Identifier',' StringLiteral')' # TextMatch
| expr POW expr # Power
| op = (ADD | SUB | BNOT | NOT) expr # Unary
// | '(' typeName ')' expr # Cast
@ -52,6 +53,7 @@ NE: '!=';
LIKE: 'like' | 'LIKE';
EXISTS: 'exists' | 'EXISTS';
TEXTMATCH: 'TextMatch';
ADD: '+';
SUB: '-';

View File

@ -13,6 +13,7 @@ null
'!='
null
null
'TextMatch'
'+'
'-'
'*'
@ -62,6 +63,7 @@ EQ
NE
LIKE
EXISTS
TEXTMATCH
ADD
SUB
MUL
@ -101,4 +103,4 @@ expr
atn:
[3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 3, 48, 131, 4, 2, 9, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 20, 10, 2, 12, 2, 14, 2, 23, 11, 2, 3, 2, 5, 2, 26, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 5, 2, 59, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 113, 10, 2, 12, 2, 14, 2, 116, 11, 2, 3, 2, 5, 2, 119, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 126, 10, 2, 12, 2, 14, 2, 129, 11, 2, 3, 2, 2, 3, 2, 3, 2, 2, 15, 4, 2, 16, 17, 29, 30, 4, 2, 34, 34, 37, 37, 4, 2, 35, 35, 38, 38, 4, 2, 36, 36, 39, 39, 4, 2, 44, 44, 46, 46, 3, 2, 18, 20, 3, 2, 16, 17, 3, 2, 22, 23, 3, 2, 8, 9, 3, 2, 10, 11, 3, 2, 8, 11, 3, 2, 12, 13, 3, 2, 31, 32, 2, 162, 2, 58, 3, 2, 2, 2, 4, 5, 8, 2, 1, 2, 5, 59, 7, 42, 2, 2, 6, 59, 7, 43, 2, 2, 7, 59, 7, 41, 2, 2, 8, 59, 7, 45, 2, 2, 9, 59, 7, 44, 2, 2, 10, 59, 7, 46, 2, 2, 11, 12, 7, 3, 2, 2, 12, 13, 5, 2, 2, 2, 13, 14, 7, 4, 2, 2, 14, 59, 3, 2, 2, 2, 15, 16, 7, 5, 2, 2, 16, 21, 5, 2, 2, 2, 17, 18, 7, 6, 2, 2, 18, 20, 5, 2, 2, 2, 19, 17, 3, 2, 2, 2, 20, 23, 3, 2, 2, 2, 21, 19, 3, 2, 2, 2, 21, 22, 3, 2, 2, 2, 22, 25, 3, 2, 2, 2, 23, 21, 3, 2, 2, 2, 24, 26, 7, 6, 2, 2, 25, 24, 3, 2, 2, 2, 25, 26, 3, 2, 2, 2, 26, 27, 3, 2, 2, 2, 27, 28, 7, 7, 2, 2, 28, 59, 3, 2, 2, 2, 29, 30, 9, 2, 2, 2, 30, 59, 5, 2, 2, 22, 31, 32, 9, 3, 2, 2, 32, 33, 7, 3, 2, 2, 33, 34, 5, 2, 2, 2, 34, 35, 7, 6, 2, 2, 35, 36, 5, 2, 2, 2, 36, 37, 7, 4, 2, 2, 37, 59, 3, 2, 2, 2, 38, 39, 9, 4, 2, 2, 39, 40, 7, 3, 2, 2, 40, 41, 5, 2, 2, 2, 41, 42, 7, 6, 2, 2, 42, 43, 5, 2, 2, 2, 43, 44, 7, 4, 2, 2, 44, 59, 3, 2, 2, 2, 45, 46, 9, 5, 2, 2, 46, 47, 7, 3, 2, 2, 47, 48, 5, 2, 2, 2, 48, 49, 7, 6, 2, 2, 49, 50, 5, 2, 2, 2, 50, 51, 7, 4, 2, 2, 51, 59, 3, 2, 2, 2, 52, 53, 7, 40, 2, 2, 53, 54, 7, 3, 2, 2, 54, 55, 9, 6, 2, 2, 55, 59, 7, 4, 2, 2, 56, 57, 7, 15, 2, 2, 57, 59, 5, 2, 2, 3, 58, 4, 3, 2, 2, 2, 58, 6, 3, 2, 2, 2, 58, 7, 3, 2, 2, 2, 58, 8, 3, 2, 2, 2, 58, 9, 3, 2, 2, 2, 58, 10, 3, 2, 2, 2, 58, 11, 3, 2, 2, 2, 58, 15, 3, 2, 2, 2, 58, 29, 3, 2, 2, 2, 58, 31, 3, 2, 2, 2, 58, 38, 3, 2, 2, 2, 58, 45, 3, 2, 2, 2, 58, 52, 3, 2, 2, 2, 58, 56, 3, 2, 2, 2, 59, 127, 3, 2, 2, 2, 60, 61, 12, 23, 2, 2, 61, 62, 7, 21, 2, 2, 62, 126, 5, 2, 2, 24, 63, 64, 12, 21, 2, 2, 64, 65, 9, 7, 2, 2, 65, 126, 5, 2, 2, 22, 66, 67, 12, 20, 2, 2, 67, 68, 9, 8, 2, 2, 68, 126, 5, 2, 2, 21, 69, 70, 12, 19, 2, 2, 70, 71, 9, 9, 2, 2, 71, 126, 5, 2, 2, 20, 72, 73, 12, 12, 2, 2, 73, 74, 9, 10, 2, 2, 74, 75, 9, 6, 2, 2, 75, 76, 9, 10, 2, 2, 76, 126, 5, 2, 2, 13, 77, 78, 12, 11, 2, 2, 78, 79, 9, 11, 2, 2, 79, 80, 9, 6, 2, 2, 80, 81, 9, 11, 2, 2, 81, 126, 5, 2, 2, 12, 82, 83, 12, 10, 2, 2, 83, 84, 9, 12, 2, 2, 84, 126, 5, 2, 2, 11, 85, 86, 12, 9, 2, 2, 86, 87, 9, 13, 2, 2, 87, 126, 5, 2, 2, 10, 88, 89, 12, 8, 2, 2, 89, 90, 7, 24, 2, 2, 90, 126, 5, 2, 2, 9, 91, 92, 12, 7, 2, 2, 92, 93, 7, 26, 2, 2, 93, 126, 5, 2, 2, 8, 94, 95, 12, 6, 2, 2, 95, 96, 7, 25, 2, 2, 96, 126, 5, 2, 2, 7, 97, 98, 12, 5, 2, 2, 98, 99, 7, 27, 2, 2, 99, 126, 5, 2, 2, 6, 100, 101, 12, 4, 2, 2, 101, 102, 7, 28, 2, 2, 102, 126, 5, 2, 2, 5, 103, 104, 12, 24, 2, 2, 104, 105, 7, 14, 2, 2, 105, 126, 7, 45, 2, 2, 106, 107, 12, 18, 2, 2, 107, 108, 9, 14, 2, 2, 108, 109, 7, 5, 2, 2, 109, 114, 5, 2, 2, 2, 110, 111, 7, 6, 2, 2, 111, 113, 5, 2, 2, 2, 112, 110, 3, 2, 2, 2, 113, 116, 3, 2, 2, 2, 114, 112, 3, 2, 2, 2, 114, 115, 3, 2, 2, 2, 115, 118, 3, 2, 2, 2, 116, 114, 3, 2, 2, 2, 117, 119, 7, 6, 2, 2, 118, 117, 3, 2, 2, 2, 118, 119, 3, 2, 2, 2, 119, 120, 3, 2, 2, 2, 120, 121, 7, 7, 2, 2, 121, 126, 3, 2, 2, 2, 122, 123, 12, 17, 2, 2, 123, 124, 9, 14, 2, 2, 124, 126, 7, 33, 2, 2, 125, 60, 3, 2, 2, 2, 125, 63, 3, 2, 2, 2, 125, 66, 3, 2, 2, 2, 125, 69, 3, 2, 2, 2, 125, 72, 3, 2, 2, 2, 125, 77, 3, 2, 2, 2, 125, 82, 3, 2, 2, 2, 125, 85, 3, 2, 2, 2, 125, 88, 3, 2, 2, 2, 125, 91, 3, 2, 2, 2, 125, 94, 3, 2, 2, 2, 125, 97, 3, 2, 2, 2, 125, 100, 3, 2, 2, 2, 125, 103, 3, 2, 2, 2, 125, 106, 3, 2, 2, 2, 125, 122, 3, 2, 2, 2, 126, 129, 3, 2, 2, 2, 127, 125, 3, 2, 2, 2, 127, 128, 3, 2, 2, 2, 128, 3, 3, 2, 2, 2, 129, 127, 3, 2, 2, 2, 9, 21, 25, 58, 114, 118, 125, 127]
[3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 3, 49, 137, 4, 2, 9, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 20, 10, 2, 12, 2, 14, 2, 23, 11, 2, 3, 2, 5, 2, 26, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 5, 2, 65, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 119, 10, 2, 12, 2, 14, 2, 122, 11, 2, 3, 2, 5, 2, 125, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 132, 10, 2, 12, 2, 14, 2, 135, 11, 2, 3, 2, 2, 3, 2, 3, 2, 2, 15, 4, 2, 17, 18, 30, 31, 4, 2, 35, 35, 38, 38, 4, 2, 36, 36, 39, 39, 4, 2, 37, 37, 40, 40, 4, 2, 45, 45, 47, 47, 3, 2, 19, 21, 3, 2, 17, 18, 3, 2, 23, 24, 3, 2, 8, 9, 3, 2, 10, 11, 3, 2, 8, 11, 3, 2, 12, 13, 3, 2, 32, 33, 2, 169, 2, 64, 3, 2, 2, 2, 4, 5, 8, 2, 1, 2, 5, 65, 7, 43, 2, 2, 6, 65, 7, 44, 2, 2, 7, 65, 7, 42, 2, 2, 8, 65, 7, 46, 2, 2, 9, 65, 7, 45, 2, 2, 10, 65, 7, 47, 2, 2, 11, 12, 7, 3, 2, 2, 12, 13, 5, 2, 2, 2, 13, 14, 7, 4, 2, 2, 14, 65, 3, 2, 2, 2, 15, 16, 7, 5, 2, 2, 16, 21, 5, 2, 2, 2, 17, 18, 7, 6, 2, 2, 18, 20, 5, 2, 2, 2, 19, 17, 3, 2, 2, 2, 20, 23, 3, 2, 2, 2, 21, 19, 3, 2, 2, 2, 21, 22, 3, 2, 2, 2, 22, 25, 3, 2, 2, 2, 23, 21, 3, 2, 2, 2, 24, 26, 7, 6, 2, 2, 25, 24, 3, 2, 2, 2, 25, 26, 3, 2, 2, 2, 26, 27, 3, 2, 2, 2, 27, 28, 7, 7, 2, 2, 28, 65, 3, 2, 2, 2, 29, 30, 7, 16, 2, 2, 30, 31, 7, 3, 2, 2, 31, 32, 7, 45, 2, 2, 32, 33, 7, 6, 2, 2, 33, 34, 7, 46, 2, 2, 34, 65, 7, 4, 2, 2, 35, 36, 9, 2, 2, 2, 36, 65, 5, 2, 2, 22, 37, 38, 9, 3, 2, 2, 38, 39, 7, 3, 2, 2, 39, 40, 5, 2, 2, 2, 40, 41, 7, 6, 2, 2, 41, 42, 5, 2, 2, 2, 42, 43, 7, 4, 2, 2, 43, 65, 3, 2, 2, 2, 44, 45, 9, 4, 2, 2, 45, 46, 7, 3, 2, 2, 46, 47, 5, 2, 2, 2, 47, 48, 7, 6, 2, 2, 48, 49, 5, 2, 2, 2, 49, 50, 7, 4, 2, 2, 50, 65, 3, 2, 2, 2, 51, 52, 9, 5, 2, 2, 52, 53, 7, 3, 2, 2, 53, 54, 5, 2, 2, 2, 54, 55, 7, 6, 2, 2, 55, 56, 5, 2, 2, 2, 56, 57, 7, 4, 2, 2, 57, 65, 3, 2, 2, 2, 58, 59, 7, 41, 2, 2, 59, 60, 7, 3, 2, 2, 60, 61, 9, 6, 2, 2, 61, 65, 7, 4, 2, 2, 62, 63, 7, 15, 2, 2, 63, 65, 5, 2, 2, 3, 64, 4, 3, 2, 2, 2, 64, 6, 3, 2, 2, 2, 64, 7, 3, 2, 2, 2, 64, 8, 3, 2, 2, 2, 64, 9, 3, 2, 2, 2, 64, 10, 3, 2, 2, 2, 64, 11, 3, 2, 2, 2, 64, 15, 3, 2, 2, 2, 64, 29, 3, 2, 2, 2, 64, 35, 3, 2, 2, 2, 64, 37, 3, 2, 2, 2, 64, 44, 3, 2, 2, 2, 64, 51, 3, 2, 2, 2, 64, 58, 3, 2, 2, 2, 64, 62, 3, 2, 2, 2, 65, 133, 3, 2, 2, 2, 66, 67, 12, 23, 2, 2, 67, 68, 7, 22, 2, 2, 68, 132, 5, 2, 2, 24, 69, 70, 12, 21, 2, 2, 70, 71, 9, 7, 2, 2, 71, 132, 5, 2, 2, 22, 72, 73, 12, 20, 2, 2, 73, 74, 9, 8, 2, 2, 74, 132, 5, 2, 2, 21, 75, 76, 12, 19, 2, 2, 76, 77, 9, 9, 2, 2, 77, 132, 5, 2, 2, 20, 78, 79, 12, 12, 2, 2, 79, 80, 9, 10, 2, 2, 80, 81, 9, 6, 2, 2, 81, 82, 9, 10, 2, 2, 82, 132, 5, 2, 2, 13, 83, 84, 12, 11, 2, 2, 84, 85, 9, 11, 2, 2, 85, 86, 9, 6, 2, 2, 86, 87, 9, 11, 2, 2, 87, 132, 5, 2, 2, 12, 88, 89, 12, 10, 2, 2, 89, 90, 9, 12, 2, 2, 90, 132, 5, 2, 2, 11, 91, 92, 12, 9, 2, 2, 92, 93, 9, 13, 2, 2, 93, 132, 5, 2, 2, 10, 94, 95, 12, 8, 2, 2, 95, 96, 7, 25, 2, 2, 96, 132, 5, 2, 2, 9, 97, 98, 12, 7, 2, 2, 98, 99, 7, 27, 2, 2, 99, 132, 5, 2, 2, 8, 100, 101, 12, 6, 2, 2, 101, 102, 7, 26, 2, 2, 102, 132, 5, 2, 2, 7, 103, 104, 12, 5, 2, 2, 104, 105, 7, 28, 2, 2, 105, 132, 5, 2, 2, 6, 106, 107, 12, 4, 2, 2, 107, 108, 7, 29, 2, 2, 108, 132, 5, 2, 2, 5, 109, 110, 12, 25, 2, 2, 110, 111, 7, 14, 2, 2, 111, 132, 7, 46, 2, 2, 112, 113, 12, 18, 2, 2, 113, 114, 9, 14, 2, 2, 114, 115, 7, 5, 2, 2, 115, 120, 5, 2, 2, 2, 116, 117, 7, 6, 2, 2, 117, 119, 5, 2, 2, 2, 118, 116, 3, 2, 2, 2, 119, 122, 3, 2, 2, 2, 120, 118, 3, 2, 2, 2, 120, 121, 3, 2, 2, 2, 121, 124, 3, 2, 2, 2, 122, 120, 3, 2, 2, 2, 123, 125, 7, 6, 2, 2, 124, 123, 3, 2, 2, 2, 124, 125, 3, 2, 2, 2, 125, 126, 3, 2, 2, 2, 126, 127, 7, 7, 2, 2, 127, 132, 3, 2, 2, 2, 128, 129, 12, 17, 2, 2, 129, 130, 9, 14, 2, 2, 130, 132, 7, 34, 2, 2, 131, 66, 3, 2, 2, 2, 131, 69, 3, 2, 2, 2, 131, 72, 3, 2, 2, 2, 131, 75, 3, 2, 2, 2, 131, 78, 3, 2, 2, 2, 131, 83, 3, 2, 2, 2, 131, 88, 3, 2, 2, 2, 131, 91, 3, 2, 2, 2, 131, 94, 3, 2, 2, 2, 131, 97, 3, 2, 2, 2, 131, 100, 3, 2, 2, 2, 131, 103, 3, 2, 2, 2, 131, 106, 3, 2, 2, 2, 131, 109, 3, 2, 2, 2, 131, 112, 3, 2, 2, 2, 131, 128, 3, 2, 2, 2, 132, 135, 3, 2, 2, 2, 133, 131, 3, 2, 2, 2, 133, 134, 3, 2, 2, 2, 134, 3, 3, 2, 2, 2, 135, 133, 3, 2, 2, 2, 9, 21, 25, 64, 120, 124, 131, 133]

View File

@ -11,39 +11,40 @@ EQ=10
NE=11
LIKE=12
EXISTS=13
ADD=14
SUB=15
MUL=16
DIV=17
MOD=18
POW=19
SHL=20
SHR=21
BAND=22
BOR=23
BXOR=24
AND=25
OR=26
BNOT=27
NOT=28
IN=29
NIN=30
EmptyTerm=31
JSONContains=32
JSONContainsAll=33
JSONContainsAny=34
ArrayContains=35
ArrayContainsAll=36
ArrayContainsAny=37
ArrayLength=38
BooleanConstant=39
IntegerConstant=40
FloatingConstant=41
Identifier=42
StringLiteral=43
JSONIdentifier=44
Whitespace=45
Newline=46
TEXTMATCH=14
ADD=15
SUB=16
MUL=17
DIV=18
MOD=19
POW=20
SHL=21
SHR=22
BAND=23
BOR=24
BXOR=25
AND=26
OR=27
BNOT=28
NOT=29
IN=30
NIN=31
EmptyTerm=32
JSONContains=33
JSONContainsAll=34
JSONContainsAny=35
ArrayContains=36
ArrayContainsAll=37
ArrayContainsAny=38
ArrayLength=39
BooleanConstant=40
IntegerConstant=41
FloatingConstant=42
Identifier=43
StringLiteral=44
JSONIdentifier=45
Whitespace=46
Newline=47
'('=1
')'=2
'['=3
@ -55,17 +56,18 @@ Newline=46
'>='=9
'=='=10
'!='=11
'+'=14
'-'=15
'*'=16
'/'=17
'%'=18
'**'=19
'<<'=20
'>>'=21
'&'=22
'|'=23
'^'=24
'~'=27
'in'=29
'not in'=30
'TextMatch'=14
'+'=15
'-'=16
'*'=17
'/'=18
'%'=19
'**'=20
'<<'=21
'>>'=22
'&'=23
'|'=24
'^'=25
'~'=28
'in'=30
'not in'=31

File diff suppressed because one or more lines are too long

View File

@ -11,39 +11,40 @@ EQ=10
NE=11
LIKE=12
EXISTS=13
ADD=14
SUB=15
MUL=16
DIV=17
MOD=18
POW=19
SHL=20
SHR=21
BAND=22
BOR=23
BXOR=24
AND=25
OR=26
BNOT=27
NOT=28
IN=29
NIN=30
EmptyTerm=31
JSONContains=32
JSONContainsAll=33
JSONContainsAny=34
ArrayContains=35
ArrayContainsAll=36
ArrayContainsAny=37
ArrayLength=38
BooleanConstant=39
IntegerConstant=40
FloatingConstant=41
Identifier=42
StringLiteral=43
JSONIdentifier=44
Whitespace=45
Newline=46
TEXTMATCH=14
ADD=15
SUB=16
MUL=17
DIV=18
MOD=19
POW=20
SHL=21
SHR=22
BAND=23
BOR=24
BXOR=25
AND=26
OR=27
BNOT=28
NOT=29
IN=30
NIN=31
EmptyTerm=32
JSONContains=33
JSONContainsAll=34
JSONContainsAny=35
ArrayContains=36
ArrayContainsAll=37
ArrayContainsAny=38
ArrayLength=39
BooleanConstant=40
IntegerConstant=41
FloatingConstant=42
Identifier=43
StringLiteral=44
JSONIdentifier=45
Whitespace=46
Newline=47
'('=1
')'=2
'['=3
@ -55,17 +56,18 @@ Newline=46
'>='=9
'=='=10
'!='=11
'+'=14
'-'=15
'*'=16
'/'=17
'%'=18
'**'=19
'<<'=20
'>>'=21
'&'=22
'|'=23
'^'=24
'~'=27
'in'=29
'not in'=30
'TextMatch'=14
'+'=15
'-'=16
'*'=17
'/'=18
'%'=19
'**'=20
'<<'=21
'>>'=22
'&'=23
'|'=24
'^'=25
'~'=28
'in'=30
'not in'=31

View File

@ -79,6 +79,10 @@ func (v *BasePlanVisitor) VisitArrayLength(ctx *ArrayLengthContext) interface{}
return v.VisitChildren(ctx)
}
func (v *BasePlanVisitor) VisitTextMatch(ctx *TextMatchContext) interface{} {
return v.VisitChildren(ctx)
}
func (v *BasePlanVisitor) VisitTerm(ctx *TermContext) interface{} {
return v.VisitChildren(ctx)
}

View File

@ -14,7 +14,7 @@ var _ = fmt.Printf
var _ = unicode.IsLetter
var serializedLexerAtn = []uint16{
3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 2, 48, 754,
3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 2, 49, 766,
8, 1, 4, 2, 9, 2, 4, 3, 9, 3, 4, 4, 9, 4, 4, 5, 9, 5, 4, 6, 9, 6, 4, 7,
9, 7, 4, 8, 9, 8, 4, 9, 9, 9, 4, 10, 9, 10, 4, 11, 9, 11, 4, 12, 9, 12,
4, 13, 9, 13, 4, 14, 9, 14, 4, 15, 9, 15, 4, 16, 9, 16, 4, 17, 9, 17, 4,
@ -28,334 +28,339 @@ var serializedLexerAtn = []uint16{
4, 55, 9, 55, 4, 56, 9, 56, 4, 57, 9, 57, 4, 58, 9, 58, 4, 59, 9, 59, 4,
60, 9, 60, 4, 61, 9, 61, 4, 62, 9, 62, 4, 63, 9, 63, 4, 64, 9, 64, 4, 65,
9, 65, 4, 66, 9, 66, 4, 67, 9, 67, 4, 68, 9, 68, 4, 69, 9, 69, 4, 70, 9,
70, 4, 71, 9, 71, 4, 72, 9, 72, 3, 2, 3, 2, 3, 3, 3, 3, 3, 4, 3, 4, 3,
5, 3, 5, 3, 6, 3, 6, 3, 7, 3, 7, 3, 8, 3, 8, 3, 8, 3, 9, 3, 9, 3, 10, 3,
10, 3, 10, 3, 11, 3, 11, 3, 11, 3, 12, 3, 12, 3, 12, 3, 13, 3, 13, 3, 13,
3, 13, 3, 13, 3, 13, 3, 13, 3, 13, 5, 13, 180, 10, 13, 3, 14, 3, 14, 3,
14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 5, 14,
194, 10, 14, 3, 15, 3, 15, 3, 16, 3, 16, 3, 17, 3, 17, 3, 18, 3, 18, 3,
19, 3, 19, 3, 20, 3, 20, 3, 20, 3, 21, 3, 21, 3, 21, 3, 22, 3, 22, 3, 22,
3, 23, 3, 23, 3, 24, 3, 24, 3, 25, 3, 25, 3, 26, 3, 26, 3, 26, 3, 26, 3,
26, 5, 26, 226, 10, 26, 3, 27, 3, 27, 3, 27, 3, 27, 5, 27, 232, 10, 27,
3, 28, 3, 28, 3, 29, 3, 29, 3, 29, 3, 29, 5, 29, 240, 10, 29, 3, 30, 3,
30, 3, 30, 3, 31, 3, 31, 3, 31, 3, 31, 3, 31, 3, 31, 3, 31, 3, 32, 3, 32,
3, 32, 7, 32, 255, 10, 32, 12, 32, 14, 32, 258, 11, 32, 3, 32, 3, 32, 3,
33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33,
3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3,
33, 3, 33, 3, 33, 3, 33, 3, 33, 5, 33, 288, 10, 33, 3, 34, 3, 34, 3, 34,
70, 4, 71, 9, 71, 4, 72, 9, 72, 4, 73, 9, 73, 3, 2, 3, 2, 3, 3, 3, 3, 3,
4, 3, 4, 3, 5, 3, 5, 3, 6, 3, 6, 3, 7, 3, 7, 3, 8, 3, 8, 3, 8, 3, 9, 3,
9, 3, 10, 3, 10, 3, 10, 3, 11, 3, 11, 3, 11, 3, 12, 3, 12, 3, 12, 3, 13,
3, 13, 3, 13, 3, 13, 3, 13, 3, 13, 3, 13, 3, 13, 5, 13, 182, 10, 13, 3,
14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14, 3, 14,
3, 14, 5, 14, 196, 10, 14, 3, 15, 3, 15, 3, 15, 3, 15, 3, 15, 3, 15, 3,
15, 3, 15, 3, 15, 3, 15, 3, 16, 3, 16, 3, 17, 3, 17, 3, 18, 3, 18, 3, 19,
3, 19, 3, 20, 3, 20, 3, 21, 3, 21, 3, 21, 3, 22, 3, 22, 3, 22, 3, 23, 3,
23, 3, 23, 3, 24, 3, 24, 3, 25, 3, 25, 3, 26, 3, 26, 3, 27, 3, 27, 3, 27,
3, 27, 3, 27, 5, 27, 238, 10, 27, 3, 28, 3, 28, 3, 28, 3, 28, 5, 28, 244,
10, 28, 3, 29, 3, 29, 3, 30, 3, 30, 3, 30, 3, 30, 5, 30, 252, 10, 30, 3,
31, 3, 31, 3, 31, 3, 32, 3, 32, 3, 32, 3, 32, 3, 32, 3, 32, 3, 32, 3, 33,
3, 33, 3, 33, 7, 33, 267, 10, 33, 12, 33, 14, 33, 270, 11, 33, 3, 33, 3,
33, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34,
3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3,
34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34,
3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 5,
34, 324, 10, 34, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35,
34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 5, 34, 300, 10, 34, 3, 35, 3, 35,
3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3,
35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35,
3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 5, 35, 360, 10, 35, 3, 36, 3, 36, 3,
36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36,
3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3,
35, 5, 35, 336, 10, 35, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36,
3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3,
36, 3, 36, 3, 36, 3, 36, 3, 36, 5, 36, 390, 10, 36, 3, 37, 3, 37, 3, 37,
3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3,
36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36,
3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 5, 36, 372, 10, 36, 3, 37, 3,
37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37,
3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3,
37, 3, 37, 5, 37, 428, 10, 37, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38,
37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 5, 37, 402, 10, 37, 3, 38, 3, 38,
3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3,
38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38,
3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 5, 38, 466,
10, 38, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39,
3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3,
38, 3, 38, 3, 38, 5, 38, 440, 10, 38, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39,
3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3,
39, 3, 39, 3, 39, 3, 39, 3, 39, 5, 39, 492, 10, 39, 3, 40, 3, 40, 3, 40,
39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39,
3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 5,
39, 478, 10, 39, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40,
3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3,
40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40,
3, 40, 3, 40, 3, 40, 5, 40, 521, 10, 40, 3, 41, 3, 41, 3, 41, 3, 41, 5,
41, 527, 10, 41, 3, 42, 3, 42, 5, 42, 531, 10, 42, 3, 43, 3, 43, 3, 43,
7, 43, 536, 10, 43, 12, 43, 14, 43, 539, 11, 43, 3, 43, 3, 43, 3, 43, 3,
43, 3, 43, 5, 43, 546, 10, 43, 3, 44, 5, 44, 549, 10, 44, 3, 44, 3, 44,
5, 44, 553, 10, 44, 3, 44, 3, 44, 3, 44, 5, 44, 558, 10, 44, 3, 44, 5,
44, 561, 10, 44, 3, 45, 3, 45, 3, 45, 3, 45, 5, 45, 567, 10, 45, 3, 45,
3, 45, 6, 45, 571, 10, 45, 13, 45, 14, 45, 572, 3, 46, 3, 46, 3, 46, 5,
46, 578, 10, 46, 3, 47, 6, 47, 581, 10, 47, 13, 47, 14, 47, 582, 3, 48,
6, 48, 586, 10, 48, 13, 48, 14, 48, 587, 3, 49, 3, 49, 3, 49, 3, 49, 3,
49, 3, 49, 3, 49, 5, 49, 597, 10, 49, 3, 50, 3, 50, 3, 50, 3, 50, 3, 50,
3, 50, 3, 50, 5, 50, 606, 10, 50, 3, 51, 3, 51, 3, 52, 3, 52, 3, 53, 3,
53, 3, 53, 6, 53, 615, 10, 53, 13, 53, 14, 53, 616, 3, 54, 3, 54, 7, 54,
621, 10, 54, 12, 54, 14, 54, 624, 11, 54, 3, 54, 5, 54, 627, 10, 54, 3,
55, 3, 55, 7, 55, 631, 10, 55, 12, 55, 14, 55, 634, 11, 55, 3, 56, 3, 56,
3, 56, 3, 56, 3, 57, 3, 57, 3, 58, 3, 58, 3, 59, 3, 59, 3, 60, 3, 60, 3,
60, 3, 60, 3, 60, 3, 61, 3, 61, 3, 61, 3, 61, 3, 61, 3, 61, 3, 61, 3, 61,
3, 61, 3, 61, 5, 61, 661, 10, 61, 3, 62, 3, 62, 5, 62, 665, 10, 62, 3,
62, 3, 62, 3, 62, 5, 62, 670, 10, 62, 3, 63, 3, 63, 3, 63, 3, 63, 5, 63,
676, 10, 63, 3, 63, 3, 63, 3, 64, 5, 64, 681, 10, 64, 3, 64, 3, 64, 3,
64, 3, 64, 3, 64, 5, 64, 688, 10, 64, 3, 65, 3, 65, 5, 65, 692, 10, 65,
3, 65, 3, 65, 3, 66, 6, 66, 697, 10, 66, 13, 66, 14, 66, 698, 3, 67, 5,
67, 702, 10, 67, 3, 67, 3, 67, 3, 67, 3, 67, 3, 67, 5, 67, 709, 10, 67,
3, 68, 6, 68, 712, 10, 68, 13, 68, 14, 68, 713, 3, 69, 3, 69, 5, 69, 718,
10, 69, 3, 69, 3, 69, 3, 70, 3, 70, 3, 70, 3, 70, 3, 70, 5, 70, 727, 10,
70, 3, 70, 5, 70, 730, 10, 70, 3, 70, 3, 70, 3, 70, 3, 70, 3, 70, 5, 70,
737, 10, 70, 3, 71, 6, 71, 740, 10, 71, 13, 71, 14, 71, 741, 3, 71, 3,
71, 3, 72, 3, 72, 5, 72, 748, 10, 72, 3, 72, 5, 72, 751, 10, 72, 3, 72,
3, 72, 2, 2, 73, 3, 3, 5, 4, 7, 5, 9, 6, 11, 7, 13, 8, 15, 9, 17, 10, 19,
11, 21, 12, 23, 13, 25, 14, 27, 15, 29, 16, 31, 17, 33, 18, 35, 19, 37,
20, 39, 21, 41, 22, 43, 23, 45, 24, 47, 25, 49, 26, 51, 27, 53, 28, 55,
29, 57, 30, 59, 31, 61, 32, 63, 33, 65, 34, 67, 35, 69, 36, 71, 37, 73,
38, 75, 39, 77, 40, 79, 41, 81, 42, 83, 43, 85, 44, 87, 45, 89, 46, 91,
2, 93, 2, 95, 2, 97, 2, 99, 2, 101, 2, 103, 2, 105, 2, 107, 2, 109, 2,
111, 2, 113, 2, 115, 2, 117, 2, 119, 2, 121, 2, 123, 2, 125, 2, 127, 2,
129, 2, 131, 2, 133, 2, 135, 2, 137, 2, 139, 2, 141, 47, 143, 48, 3, 2,
18, 5, 2, 78, 78, 87, 87, 119, 119, 6, 2, 12, 12, 15, 15, 36, 36, 94, 94,
6, 2, 12, 12, 15, 15, 41, 41, 94, 94, 5, 2, 67, 92, 97, 97, 99, 124, 3,
2, 50, 59, 4, 2, 68, 68, 100, 100, 3, 2, 50, 51, 4, 2, 90, 90, 122, 122,
3, 2, 51, 59, 3, 2, 50, 57, 5, 2, 50, 59, 67, 72, 99, 104, 4, 2, 71, 71,
103, 103, 4, 2, 45, 45, 47, 47, 4, 2, 82, 82, 114, 114, 12, 2, 36, 36,
41, 41, 65, 65, 94, 94, 99, 100, 104, 104, 112, 112, 116, 116, 118, 118,
120, 120, 4, 2, 11, 11, 34, 34, 2, 793, 2, 3, 3, 2, 2, 2, 2, 5, 3, 2, 2,
2, 2, 7, 3, 2, 2, 2, 2, 9, 3, 2, 2, 2, 2, 11, 3, 2, 2, 2, 2, 13, 3, 2,
2, 2, 2, 15, 3, 2, 2, 2, 2, 17, 3, 2, 2, 2, 2, 19, 3, 2, 2, 2, 2, 21, 3,
2, 2, 2, 2, 23, 3, 2, 2, 2, 2, 25, 3, 2, 2, 2, 2, 27, 3, 2, 2, 2, 2, 29,
3, 2, 2, 2, 2, 31, 3, 2, 2, 2, 2, 33, 3, 2, 2, 2, 2, 35, 3, 2, 2, 2, 2,
37, 3, 2, 2, 2, 2, 39, 3, 2, 2, 2, 2, 41, 3, 2, 2, 2, 2, 43, 3, 2, 2, 2,
2, 45, 3, 2, 2, 2, 2, 47, 3, 2, 2, 2, 2, 49, 3, 2, 2, 2, 2, 51, 3, 2, 2,
2, 2, 53, 3, 2, 2, 2, 2, 55, 3, 2, 2, 2, 2, 57, 3, 2, 2, 2, 2, 59, 3, 2,
2, 2, 2, 61, 3, 2, 2, 2, 2, 63, 3, 2, 2, 2, 2, 65, 3, 2, 2, 2, 2, 67, 3,
2, 2, 2, 2, 69, 3, 2, 2, 2, 2, 71, 3, 2, 2, 2, 2, 73, 3, 2, 2, 2, 2, 75,
3, 2, 2, 2, 2, 77, 3, 2, 2, 2, 2, 79, 3, 2, 2, 2, 2, 81, 3, 2, 2, 2, 2,
83, 3, 2, 2, 2, 2, 85, 3, 2, 2, 2, 2, 87, 3, 2, 2, 2, 2, 89, 3, 2, 2, 2,
2, 141, 3, 2, 2, 2, 2, 143, 3, 2, 2, 2, 3, 145, 3, 2, 2, 2, 5, 147, 3,
2, 2, 2, 7, 149, 3, 2, 2, 2, 9, 151, 3, 2, 2, 2, 11, 153, 3, 2, 2, 2, 13,
155, 3, 2, 2, 2, 15, 157, 3, 2, 2, 2, 17, 160, 3, 2, 2, 2, 19, 162, 3,
2, 2, 2, 21, 165, 3, 2, 2, 2, 23, 168, 3, 2, 2, 2, 25, 179, 3, 2, 2, 2,
27, 193, 3, 2, 2, 2, 29, 195, 3, 2, 2, 2, 31, 197, 3, 2, 2, 2, 33, 199,
3, 2, 2, 2, 35, 201, 3, 2, 2, 2, 37, 203, 3, 2, 2, 2, 39, 205, 3, 2, 2,
2, 41, 208, 3, 2, 2, 2, 43, 211, 3, 2, 2, 2, 45, 214, 3, 2, 2, 2, 47, 216,
3, 2, 2, 2, 49, 218, 3, 2, 2, 2, 51, 225, 3, 2, 2, 2, 53, 231, 3, 2, 2,
2, 55, 233, 3, 2, 2, 2, 57, 239, 3, 2, 2, 2, 59, 241, 3, 2, 2, 2, 61, 244,
3, 2, 2, 2, 63, 251, 3, 2, 2, 2, 65, 287, 3, 2, 2, 2, 67, 323, 3, 2, 2,
2, 69, 359, 3, 2, 2, 2, 71, 389, 3, 2, 2, 2, 73, 427, 3, 2, 2, 2, 75, 465,
3, 2, 2, 2, 77, 491, 3, 2, 2, 2, 79, 520, 3, 2, 2, 2, 81, 526, 3, 2, 2,
2, 83, 530, 3, 2, 2, 2, 85, 545, 3, 2, 2, 2, 87, 548, 3, 2, 2, 2, 89, 562,
3, 2, 2, 2, 91, 577, 3, 2, 2, 2, 93, 580, 3, 2, 2, 2, 95, 585, 3, 2, 2,
2, 97, 596, 3, 2, 2, 2, 99, 605, 3, 2, 2, 2, 101, 607, 3, 2, 2, 2, 103,
609, 3, 2, 2, 2, 105, 611, 3, 2, 2, 2, 107, 626, 3, 2, 2, 2, 109, 628,
3, 2, 2, 2, 111, 635, 3, 2, 2, 2, 113, 639, 3, 2, 2, 2, 115, 641, 3, 2,
2, 2, 117, 643, 3, 2, 2, 2, 119, 645, 3, 2, 2, 2, 121, 660, 3, 2, 2, 2,
123, 669, 3, 2, 2, 2, 125, 671, 3, 2, 2, 2, 127, 687, 3, 2, 2, 2, 129,
689, 3, 2, 2, 2, 131, 696, 3, 2, 2, 2, 133, 708, 3, 2, 2, 2, 135, 711,
3, 2, 2, 2, 137, 715, 3, 2, 2, 2, 139, 736, 3, 2, 2, 2, 141, 739, 3, 2,
2, 2, 143, 750, 3, 2, 2, 2, 145, 146, 7, 42, 2, 2, 146, 4, 3, 2, 2, 2,
147, 148, 7, 43, 2, 2, 148, 6, 3, 2, 2, 2, 149, 150, 7, 93, 2, 2, 150,
8, 3, 2, 2, 2, 151, 152, 7, 46, 2, 2, 152, 10, 3, 2, 2, 2, 153, 154, 7,
95, 2, 2, 154, 12, 3, 2, 2, 2, 155, 156, 7, 62, 2, 2, 156, 14, 3, 2, 2,
2, 157, 158, 7, 62, 2, 2, 158, 159, 7, 63, 2, 2, 159, 16, 3, 2, 2, 2, 160,
161, 7, 64, 2, 2, 161, 18, 3, 2, 2, 2, 162, 163, 7, 64, 2, 2, 163, 164,
7, 63, 2, 2, 164, 20, 3, 2, 2, 2, 165, 166, 7, 63, 2, 2, 166, 167, 7, 63,
2, 2, 167, 22, 3, 2, 2, 2, 168, 169, 7, 35, 2, 2, 169, 170, 7, 63, 2, 2,
170, 24, 3, 2, 2, 2, 171, 172, 7, 110, 2, 2, 172, 173, 7, 107, 2, 2, 173,
174, 7, 109, 2, 2, 174, 180, 7, 103, 2, 2, 175, 176, 7, 78, 2, 2, 176,
177, 7, 75, 2, 2, 177, 178, 7, 77, 2, 2, 178, 180, 7, 71, 2, 2, 179, 171,
3, 2, 2, 2, 179, 175, 3, 2, 2, 2, 180, 26, 3, 2, 2, 2, 181, 182, 7, 103,
2, 2, 182, 183, 7, 122, 2, 2, 183, 184, 7, 107, 2, 2, 184, 185, 7, 117,
2, 2, 185, 186, 7, 118, 2, 2, 186, 194, 7, 117, 2, 2, 187, 188, 7, 71,
2, 2, 188, 189, 7, 90, 2, 2, 189, 190, 7, 75, 2, 2, 190, 191, 7, 85, 2,
2, 191, 192, 7, 86, 2, 2, 192, 194, 7, 85, 2, 2, 193, 181, 3, 2, 2, 2,
193, 187, 3, 2, 2, 2, 194, 28, 3, 2, 2, 2, 195, 196, 7, 45, 2, 2, 196,
30, 3, 2, 2, 2, 197, 198, 7, 47, 2, 2, 198, 32, 3, 2, 2, 2, 199, 200, 7,
44, 2, 2, 200, 34, 3, 2, 2, 2, 201, 202, 7, 49, 2, 2, 202, 36, 3, 2, 2,
2, 203, 204, 7, 39, 2, 2, 204, 38, 3, 2, 2, 2, 205, 206, 7, 44, 2, 2, 206,
207, 7, 44, 2, 2, 207, 40, 3, 2, 2, 2, 208, 209, 7, 62, 2, 2, 209, 210,
7, 62, 2, 2, 210, 42, 3, 2, 2, 2, 211, 212, 7, 64, 2, 2, 212, 213, 7, 64,
2, 2, 213, 44, 3, 2, 2, 2, 214, 215, 7, 40, 2, 2, 215, 46, 3, 2, 2, 2,
216, 217, 7, 126, 2, 2, 217, 48, 3, 2, 2, 2, 218, 219, 7, 96, 2, 2, 219,
50, 3, 2, 2, 2, 220, 221, 7, 40, 2, 2, 221, 226, 7, 40, 2, 2, 222, 223,
7, 99, 2, 2, 223, 224, 7, 112, 2, 2, 224, 226, 7, 102, 2, 2, 225, 220,
3, 2, 2, 2, 225, 222, 3, 2, 2, 2, 226, 52, 3, 2, 2, 2, 227, 228, 7, 126,
2, 2, 228, 232, 7, 126, 2, 2, 229, 230, 7, 113, 2, 2, 230, 232, 7, 116,
2, 2, 231, 227, 3, 2, 2, 2, 231, 229, 3, 2, 2, 2, 232, 54, 3, 2, 2, 2,
233, 234, 7, 128, 2, 2, 234, 56, 3, 2, 2, 2, 235, 240, 7, 35, 2, 2, 236,
237, 7, 112, 2, 2, 237, 238, 7, 113, 2, 2, 238, 240, 7, 118, 2, 2, 239,
235, 3, 2, 2, 2, 239, 236, 3, 2, 2, 2, 240, 58, 3, 2, 2, 2, 241, 242, 7,
107, 2, 2, 242, 243, 7, 112, 2, 2, 243, 60, 3, 2, 2, 2, 244, 245, 7, 112,
2, 2, 245, 246, 7, 113, 2, 2, 246, 247, 7, 118, 2, 2, 247, 248, 7, 34,
2, 2, 248, 249, 7, 107, 2, 2, 249, 250, 7, 112, 2, 2, 250, 62, 3, 2, 2,
2, 251, 256, 7, 93, 2, 2, 252, 255, 5, 141, 71, 2, 253, 255, 5, 143, 72,
2, 254, 252, 3, 2, 2, 2, 254, 253, 3, 2, 2, 2, 255, 258, 3, 2, 2, 2, 256,
254, 3, 2, 2, 2, 256, 257, 3, 2, 2, 2, 257, 259, 3, 2, 2, 2, 258, 256,
3, 2, 2, 2, 259, 260, 7, 95, 2, 2, 260, 64, 3, 2, 2, 2, 261, 262, 7, 108,
2, 2, 262, 263, 7, 117, 2, 2, 263, 264, 7, 113, 2, 2, 264, 265, 7, 112,
2, 2, 265, 266, 7, 97, 2, 2, 266, 267, 7, 101, 2, 2, 267, 268, 7, 113,
2, 2, 268, 269, 7, 112, 2, 2, 269, 270, 7, 118, 2, 2, 270, 271, 7, 99,
2, 2, 271, 272, 7, 107, 2, 2, 272, 273, 7, 112, 2, 2, 273, 288, 7, 117,
2, 2, 274, 275, 7, 76, 2, 2, 275, 276, 7, 85, 2, 2, 276, 277, 7, 81, 2,
2, 277, 278, 7, 80, 2, 2, 278, 279, 7, 97, 2, 2, 279, 280, 7, 69, 2, 2,
280, 281, 7, 81, 2, 2, 281, 282, 7, 80, 2, 2, 282, 283, 7, 86, 2, 2, 283,
284, 7, 67, 2, 2, 284, 285, 7, 75, 2, 2, 285, 286, 7, 80, 2, 2, 286, 288,
7, 85, 2, 2, 287, 261, 3, 2, 2, 2, 287, 274, 3, 2, 2, 2, 288, 66, 3, 2,
2, 2, 289, 290, 7, 108, 2, 2, 290, 291, 7, 117, 2, 2, 291, 292, 7, 113,
2, 2, 292, 293, 7, 112, 2, 2, 293, 294, 7, 97, 2, 2, 294, 295, 7, 101,
2, 2, 295, 296, 7, 113, 2, 2, 296, 297, 7, 112, 2, 2, 297, 298, 7, 118,
2, 2, 298, 299, 7, 99, 2, 2, 299, 300, 7, 107, 2, 2, 300, 301, 7, 112,
2, 2, 301, 302, 7, 117, 2, 2, 302, 303, 7, 97, 2, 2, 303, 304, 7, 99, 2,
2, 304, 305, 7, 110, 2, 2, 305, 324, 7, 110, 2, 2, 306, 307, 7, 76, 2,
2, 307, 308, 7, 85, 2, 2, 308, 309, 7, 81, 2, 2, 309, 310, 7, 80, 2, 2,
310, 311, 7, 97, 2, 2, 311, 312, 7, 69, 2, 2, 312, 313, 7, 81, 2, 2, 313,
314, 7, 80, 2, 2, 314, 315, 7, 86, 2, 2, 315, 316, 7, 67, 2, 2, 316, 317,
7, 75, 2, 2, 317, 318, 7, 80, 2, 2, 318, 319, 7, 85, 2, 2, 319, 320, 7,
97, 2, 2, 320, 321, 7, 67, 2, 2, 321, 322, 7, 78, 2, 2, 322, 324, 7, 78,
2, 2, 323, 289, 3, 2, 2, 2, 323, 306, 3, 2, 2, 2, 324, 68, 3, 2, 2, 2,
325, 326, 7, 108, 2, 2, 326, 327, 7, 117, 2, 2, 327, 328, 7, 113, 2, 2,
328, 329, 7, 112, 2, 2, 329, 330, 7, 97, 2, 2, 330, 331, 7, 101, 2, 2,
331, 332, 7, 113, 2, 2, 332, 333, 7, 112, 2, 2, 333, 334, 7, 118, 2, 2,
334, 335, 7, 99, 2, 2, 335, 336, 7, 107, 2, 2, 336, 337, 7, 112, 2, 2,
337, 338, 7, 117, 2, 2, 338, 339, 7, 97, 2, 2, 339, 340, 7, 99, 2, 2, 340,
341, 7, 112, 2, 2, 341, 360, 7, 123, 2, 2, 342, 343, 7, 76, 2, 2, 343,
344, 7, 85, 2, 2, 344, 345, 7, 81, 2, 2, 345, 346, 7, 80, 2, 2, 346, 347,
7, 97, 2, 2, 347, 348, 7, 69, 2, 2, 348, 349, 7, 81, 2, 2, 349, 350, 7,
80, 2, 2, 350, 351, 7, 86, 2, 2, 351, 352, 7, 67, 2, 2, 352, 353, 7, 75,
2, 2, 353, 354, 7, 80, 2, 2, 354, 355, 7, 85, 2, 2, 355, 356, 7, 97, 2,
2, 356, 357, 7, 67, 2, 2, 357, 358, 7, 80, 2, 2, 358, 360, 7, 91, 2, 2,
359, 325, 3, 2, 2, 2, 359, 342, 3, 2, 2, 2, 360, 70, 3, 2, 2, 2, 361, 362,
7, 99, 2, 2, 362, 363, 7, 116, 2, 2, 363, 364, 7, 116, 2, 2, 364, 365,
7, 99, 2, 2, 365, 366, 7, 123, 2, 2, 366, 367, 7, 97, 2, 2, 367, 368, 7,
101, 2, 2, 368, 369, 7, 113, 2, 2, 369, 370, 7, 112, 2, 2, 370, 371, 7,
118, 2, 2, 371, 372, 7, 99, 2, 2, 372, 373, 7, 107, 2, 2, 373, 374, 7,
112, 2, 2, 374, 390, 7, 117, 2, 2, 375, 376, 7, 67, 2, 2, 376, 377, 7,
84, 2, 2, 377, 378, 7, 84, 2, 2, 378, 379, 7, 67, 2, 2, 379, 380, 7, 91,
2, 2, 380, 381, 7, 97, 2, 2, 381, 382, 7, 69, 2, 2, 382, 383, 7, 81, 2,
2, 383, 384, 7, 80, 2, 2, 384, 385, 7, 86, 2, 2, 385, 386, 7, 67, 2, 2,
386, 387, 7, 75, 2, 2, 387, 388, 7, 80, 2, 2, 388, 390, 7, 85, 2, 2, 389,
361, 3, 2, 2, 2, 389, 375, 3, 2, 2, 2, 390, 72, 3, 2, 2, 2, 391, 392, 7,
99, 2, 2, 392, 393, 7, 116, 2, 2, 393, 394, 7, 116, 2, 2, 394, 395, 7,
99, 2, 2, 395, 396, 7, 123, 2, 2, 396, 397, 7, 97, 2, 2, 397, 398, 7, 101,
2, 2, 398, 399, 7, 113, 2, 2, 399, 400, 7, 112, 2, 2, 400, 401, 7, 118,
2, 2, 401, 402, 7, 99, 2, 2, 402, 403, 7, 107, 2, 2, 403, 404, 7, 112,
2, 2, 404, 405, 7, 117, 2, 2, 405, 406, 7, 97, 2, 2, 406, 407, 7, 99, 2,
2, 407, 408, 7, 110, 2, 2, 408, 428, 7, 110, 2, 2, 409, 410, 7, 67, 2,
2, 410, 411, 7, 84, 2, 2, 411, 412, 7, 84, 2, 2, 412, 413, 7, 67, 2, 2,
413, 414, 7, 91, 2, 2, 414, 415, 7, 97, 2, 2, 415, 416, 7, 69, 2, 2, 416,
417, 7, 81, 2, 2, 417, 418, 7, 80, 2, 2, 418, 419, 7, 86, 2, 2, 419, 420,
7, 67, 2, 2, 420, 421, 7, 75, 2, 2, 421, 422, 7, 80, 2, 2, 422, 423, 7,
85, 2, 2, 423, 424, 7, 97, 2, 2, 424, 425, 7, 67, 2, 2, 425, 426, 7, 78,
2, 2, 426, 428, 7, 78, 2, 2, 427, 391, 3, 2, 2, 2, 427, 409, 3, 2, 2, 2,
428, 74, 3, 2, 2, 2, 429, 430, 7, 99, 2, 2, 430, 431, 7, 116, 2, 2, 431,
432, 7, 116, 2, 2, 432, 433, 7, 99, 2, 2, 433, 434, 7, 123, 2, 2, 434,
435, 7, 97, 2, 2, 435, 436, 7, 101, 2, 2, 436, 437, 7, 113, 2, 2, 437,
438, 7, 112, 2, 2, 438, 439, 7, 118, 2, 2, 439, 440, 7, 99, 2, 2, 440,
441, 7, 107, 2, 2, 441, 442, 7, 112, 2, 2, 442, 443, 7, 117, 2, 2, 443,
444, 7, 97, 2, 2, 444, 445, 7, 99, 2, 2, 445, 446, 7, 112, 2, 2, 446, 466,
7, 123, 2, 2, 447, 448, 7, 67, 2, 2, 448, 449, 7, 84, 2, 2, 449, 450, 7,
84, 2, 2, 450, 451, 7, 67, 2, 2, 451, 452, 7, 91, 2, 2, 452, 453, 7, 97,
2, 2, 453, 454, 7, 69, 2, 2, 454, 455, 7, 81, 2, 2, 455, 456, 7, 80, 2,
2, 456, 457, 7, 86, 2, 2, 457, 458, 7, 67, 2, 2, 458, 459, 7, 75, 2, 2,
459, 460, 7, 80, 2, 2, 460, 461, 7, 85, 2, 2, 461, 462, 7, 97, 2, 2, 462,
463, 7, 67, 2, 2, 463, 464, 7, 80, 2, 2, 464, 466, 7, 91, 2, 2, 465, 429,
3, 2, 2, 2, 465, 447, 3, 2, 2, 2, 466, 76, 3, 2, 2, 2, 467, 468, 7, 99,
2, 2, 468, 469, 7, 116, 2, 2, 469, 470, 7, 116, 2, 2, 470, 471, 7, 99,
2, 2, 471, 472, 7, 123, 2, 2, 472, 473, 7, 97, 2, 2, 473, 474, 7, 110,
2, 2, 474, 475, 7, 103, 2, 2, 475, 476, 7, 112, 2, 2, 476, 477, 7, 105,
2, 2, 477, 478, 7, 118, 2, 2, 478, 492, 7, 106, 2, 2, 479, 480, 7, 67,
2, 2, 480, 481, 7, 84, 2, 2, 481, 482, 7, 84, 2, 2, 482, 483, 7, 67, 2,
2, 483, 484, 7, 91, 2, 2, 484, 485, 7, 97, 2, 2, 485, 486, 7, 78, 2, 2,
486, 487, 7, 71, 2, 2, 487, 488, 7, 80, 2, 2, 488, 489, 7, 73, 2, 2, 489,
490, 7, 86, 2, 2, 490, 492, 7, 74, 2, 2, 491, 467, 3, 2, 2, 2, 491, 479,
3, 2, 2, 2, 492, 78, 3, 2, 2, 2, 493, 494, 7, 118, 2, 2, 494, 495, 7, 116,
2, 2, 495, 496, 7, 119, 2, 2, 496, 521, 7, 103, 2, 2, 497, 498, 7, 86,
2, 2, 498, 499, 7, 116, 2, 2, 499, 500, 7, 119, 2, 2, 500, 521, 7, 103,
2, 2, 501, 502, 7, 86, 2, 2, 502, 503, 7, 84, 2, 2, 503, 504, 7, 87, 2,
2, 504, 521, 7, 71, 2, 2, 505, 506, 7, 104, 2, 2, 506, 507, 7, 99, 2, 2,
507, 508, 7, 110, 2, 2, 508, 509, 7, 117, 2, 2, 509, 521, 7, 103, 2, 2,
510, 511, 7, 72, 2, 2, 511, 512, 7, 99, 2, 2, 512, 513, 7, 110, 2, 2, 513,
514, 7, 117, 2, 2, 514, 521, 7, 103, 2, 2, 515, 516, 7, 72, 2, 2, 516,
517, 7, 67, 2, 2, 517, 518, 7, 78, 2, 2, 518, 519, 7, 85, 2, 2, 519, 521,
7, 71, 2, 2, 520, 493, 3, 2, 2, 2, 520, 497, 3, 2, 2, 2, 520, 501, 3, 2,
2, 2, 520, 505, 3, 2, 2, 2, 520, 510, 3, 2, 2, 2, 520, 515, 3, 2, 2, 2,
521, 80, 3, 2, 2, 2, 522, 527, 5, 107, 54, 2, 523, 527, 5, 109, 55, 2,
524, 527, 5, 111, 56, 2, 525, 527, 5, 105, 53, 2, 526, 522, 3, 2, 2, 2,
526, 523, 3, 2, 2, 2, 526, 524, 3, 2, 2, 2, 526, 525, 3, 2, 2, 2, 527,
82, 3, 2, 2, 2, 528, 531, 5, 123, 62, 2, 529, 531, 5, 125, 63, 2, 530,
528, 3, 2, 2, 2, 530, 529, 3, 2, 2, 2, 531, 84, 3, 2, 2, 2, 532, 537, 5,
101, 51, 2, 533, 536, 5, 101, 51, 2, 534, 536, 5, 103, 52, 2, 535, 533,
3, 2, 2, 2, 535, 534, 3, 2, 2, 2, 536, 539, 3, 2, 2, 2, 537, 535, 3, 2,
2, 2, 537, 538, 3, 2, 2, 2, 538, 546, 3, 2, 2, 2, 539, 537, 3, 2, 2, 2,
540, 541, 7, 38, 2, 2, 541, 542, 7, 111, 2, 2, 542, 543, 7, 103, 2, 2,
543, 544, 7, 118, 2, 2, 544, 546, 7, 99, 2, 2, 545, 532, 3, 2, 2, 2, 545,
540, 3, 2, 2, 2, 546, 86, 3, 2, 2, 2, 547, 549, 5, 91, 46, 2, 548, 547,
3, 2, 2, 2, 548, 549, 3, 2, 2, 2, 549, 560, 3, 2, 2, 2, 550, 552, 7, 36,
2, 2, 551, 553, 5, 93, 47, 2, 552, 551, 3, 2, 2, 2, 552, 553, 3, 2, 2,
2, 553, 554, 3, 2, 2, 2, 554, 561, 7, 36, 2, 2, 555, 557, 7, 41, 2, 2,
556, 558, 5, 95, 48, 2, 557, 556, 3, 2, 2, 2, 557, 558, 3, 2, 2, 2, 558,
559, 3, 2, 2, 2, 559, 561, 7, 41, 2, 2, 560, 550, 3, 2, 2, 2, 560, 555,
3, 2, 2, 2, 561, 88, 3, 2, 2, 2, 562, 570, 5, 85, 43, 2, 563, 566, 7, 93,
2, 2, 564, 567, 5, 87, 44, 2, 565, 567, 5, 107, 54, 2, 566, 564, 3, 2,
2, 2, 566, 565, 3, 2, 2, 2, 567, 568, 3, 2, 2, 2, 568, 569, 7, 95, 2, 2,
569, 571, 3, 2, 2, 2, 570, 563, 3, 2, 2, 2, 571, 572, 3, 2, 2, 2, 572,
570, 3, 2, 2, 2, 572, 573, 3, 2, 2, 2, 573, 90, 3, 2, 2, 2, 574, 575, 7,
119, 2, 2, 575, 578, 7, 58, 2, 2, 576, 578, 9, 2, 2, 2, 577, 574, 3, 2,
2, 2, 577, 576, 3, 2, 2, 2, 578, 92, 3, 2, 2, 2, 579, 581, 5, 97, 49, 2,
580, 579, 3, 2, 2, 2, 581, 582, 3, 2, 2, 2, 582, 580, 3, 2, 2, 2, 582,
583, 3, 2, 2, 2, 583, 94, 3, 2, 2, 2, 584, 586, 5, 99, 50, 2, 585, 584,
3, 2, 2, 2, 586, 587, 3, 2, 2, 2, 587, 585, 3, 2, 2, 2, 587, 588, 3, 2,
2, 2, 588, 96, 3, 2, 2, 2, 589, 597, 10, 3, 2, 2, 590, 597, 5, 139, 70,
2, 591, 592, 7, 94, 2, 2, 592, 597, 7, 12, 2, 2, 593, 594, 7, 94, 2, 2,
594, 595, 7, 15, 2, 2, 595, 597, 7, 12, 2, 2, 596, 589, 3, 2, 2, 2, 596,
590, 3, 2, 2, 2, 596, 591, 3, 2, 2, 2, 596, 593, 3, 2, 2, 2, 597, 98, 3,
2, 2, 2, 598, 606, 10, 4, 2, 2, 599, 606, 5, 139, 70, 2, 600, 601, 7, 94,
2, 2, 601, 606, 7, 12, 2, 2, 602, 603, 7, 94, 2, 2, 603, 604, 7, 15, 2,
2, 604, 606, 7, 12, 2, 2, 605, 598, 3, 2, 2, 2, 605, 599, 3, 2, 2, 2, 605,
600, 3, 2, 2, 2, 605, 602, 3, 2, 2, 2, 606, 100, 3, 2, 2, 2, 607, 608,
9, 5, 2, 2, 608, 102, 3, 2, 2, 2, 609, 610, 9, 6, 2, 2, 610, 104, 3, 2,
2, 2, 611, 612, 7, 50, 2, 2, 612, 614, 9, 7, 2, 2, 613, 615, 9, 8, 2, 2,
614, 613, 3, 2, 2, 2, 615, 616, 3, 2, 2, 2, 616, 614, 3, 2, 2, 2, 616,
617, 3, 2, 2, 2, 617, 106, 3, 2, 2, 2, 618, 622, 5, 113, 57, 2, 619, 621,
5, 103, 52, 2, 620, 619, 3, 2, 2, 2, 621, 624, 3, 2, 2, 2, 622, 620, 3,
2, 2, 2, 622, 623, 3, 2, 2, 2, 623, 627, 3, 2, 2, 2, 624, 622, 3, 2, 2,
2, 625, 627, 7, 50, 2, 2, 626, 618, 3, 2, 2, 2, 626, 625, 3, 2, 2, 2, 627,
108, 3, 2, 2, 2, 628, 632, 7, 50, 2, 2, 629, 631, 5, 115, 58, 2, 630, 629,
3, 2, 2, 2, 631, 634, 3, 2, 2, 2, 632, 630, 3, 2, 2, 2, 632, 633, 3, 2,
2, 2, 633, 110, 3, 2, 2, 2, 634, 632, 3, 2, 2, 2, 635, 636, 7, 50, 2, 2,
636, 637, 9, 9, 2, 2, 637, 638, 5, 135, 68, 2, 638, 112, 3, 2, 2, 2, 639,
640, 9, 10, 2, 2, 640, 114, 3, 2, 2, 2, 641, 642, 9, 11, 2, 2, 642, 116,
3, 2, 2, 2, 643, 644, 9, 12, 2, 2, 644, 118, 3, 2, 2, 2, 645, 646, 5, 117,
59, 2, 646, 647, 5, 117, 59, 2, 647, 648, 5, 117, 59, 2, 648, 649, 5, 117,
59, 2, 649, 120, 3, 2, 2, 2, 650, 651, 7, 94, 2, 2, 651, 652, 7, 119, 2,
2, 652, 653, 3, 2, 2, 2, 653, 661, 5, 119, 60, 2, 654, 655, 7, 94, 2, 2,
655, 656, 7, 87, 2, 2, 656, 657, 3, 2, 2, 2, 657, 658, 5, 119, 60, 2, 658,
659, 5, 119, 60, 2, 659, 661, 3, 2, 2, 2, 660, 650, 3, 2, 2, 2, 660, 654,
3, 2, 2, 2, 661, 122, 3, 2, 2, 2, 662, 664, 5, 127, 64, 2, 663, 665, 5,
129, 65, 2, 664, 663, 3, 2, 2, 2, 664, 665, 3, 2, 2, 2, 665, 670, 3, 2,
2, 2, 666, 667, 5, 131, 66, 2, 667, 668, 5, 129, 65, 2, 668, 670, 3, 2,
2, 2, 669, 662, 3, 2, 2, 2, 669, 666, 3, 2, 2, 2, 670, 124, 3, 2, 2, 2,
671, 672, 7, 50, 2, 2, 672, 675, 9, 9, 2, 2, 673, 676, 5, 133, 67, 2, 674,
676, 5, 135, 68, 2, 675, 673, 3, 2, 2, 2, 675, 674, 3, 2, 2, 2, 676, 677,
3, 2, 2, 2, 677, 678, 5, 137, 69, 2, 678, 126, 3, 2, 2, 2, 679, 681, 5,
131, 66, 2, 680, 679, 3, 2, 2, 2, 680, 681, 3, 2, 2, 2, 681, 682, 3, 2,
2, 2, 682, 683, 7, 48, 2, 2, 683, 688, 5, 131, 66, 2, 684, 685, 5, 131,
66, 2, 685, 686, 7, 48, 2, 2, 686, 688, 3, 2, 2, 2, 687, 680, 3, 2, 2,
2, 687, 684, 3, 2, 2, 2, 688, 128, 3, 2, 2, 2, 689, 691, 9, 13, 2, 2, 690,
692, 9, 14, 2, 2, 691, 690, 3, 2, 2, 2, 691, 692, 3, 2, 2, 2, 692, 693,
3, 2, 2, 2, 693, 694, 5, 131, 66, 2, 694, 130, 3, 2, 2, 2, 695, 697, 5,
103, 52, 2, 696, 695, 3, 2, 2, 2, 697, 698, 3, 2, 2, 2, 698, 696, 3, 2,
2, 2, 698, 699, 3, 2, 2, 2, 699, 132, 3, 2, 2, 2, 700, 702, 5, 135, 68,
2, 701, 700, 3, 2, 2, 2, 701, 702, 3, 2, 2, 2, 702, 703, 3, 2, 2, 2, 703,
704, 7, 48, 2, 2, 704, 709, 5, 135, 68, 2, 705, 706, 5, 135, 68, 2, 706,
707, 7, 48, 2, 2, 707, 709, 3, 2, 2, 2, 708, 701, 3, 2, 2, 2, 708, 705,
3, 2, 2, 2, 709, 134, 3, 2, 2, 2, 710, 712, 5, 117, 59, 2, 711, 710, 3,
2, 2, 2, 712, 713, 3, 2, 2, 2, 713, 711, 3, 2, 2, 2, 713, 714, 3, 2, 2,
2, 714, 136, 3, 2, 2, 2, 715, 717, 9, 15, 2, 2, 716, 718, 9, 14, 2, 2,
717, 716, 3, 2, 2, 2, 717, 718, 3, 2, 2, 2, 718, 719, 3, 2, 2, 2, 719,
720, 5, 131, 66, 2, 720, 138, 3, 2, 2, 2, 721, 722, 7, 94, 2, 2, 722, 737,
9, 16, 2, 2, 723, 724, 7, 94, 2, 2, 724, 726, 5, 115, 58, 2, 725, 727,
5, 115, 58, 2, 726, 725, 3, 2, 2, 2, 726, 727, 3, 2, 2, 2, 727, 729, 3,
2, 2, 2, 728, 730, 5, 115, 58, 2, 729, 728, 3, 2, 2, 2, 729, 730, 3, 2,
2, 2, 730, 737, 3, 2, 2, 2, 731, 732, 7, 94, 2, 2, 732, 733, 7, 122, 2,
2, 733, 734, 3, 2, 2, 2, 734, 737, 5, 135, 68, 2, 735, 737, 5, 121, 61,
2, 736, 721, 3, 2, 2, 2, 736, 723, 3, 2, 2, 2, 736, 731, 3, 2, 2, 2, 736,
735, 3, 2, 2, 2, 737, 140, 3, 2, 2, 2, 738, 740, 9, 17, 2, 2, 739, 738,
3, 2, 2, 2, 740, 741, 3, 2, 2, 2, 741, 739, 3, 2, 2, 2, 741, 742, 3, 2,
2, 2, 742, 743, 3, 2, 2, 2, 743, 744, 8, 71, 2, 2, 744, 142, 3, 2, 2, 2,
745, 747, 7, 15, 2, 2, 746, 748, 7, 12, 2, 2, 747, 746, 3, 2, 2, 2, 747,
748, 3, 2, 2, 2, 748, 751, 3, 2, 2, 2, 749, 751, 7, 12, 2, 2, 750, 745,
3, 2, 2, 2, 750, 749, 3, 2, 2, 2, 751, 752, 3, 2, 2, 2, 752, 753, 8, 72,
2, 2, 753, 144, 3, 2, 2, 2, 56, 2, 179, 193, 225, 231, 239, 254, 256, 287,
323, 359, 389, 427, 465, 491, 520, 526, 530, 535, 537, 545, 548, 552, 557,
560, 566, 572, 577, 582, 587, 596, 605, 616, 622, 626, 632, 660, 664, 669,
675, 680, 687, 691, 698, 701, 708, 713, 717, 726, 729, 736, 741, 747, 750,
40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 5, 40, 504, 10, 40, 3, 41, 3, 41,
3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3,
41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41,
3, 41, 3, 41, 3, 41, 3, 41, 5, 41, 533, 10, 41, 3, 42, 3, 42, 3, 42, 3,
42, 5, 42, 539, 10, 42, 3, 43, 3, 43, 5, 43, 543, 10, 43, 3, 44, 3, 44,
3, 44, 7, 44, 548, 10, 44, 12, 44, 14, 44, 551, 11, 44, 3, 44, 3, 44, 3,
44, 3, 44, 3, 44, 5, 44, 558, 10, 44, 3, 45, 5, 45, 561, 10, 45, 3, 45,
3, 45, 5, 45, 565, 10, 45, 3, 45, 3, 45, 3, 45, 5, 45, 570, 10, 45, 3,
45, 5, 45, 573, 10, 45, 3, 46, 3, 46, 3, 46, 3, 46, 5, 46, 579, 10, 46,
3, 46, 3, 46, 6, 46, 583, 10, 46, 13, 46, 14, 46, 584, 3, 47, 3, 47, 3,
47, 5, 47, 590, 10, 47, 3, 48, 6, 48, 593, 10, 48, 13, 48, 14, 48, 594,
3, 49, 6, 49, 598, 10, 49, 13, 49, 14, 49, 599, 3, 50, 3, 50, 3, 50, 3,
50, 3, 50, 3, 50, 3, 50, 5, 50, 609, 10, 50, 3, 51, 3, 51, 3, 51, 3, 51,
3, 51, 3, 51, 3, 51, 5, 51, 618, 10, 51, 3, 52, 3, 52, 3, 53, 3, 53, 3,
54, 3, 54, 3, 54, 6, 54, 627, 10, 54, 13, 54, 14, 54, 628, 3, 55, 3, 55,
7, 55, 633, 10, 55, 12, 55, 14, 55, 636, 11, 55, 3, 55, 5, 55, 639, 10,
55, 3, 56, 3, 56, 7, 56, 643, 10, 56, 12, 56, 14, 56, 646, 11, 56, 3, 57,
3, 57, 3, 57, 3, 57, 3, 58, 3, 58, 3, 59, 3, 59, 3, 60, 3, 60, 3, 61, 3,
61, 3, 61, 3, 61, 3, 61, 3, 62, 3, 62, 3, 62, 3, 62, 3, 62, 3, 62, 3, 62,
3, 62, 3, 62, 3, 62, 5, 62, 673, 10, 62, 3, 63, 3, 63, 5, 63, 677, 10,
63, 3, 63, 3, 63, 3, 63, 5, 63, 682, 10, 63, 3, 64, 3, 64, 3, 64, 3, 64,
5, 64, 688, 10, 64, 3, 64, 3, 64, 3, 65, 5, 65, 693, 10, 65, 3, 65, 3,
65, 3, 65, 3, 65, 3, 65, 5, 65, 700, 10, 65, 3, 66, 3, 66, 5, 66, 704,
10, 66, 3, 66, 3, 66, 3, 67, 6, 67, 709, 10, 67, 13, 67, 14, 67, 710, 3,
68, 5, 68, 714, 10, 68, 3, 68, 3, 68, 3, 68, 3, 68, 3, 68, 5, 68, 721,
10, 68, 3, 69, 6, 69, 724, 10, 69, 13, 69, 14, 69, 725, 3, 70, 3, 70, 5,
70, 730, 10, 70, 3, 70, 3, 70, 3, 71, 3, 71, 3, 71, 3, 71, 3, 71, 5, 71,
739, 10, 71, 3, 71, 5, 71, 742, 10, 71, 3, 71, 3, 71, 3, 71, 3, 71, 3,
71, 5, 71, 749, 10, 71, 3, 72, 6, 72, 752, 10, 72, 13, 72, 14, 72, 753,
3, 72, 3, 72, 3, 73, 3, 73, 5, 73, 760, 10, 73, 3, 73, 5, 73, 763, 10,
73, 3, 73, 3, 73, 2, 2, 74, 3, 3, 5, 4, 7, 5, 9, 6, 11, 7, 13, 8, 15, 9,
17, 10, 19, 11, 21, 12, 23, 13, 25, 14, 27, 15, 29, 16, 31, 17, 33, 18,
35, 19, 37, 20, 39, 21, 41, 22, 43, 23, 45, 24, 47, 25, 49, 26, 51, 27,
53, 28, 55, 29, 57, 30, 59, 31, 61, 32, 63, 33, 65, 34, 67, 35, 69, 36,
71, 37, 73, 38, 75, 39, 77, 40, 79, 41, 81, 42, 83, 43, 85, 44, 87, 45,
89, 46, 91, 47, 93, 2, 95, 2, 97, 2, 99, 2, 101, 2, 103, 2, 105, 2, 107,
2, 109, 2, 111, 2, 113, 2, 115, 2, 117, 2, 119, 2, 121, 2, 123, 2, 125,
2, 127, 2, 129, 2, 131, 2, 133, 2, 135, 2, 137, 2, 139, 2, 141, 2, 143,
48, 145, 49, 3, 2, 18, 5, 2, 78, 78, 87, 87, 119, 119, 6, 2, 12, 12, 15,
15, 36, 36, 94, 94, 6, 2, 12, 12, 15, 15, 41, 41, 94, 94, 5, 2, 67, 92,
97, 97, 99, 124, 3, 2, 50, 59, 4, 2, 68, 68, 100, 100, 3, 2, 50, 51, 4,
2, 90, 90, 122, 122, 3, 2, 51, 59, 3, 2, 50, 57, 5, 2, 50, 59, 67, 72,
99, 104, 4, 2, 71, 71, 103, 103, 4, 2, 45, 45, 47, 47, 4, 2, 82, 82, 114,
114, 12, 2, 36, 36, 41, 41, 65, 65, 94, 94, 99, 100, 104, 104, 112, 112,
116, 116, 118, 118, 120, 120, 4, 2, 11, 11, 34, 34, 2, 805, 2, 3, 3, 2,
2, 2, 2, 5, 3, 2, 2, 2, 2, 7, 3, 2, 2, 2, 2, 9, 3, 2, 2, 2, 2, 11, 3, 2,
2, 2, 2, 13, 3, 2, 2, 2, 2, 15, 3, 2, 2, 2, 2, 17, 3, 2, 2, 2, 2, 19, 3,
2, 2, 2, 2, 21, 3, 2, 2, 2, 2, 23, 3, 2, 2, 2, 2, 25, 3, 2, 2, 2, 2, 27,
3, 2, 2, 2, 2, 29, 3, 2, 2, 2, 2, 31, 3, 2, 2, 2, 2, 33, 3, 2, 2, 2, 2,
35, 3, 2, 2, 2, 2, 37, 3, 2, 2, 2, 2, 39, 3, 2, 2, 2, 2, 41, 3, 2, 2, 2,
2, 43, 3, 2, 2, 2, 2, 45, 3, 2, 2, 2, 2, 47, 3, 2, 2, 2, 2, 49, 3, 2, 2,
2, 2, 51, 3, 2, 2, 2, 2, 53, 3, 2, 2, 2, 2, 55, 3, 2, 2, 2, 2, 57, 3, 2,
2, 2, 2, 59, 3, 2, 2, 2, 2, 61, 3, 2, 2, 2, 2, 63, 3, 2, 2, 2, 2, 65, 3,
2, 2, 2, 2, 67, 3, 2, 2, 2, 2, 69, 3, 2, 2, 2, 2, 71, 3, 2, 2, 2, 2, 73,
3, 2, 2, 2, 2, 75, 3, 2, 2, 2, 2, 77, 3, 2, 2, 2, 2, 79, 3, 2, 2, 2, 2,
81, 3, 2, 2, 2, 2, 83, 3, 2, 2, 2, 2, 85, 3, 2, 2, 2, 2, 87, 3, 2, 2, 2,
2, 89, 3, 2, 2, 2, 2, 91, 3, 2, 2, 2, 2, 143, 3, 2, 2, 2, 2, 145, 3, 2,
2, 2, 3, 147, 3, 2, 2, 2, 5, 149, 3, 2, 2, 2, 7, 151, 3, 2, 2, 2, 9, 153,
3, 2, 2, 2, 11, 155, 3, 2, 2, 2, 13, 157, 3, 2, 2, 2, 15, 159, 3, 2, 2,
2, 17, 162, 3, 2, 2, 2, 19, 164, 3, 2, 2, 2, 21, 167, 3, 2, 2, 2, 23, 170,
3, 2, 2, 2, 25, 181, 3, 2, 2, 2, 27, 195, 3, 2, 2, 2, 29, 197, 3, 2, 2,
2, 31, 207, 3, 2, 2, 2, 33, 209, 3, 2, 2, 2, 35, 211, 3, 2, 2, 2, 37, 213,
3, 2, 2, 2, 39, 215, 3, 2, 2, 2, 41, 217, 3, 2, 2, 2, 43, 220, 3, 2, 2,
2, 45, 223, 3, 2, 2, 2, 47, 226, 3, 2, 2, 2, 49, 228, 3, 2, 2, 2, 51, 230,
3, 2, 2, 2, 53, 237, 3, 2, 2, 2, 55, 243, 3, 2, 2, 2, 57, 245, 3, 2, 2,
2, 59, 251, 3, 2, 2, 2, 61, 253, 3, 2, 2, 2, 63, 256, 3, 2, 2, 2, 65, 263,
3, 2, 2, 2, 67, 299, 3, 2, 2, 2, 69, 335, 3, 2, 2, 2, 71, 371, 3, 2, 2,
2, 73, 401, 3, 2, 2, 2, 75, 439, 3, 2, 2, 2, 77, 477, 3, 2, 2, 2, 79, 503,
3, 2, 2, 2, 81, 532, 3, 2, 2, 2, 83, 538, 3, 2, 2, 2, 85, 542, 3, 2, 2,
2, 87, 557, 3, 2, 2, 2, 89, 560, 3, 2, 2, 2, 91, 574, 3, 2, 2, 2, 93, 589,
3, 2, 2, 2, 95, 592, 3, 2, 2, 2, 97, 597, 3, 2, 2, 2, 99, 608, 3, 2, 2,
2, 101, 617, 3, 2, 2, 2, 103, 619, 3, 2, 2, 2, 105, 621, 3, 2, 2, 2, 107,
623, 3, 2, 2, 2, 109, 638, 3, 2, 2, 2, 111, 640, 3, 2, 2, 2, 113, 647,
3, 2, 2, 2, 115, 651, 3, 2, 2, 2, 117, 653, 3, 2, 2, 2, 119, 655, 3, 2,
2, 2, 121, 657, 3, 2, 2, 2, 123, 672, 3, 2, 2, 2, 125, 681, 3, 2, 2, 2,
127, 683, 3, 2, 2, 2, 129, 699, 3, 2, 2, 2, 131, 701, 3, 2, 2, 2, 133,
708, 3, 2, 2, 2, 135, 720, 3, 2, 2, 2, 137, 723, 3, 2, 2, 2, 139, 727,
3, 2, 2, 2, 141, 748, 3, 2, 2, 2, 143, 751, 3, 2, 2, 2, 145, 762, 3, 2,
2, 2, 147, 148, 7, 42, 2, 2, 148, 4, 3, 2, 2, 2, 149, 150, 7, 43, 2, 2,
150, 6, 3, 2, 2, 2, 151, 152, 7, 93, 2, 2, 152, 8, 3, 2, 2, 2, 153, 154,
7, 46, 2, 2, 154, 10, 3, 2, 2, 2, 155, 156, 7, 95, 2, 2, 156, 12, 3, 2,
2, 2, 157, 158, 7, 62, 2, 2, 158, 14, 3, 2, 2, 2, 159, 160, 7, 62, 2, 2,
160, 161, 7, 63, 2, 2, 161, 16, 3, 2, 2, 2, 162, 163, 7, 64, 2, 2, 163,
18, 3, 2, 2, 2, 164, 165, 7, 64, 2, 2, 165, 166, 7, 63, 2, 2, 166, 20,
3, 2, 2, 2, 167, 168, 7, 63, 2, 2, 168, 169, 7, 63, 2, 2, 169, 22, 3, 2,
2, 2, 170, 171, 7, 35, 2, 2, 171, 172, 7, 63, 2, 2, 172, 24, 3, 2, 2, 2,
173, 174, 7, 110, 2, 2, 174, 175, 7, 107, 2, 2, 175, 176, 7, 109, 2, 2,
176, 182, 7, 103, 2, 2, 177, 178, 7, 78, 2, 2, 178, 179, 7, 75, 2, 2, 179,
180, 7, 77, 2, 2, 180, 182, 7, 71, 2, 2, 181, 173, 3, 2, 2, 2, 181, 177,
3, 2, 2, 2, 182, 26, 3, 2, 2, 2, 183, 184, 7, 103, 2, 2, 184, 185, 7, 122,
2, 2, 185, 186, 7, 107, 2, 2, 186, 187, 7, 117, 2, 2, 187, 188, 7, 118,
2, 2, 188, 196, 7, 117, 2, 2, 189, 190, 7, 71, 2, 2, 190, 191, 7, 90, 2,
2, 191, 192, 7, 75, 2, 2, 192, 193, 7, 85, 2, 2, 193, 194, 7, 86, 2, 2,
194, 196, 7, 85, 2, 2, 195, 183, 3, 2, 2, 2, 195, 189, 3, 2, 2, 2, 196,
28, 3, 2, 2, 2, 197, 198, 7, 86, 2, 2, 198, 199, 7, 103, 2, 2, 199, 200,
7, 122, 2, 2, 200, 201, 7, 118, 2, 2, 201, 202, 7, 79, 2, 2, 202, 203,
7, 99, 2, 2, 203, 204, 7, 118, 2, 2, 204, 205, 7, 101, 2, 2, 205, 206,
7, 106, 2, 2, 206, 30, 3, 2, 2, 2, 207, 208, 7, 45, 2, 2, 208, 32, 3, 2,
2, 2, 209, 210, 7, 47, 2, 2, 210, 34, 3, 2, 2, 2, 211, 212, 7, 44, 2, 2,
212, 36, 3, 2, 2, 2, 213, 214, 7, 49, 2, 2, 214, 38, 3, 2, 2, 2, 215, 216,
7, 39, 2, 2, 216, 40, 3, 2, 2, 2, 217, 218, 7, 44, 2, 2, 218, 219, 7, 44,
2, 2, 219, 42, 3, 2, 2, 2, 220, 221, 7, 62, 2, 2, 221, 222, 7, 62, 2, 2,
222, 44, 3, 2, 2, 2, 223, 224, 7, 64, 2, 2, 224, 225, 7, 64, 2, 2, 225,
46, 3, 2, 2, 2, 226, 227, 7, 40, 2, 2, 227, 48, 3, 2, 2, 2, 228, 229, 7,
126, 2, 2, 229, 50, 3, 2, 2, 2, 230, 231, 7, 96, 2, 2, 231, 52, 3, 2, 2,
2, 232, 233, 7, 40, 2, 2, 233, 238, 7, 40, 2, 2, 234, 235, 7, 99, 2, 2,
235, 236, 7, 112, 2, 2, 236, 238, 7, 102, 2, 2, 237, 232, 3, 2, 2, 2, 237,
234, 3, 2, 2, 2, 238, 54, 3, 2, 2, 2, 239, 240, 7, 126, 2, 2, 240, 244,
7, 126, 2, 2, 241, 242, 7, 113, 2, 2, 242, 244, 7, 116, 2, 2, 243, 239,
3, 2, 2, 2, 243, 241, 3, 2, 2, 2, 244, 56, 3, 2, 2, 2, 245, 246, 7, 128,
2, 2, 246, 58, 3, 2, 2, 2, 247, 252, 7, 35, 2, 2, 248, 249, 7, 112, 2,
2, 249, 250, 7, 113, 2, 2, 250, 252, 7, 118, 2, 2, 251, 247, 3, 2, 2, 2,
251, 248, 3, 2, 2, 2, 252, 60, 3, 2, 2, 2, 253, 254, 7, 107, 2, 2, 254,
255, 7, 112, 2, 2, 255, 62, 3, 2, 2, 2, 256, 257, 7, 112, 2, 2, 257, 258,
7, 113, 2, 2, 258, 259, 7, 118, 2, 2, 259, 260, 7, 34, 2, 2, 260, 261,
7, 107, 2, 2, 261, 262, 7, 112, 2, 2, 262, 64, 3, 2, 2, 2, 263, 268, 7,
93, 2, 2, 264, 267, 5, 143, 72, 2, 265, 267, 5, 145, 73, 2, 266, 264, 3,
2, 2, 2, 266, 265, 3, 2, 2, 2, 267, 270, 3, 2, 2, 2, 268, 266, 3, 2, 2,
2, 268, 269, 3, 2, 2, 2, 269, 271, 3, 2, 2, 2, 270, 268, 3, 2, 2, 2, 271,
272, 7, 95, 2, 2, 272, 66, 3, 2, 2, 2, 273, 274, 7, 108, 2, 2, 274, 275,
7, 117, 2, 2, 275, 276, 7, 113, 2, 2, 276, 277, 7, 112, 2, 2, 277, 278,
7, 97, 2, 2, 278, 279, 7, 101, 2, 2, 279, 280, 7, 113, 2, 2, 280, 281,
7, 112, 2, 2, 281, 282, 7, 118, 2, 2, 282, 283, 7, 99, 2, 2, 283, 284,
7, 107, 2, 2, 284, 285, 7, 112, 2, 2, 285, 300, 7, 117, 2, 2, 286, 287,
7, 76, 2, 2, 287, 288, 7, 85, 2, 2, 288, 289, 7, 81, 2, 2, 289, 290, 7,
80, 2, 2, 290, 291, 7, 97, 2, 2, 291, 292, 7, 69, 2, 2, 292, 293, 7, 81,
2, 2, 293, 294, 7, 80, 2, 2, 294, 295, 7, 86, 2, 2, 295, 296, 7, 67, 2,
2, 296, 297, 7, 75, 2, 2, 297, 298, 7, 80, 2, 2, 298, 300, 7, 85, 2, 2,
299, 273, 3, 2, 2, 2, 299, 286, 3, 2, 2, 2, 300, 68, 3, 2, 2, 2, 301, 302,
7, 108, 2, 2, 302, 303, 7, 117, 2, 2, 303, 304, 7, 113, 2, 2, 304, 305,
7, 112, 2, 2, 305, 306, 7, 97, 2, 2, 306, 307, 7, 101, 2, 2, 307, 308,
7, 113, 2, 2, 308, 309, 7, 112, 2, 2, 309, 310, 7, 118, 2, 2, 310, 311,
7, 99, 2, 2, 311, 312, 7, 107, 2, 2, 312, 313, 7, 112, 2, 2, 313, 314,
7, 117, 2, 2, 314, 315, 7, 97, 2, 2, 315, 316, 7, 99, 2, 2, 316, 317, 7,
110, 2, 2, 317, 336, 7, 110, 2, 2, 318, 319, 7, 76, 2, 2, 319, 320, 7,
85, 2, 2, 320, 321, 7, 81, 2, 2, 321, 322, 7, 80, 2, 2, 322, 323, 7, 97,
2, 2, 323, 324, 7, 69, 2, 2, 324, 325, 7, 81, 2, 2, 325, 326, 7, 80, 2,
2, 326, 327, 7, 86, 2, 2, 327, 328, 7, 67, 2, 2, 328, 329, 7, 75, 2, 2,
329, 330, 7, 80, 2, 2, 330, 331, 7, 85, 2, 2, 331, 332, 7, 97, 2, 2, 332,
333, 7, 67, 2, 2, 333, 334, 7, 78, 2, 2, 334, 336, 7, 78, 2, 2, 335, 301,
3, 2, 2, 2, 335, 318, 3, 2, 2, 2, 336, 70, 3, 2, 2, 2, 337, 338, 7, 108,
2, 2, 338, 339, 7, 117, 2, 2, 339, 340, 7, 113, 2, 2, 340, 341, 7, 112,
2, 2, 341, 342, 7, 97, 2, 2, 342, 343, 7, 101, 2, 2, 343, 344, 7, 113,
2, 2, 344, 345, 7, 112, 2, 2, 345, 346, 7, 118, 2, 2, 346, 347, 7, 99,
2, 2, 347, 348, 7, 107, 2, 2, 348, 349, 7, 112, 2, 2, 349, 350, 7, 117,
2, 2, 350, 351, 7, 97, 2, 2, 351, 352, 7, 99, 2, 2, 352, 353, 7, 112, 2,
2, 353, 372, 7, 123, 2, 2, 354, 355, 7, 76, 2, 2, 355, 356, 7, 85, 2, 2,
356, 357, 7, 81, 2, 2, 357, 358, 7, 80, 2, 2, 358, 359, 7, 97, 2, 2, 359,
360, 7, 69, 2, 2, 360, 361, 7, 81, 2, 2, 361, 362, 7, 80, 2, 2, 362, 363,
7, 86, 2, 2, 363, 364, 7, 67, 2, 2, 364, 365, 7, 75, 2, 2, 365, 366, 7,
80, 2, 2, 366, 367, 7, 85, 2, 2, 367, 368, 7, 97, 2, 2, 368, 369, 7, 67,
2, 2, 369, 370, 7, 80, 2, 2, 370, 372, 7, 91, 2, 2, 371, 337, 3, 2, 2,
2, 371, 354, 3, 2, 2, 2, 372, 72, 3, 2, 2, 2, 373, 374, 7, 99, 2, 2, 374,
375, 7, 116, 2, 2, 375, 376, 7, 116, 2, 2, 376, 377, 7, 99, 2, 2, 377,
378, 7, 123, 2, 2, 378, 379, 7, 97, 2, 2, 379, 380, 7, 101, 2, 2, 380,
381, 7, 113, 2, 2, 381, 382, 7, 112, 2, 2, 382, 383, 7, 118, 2, 2, 383,
384, 7, 99, 2, 2, 384, 385, 7, 107, 2, 2, 385, 386, 7, 112, 2, 2, 386,
402, 7, 117, 2, 2, 387, 388, 7, 67, 2, 2, 388, 389, 7, 84, 2, 2, 389, 390,
7, 84, 2, 2, 390, 391, 7, 67, 2, 2, 391, 392, 7, 91, 2, 2, 392, 393, 7,
97, 2, 2, 393, 394, 7, 69, 2, 2, 394, 395, 7, 81, 2, 2, 395, 396, 7, 80,
2, 2, 396, 397, 7, 86, 2, 2, 397, 398, 7, 67, 2, 2, 398, 399, 7, 75, 2,
2, 399, 400, 7, 80, 2, 2, 400, 402, 7, 85, 2, 2, 401, 373, 3, 2, 2, 2,
401, 387, 3, 2, 2, 2, 402, 74, 3, 2, 2, 2, 403, 404, 7, 99, 2, 2, 404,
405, 7, 116, 2, 2, 405, 406, 7, 116, 2, 2, 406, 407, 7, 99, 2, 2, 407,
408, 7, 123, 2, 2, 408, 409, 7, 97, 2, 2, 409, 410, 7, 101, 2, 2, 410,
411, 7, 113, 2, 2, 411, 412, 7, 112, 2, 2, 412, 413, 7, 118, 2, 2, 413,
414, 7, 99, 2, 2, 414, 415, 7, 107, 2, 2, 415, 416, 7, 112, 2, 2, 416,
417, 7, 117, 2, 2, 417, 418, 7, 97, 2, 2, 418, 419, 7, 99, 2, 2, 419, 420,
7, 110, 2, 2, 420, 440, 7, 110, 2, 2, 421, 422, 7, 67, 2, 2, 422, 423,
7, 84, 2, 2, 423, 424, 7, 84, 2, 2, 424, 425, 7, 67, 2, 2, 425, 426, 7,
91, 2, 2, 426, 427, 7, 97, 2, 2, 427, 428, 7, 69, 2, 2, 428, 429, 7, 81,
2, 2, 429, 430, 7, 80, 2, 2, 430, 431, 7, 86, 2, 2, 431, 432, 7, 67, 2,
2, 432, 433, 7, 75, 2, 2, 433, 434, 7, 80, 2, 2, 434, 435, 7, 85, 2, 2,
435, 436, 7, 97, 2, 2, 436, 437, 7, 67, 2, 2, 437, 438, 7, 78, 2, 2, 438,
440, 7, 78, 2, 2, 439, 403, 3, 2, 2, 2, 439, 421, 3, 2, 2, 2, 440, 76,
3, 2, 2, 2, 441, 442, 7, 99, 2, 2, 442, 443, 7, 116, 2, 2, 443, 444, 7,
116, 2, 2, 444, 445, 7, 99, 2, 2, 445, 446, 7, 123, 2, 2, 446, 447, 7,
97, 2, 2, 447, 448, 7, 101, 2, 2, 448, 449, 7, 113, 2, 2, 449, 450, 7,
112, 2, 2, 450, 451, 7, 118, 2, 2, 451, 452, 7, 99, 2, 2, 452, 453, 7,
107, 2, 2, 453, 454, 7, 112, 2, 2, 454, 455, 7, 117, 2, 2, 455, 456, 7,
97, 2, 2, 456, 457, 7, 99, 2, 2, 457, 458, 7, 112, 2, 2, 458, 478, 7, 123,
2, 2, 459, 460, 7, 67, 2, 2, 460, 461, 7, 84, 2, 2, 461, 462, 7, 84, 2,
2, 462, 463, 7, 67, 2, 2, 463, 464, 7, 91, 2, 2, 464, 465, 7, 97, 2, 2,
465, 466, 7, 69, 2, 2, 466, 467, 7, 81, 2, 2, 467, 468, 7, 80, 2, 2, 468,
469, 7, 86, 2, 2, 469, 470, 7, 67, 2, 2, 470, 471, 7, 75, 2, 2, 471, 472,
7, 80, 2, 2, 472, 473, 7, 85, 2, 2, 473, 474, 7, 97, 2, 2, 474, 475, 7,
67, 2, 2, 475, 476, 7, 80, 2, 2, 476, 478, 7, 91, 2, 2, 477, 441, 3, 2,
2, 2, 477, 459, 3, 2, 2, 2, 478, 78, 3, 2, 2, 2, 479, 480, 7, 99, 2, 2,
480, 481, 7, 116, 2, 2, 481, 482, 7, 116, 2, 2, 482, 483, 7, 99, 2, 2,
483, 484, 7, 123, 2, 2, 484, 485, 7, 97, 2, 2, 485, 486, 7, 110, 2, 2,
486, 487, 7, 103, 2, 2, 487, 488, 7, 112, 2, 2, 488, 489, 7, 105, 2, 2,
489, 490, 7, 118, 2, 2, 490, 504, 7, 106, 2, 2, 491, 492, 7, 67, 2, 2,
492, 493, 7, 84, 2, 2, 493, 494, 7, 84, 2, 2, 494, 495, 7, 67, 2, 2, 495,
496, 7, 91, 2, 2, 496, 497, 7, 97, 2, 2, 497, 498, 7, 78, 2, 2, 498, 499,
7, 71, 2, 2, 499, 500, 7, 80, 2, 2, 500, 501, 7, 73, 2, 2, 501, 502, 7,
86, 2, 2, 502, 504, 7, 74, 2, 2, 503, 479, 3, 2, 2, 2, 503, 491, 3, 2,
2, 2, 504, 80, 3, 2, 2, 2, 505, 506, 7, 118, 2, 2, 506, 507, 7, 116, 2,
2, 507, 508, 7, 119, 2, 2, 508, 533, 7, 103, 2, 2, 509, 510, 7, 86, 2,
2, 510, 511, 7, 116, 2, 2, 511, 512, 7, 119, 2, 2, 512, 533, 7, 103, 2,
2, 513, 514, 7, 86, 2, 2, 514, 515, 7, 84, 2, 2, 515, 516, 7, 87, 2, 2,
516, 533, 7, 71, 2, 2, 517, 518, 7, 104, 2, 2, 518, 519, 7, 99, 2, 2, 519,
520, 7, 110, 2, 2, 520, 521, 7, 117, 2, 2, 521, 533, 7, 103, 2, 2, 522,
523, 7, 72, 2, 2, 523, 524, 7, 99, 2, 2, 524, 525, 7, 110, 2, 2, 525, 526,
7, 117, 2, 2, 526, 533, 7, 103, 2, 2, 527, 528, 7, 72, 2, 2, 528, 529,
7, 67, 2, 2, 529, 530, 7, 78, 2, 2, 530, 531, 7, 85, 2, 2, 531, 533, 7,
71, 2, 2, 532, 505, 3, 2, 2, 2, 532, 509, 3, 2, 2, 2, 532, 513, 3, 2, 2,
2, 532, 517, 3, 2, 2, 2, 532, 522, 3, 2, 2, 2, 532, 527, 3, 2, 2, 2, 533,
82, 3, 2, 2, 2, 534, 539, 5, 109, 55, 2, 535, 539, 5, 111, 56, 2, 536,
539, 5, 113, 57, 2, 537, 539, 5, 107, 54, 2, 538, 534, 3, 2, 2, 2, 538,
535, 3, 2, 2, 2, 538, 536, 3, 2, 2, 2, 538, 537, 3, 2, 2, 2, 539, 84, 3,
2, 2, 2, 540, 543, 5, 125, 63, 2, 541, 543, 5, 127, 64, 2, 542, 540, 3,
2, 2, 2, 542, 541, 3, 2, 2, 2, 543, 86, 3, 2, 2, 2, 544, 549, 5, 103, 52,
2, 545, 548, 5, 103, 52, 2, 546, 548, 5, 105, 53, 2, 547, 545, 3, 2, 2,
2, 547, 546, 3, 2, 2, 2, 548, 551, 3, 2, 2, 2, 549, 547, 3, 2, 2, 2, 549,
550, 3, 2, 2, 2, 550, 558, 3, 2, 2, 2, 551, 549, 3, 2, 2, 2, 552, 553,
7, 38, 2, 2, 553, 554, 7, 111, 2, 2, 554, 555, 7, 103, 2, 2, 555, 556,
7, 118, 2, 2, 556, 558, 7, 99, 2, 2, 557, 544, 3, 2, 2, 2, 557, 552, 3,
2, 2, 2, 558, 88, 3, 2, 2, 2, 559, 561, 5, 93, 47, 2, 560, 559, 3, 2, 2,
2, 560, 561, 3, 2, 2, 2, 561, 572, 3, 2, 2, 2, 562, 564, 7, 36, 2, 2, 563,
565, 5, 95, 48, 2, 564, 563, 3, 2, 2, 2, 564, 565, 3, 2, 2, 2, 565, 566,
3, 2, 2, 2, 566, 573, 7, 36, 2, 2, 567, 569, 7, 41, 2, 2, 568, 570, 5,
97, 49, 2, 569, 568, 3, 2, 2, 2, 569, 570, 3, 2, 2, 2, 570, 571, 3, 2,
2, 2, 571, 573, 7, 41, 2, 2, 572, 562, 3, 2, 2, 2, 572, 567, 3, 2, 2, 2,
573, 90, 3, 2, 2, 2, 574, 582, 5, 87, 44, 2, 575, 578, 7, 93, 2, 2, 576,
579, 5, 89, 45, 2, 577, 579, 5, 109, 55, 2, 578, 576, 3, 2, 2, 2, 578,
577, 3, 2, 2, 2, 579, 580, 3, 2, 2, 2, 580, 581, 7, 95, 2, 2, 581, 583,
3, 2, 2, 2, 582, 575, 3, 2, 2, 2, 583, 584, 3, 2, 2, 2, 584, 582, 3, 2,
2, 2, 584, 585, 3, 2, 2, 2, 585, 92, 3, 2, 2, 2, 586, 587, 7, 119, 2, 2,
587, 590, 7, 58, 2, 2, 588, 590, 9, 2, 2, 2, 589, 586, 3, 2, 2, 2, 589,
588, 3, 2, 2, 2, 590, 94, 3, 2, 2, 2, 591, 593, 5, 99, 50, 2, 592, 591,
3, 2, 2, 2, 593, 594, 3, 2, 2, 2, 594, 592, 3, 2, 2, 2, 594, 595, 3, 2,
2, 2, 595, 96, 3, 2, 2, 2, 596, 598, 5, 101, 51, 2, 597, 596, 3, 2, 2,
2, 598, 599, 3, 2, 2, 2, 599, 597, 3, 2, 2, 2, 599, 600, 3, 2, 2, 2, 600,
98, 3, 2, 2, 2, 601, 609, 10, 3, 2, 2, 602, 609, 5, 141, 71, 2, 603, 604,
7, 94, 2, 2, 604, 609, 7, 12, 2, 2, 605, 606, 7, 94, 2, 2, 606, 607, 7,
15, 2, 2, 607, 609, 7, 12, 2, 2, 608, 601, 3, 2, 2, 2, 608, 602, 3, 2,
2, 2, 608, 603, 3, 2, 2, 2, 608, 605, 3, 2, 2, 2, 609, 100, 3, 2, 2, 2,
610, 618, 10, 4, 2, 2, 611, 618, 5, 141, 71, 2, 612, 613, 7, 94, 2, 2,
613, 618, 7, 12, 2, 2, 614, 615, 7, 94, 2, 2, 615, 616, 7, 15, 2, 2, 616,
618, 7, 12, 2, 2, 617, 610, 3, 2, 2, 2, 617, 611, 3, 2, 2, 2, 617, 612,
3, 2, 2, 2, 617, 614, 3, 2, 2, 2, 618, 102, 3, 2, 2, 2, 619, 620, 9, 5,
2, 2, 620, 104, 3, 2, 2, 2, 621, 622, 9, 6, 2, 2, 622, 106, 3, 2, 2, 2,
623, 624, 7, 50, 2, 2, 624, 626, 9, 7, 2, 2, 625, 627, 9, 8, 2, 2, 626,
625, 3, 2, 2, 2, 627, 628, 3, 2, 2, 2, 628, 626, 3, 2, 2, 2, 628, 629,
3, 2, 2, 2, 629, 108, 3, 2, 2, 2, 630, 634, 5, 115, 58, 2, 631, 633, 5,
105, 53, 2, 632, 631, 3, 2, 2, 2, 633, 636, 3, 2, 2, 2, 634, 632, 3, 2,
2, 2, 634, 635, 3, 2, 2, 2, 635, 639, 3, 2, 2, 2, 636, 634, 3, 2, 2, 2,
637, 639, 7, 50, 2, 2, 638, 630, 3, 2, 2, 2, 638, 637, 3, 2, 2, 2, 639,
110, 3, 2, 2, 2, 640, 644, 7, 50, 2, 2, 641, 643, 5, 117, 59, 2, 642, 641,
3, 2, 2, 2, 643, 646, 3, 2, 2, 2, 644, 642, 3, 2, 2, 2, 644, 645, 3, 2,
2, 2, 645, 112, 3, 2, 2, 2, 646, 644, 3, 2, 2, 2, 647, 648, 7, 50, 2, 2,
648, 649, 9, 9, 2, 2, 649, 650, 5, 137, 69, 2, 650, 114, 3, 2, 2, 2, 651,
652, 9, 10, 2, 2, 652, 116, 3, 2, 2, 2, 653, 654, 9, 11, 2, 2, 654, 118,
3, 2, 2, 2, 655, 656, 9, 12, 2, 2, 656, 120, 3, 2, 2, 2, 657, 658, 5, 119,
60, 2, 658, 659, 5, 119, 60, 2, 659, 660, 5, 119, 60, 2, 660, 661, 5, 119,
60, 2, 661, 122, 3, 2, 2, 2, 662, 663, 7, 94, 2, 2, 663, 664, 7, 119, 2,
2, 664, 665, 3, 2, 2, 2, 665, 673, 5, 121, 61, 2, 666, 667, 7, 94, 2, 2,
667, 668, 7, 87, 2, 2, 668, 669, 3, 2, 2, 2, 669, 670, 5, 121, 61, 2, 670,
671, 5, 121, 61, 2, 671, 673, 3, 2, 2, 2, 672, 662, 3, 2, 2, 2, 672, 666,
3, 2, 2, 2, 673, 124, 3, 2, 2, 2, 674, 676, 5, 129, 65, 2, 675, 677, 5,
131, 66, 2, 676, 675, 3, 2, 2, 2, 676, 677, 3, 2, 2, 2, 677, 682, 3, 2,
2, 2, 678, 679, 5, 133, 67, 2, 679, 680, 5, 131, 66, 2, 680, 682, 3, 2,
2, 2, 681, 674, 3, 2, 2, 2, 681, 678, 3, 2, 2, 2, 682, 126, 3, 2, 2, 2,
683, 684, 7, 50, 2, 2, 684, 687, 9, 9, 2, 2, 685, 688, 5, 135, 68, 2, 686,
688, 5, 137, 69, 2, 687, 685, 3, 2, 2, 2, 687, 686, 3, 2, 2, 2, 688, 689,
3, 2, 2, 2, 689, 690, 5, 139, 70, 2, 690, 128, 3, 2, 2, 2, 691, 693, 5,
133, 67, 2, 692, 691, 3, 2, 2, 2, 692, 693, 3, 2, 2, 2, 693, 694, 3, 2,
2, 2, 694, 695, 7, 48, 2, 2, 695, 700, 5, 133, 67, 2, 696, 697, 5, 133,
67, 2, 697, 698, 7, 48, 2, 2, 698, 700, 3, 2, 2, 2, 699, 692, 3, 2, 2,
2, 699, 696, 3, 2, 2, 2, 700, 130, 3, 2, 2, 2, 701, 703, 9, 13, 2, 2, 702,
704, 9, 14, 2, 2, 703, 702, 3, 2, 2, 2, 703, 704, 3, 2, 2, 2, 704, 705,
3, 2, 2, 2, 705, 706, 5, 133, 67, 2, 706, 132, 3, 2, 2, 2, 707, 709, 5,
105, 53, 2, 708, 707, 3, 2, 2, 2, 709, 710, 3, 2, 2, 2, 710, 708, 3, 2,
2, 2, 710, 711, 3, 2, 2, 2, 711, 134, 3, 2, 2, 2, 712, 714, 5, 137, 69,
2, 713, 712, 3, 2, 2, 2, 713, 714, 3, 2, 2, 2, 714, 715, 3, 2, 2, 2, 715,
716, 7, 48, 2, 2, 716, 721, 5, 137, 69, 2, 717, 718, 5, 137, 69, 2, 718,
719, 7, 48, 2, 2, 719, 721, 3, 2, 2, 2, 720, 713, 3, 2, 2, 2, 720, 717,
3, 2, 2, 2, 721, 136, 3, 2, 2, 2, 722, 724, 5, 119, 60, 2, 723, 722, 3,
2, 2, 2, 724, 725, 3, 2, 2, 2, 725, 723, 3, 2, 2, 2, 725, 726, 3, 2, 2,
2, 726, 138, 3, 2, 2, 2, 727, 729, 9, 15, 2, 2, 728, 730, 9, 14, 2, 2,
729, 728, 3, 2, 2, 2, 729, 730, 3, 2, 2, 2, 730, 731, 3, 2, 2, 2, 731,
732, 5, 133, 67, 2, 732, 140, 3, 2, 2, 2, 733, 734, 7, 94, 2, 2, 734, 749,
9, 16, 2, 2, 735, 736, 7, 94, 2, 2, 736, 738, 5, 117, 59, 2, 737, 739,
5, 117, 59, 2, 738, 737, 3, 2, 2, 2, 738, 739, 3, 2, 2, 2, 739, 741, 3,
2, 2, 2, 740, 742, 5, 117, 59, 2, 741, 740, 3, 2, 2, 2, 741, 742, 3, 2,
2, 2, 742, 749, 3, 2, 2, 2, 743, 744, 7, 94, 2, 2, 744, 745, 7, 122, 2,
2, 745, 746, 3, 2, 2, 2, 746, 749, 5, 137, 69, 2, 747, 749, 5, 123, 62,
2, 748, 733, 3, 2, 2, 2, 748, 735, 3, 2, 2, 2, 748, 743, 3, 2, 2, 2, 748,
747, 3, 2, 2, 2, 749, 142, 3, 2, 2, 2, 750, 752, 9, 17, 2, 2, 751, 750,
3, 2, 2, 2, 752, 753, 3, 2, 2, 2, 753, 751, 3, 2, 2, 2, 753, 754, 3, 2,
2, 2, 754, 755, 3, 2, 2, 2, 755, 756, 8, 72, 2, 2, 756, 144, 3, 2, 2, 2,
757, 759, 7, 15, 2, 2, 758, 760, 7, 12, 2, 2, 759, 758, 3, 2, 2, 2, 759,
760, 3, 2, 2, 2, 760, 763, 3, 2, 2, 2, 761, 763, 7, 12, 2, 2, 762, 757,
3, 2, 2, 2, 762, 761, 3, 2, 2, 2, 763, 764, 3, 2, 2, 2, 764, 765, 8, 73,
2, 2, 765, 146, 3, 2, 2, 2, 56, 2, 181, 195, 237, 243, 251, 266, 268, 299,
335, 371, 401, 439, 477, 503, 532, 538, 542, 547, 549, 557, 560, 564, 569,
572, 578, 584, 589, 594, 599, 608, 617, 628, 634, 638, 644, 672, 676, 681,
687, 692, 699, 703, 710, 713, 720, 725, 729, 738, 741, 748, 753, 759, 762,
3, 8, 2, 2,
}
@ -369,14 +374,14 @@ var lexerModeNames = []string{
var lexerLiteralNames = []string{
"", "'('", "')'", "'['", "','", "']'", "'<'", "'<='", "'>'", "'>='", "'=='",
"'!='", "", "", "'+'", "'-'", "'*'", "'/'", "'%'", "'**'", "'<<'", "'>>'",
"'&'", "'|'", "'^'", "", "", "'~'", "", "'in'", "'not in'",
"'!='", "", "", "'TextMatch'", "'+'", "'-'", "'*'", "'/'", "'%'", "'**'",
"'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "'~'", "", "'in'", "'not in'",
}
var lexerSymbolicNames = []string{
"", "", "", "", "", "", "LT", "LE", "GT", "GE", "EQ", "NE", "LIKE", "EXISTS",
"ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND", "BOR",
"BXOR", "AND", "OR", "BNOT", "NOT", "IN", "NIN", "EmptyTerm", "JSONContains",
"TEXTMATCH", "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND",
"BOR", "BXOR", "AND", "OR", "BNOT", "NOT", "IN", "NIN", "EmptyTerm", "JSONContains",
"JSONContainsAll", "JSONContainsAny", "ArrayContains", "ArrayContainsAll",
"ArrayContainsAny", "ArrayLength", "BooleanConstant", "IntegerConstant",
"FloatingConstant", "Identifier", "StringLiteral", "JSONIdentifier", "Whitespace",
@ -385,18 +390,18 @@ var lexerSymbolicNames = []string{
var lexerRuleNames = []string{
"T__0", "T__1", "T__2", "T__3", "T__4", "LT", "LE", "GT", "GE", "EQ", "NE",
"LIKE", "EXISTS", "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR",
"BAND", "BOR", "BXOR", "AND", "OR", "BNOT", "NOT", "IN", "NIN", "EmptyTerm",
"JSONContains", "JSONContainsAll", "JSONContainsAny", "ArrayContains",
"ArrayContainsAll", "ArrayContainsAny", "ArrayLength", "BooleanConstant",
"IntegerConstant", "FloatingConstant", "Identifier", "StringLiteral", "JSONIdentifier",
"EncodingPrefix", "DoubleSCharSequence", "SingleSCharSequence", "DoubleSChar",
"SingleSChar", "Nondigit", "Digit", "BinaryConstant", "DecimalConstant",
"OctalConstant", "HexadecimalConstant", "NonzeroDigit", "OctalDigit", "HexadecimalDigit",
"HexQuad", "UniversalCharacterName", "DecimalFloatingConstant", "HexadecimalFloatingConstant",
"FractionalConstant", "ExponentPart", "DigitSequence", "HexadecimalFractionalConstant",
"HexadecimalDigitSequence", "BinaryExponentPart", "EscapeSequence", "Whitespace",
"Newline",
"LIKE", "EXISTS", "TEXTMATCH", "ADD", "SUB", "MUL", "DIV", "MOD", "POW",
"SHL", "SHR", "BAND", "BOR", "BXOR", "AND", "OR", "BNOT", "NOT", "IN",
"NIN", "EmptyTerm", "JSONContains", "JSONContainsAll", "JSONContainsAny",
"ArrayContains", "ArrayContainsAll", "ArrayContainsAny", "ArrayLength",
"BooleanConstant", "IntegerConstant", "FloatingConstant", "Identifier",
"StringLiteral", "JSONIdentifier", "EncodingPrefix", "DoubleSCharSequence",
"SingleSCharSequence", "DoubleSChar", "SingleSChar", "Nondigit", "Digit",
"BinaryConstant", "DecimalConstant", "OctalConstant", "HexadecimalConstant",
"NonzeroDigit", "OctalDigit", "HexadecimalDigit", "HexQuad", "UniversalCharacterName",
"DecimalFloatingConstant", "HexadecimalFloatingConstant", "FractionalConstant",
"ExponentPart", "DigitSequence", "HexadecimalFractionalConstant", "HexadecimalDigitSequence",
"BinaryExponentPart", "EscapeSequence", "Whitespace", "Newline",
}
type PlanLexer struct {
@ -449,37 +454,38 @@ const (
PlanLexerNE = 11
PlanLexerLIKE = 12
PlanLexerEXISTS = 13
PlanLexerADD = 14
PlanLexerSUB = 15
PlanLexerMUL = 16
PlanLexerDIV = 17
PlanLexerMOD = 18
PlanLexerPOW = 19
PlanLexerSHL = 20
PlanLexerSHR = 21
PlanLexerBAND = 22
PlanLexerBOR = 23
PlanLexerBXOR = 24
PlanLexerAND = 25
PlanLexerOR = 26
PlanLexerBNOT = 27
PlanLexerNOT = 28
PlanLexerIN = 29
PlanLexerNIN = 30
PlanLexerEmptyTerm = 31
PlanLexerJSONContains = 32
PlanLexerJSONContainsAll = 33
PlanLexerJSONContainsAny = 34
PlanLexerArrayContains = 35
PlanLexerArrayContainsAll = 36
PlanLexerArrayContainsAny = 37
PlanLexerArrayLength = 38
PlanLexerBooleanConstant = 39
PlanLexerIntegerConstant = 40
PlanLexerFloatingConstant = 41
PlanLexerIdentifier = 42
PlanLexerStringLiteral = 43
PlanLexerJSONIdentifier = 44
PlanLexerWhitespace = 45
PlanLexerNewline = 46
PlanLexerTEXTMATCH = 14
PlanLexerADD = 15
PlanLexerSUB = 16
PlanLexerMUL = 17
PlanLexerDIV = 18
PlanLexerMOD = 19
PlanLexerPOW = 20
PlanLexerSHL = 21
PlanLexerSHR = 22
PlanLexerBAND = 23
PlanLexerBOR = 24
PlanLexerBXOR = 25
PlanLexerAND = 26
PlanLexerOR = 27
PlanLexerBNOT = 28
PlanLexerNOT = 29
PlanLexerIN = 30
PlanLexerNIN = 31
PlanLexerEmptyTerm = 32
PlanLexerJSONContains = 33
PlanLexerJSONContainsAll = 34
PlanLexerJSONContainsAny = 35
PlanLexerArrayContains = 36
PlanLexerArrayContainsAll = 37
PlanLexerArrayContainsAny = 38
PlanLexerArrayLength = 39
PlanLexerBooleanConstant = 40
PlanLexerIntegerConstant = 41
PlanLexerFloatingConstant = 42
PlanLexerIdentifier = 43
PlanLexerStringLiteral = 44
PlanLexerJSONIdentifier = 45
PlanLexerWhitespace = 46
PlanLexerNewline = 47
)

View File

@ -15,79 +15,82 @@ var _ = reflect.Copy
var _ = strconv.Itoa
var parserATN = []uint16{
3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 3, 48, 131,
3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 3, 49, 137,
4, 2, 9, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 20, 10, 2, 12, 2, 14, 2, 23, 11, 2,
3, 2, 5, 2, 26, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 5, 2,
59, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 5, 2, 65, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 113, 10, 2, 12, 2, 14, 2, 116,
11, 2, 3, 2, 5, 2, 119, 10, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 126,
10, 2, 12, 2, 14, 2, 129, 11, 2, 3, 2, 2, 3, 2, 3, 2, 2, 15, 4, 2, 16,
17, 29, 30, 4, 2, 34, 34, 37, 37, 4, 2, 35, 35, 38, 38, 4, 2, 36, 36, 39,
39, 4, 2, 44, 44, 46, 46, 3, 2, 18, 20, 3, 2, 16, 17, 3, 2, 22, 23, 3,
2, 8, 9, 3, 2, 10, 11, 3, 2, 8, 11, 3, 2, 12, 13, 3, 2, 31, 32, 2, 162,
2, 58, 3, 2, 2, 2, 4, 5, 8, 2, 1, 2, 5, 59, 7, 42, 2, 2, 6, 59, 7, 43,
2, 2, 7, 59, 7, 41, 2, 2, 8, 59, 7, 45, 2, 2, 9, 59, 7, 44, 2, 2, 10, 59,
7, 46, 2, 2, 11, 12, 7, 3, 2, 2, 12, 13, 5, 2, 2, 2, 13, 14, 7, 4, 2, 2,
14, 59, 3, 2, 2, 2, 15, 16, 7, 5, 2, 2, 16, 21, 5, 2, 2, 2, 17, 18, 7,
6, 2, 2, 18, 20, 5, 2, 2, 2, 19, 17, 3, 2, 2, 2, 20, 23, 3, 2, 2, 2, 21,
19, 3, 2, 2, 2, 21, 22, 3, 2, 2, 2, 22, 25, 3, 2, 2, 2, 23, 21, 3, 2, 2,
2, 24, 26, 7, 6, 2, 2, 25, 24, 3, 2, 2, 2, 25, 26, 3, 2, 2, 2, 26, 27,
3, 2, 2, 2, 27, 28, 7, 7, 2, 2, 28, 59, 3, 2, 2, 2, 29, 30, 9, 2, 2, 2,
30, 59, 5, 2, 2, 22, 31, 32, 9, 3, 2, 2, 32, 33, 7, 3, 2, 2, 33, 34, 5,
2, 2, 2, 34, 35, 7, 6, 2, 2, 35, 36, 5, 2, 2, 2, 36, 37, 7, 4, 2, 2, 37,
59, 3, 2, 2, 2, 38, 39, 9, 4, 2, 2, 39, 40, 7, 3, 2, 2, 40, 41, 5, 2, 2,
2, 41, 42, 7, 6, 2, 2, 42, 43, 5, 2, 2, 2, 43, 44, 7, 4, 2, 2, 44, 59,
3, 2, 2, 2, 45, 46, 9, 5, 2, 2, 46, 47, 7, 3, 2, 2, 47, 48, 5, 2, 2, 2,
48, 49, 7, 6, 2, 2, 49, 50, 5, 2, 2, 2, 50, 51, 7, 4, 2, 2, 51, 59, 3,
2, 2, 2, 52, 53, 7, 40, 2, 2, 53, 54, 7, 3, 2, 2, 54, 55, 9, 6, 2, 2, 55,
59, 7, 4, 2, 2, 56, 57, 7, 15, 2, 2, 57, 59, 5, 2, 2, 3, 58, 4, 3, 2, 2,
2, 58, 6, 3, 2, 2, 2, 58, 7, 3, 2, 2, 2, 58, 8, 3, 2, 2, 2, 58, 9, 3, 2,
2, 2, 58, 10, 3, 2, 2, 2, 58, 11, 3, 2, 2, 2, 58, 15, 3, 2, 2, 2, 58, 29,
3, 2, 2, 2, 58, 31, 3, 2, 2, 2, 58, 38, 3, 2, 2, 2, 58, 45, 3, 2, 2, 2,
58, 52, 3, 2, 2, 2, 58, 56, 3, 2, 2, 2, 59, 127, 3, 2, 2, 2, 60, 61, 12,
23, 2, 2, 61, 62, 7, 21, 2, 2, 62, 126, 5, 2, 2, 24, 63, 64, 12, 21, 2,
2, 64, 65, 9, 7, 2, 2, 65, 126, 5, 2, 2, 22, 66, 67, 12, 20, 2, 2, 67,
68, 9, 8, 2, 2, 68, 126, 5, 2, 2, 21, 69, 70, 12, 19, 2, 2, 70, 71, 9,
9, 2, 2, 71, 126, 5, 2, 2, 20, 72, 73, 12, 12, 2, 2, 73, 74, 9, 10, 2,
2, 74, 75, 9, 6, 2, 2, 75, 76, 9, 10, 2, 2, 76, 126, 5, 2, 2, 13, 77, 78,
12, 11, 2, 2, 78, 79, 9, 11, 2, 2, 79, 80, 9, 6, 2, 2, 80, 81, 9, 11, 2,
2, 81, 126, 5, 2, 2, 12, 82, 83, 12, 10, 2, 2, 83, 84, 9, 12, 2, 2, 84,
126, 5, 2, 2, 11, 85, 86, 12, 9, 2, 2, 86, 87, 9, 13, 2, 2, 87, 126, 5,
2, 2, 10, 88, 89, 12, 8, 2, 2, 89, 90, 7, 24, 2, 2, 90, 126, 5, 2, 2, 9,
91, 92, 12, 7, 2, 2, 92, 93, 7, 26, 2, 2, 93, 126, 5, 2, 2, 8, 94, 95,
12, 6, 2, 2, 95, 96, 7, 25, 2, 2, 96, 126, 5, 2, 2, 7, 97, 98, 12, 5, 2,
2, 98, 99, 7, 27, 2, 2, 99, 126, 5, 2, 2, 6, 100, 101, 12, 4, 2, 2, 101,
102, 7, 28, 2, 2, 102, 126, 5, 2, 2, 5, 103, 104, 12, 24, 2, 2, 104, 105,
7, 14, 2, 2, 105, 126, 7, 45, 2, 2, 106, 107, 12, 18, 2, 2, 107, 108, 9,
14, 2, 2, 108, 109, 7, 5, 2, 2, 109, 114, 5, 2, 2, 2, 110, 111, 7, 6, 2,
2, 111, 113, 5, 2, 2, 2, 112, 110, 3, 2, 2, 2, 113, 116, 3, 2, 2, 2, 114,
112, 3, 2, 2, 2, 114, 115, 3, 2, 2, 2, 115, 118, 3, 2, 2, 2, 116, 114,
3, 2, 2, 2, 117, 119, 7, 6, 2, 2, 118, 117, 3, 2, 2, 2, 118, 119, 3, 2,
2, 2, 119, 120, 3, 2, 2, 2, 120, 121, 7, 7, 2, 2, 121, 126, 3, 2, 2, 2,
122, 123, 12, 17, 2, 2, 123, 124, 9, 14, 2, 2, 124, 126, 7, 33, 2, 2, 125,
60, 3, 2, 2, 2, 125, 63, 3, 2, 2, 2, 125, 66, 3, 2, 2, 2, 125, 69, 3, 2,
2, 2, 125, 72, 3, 2, 2, 2, 125, 77, 3, 2, 2, 2, 125, 82, 3, 2, 2, 2, 125,
85, 3, 2, 2, 2, 125, 88, 3, 2, 2, 2, 125, 91, 3, 2, 2, 2, 125, 94, 3, 2,
2, 2, 125, 97, 3, 2, 2, 2, 125, 100, 3, 2, 2, 2, 125, 103, 3, 2, 2, 2,
125, 106, 3, 2, 2, 2, 125, 122, 3, 2, 2, 2, 126, 129, 3, 2, 2, 2, 127,
125, 3, 2, 2, 2, 127, 128, 3, 2, 2, 2, 128, 3, 3, 2, 2, 2, 129, 127, 3,
2, 2, 2, 9, 21, 25, 58, 114, 118, 125, 127,
3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2, 3, 2,
7, 2, 119, 10, 2, 12, 2, 14, 2, 122, 11, 2, 3, 2, 5, 2, 125, 10, 2, 3,
2, 3, 2, 3, 2, 3, 2, 3, 2, 7, 2, 132, 10, 2, 12, 2, 14, 2, 135, 11, 2,
3, 2, 2, 3, 2, 3, 2, 2, 15, 4, 2, 17, 18, 30, 31, 4, 2, 35, 35, 38, 38,
4, 2, 36, 36, 39, 39, 4, 2, 37, 37, 40, 40, 4, 2, 45, 45, 47, 47, 3, 2,
19, 21, 3, 2, 17, 18, 3, 2, 23, 24, 3, 2, 8, 9, 3, 2, 10, 11, 3, 2, 8,
11, 3, 2, 12, 13, 3, 2, 32, 33, 2, 169, 2, 64, 3, 2, 2, 2, 4, 5, 8, 2,
1, 2, 5, 65, 7, 43, 2, 2, 6, 65, 7, 44, 2, 2, 7, 65, 7, 42, 2, 2, 8, 65,
7, 46, 2, 2, 9, 65, 7, 45, 2, 2, 10, 65, 7, 47, 2, 2, 11, 12, 7, 3, 2,
2, 12, 13, 5, 2, 2, 2, 13, 14, 7, 4, 2, 2, 14, 65, 3, 2, 2, 2, 15, 16,
7, 5, 2, 2, 16, 21, 5, 2, 2, 2, 17, 18, 7, 6, 2, 2, 18, 20, 5, 2, 2, 2,
19, 17, 3, 2, 2, 2, 20, 23, 3, 2, 2, 2, 21, 19, 3, 2, 2, 2, 21, 22, 3,
2, 2, 2, 22, 25, 3, 2, 2, 2, 23, 21, 3, 2, 2, 2, 24, 26, 7, 6, 2, 2, 25,
24, 3, 2, 2, 2, 25, 26, 3, 2, 2, 2, 26, 27, 3, 2, 2, 2, 27, 28, 7, 7, 2,
2, 28, 65, 3, 2, 2, 2, 29, 30, 7, 16, 2, 2, 30, 31, 7, 3, 2, 2, 31, 32,
7, 45, 2, 2, 32, 33, 7, 6, 2, 2, 33, 34, 7, 46, 2, 2, 34, 65, 7, 4, 2,
2, 35, 36, 9, 2, 2, 2, 36, 65, 5, 2, 2, 22, 37, 38, 9, 3, 2, 2, 38, 39,
7, 3, 2, 2, 39, 40, 5, 2, 2, 2, 40, 41, 7, 6, 2, 2, 41, 42, 5, 2, 2, 2,
42, 43, 7, 4, 2, 2, 43, 65, 3, 2, 2, 2, 44, 45, 9, 4, 2, 2, 45, 46, 7,
3, 2, 2, 46, 47, 5, 2, 2, 2, 47, 48, 7, 6, 2, 2, 48, 49, 5, 2, 2, 2, 49,
50, 7, 4, 2, 2, 50, 65, 3, 2, 2, 2, 51, 52, 9, 5, 2, 2, 52, 53, 7, 3, 2,
2, 53, 54, 5, 2, 2, 2, 54, 55, 7, 6, 2, 2, 55, 56, 5, 2, 2, 2, 56, 57,
7, 4, 2, 2, 57, 65, 3, 2, 2, 2, 58, 59, 7, 41, 2, 2, 59, 60, 7, 3, 2, 2,
60, 61, 9, 6, 2, 2, 61, 65, 7, 4, 2, 2, 62, 63, 7, 15, 2, 2, 63, 65, 5,
2, 2, 3, 64, 4, 3, 2, 2, 2, 64, 6, 3, 2, 2, 2, 64, 7, 3, 2, 2, 2, 64, 8,
3, 2, 2, 2, 64, 9, 3, 2, 2, 2, 64, 10, 3, 2, 2, 2, 64, 11, 3, 2, 2, 2,
64, 15, 3, 2, 2, 2, 64, 29, 3, 2, 2, 2, 64, 35, 3, 2, 2, 2, 64, 37, 3,
2, 2, 2, 64, 44, 3, 2, 2, 2, 64, 51, 3, 2, 2, 2, 64, 58, 3, 2, 2, 2, 64,
62, 3, 2, 2, 2, 65, 133, 3, 2, 2, 2, 66, 67, 12, 23, 2, 2, 67, 68, 7, 22,
2, 2, 68, 132, 5, 2, 2, 24, 69, 70, 12, 21, 2, 2, 70, 71, 9, 7, 2, 2, 71,
132, 5, 2, 2, 22, 72, 73, 12, 20, 2, 2, 73, 74, 9, 8, 2, 2, 74, 132, 5,
2, 2, 21, 75, 76, 12, 19, 2, 2, 76, 77, 9, 9, 2, 2, 77, 132, 5, 2, 2, 20,
78, 79, 12, 12, 2, 2, 79, 80, 9, 10, 2, 2, 80, 81, 9, 6, 2, 2, 81, 82,
9, 10, 2, 2, 82, 132, 5, 2, 2, 13, 83, 84, 12, 11, 2, 2, 84, 85, 9, 11,
2, 2, 85, 86, 9, 6, 2, 2, 86, 87, 9, 11, 2, 2, 87, 132, 5, 2, 2, 12, 88,
89, 12, 10, 2, 2, 89, 90, 9, 12, 2, 2, 90, 132, 5, 2, 2, 11, 91, 92, 12,
9, 2, 2, 92, 93, 9, 13, 2, 2, 93, 132, 5, 2, 2, 10, 94, 95, 12, 8, 2, 2,
95, 96, 7, 25, 2, 2, 96, 132, 5, 2, 2, 9, 97, 98, 12, 7, 2, 2, 98, 99,
7, 27, 2, 2, 99, 132, 5, 2, 2, 8, 100, 101, 12, 6, 2, 2, 101, 102, 7, 26,
2, 2, 102, 132, 5, 2, 2, 7, 103, 104, 12, 5, 2, 2, 104, 105, 7, 28, 2,
2, 105, 132, 5, 2, 2, 6, 106, 107, 12, 4, 2, 2, 107, 108, 7, 29, 2, 2,
108, 132, 5, 2, 2, 5, 109, 110, 12, 25, 2, 2, 110, 111, 7, 14, 2, 2, 111,
132, 7, 46, 2, 2, 112, 113, 12, 18, 2, 2, 113, 114, 9, 14, 2, 2, 114, 115,
7, 5, 2, 2, 115, 120, 5, 2, 2, 2, 116, 117, 7, 6, 2, 2, 117, 119, 5, 2,
2, 2, 118, 116, 3, 2, 2, 2, 119, 122, 3, 2, 2, 2, 120, 118, 3, 2, 2, 2,
120, 121, 3, 2, 2, 2, 121, 124, 3, 2, 2, 2, 122, 120, 3, 2, 2, 2, 123,
125, 7, 6, 2, 2, 124, 123, 3, 2, 2, 2, 124, 125, 3, 2, 2, 2, 125, 126,
3, 2, 2, 2, 126, 127, 7, 7, 2, 2, 127, 132, 3, 2, 2, 2, 128, 129, 12, 17,
2, 2, 129, 130, 9, 14, 2, 2, 130, 132, 7, 34, 2, 2, 131, 66, 3, 2, 2, 2,
131, 69, 3, 2, 2, 2, 131, 72, 3, 2, 2, 2, 131, 75, 3, 2, 2, 2, 131, 78,
3, 2, 2, 2, 131, 83, 3, 2, 2, 2, 131, 88, 3, 2, 2, 2, 131, 91, 3, 2, 2,
2, 131, 94, 3, 2, 2, 2, 131, 97, 3, 2, 2, 2, 131, 100, 3, 2, 2, 2, 131,
103, 3, 2, 2, 2, 131, 106, 3, 2, 2, 2, 131, 109, 3, 2, 2, 2, 131, 112,
3, 2, 2, 2, 131, 128, 3, 2, 2, 2, 132, 135, 3, 2, 2, 2, 133, 131, 3, 2,
2, 2, 133, 134, 3, 2, 2, 2, 134, 3, 3, 2, 2, 2, 135, 133, 3, 2, 2, 2, 9,
21, 25, 64, 120, 124, 131, 133,
}
var literalNames = []string{
"", "'('", "')'", "'['", "','", "']'", "'<'", "'<='", "'>'", "'>='", "'=='",
"'!='", "", "", "'+'", "'-'", "'*'", "'/'", "'%'", "'**'", "'<<'", "'>>'",
"'&'", "'|'", "'^'", "", "", "'~'", "", "'in'", "'not in'",
"'!='", "", "", "'TextMatch'", "'+'", "'-'", "'*'", "'/'", "'%'", "'**'",
"'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "'~'", "", "'in'", "'not in'",
}
var symbolicNames = []string{
"", "", "", "", "", "", "LT", "LE", "GT", "GE", "EQ", "NE", "LIKE", "EXISTS",
"ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND", "BOR",
"BXOR", "AND", "OR", "BNOT", "NOT", "IN", "NIN", "EmptyTerm", "JSONContains",
"TEXTMATCH", "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND",
"BOR", "BXOR", "AND", "OR", "BNOT", "NOT", "IN", "NIN", "EmptyTerm", "JSONContains",
"JSONContainsAll", "JSONContainsAny", "ArrayContains", "ArrayContainsAll",
"ArrayContainsAny", "ArrayLength", "BooleanConstant", "IntegerConstant",
"FloatingConstant", "Identifier", "StringLiteral", "JSONIdentifier", "Whitespace",
@ -143,39 +146,40 @@ const (
PlanParserNE = 11
PlanParserLIKE = 12
PlanParserEXISTS = 13
PlanParserADD = 14
PlanParserSUB = 15
PlanParserMUL = 16
PlanParserDIV = 17
PlanParserMOD = 18
PlanParserPOW = 19
PlanParserSHL = 20
PlanParserSHR = 21
PlanParserBAND = 22
PlanParserBOR = 23
PlanParserBXOR = 24
PlanParserAND = 25
PlanParserOR = 26
PlanParserBNOT = 27
PlanParserNOT = 28
PlanParserIN = 29
PlanParserNIN = 30
PlanParserEmptyTerm = 31
PlanParserJSONContains = 32
PlanParserJSONContainsAll = 33
PlanParserJSONContainsAny = 34
PlanParserArrayContains = 35
PlanParserArrayContainsAll = 36
PlanParserArrayContainsAny = 37
PlanParserArrayLength = 38
PlanParserBooleanConstant = 39
PlanParserIntegerConstant = 40
PlanParserFloatingConstant = 41
PlanParserIdentifier = 42
PlanParserStringLiteral = 43
PlanParserJSONIdentifier = 44
PlanParserWhitespace = 45
PlanParserNewline = 46
PlanParserTEXTMATCH = 14
PlanParserADD = 15
PlanParserSUB = 16
PlanParserMUL = 17
PlanParserDIV = 18
PlanParserMOD = 19
PlanParserPOW = 20
PlanParserSHL = 21
PlanParserSHR = 22
PlanParserBAND = 23
PlanParserBOR = 24
PlanParserBXOR = 25
PlanParserAND = 26
PlanParserOR = 27
PlanParserBNOT = 28
PlanParserNOT = 29
PlanParserIN = 30
PlanParserNIN = 31
PlanParserEmptyTerm = 32
PlanParserJSONContains = 33
PlanParserJSONContainsAll = 34
PlanParserJSONContainsAny = 35
PlanParserArrayContains = 36
PlanParserArrayContainsAll = 37
PlanParserArrayContainsAny = 38
PlanParserArrayLength = 39
PlanParserBooleanConstant = 40
PlanParserIntegerConstant = 41
PlanParserFloatingConstant = 42
PlanParserIdentifier = 43
PlanParserStringLiteral = 44
PlanParserJSONIdentifier = 45
PlanParserWhitespace = 46
PlanParserNewline = 47
)
// PlanParserRULE_expr is the PlanParser rule.
@ -1156,6 +1160,46 @@ func (s *ArrayLengthContext) Accept(visitor antlr.ParseTreeVisitor) interface{}
}
}
type TextMatchContext struct {
*ExprContext
}
func NewTextMatchContext(parser antlr.Parser, ctx antlr.ParserRuleContext) *TextMatchContext {
var p = new(TextMatchContext)
p.ExprContext = NewEmptyExprContext()
p.parser = parser
p.CopyFrom(ctx.(*ExprContext))
return p
}
func (s *TextMatchContext) GetRuleContext() antlr.RuleContext {
return s
}
func (s *TextMatchContext) TEXTMATCH() antlr.TerminalNode {
return s.GetToken(PlanParserTEXTMATCH, 0)
}
func (s *TextMatchContext) Identifier() antlr.TerminalNode {
return s.GetToken(PlanParserIdentifier, 0)
}
func (s *TextMatchContext) StringLiteral() antlr.TerminalNode {
return s.GetToken(PlanParserStringLiteral, 0)
}
func (s *TextMatchContext) Accept(visitor antlr.ParseTreeVisitor) interface{} {
switch t := visitor.(type) {
case PlanVisitor:
return t.VisitTextMatch(s)
default:
return t.VisitChildren(s)
}
}
type TermContext struct {
*ExprContext
op antlr.Token
@ -1860,7 +1904,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
var _alt int
p.EnterOuterAlt(localctx, 1)
p.SetState(56)
p.SetState(62)
p.GetErrorHandler().Sync(p)
switch p.GetTokenStream().LA(1) {
@ -1984,12 +2028,41 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
p.Match(PlanParserT__4)
}
case PlanParserTEXTMATCH:
localctx = NewTextMatchContext(p, localctx)
p.SetParserRuleContext(localctx)
_prevctx = localctx
{
p.SetState(27)
p.Match(PlanParserTEXTMATCH)
}
{
p.SetState(28)
p.Match(PlanParserT__0)
}
{
p.SetState(29)
p.Match(PlanParserIdentifier)
}
{
p.SetState(30)
p.Match(PlanParserT__3)
}
{
p.SetState(31)
p.Match(PlanParserStringLiteral)
}
{
p.SetState(32)
p.Match(PlanParserT__1)
}
case PlanParserADD, PlanParserSUB, PlanParserBNOT, PlanParserNOT:
localctx = NewUnaryContext(p, localctx)
p.SetParserRuleContext(localctx)
_prevctx = localctx
{
p.SetState(27)
p.SetState(33)
var _lt = p.GetTokenStream().LT(1)
@ -2007,7 +2080,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(28)
p.SetState(34)
p.expr(20)
}
@ -2016,7 +2089,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
p.SetParserRuleContext(localctx)
_prevctx = localctx
{
p.SetState(29)
p.SetState(35)
_la = p.GetTokenStream().LA(1)
if !(_la == PlanParserJSONContains || _la == PlanParserArrayContains) {
@ -2027,23 +2100,23 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(30)
p.SetState(36)
p.Match(PlanParserT__0)
}
{
p.SetState(31)
p.SetState(37)
p.expr(0)
}
{
p.SetState(32)
p.SetState(38)
p.Match(PlanParserT__3)
}
{
p.SetState(33)
p.SetState(39)
p.expr(0)
}
{
p.SetState(34)
p.SetState(40)
p.Match(PlanParserT__1)
}
@ -2052,7 +2125,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
p.SetParserRuleContext(localctx)
_prevctx = localctx
{
p.SetState(36)
p.SetState(42)
_la = p.GetTokenStream().LA(1)
if !(_la == PlanParserJSONContainsAll || _la == PlanParserArrayContainsAll) {
@ -2063,23 +2136,23 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(37)
p.SetState(43)
p.Match(PlanParserT__0)
}
{
p.SetState(38)
p.SetState(44)
p.expr(0)
}
{
p.SetState(39)
p.SetState(45)
p.Match(PlanParserT__3)
}
{
p.SetState(40)
p.SetState(46)
p.expr(0)
}
{
p.SetState(41)
p.SetState(47)
p.Match(PlanParserT__1)
}
@ -2088,7 +2161,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
p.SetParserRuleContext(localctx)
_prevctx = localctx
{
p.SetState(43)
p.SetState(49)
_la = p.GetTokenStream().LA(1)
if !(_la == PlanParserJSONContainsAny || _la == PlanParserArrayContainsAny) {
@ -2099,23 +2172,23 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(44)
p.SetState(50)
p.Match(PlanParserT__0)
}
{
p.SetState(45)
p.SetState(51)
p.expr(0)
}
{
p.SetState(46)
p.SetState(52)
p.Match(PlanParserT__3)
}
{
p.SetState(47)
p.SetState(53)
p.expr(0)
}
{
p.SetState(48)
p.SetState(54)
p.Match(PlanParserT__1)
}
@ -2124,15 +2197,15 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
p.SetParserRuleContext(localctx)
_prevctx = localctx
{
p.SetState(50)
p.SetState(56)
p.Match(PlanParserArrayLength)
}
{
p.SetState(51)
p.SetState(57)
p.Match(PlanParserT__0)
}
{
p.SetState(52)
p.SetState(58)
_la = p.GetTokenStream().LA(1)
if !(_la == PlanParserIdentifier || _la == PlanParserJSONIdentifier) {
@ -2143,7 +2216,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(53)
p.SetState(59)
p.Match(PlanParserT__1)
}
@ -2152,11 +2225,11 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
p.SetParserRuleContext(localctx)
_prevctx = localctx
{
p.SetState(54)
p.SetState(60)
p.Match(PlanParserEXISTS)
}
{
p.SetState(55)
p.SetState(61)
p.expr(1)
}
@ -2164,7 +2237,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
panic(antlr.NewNoViableAltException(p, nil, nil, nil, nil, nil))
}
p.GetParserRuleContext().SetStop(p.GetTokenStream().LT(-1))
p.SetState(125)
p.SetState(131)
p.GetErrorHandler().Sync(p)
_alt = p.GetInterpreter().AdaptivePredict(p.GetTokenStream(), 6, p.GetParserRuleContext())
@ -2174,36 +2247,36 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
p.TriggerExitRuleEvent()
}
_prevctx = localctx
p.SetState(123)
p.SetState(129)
p.GetErrorHandler().Sync(p)
switch p.GetInterpreter().AdaptivePredict(p.GetTokenStream(), 5, p.GetParserRuleContext()) {
case 1:
localctx = NewPowerContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(58)
p.SetState(64)
if !(p.Precpred(p.GetParserRuleContext(), 21)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 21)", ""))
}
{
p.SetState(59)
p.SetState(65)
p.Match(PlanParserPOW)
}
{
p.SetState(60)
p.SetState(66)
p.expr(22)
}
case 2:
localctx = NewMulDivModContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(61)
p.SetState(67)
if !(p.Precpred(p.GetParserRuleContext(), 19)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 19)", ""))
}
{
p.SetState(62)
p.SetState(68)
var _lt = p.GetTokenStream().LT(1)
@ -2221,20 +2294,20 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(63)
p.SetState(69)
p.expr(20)
}
case 3:
localctx = NewAddSubContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(64)
p.SetState(70)
if !(p.Precpred(p.GetParserRuleContext(), 18)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 18)", ""))
}
{
p.SetState(65)
p.SetState(71)
var _lt = p.GetTokenStream().LT(1)
@ -2252,20 +2325,20 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(66)
p.SetState(72)
p.expr(19)
}
case 4:
localctx = NewShiftContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(67)
p.SetState(73)
if !(p.Precpred(p.GetParserRuleContext(), 17)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 17)", ""))
}
{
p.SetState(68)
p.SetState(74)
var _lt = p.GetTokenStream().LT(1)
@ -2283,20 +2356,20 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(69)
p.SetState(75)
p.expr(18)
}
case 5:
localctx = NewRangeContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(70)
p.SetState(76)
if !(p.Precpred(p.GetParserRuleContext(), 10)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 10)", ""))
}
{
p.SetState(71)
p.SetState(77)
var _lt = p.GetTokenStream().LT(1)
@ -2314,7 +2387,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(72)
p.SetState(78)
_la = p.GetTokenStream().LA(1)
if !(_la == PlanParserIdentifier || _la == PlanParserJSONIdentifier) {
@ -2325,7 +2398,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(73)
p.SetState(79)
var _lt = p.GetTokenStream().LT(1)
@ -2343,20 +2416,20 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(74)
p.SetState(80)
p.expr(11)
}
case 6:
localctx = NewReverseRangeContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(75)
p.SetState(81)
if !(p.Precpred(p.GetParserRuleContext(), 9)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 9)", ""))
}
{
p.SetState(76)
p.SetState(82)
var _lt = p.GetTokenStream().LT(1)
@ -2374,7 +2447,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(77)
p.SetState(83)
_la = p.GetTokenStream().LA(1)
if !(_la == PlanParserIdentifier || _la == PlanParserJSONIdentifier) {
@ -2385,7 +2458,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(78)
p.SetState(84)
var _lt = p.GetTokenStream().LT(1)
@ -2403,20 +2476,20 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(79)
p.SetState(85)
p.expr(10)
}
case 7:
localctx = NewRelationalContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(80)
p.SetState(86)
if !(p.Precpred(p.GetParserRuleContext(), 8)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 8)", ""))
}
{
p.SetState(81)
p.SetState(87)
var _lt = p.GetTokenStream().LT(1)
@ -2434,20 +2507,20 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(82)
p.SetState(88)
p.expr(9)
}
case 8:
localctx = NewEqualityContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(83)
p.SetState(89)
if !(p.Precpred(p.GetParserRuleContext(), 7)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 7)", ""))
}
{
p.SetState(84)
p.SetState(90)
var _lt = p.GetTokenStream().LT(1)
@ -2465,122 +2538,122 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(85)
p.SetState(91)
p.expr(8)
}
case 9:
localctx = NewBitAndContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(86)
p.SetState(92)
if !(p.Precpred(p.GetParserRuleContext(), 6)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 6)", ""))
}
{
p.SetState(87)
p.SetState(93)
p.Match(PlanParserBAND)
}
{
p.SetState(88)
p.SetState(94)
p.expr(7)
}
case 10:
localctx = NewBitXorContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(89)
p.SetState(95)
if !(p.Precpred(p.GetParserRuleContext(), 5)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 5)", ""))
}
{
p.SetState(90)
p.SetState(96)
p.Match(PlanParserBXOR)
}
{
p.SetState(91)
p.SetState(97)
p.expr(6)
}
case 11:
localctx = NewBitOrContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(92)
p.SetState(98)
if !(p.Precpred(p.GetParserRuleContext(), 4)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 4)", ""))
}
{
p.SetState(93)
p.SetState(99)
p.Match(PlanParserBOR)
}
{
p.SetState(94)
p.SetState(100)
p.expr(5)
}
case 12:
localctx = NewLogicalAndContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(95)
p.SetState(101)
if !(p.Precpred(p.GetParserRuleContext(), 3)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 3)", ""))
}
{
p.SetState(96)
p.SetState(102)
p.Match(PlanParserAND)
}
{
p.SetState(97)
p.SetState(103)
p.expr(4)
}
case 13:
localctx = NewLogicalOrContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(98)
p.SetState(104)
if !(p.Precpred(p.GetParserRuleContext(), 2)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 2)", ""))
}
{
p.SetState(99)
p.SetState(105)
p.Match(PlanParserOR)
}
{
p.SetState(100)
p.SetState(106)
p.expr(3)
}
case 14:
localctx = NewLikeContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(101)
p.SetState(107)
if !(p.Precpred(p.GetParserRuleContext(), 22)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 22)", ""))
if !(p.Precpred(p.GetParserRuleContext(), 23)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 23)", ""))
}
{
p.SetState(102)
p.SetState(108)
p.Match(PlanParserLIKE)
}
{
p.SetState(103)
p.SetState(109)
p.Match(PlanParserStringLiteral)
}
case 15:
localctx = NewTermContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(104)
p.SetState(110)
if !(p.Precpred(p.GetParserRuleContext(), 16)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 16)", ""))
}
{
p.SetState(105)
p.SetState(111)
var _lt = p.GetTokenStream().LT(1)
@ -2599,59 +2672,59 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
{
p.SetState(106)
p.SetState(112)
p.Match(PlanParserT__2)
}
{
p.SetState(107)
p.SetState(113)
p.expr(0)
}
p.SetState(112)
p.SetState(118)
p.GetErrorHandler().Sync(p)
_alt = p.GetInterpreter().AdaptivePredict(p.GetTokenStream(), 3, p.GetParserRuleContext())
for _alt != 2 && _alt != antlr.ATNInvalidAltNumber {
if _alt == 1 {
{
p.SetState(108)
p.SetState(114)
p.Match(PlanParserT__3)
}
{
p.SetState(109)
p.SetState(115)
p.expr(0)
}
}
p.SetState(114)
p.SetState(120)
p.GetErrorHandler().Sync(p)
_alt = p.GetInterpreter().AdaptivePredict(p.GetTokenStream(), 3, p.GetParserRuleContext())
}
p.SetState(116)
p.SetState(122)
p.GetErrorHandler().Sync(p)
_la = p.GetTokenStream().LA(1)
if _la == PlanParserT__3 {
{
p.SetState(115)
p.SetState(121)
p.Match(PlanParserT__3)
}
}
{
p.SetState(118)
p.SetState(124)
p.Match(PlanParserT__4)
}
case 16:
localctx = NewEmptyTermContext(p, NewExprContext(p, _parentctx, _parentState))
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
p.SetState(120)
p.SetState(126)
if !(p.Precpred(p.GetParserRuleContext(), 15)) {
panic(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 15)", ""))
}
{
p.SetState(121)
p.SetState(127)
var _lt = p.GetTokenStream().LT(1)
@ -2669,14 +2742,14 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
}
}
{
p.SetState(122)
p.SetState(128)
p.Match(PlanParserEmptyTerm)
}
}
}
p.SetState(127)
p.SetState(133)
p.GetErrorHandler().Sync(p)
_alt = p.GetInterpreter().AdaptivePredict(p.GetTokenStream(), 6, p.GetParserRuleContext())
}
@ -2740,7 +2813,7 @@ func (p *PlanParser) Expr_Sempred(localctx antlr.RuleContext, predIndex int) boo
return p.Precpred(p.GetParserRuleContext(), 2)
case 13:
return p.Precpred(p.GetParserRuleContext(), 22)
return p.Precpred(p.GetParserRuleContext(), 23)
case 14:
return p.Precpred(p.GetParserRuleContext(), 16)

View File

@ -61,6 +61,9 @@ type PlanVisitor interface {
// Visit a parse tree produced by PlanParser#ArrayLength.
VisitArrayLength(ctx *ArrayLengthContext) interface{}
// Visit a parse tree produced by PlanParser#TextMatch.
VisitTextMatch(ctx *TextMatchContext) interface{}
// Visit a parse tree produced by PlanParser#Term.
VisitTerm(ctx *TermContext) interface{}

View File

@ -486,6 +486,34 @@ func (v *ParserVisitor) VisitLike(ctx *parser.LikeContext) interface{} {
}
}
func (v *ParserVisitor) VisitTextMatch(ctx *parser.TextMatchContext) interface{} {
column, err := v.translateIdentifier(ctx.Identifier().GetText())
if err != nil {
return err
}
if !typeutil.IsStringType(column.dataType) {
return fmt.Errorf("text match operation on non-string is unsupported")
}
queryText, err := convertEscapeSingle(ctx.StringLiteral().GetText())
if err != nil {
return err
}
return &ExprWithType{
expr: &planpb.Expr{
Expr: &planpb.Expr_UnaryRangeExpr{
UnaryRangeExpr: &planpb.UnaryRangeExpr{
ColumnInfo: toColumnInfo(column),
Op: planpb.OpType_TextMatch,
Value: NewString(queryText),
},
},
},
dataType: schemapb.DataType_Bool,
}
}
// VisitTerm translates expr to term plan.
func (v *ParserVisitor) VisitTerm(ctx *parser.TermContext) interface{} {
child := ctx.Expr(0).Accept(v)

View File

@ -173,6 +173,27 @@ func TestExpr_Like(t *testing.T) {
//}
}
func TestExpr_TextMatch(t *testing.T) {
schema := newTestSchema()
helper, err := typeutil.CreateSchemaHelper(schema)
assert.NoError(t, err)
exprStrs := []string{
`TextMatch(VarCharField, "query")`,
}
for _, exprStr := range exprStrs {
assertValidExpr(t, helper, exprStr)
}
unsupported := []string{
`TextMatch(not_exist, "query")`,
`TextMatch(BoolField, "query")`,
}
for _, exprStr := range unsupported {
assertInvalidExpr(t, helper, exprStr)
}
}
func TestExpr_BinaryRange(t *testing.T) {
schema := newTestSchema()
helper, err := typeutil.CreateSchemaHelper(schema)

View File

@ -439,6 +439,7 @@ message TextIndexStats {
repeated string files = 3;
int64 log_size = 4;
int64 memory_size = 5;
int64 buildID = 6;
}
message Binlog {

View File

@ -81,3 +81,13 @@ message BuildIndexInfo {
repeated OptionalFieldInfo opt_fields = 19;
bool partition_key_isolation = 20;
}
message LoadTextIndexInfo {
int64 FieldID = 1;
int64 version = 2;
int64 buildID = 3;
repeated string files = 4;
schema.FieldSchema schema = 5;
int64 collectionID = 6;
int64 partitionID = 7;
}

View File

@ -18,6 +18,7 @@ enum OpType {
Range = 10; // for case 1 < a < b
In = 11; // TODO:: used for term expr
NotIn = 12;
TextMatch = 13; // text match
};
enum ArithOpType {

View File

@ -364,6 +364,7 @@ message SegmentLoadInfo {
data.SegmentLevel level = 17;
int64 storageVersion = 18;
bool is_sorted = 19;
map<int64, data.TextIndexStats> textStatsLogs = 20;
}
message FieldIndexInfo {

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/ctokenizer"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/mq/msgstream"
@ -396,6 +397,10 @@ func (t *createCollectionTask) PreExecute(ctx context.Context) error {
if err = ValidateAutoIndexMmapConfig(isVectorType, indexParams); err != nil {
return err
}
if err := ctokenizer.ValidateTextSchema(field); err != nil {
return err
}
}
if err := validateMultipleVectorFields(t.schema); err != nil {

View File

@ -87,6 +87,7 @@ func PackSegmentLoadInfo(segment *datapb.SegmentInfo, channelCheckpoint *msgpb.M
Level: segment.GetLevel(),
StorageVersion: segment.GetStorageVersion(),
IsSorted: segment.GetIsSorted(),
TextStatsLogs: segment.GetTextStatsLogs(),
}
return loadInfo
}

View File

@ -44,6 +44,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/cgopb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexcgopb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/segcorepb"
"github.com/milvus-io/milvus/internal/querycoordv2/params"
@ -1182,6 +1183,38 @@ func (s *LocalSegment) LoadIndex(ctx context.Context, indexInfo *querypb.FieldIn
return nil
}
func (s *LocalSegment) LoadTextIndex(ctx context.Context, textLogs *datapb.TextIndexStats, schemaHelper *typeutil.SchemaHelper) error {
log.Ctx(ctx).Info("load text index", zap.Int64("field id", textLogs.GetFieldID()), zap.Any("text logs", textLogs))
f, err := schemaHelper.GetFieldFromID(textLogs.GetFieldID())
if err != nil {
return err
}
cgoProto := &indexcgopb.LoadTextIndexInfo{
FieldID: textLogs.GetFieldID(),
Version: textLogs.GetVersion(),
BuildID: textLogs.GetBuildID(),
Files: textLogs.GetFiles(),
Schema: f,
CollectionID: s.Collection(),
PartitionID: s.Partition(),
}
marshaled, err := proto.Marshal(cgoProto)
if err != nil {
return err
}
var status C.CStatus
_, _ = GetLoadPool().Submit(func() (any, error) {
status = C.LoadTextIndex(s.ptr, (*C.uint8_t)(unsafe.Pointer(&marshaled[0])), (C.uint64_t)(len(marshaled)))
return nil, nil
}).Await()
return HandleCStatus(ctx, &status, "LoadTextIndex failed")
}
func (s *LocalSegment) UpdateIndexInfo(ctx context.Context, indexInfo *querypb.FieldIndexInfo, info *LoadIndexInfo) error {
log := log.Ctx(ctx).With(
zap.Int64("collectionID", s.Collection()),
@ -1295,6 +1328,24 @@ func (s *LocalSegment) UpdateFieldRawDataSize(ctx context.Context, numRows int64
return nil
}
func (s *LocalSegment) CreateTextIndex(ctx context.Context, fieldID int64) error {
var status C.CStatus
log.Ctx(ctx).Info("create text index for segment", zap.Int64("segmentID", s.ID()), zap.Int64("fieldID", fieldID))
GetDynamicPool().Submit(func() (any, error) {
status = C.CreateTextIndex(s.ptr, C.int64_t(fieldID))
return nil, nil
}).Await()
if err := HandleCStatus(ctx, &status, "CreateTextIndex failed"); err != nil {
return err
}
log.Ctx(ctx).Info("create text index for segment done", zap.Int64("segmentID", s.ID()), zap.Int64("fieldID", fieldID))
return nil
}
type ReleaseScope int
const (

View File

@ -611,6 +611,59 @@ func separateIndexAndBinlog(loadInfo *querypb.SegmentLoadInfo) (map[int64]*Index
return indexedFieldInfos, fieldBinlogs
}
func separateLoadInfoV2(loadInfo *querypb.SegmentLoadInfo, schema *schemapb.CollectionSchema) (
map[int64]*IndexedFieldInfo, // indexed info
[]*datapb.FieldBinlog, // fields info
map[int64]*datapb.TextIndexStats, // text indexed info
map[int64]struct{}, // unindexed text fields
) {
fieldID2IndexInfo := make(map[int64]*querypb.FieldIndexInfo)
for _, indexInfo := range loadInfo.IndexInfos {
if len(indexInfo.GetIndexFilePaths()) > 0 {
fieldID := indexInfo.FieldID
fieldID2IndexInfo[fieldID] = indexInfo
}
}
indexedFieldInfos := make(map[int64]*IndexedFieldInfo)
fieldBinlogs := make([]*datapb.FieldBinlog, 0, len(loadInfo.BinlogPaths))
for _, fieldBinlog := range loadInfo.BinlogPaths {
fieldID := fieldBinlog.FieldID
// check num rows of data meta and index meta are consistent
if indexInfo, ok := fieldID2IndexInfo[fieldID]; ok {
fieldInfo := &IndexedFieldInfo{
FieldBinlog: fieldBinlog,
IndexInfo: indexInfo,
}
indexedFieldInfos[fieldID] = fieldInfo
} else {
fieldBinlogs = append(fieldBinlogs, fieldBinlog)
}
}
textIndexedInfo := make(map[int64]*datapb.TextIndexStats, len(loadInfo.GetTextStatsLogs()))
for _, fieldStatsLog := range loadInfo.GetTextStatsLogs() {
textLog, ok := textIndexedInfo[fieldStatsLog.FieldID]
if !ok {
textIndexedInfo[fieldStatsLog.FieldID] = fieldStatsLog
} else if fieldStatsLog.GetVersion() > textLog.GetVersion() {
textIndexedInfo[fieldStatsLog.FieldID] = fieldStatsLog
}
}
unindexedTextFields := make(map[int64]struct{})
for _, field := range schema.GetFields() {
h := typeutil.CreateFieldSchemaHelper(field)
_, textIndexExist := textIndexedInfo[field.GetFieldID()]
if h.EnableMatch() && !textIndexExist {
unindexedTextFields[field.GetFieldID()] = struct{}{}
}
}
return indexedFieldInfos, fieldBinlogs, textIndexedInfo, unindexedTextFields
}
func (loader *segmentLoader) loadSealedSegment(ctx context.Context, loadInfo *querypb.SegmentLoadInfo, segment *LocalSegment) (err error) {
// TODO: we should create a transaction-like api to load segment for segment interface,
// but not do many things in segment loader.
@ -631,9 +684,8 @@ func (loader *segmentLoader) loadSealedSegment(ctx context.Context, loadInfo *qu
}()
collection := segment.GetCollection()
indexedFieldInfos, fieldBinlogs := separateIndexAndBinlog(loadInfo)
schemaHelper, _ := typeutil.CreateSchemaHelper(collection.Schema())
indexedFieldInfos, fieldBinlogs, textIndexes, unindexedTextFields := separateLoadInfoV2(loadInfo, collection.Schema())
if err := segment.AddFieldDataInfo(ctx, loadInfo.GetNumOfRows(), loadInfo.GetBinlogPaths()); err != nil {
return err
}
@ -642,6 +694,8 @@ func (loader *segmentLoader) loadSealedSegment(ctx context.Context, loadInfo *qu
tr := timerecord.NewTimeRecorder("segmentLoader.loadSealedSegment")
log.Info("Start loading fields...",
zap.Int64s("indexedFields", lo.Keys(indexedFieldInfos)),
zap.Int64s("indexed text fields", lo.Keys(textIndexes)),
zap.Int64s("unindexed text fields", lo.Keys(unindexedTextFields)),
)
if err := loader.loadFieldsIndex(ctx, schemaHelper, segment, loadInfo.GetNumOfRows(), indexedFieldInfos); err != nil {
return err
@ -673,6 +727,21 @@ func (loader *segmentLoader) loadSealedSegment(ctx context.Context, loadInfo *qu
}
loadRawDataSpan := tr.RecordSpan()
// load text indexes.
for _, info := range textIndexes {
if err := segment.LoadTextIndex(ctx, info, schemaHelper); err != nil {
return err
}
}
loadTextIndexesSpan := tr.RecordSpan()
// create index for unindexed text fields.
for fieldID := range unindexedTextFields {
if err := segment.CreateTextIndex(ctx, fieldID); err != nil {
return err
}
}
// 4. rectify entries number for binlog in very rare cases
// https://github.com/milvus-io/milvus/23654
// legacy entry num = 0
@ -685,6 +754,7 @@ func (loader *segmentLoader) loadSealedSegment(ctx context.Context, loadInfo *qu
zap.Duration("complementScalarDataSpan", complementScalarDataSpan),
zap.Duration("loadRawDataSpan", loadRawDataSpan),
zap.Duration("patchEntryNumberSpan", patchEntryNumberSpan),
zap.Duration("loadTextIndexesSpan", loadTextIndexesSpan),
)
return nil
}

View File

@ -0,0 +1,33 @@
package ctokenizer
/*
#cgo pkg-config: milvus_core
#include <stdlib.h> // free
#include "segcore/tokenizer_c.h"
*/
import "C"
import (
"fmt"
"unsafe"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
func ValidateTextSchema(fieldSchema *schemapb.FieldSchema) error {
h := typeutil.CreateFieldSchemaHelper(fieldSchema)
if !h.EnableMatch() {
return nil
}
bs, err := proto.Marshal(fieldSchema)
if err != nil {
return fmt.Errorf("failed to marshal field schema: %w", err)
}
status := C.validate_text_schema((*C.uint8_t)(unsafe.Pointer(&bs[0])), (C.uint64_t)(len(bs)))
return HandleCStatus(&status, "failed to validate text schema")
}

View File

@ -0,0 +1,79 @@
package ctokenizer
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
)
func TestValidateTextSchema(t *testing.T) {
type args struct {
fieldSchema *schemapb.FieldSchema
}
tests := []struct {
name string
args args
errIsNil bool
}{
{
args: args{
fieldSchema: &schemapb.FieldSchema{
FieldID: 101,
TypeParams: []*commonpb.KeyValuePair{},
},
},
errIsNil: true,
},
{
// default
args: args{
fieldSchema: &schemapb.FieldSchema{
FieldID: 101,
TypeParams: []*commonpb.KeyValuePair{
{Key: "enable_match", Value: "true"},
},
},
},
errIsNil: true,
},
{
// default
args: args{
fieldSchema: &schemapb.FieldSchema{
FieldID: 101,
TypeParams: []*commonpb.KeyValuePair{
{Key: "enable_match", Value: "true"},
{Key: "analyzer_params", Value: `{"tokenizer": "default"}`},
},
},
},
errIsNil: true,
},
{
// jieba
args: args{
fieldSchema: &schemapb.FieldSchema{
FieldID: 101,
TypeParams: []*commonpb.KeyValuePair{
{Key: "enable_match", Value: "true"},
{Key: "analyzer_params", Value: `{"tokenizer": "jieba"}`},
},
},
},
errIsNil: true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
err := ValidateTextSchema(tt.args.fieldSchema)
if tt.errIsNil {
assert.Nil(t, err)
} else {
assert.NotNil(t, err)
}
})
}
}

View File

@ -126,6 +126,43 @@ func CreateIndex(ctx context.Context, buildIndexInfo *indexcgopb.BuildIndexInfo)
return index, nil
}
func CreateTextIndex(ctx context.Context, buildIndexInfo *indexcgopb.BuildIndexInfo) (map[string]int64, error) {
buildIndexInfoBlob, err := proto.Marshal(buildIndexInfo)
if err != nil {
log.Ctx(ctx).Warn("marshal buildIndexInfo failed",
zap.String("clusterID", buildIndexInfo.GetClusterID()),
zap.Int64("buildID", buildIndexInfo.GetBuildID()),
zap.Error(err))
return nil, err
}
var cBinarySet C.CBinarySet
status := C.BuildTextIndex(&cBinarySet, (*C.uint8_t)(unsafe.Pointer(&buildIndexInfoBlob[0])), (C.uint64_t)(len(buildIndexInfoBlob)))
if err := HandleCStatus(&status, "failed to build text index"); err != nil {
return nil, err
}
defer func() {
if cBinarySet != nil {
C.DeleteBinarySet(cBinarySet)
}
}()
res := make(map[string]int64)
indexFilePaths, err := GetBinarySetKeys(cBinarySet)
if err != nil {
return nil, err
}
for _, path := range indexFilePaths {
size, err := GetBinarySetSize(cBinarySet, path)
if err != nil {
return nil, err
}
res[path] = size
}
return res, nil
}
// TODO: this seems to be used only for test. We should mark the method
// name with ForTest, or maybe move to test file.
func (index *CgoIndex) Build(dataset *Dataset) error {

View File

@ -186,6 +186,7 @@ func InitMmapManager(params *paramtable.ComponentParam) error {
disk_limit: C.uint64_t(diskLimit),
fix_file_size: C.uint64_t(mmapFileSize),
growing_enable_mmap: C.bool(params.QueryNodeCfg.GrowingMmapEnabled.GetAsBool()),
enable_mmap: C.bool(params.QueryNodeCfg.MmapEnabled.GetAsBool()),
}
status := C.InitMmapManager(mmapConfig)
return HandleCStatus(&status, "InitMmapManager failed")

View File

@ -0,0 +1,62 @@
package typeutil
import (
"fmt"
"strconv"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/common"
)
type FieldSchemaHelper struct {
schema *schemapb.FieldSchema
typeParams *kvPairsHelper[string, string]
indexParams *kvPairsHelper[string, string]
}
func (h *FieldSchemaHelper) GetDim() (int64, error) {
if !IsVectorType(h.schema.GetDataType()) {
return 0, fmt.Errorf("%s is not of vector type", h.schema.GetDataType())
}
if IsSparseFloatVectorType(h.schema.GetDataType()) {
return 0, fmt.Errorf("typeutil.GetDim should not invoke on sparse vector type")
}
getDim := func(kvPairs *kvPairsHelper[string, string]) (int64, error) {
dimStr, err := kvPairs.Get(common.DimKey)
if err != nil {
return 0, fmt.Errorf("dim not found")
}
dim, err := strconv.Atoi(dimStr)
if err != nil {
return 0, fmt.Errorf("invalid dimension: %s", dimStr)
}
return int64(dim), nil
}
if dim, err := getDim(h.typeParams); err == nil {
return dim, nil
}
return getDim(h.indexParams)
}
func (h *FieldSchemaHelper) EnableMatch() bool {
if !IsStringType(h.schema.GetDataType()) {
return false
}
s, err := h.typeParams.Get("enable_match")
if err != nil {
return false
}
enable, err := strconv.ParseBool(s)
return err == nil && enable
}
func CreateFieldSchemaHelper(schema *schemapb.FieldSchema) *FieldSchemaHelper {
return &FieldSchemaHelper{
schema: schema,
typeParams: NewKvPairs(schema.GetTypeParams()),
indexParams: NewKvPairs(schema.GetIndexParams()),
}
}

View File

@ -1,29 +1,10 @@
package typeutil
import (
"fmt"
"strconv"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/common"
)
// GetDim get dimension of field. Maybe also helpful outside.
func GetDim(field *schemapb.FieldSchema) (int64, error) {
if !IsVectorType(field.GetDataType()) {
return 0, fmt.Errorf("%s is not of vector type", field.GetDataType())
}
if IsSparseFloatVectorType(field.GetDataType()) {
return 0, fmt.Errorf("typeutil.GetDim should not invoke on sparse vector type")
}
h := NewKvPairs(append(field.GetIndexParams(), field.GetTypeParams()...))
dimStr, err := h.Get(common.DimKey)
if err != nil {
return 0, fmt.Errorf("dim not found")
}
dim, err := strconv.Atoi(dimStr)
if err != nil {
return 0, fmt.Errorf("invalid dimension: %s", dimStr)
}
return int64(dim), nil
return CreateFieldSchemaHelper(field).GetDim()
}

View File

@ -16,7 +16,9 @@
package typeutil
import "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
)
// CompareIndexParams compares indexParam1 with indexParam2. When all keys of indexParam1 exist in indexParam2,
// and the corresponding value are the same as in indexParam2, return true

View File

@ -18,6 +18,10 @@ func (h *kvPairsHelper[K, V]) Get(k K) (V, error) {
return v, nil
}
func (h *kvPairsHelper[K, V]) GetAll() map[K]V {
return h.kvPairs
}
func NewKvPairs(pairs []*commonpb.KeyValuePair) *kvPairsHelper[string, string] {
helper := &kvPairsHelper[string, string]{
kvPairs: make(map[string]string),