[feature](function)add search functions: multi_search_all_positions & multi_match_any (#13763)

Co-authored-by: yiliang qiu <yiliang.qiu@qq.com>
This commit is contained in:
Yiliang Qiu
2022-11-07 11:50:55 +08:00
committed by GitHub
parent 7ffe88b579
commit e8d2fb6778
14 changed files with 1143 additions and 0 deletions

View File

@ -211,6 +211,8 @@ set(VEC_FILES
functions/least_greast.cpp
functions/function_fake.cpp
functions/url/function_url.cpp
functions/functions_multi_string_position.cpp
functions/functions_multi_string_search.cpp
olap/vgeneric_iterators.cpp
olap/vcollect_iterator.cpp
olap/block_reader.cpp

View File

@ -0,0 +1,236 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
// This file is copied from
// https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsMultiStringPosition.h
// and modified by Doris
#include "function.h"
#include "function_helpers.h"
#include "vec/columns/column_array.h"
#include "vec/columns/column_fixed_length_object.h"
#include "vec/columns/column_string.h"
#include "vec/columns/column_vector.h"
#include "vec/common/pod_array.h"
#include "vec/common/volnitsky.h"
#include "vec/data_types/data_type_array.h"
#include "vec/data_types/data_type_number.h"
#include "vec/data_types/data_type_string.h"
#include "vec/functions/simple_function_factory.h"
namespace doris::vectorized {
template <typename Impl>
class FunctionMultiStringPosition : public IFunction {
public:
static constexpr auto name = Impl::name;
static FunctionPtr create() { return std::make_shared<FunctionMultiStringPosition>(); }
String get_name() const override { return name; }
size_t get_number_of_arguments() const override { return 2; }
bool use_default_implementation_for_constants() const override { return true; }
DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
return std::make_shared<DataTypeArray>(make_nullable(std::make_shared<DataTypeInt32>()));
}
Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
size_t result, size_t input_rows_count) override {
ColumnPtr haystack_ptr = block.get_by_position(arguments[0]).column;
ColumnPtr needles_ptr = block.get_by_position(arguments[1]).column;
const ColumnString* col_haystack_vector =
check_and_get_column<ColumnString>(&*haystack_ptr);
const ColumnConst* col_haystack_const =
check_and_get_column_const<ColumnString>(&*haystack_ptr);
const ColumnArray* col_needles_vector =
check_and_get_column<ColumnArray>(needles_ptr.get());
const ColumnConst* col_needles_const =
check_and_get_column_const<ColumnArray>(needles_ptr.get());
if (col_haystack_const && col_needles_vector)
return Status::InvalidArgument(
"function '{}' doesn't support search with non-constant needles "
"in constant haystack",
name);
using ResultType = typename Impl::ResultType;
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create();
auto& vec_res = col_res->get_data();
auto& offsets_res = col_offsets->get_data();
Status status;
if (col_needles_const)
status = Impl::vector_constant(
col_haystack_vector->get_chars(), col_haystack_vector->get_offsets(),
col_needles_const->get_value<Array>(), vec_res, offsets_res);
else
status = Impl::vector_vector(col_haystack_vector->get_chars(),
col_haystack_vector->get_offsets(),
col_needles_vector->get_data(),
col_needles_vector->get_offsets(), vec_res, offsets_res);
if (!status.ok()) return status;
auto nullable_col =
ColumnNullable::create(std::move(col_res), ColumnUInt8::create(col_res->size(), 0));
block.get_by_position(result).column =
ColumnArray::create(std::move(nullable_col), std::move(col_offsets));
return status;
}
};
template <typename Impl>
struct FunctionMultiSearchAllPositionsImpl {
using ResultType = Int32;
static constexpr auto name = "multi_search_all_positions";
static Status vector_constant(const ColumnString::Chars& haystack_data,
const ColumnString::Offsets& haystack_offsets,
const Array& needles_arr, PaddedPODArray<Int32>& vec_res,
PaddedPODArray<UInt64>& offsets_res) {
if (needles_arr.size() > std::numeric_limits<UInt8>::max())
return Status::InvalidArgument(
"number of arguments for function {} doesn't match: "
"passed {}, should be at most 255",
name, needles_arr.size());
std::vector<StringRef> needles;
needles.reserve(needles_arr.size());
for (const auto& needle : needles_arr) needles.emplace_back(needle.get<StringRef>());
auto res_callback = [](const UInt8* start, const UInt8* end) -> Int32 {
return 1 + Impl::count_chars(reinterpret_cast<const char*>(start),
reinterpret_cast<const char*>(end));
};
auto searcher = Impl::create_multi_searcher(needles);
const size_t haystack_size = haystack_offsets.size();
const size_t needles_size = needles.size();
vec_res.resize(haystack_size * needles.size());
offsets_res.resize(haystack_size);
std::fill(vec_res.begin(), vec_res.end(), 0);
while (searcher.hasMoreToSearch()) {
size_t prev_haystack_offset = 0;
for (size_t j = 0, from = 0; j < haystack_size; ++j, from += needles_size) {
const auto* haystack = &haystack_data[prev_haystack_offset];
const auto* haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset;
searcher.searchOneAll(haystack, haystack_end, &vec_res[from], res_callback);
prev_haystack_offset = haystack_offsets[j];
}
}
size_t accum = needles_size;
for (size_t i = 0; i < haystack_size; ++i) {
offsets_res[i] = accum;
accum += needles_size;
}
return Status::OK();
}
static Status vector_vector(const ColumnString::Chars& haystack_data,
const ColumnString::Offsets& haystack_offsets,
const IColumn& needles_data,
const ColumnArray::Offsets64& needles_offsets,
PaddedPODArray<Int32>& vec_res,
PaddedPODArray<UInt64>& offsets_res) {
size_t prev_haystack_offset = 0;
size_t prev_needles_offset = 0;
auto res_callback = [](const UInt8* start, const UInt8* end) -> Int32 {
return 1 + Impl::count_chars(reinterpret_cast<const char*>(start),
reinterpret_cast<const char*>(end));
};
offsets_res.reserve(haystack_offsets.size());
auto& nested_column =
vectorized::check_and_get_column<vectorized::ColumnNullable>(needles_data)
->get_nested_column();
const ColumnString* needles_data_string = check_and_get_column<ColumnString>(nested_column);
std::vector<StringRef> needles;
for (size_t i = 0; i < haystack_offsets.size(); ++i) {
needles.reserve(needles_offsets[i] - prev_needles_offset);
for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) {
needles.emplace_back(needles_data_string->get_data_at(j));
}
const size_t needles_size = needles.size();
if (needles_size > std::numeric_limits<UInt8>::max())
return Status::InvalidArgument(
"number of arguments for function {} doesn't match: "
"passed {}, should be at most 255",
name, needles_size);
vec_res.resize(vec_res.size() + needles_size);
auto searcher = Impl::create_multi_searcher(needles);
std::fill(vec_res.begin() + vec_res.size() - needles_size, vec_res.end(), 0);
while (searcher.hasMoreToSearch()) {
const auto* haystack = &haystack_data[prev_haystack_offset];
const auto* haystack_end = haystack + haystack_offsets[i] - prev_haystack_offset;
searcher.searchOneAll(haystack, haystack_end,
&vec_res[vec_res.size() - needles_size], res_callback);
}
if (offsets_res.empty())
offsets_res.push_back(needles_size);
else
offsets_res.push_back(offsets_res.back() + needles_size);
prev_haystack_offset = haystack_offsets[i];
prev_needles_offset = needles_offsets[i];
needles.clear();
}
return Status::OK();
}
};
struct MultiSearcherImpl {
using MultiSearcher = MultiVolnitsky;
static MultiSearcher create_multi_searcher(const std::vector<StringRef>& needles) {
return MultiSearcher(needles);
}
static size_t count_chars(const char* begin, const char* end) { return end - begin; }
};
using FunctionMultiSearchAllPositions =
FunctionMultiStringPosition<FunctionMultiSearchAllPositionsImpl<MultiSearcherImpl>>;
void register_function_multi_string_position(SimpleFunctionFactory& factory) {
factory.register_function<FunctionMultiSearchAllPositions>();
}
} // namespace doris::vectorized

View File

@ -0,0 +1,283 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
// This file is copied from
// https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsMultiStringSearch.h
// and modified by Doris
#include <hs/hs.h>
#include "function.h"
#include "function_helpers.h"
#include "regexps.h"
#include "vec/columns/column_array.h"
#include "vec/columns/column_const.h"
#include "vec/columns/column_fixed_length_object.h"
#include "vec/columns/column_string.h"
#include "vec/columns/column_vector.h"
#include "vec/common/pod_array.h"
#include "vec/data_types/data_type_array.h"
#include "vec/data_types/data_type_number.h"
#include "vec/data_types/data_type_string.h"
#include "vec/functions/simple_function_factory.h"
namespace doris::vectorized {
template <typename Impl>
class FunctionsMultiStringSearch : public IFunction {
public:
static constexpr auto name = Impl::name;
static FunctionPtr create() { return std::make_shared<FunctionsMultiStringSearch>(); }
String get_name() const override { return name; }
size_t get_number_of_arguments() const override { return 2; }
bool use_default_implementation_for_constants() const override { return true; }
DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
return Impl::get_return_type();
}
Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
size_t result, size_t input_rows_count) override {
ColumnPtr haystack_ptr = block.get_by_position(arguments[0]).column;
ColumnPtr needles_ptr = block.get_by_position(arguments[1]).column;
const ColumnString* col_haystack_vector =
check_and_get_column<ColumnString>(&*haystack_ptr);
const ColumnConst* col_haystack_const =
check_and_get_column_const<ColumnString>(&*haystack_ptr);
const ColumnArray* col_needles_vector =
check_and_get_column<ColumnArray>(needles_ptr.get());
const ColumnConst* col_needles_const =
check_and_get_column_const<ColumnArray>(needles_ptr.get());
if (col_haystack_const && col_needles_vector)
return Status::InvalidArgument(
"function '{}' doesn't support search with non-constant needles "
"in constant haystack",
name);
using ResultType = typename Impl::ResultType;
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create();
auto& vec_res = col_res->get_data();
auto& offsets_res = col_offsets->get_data();
Status status;
if (col_needles_const)
status = Impl::vector_constant(
col_haystack_vector->get_chars(), col_haystack_vector->get_offsets(),
col_needles_const->get_value<Array>(), vec_res, offsets_res, allow_hyperscan_,
max_hyperscan_regexp_length_, max_hyperscan_regexp_total_length_);
else
status = Impl::vector_vector(
col_haystack_vector->get_chars(), col_haystack_vector->get_offsets(),
col_needles_vector->get_data(), col_needles_vector->get_offsets(), vec_res,
offsets_res, allow_hyperscan_, max_hyperscan_regexp_length_,
max_hyperscan_regexp_total_length_);
if (!status.ok()) return status;
if constexpr (Impl::is_column_array)
block.get_by_position(result).column =
ColumnArray::create(std::move(col_res), std::move(col_offsets));
else
block.replace_by_position(result, std::move(col_res));
return status;
}
private:
const bool allow_hyperscan_ = true;
const size_t max_hyperscan_regexp_length_ = 0; // not limited
const size_t max_hyperscan_regexp_total_length_ = 0; // not limited
};
/// For more readable instantiations of MultiMatchAnyImpl<>
struct MultiMatchTraits {
enum class Find { Any, AnyIndex };
};
template <typename ResultType_, MultiMatchTraits::Find Find, bool WithEditDistance>
struct FunctionMultiMatchAnyImpl {
using ResultType = ResultType_;
static constexpr bool FindAny = (Find == MultiMatchTraits::Find::Any);
static constexpr bool FindAnyIndex = (Find == MultiMatchTraits::Find::AnyIndex);
static constexpr auto name = "multi_match_any";
static constexpr bool is_column_array = false;
static auto get_return_type() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static Status vector_constant(const ColumnString::Chars& haystack_data,
const ColumnString::Offsets& haystack_offsets,
const Array& needles_arr, PaddedPODArray<ResultType>& res,
PaddedPODArray<UInt64>& offsets, bool allow_hyperscan,
size_t max_hyperscan_regexp_length,
size_t max_hyperscan_regexp_total_length) {
if (!allow_hyperscan) return Status::InvalidArgument("Hyperscan functions are disabled");
std::vector<StringRef> needles;
needles.reserve(needles_arr.size());
for (const auto& needle : needles_arr) needles.emplace_back(needle.get<StringRef>());
res.resize(haystack_offsets.size());
if (needles_arr.empty()) {
std::fill(res.begin(), res.end(), 0);
return Status::OK();
}
multiregexps::DeferredConstructedRegexpsPtr deferred_constructed_regexps =
multiregexps::getOrSet</*SaveIndices*/ FindAnyIndex, WithEditDistance>(
needles, std::nullopt);
multiregexps::Regexps* regexps = deferred_constructed_regexps->get();
hs_scratch_t* scratch = nullptr;
hs_error_t err = hs_clone_scratch(regexps->getScratch(), &scratch);
if (err != HS_SUCCESS)
return Status::InternalError("could not clone scratch space for vectorscan");
multiregexps::ScratchPtr smart_scratch(scratch);
auto on_match = []([[maybe_unused]] unsigned int id,
unsigned long long /* from */, // NOLINT
unsigned long long /* to */, // NOLINT
unsigned int /* flags */, void* context) -> int {
if constexpr (FindAnyIndex)
*reinterpret_cast<ResultType*>(context) = id;
else if constexpr (FindAny)
*reinterpret_cast<ResultType*>(context) = 1;
/// Once we hit the callback, there is no need to search for others.
return 1;
};
const size_t haystack_offsets_size = haystack_offsets.size();
UInt64 offset = 0;
for (size_t i = 0; i < haystack_offsets_size; ++i) {
UInt64 length = haystack_offsets[i] - offset;
/// vectorscan restriction.
if (length > std::numeric_limits<UInt32>::max())
return Status::InternalError("too long string to search");
/// zero the result, scan, check, update the offset.
res[i] = 0;
err = hs_scan(regexps->getDB(),
reinterpret_cast<const char*>(haystack_data.data()) + offset,
static_cast<unsigned>(length), 0, smart_scratch.get(), on_match, &res[i]);
if (err != HS_SUCCESS && err != HS_SCAN_TERMINATED)
return Status::InternalError("failed to scan with vectorscan");
offset = haystack_offsets[i];
}
return Status::OK();
}
static Status vector_vector(const ColumnString::Chars& haystack_data,
const ColumnString::Offsets& haystack_offsets,
const IColumn& needles_data,
const ColumnArray::Offsets64& needles_offsets,
PaddedPODArray<ResultType>& res, PaddedPODArray<UInt64>& offsets,
bool allow_hyperscan, size_t max_hyperscan_regexp_length,
size_t max_hyperscan_regexp_total_length) {
if (!allow_hyperscan) return Status::InvalidArgument("Hyperscan functions are disabled");
res.resize(haystack_offsets.size());
size_t prev_haystack_offset = 0;
size_t prev_needles_offset = 0;
auto& nested_column =
vectorized::check_and_get_column<vectorized::ColumnNullable>(needles_data)
->get_nested_column();
const ColumnString* needles_data_string = check_and_get_column<ColumnString>(nested_column);
std::vector<StringRef> needles;
for (size_t i = 0; i < haystack_offsets.size(); ++i) {
needles.reserve(needles_offsets[i] - prev_needles_offset);
for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) {
needles.emplace_back(needles_data_string->get_data_at(j));
}
if (needles.empty()) {
res[i] = 0;
prev_haystack_offset = haystack_offsets[i];
prev_needles_offset = needles_offsets[i];
continue;
}
multiregexps::DeferredConstructedRegexpsPtr deferred_constructed_regexps =
multiregexps::getOrSet</*SaveIndices*/ FindAnyIndex, WithEditDistance>(
needles, std::nullopt);
multiregexps::Regexps* regexps = deferred_constructed_regexps->get();
hs_scratch_t* scratch = nullptr;
hs_error_t err = hs_clone_scratch(regexps->getScratch(), &scratch);
if (err != HS_SUCCESS)
return Status::InternalError("could not clone scratch space for vectorscan");
multiregexps::ScratchPtr smart_scratch(scratch);
auto on_match = []([[maybe_unused]] unsigned int id,
unsigned long long /* from */, // NOLINT
unsigned long long /* to */, // NOLINT
unsigned int /* flags */, void* context) -> int {
if constexpr (FindAnyIndex)
*reinterpret_cast<ResultType*>(context) = id;
else if constexpr (FindAny)
*reinterpret_cast<ResultType*>(context) = 1;
/// Once we hit the callback, there is no need to search for others.
return 1;
};
const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset;
/// vectorscan restriction.
if (cur_haystack_length > std::numeric_limits<UInt32>::max())
return Status::InternalError("too long string to search");
/// zero the result, scan, check, update the offset.
res[i] = 0;
err = hs_scan(
regexps->getDB(),
reinterpret_cast<const char*>(haystack_data.data()) + prev_haystack_offset,
static_cast<unsigned>(cur_haystack_length), 0, smart_scratch.get(), on_match,
&res[i]);
if (err != HS_SUCCESS && err != HS_SCAN_TERMINATED)
return Status::InternalError("failed to scan with vectorscan");
prev_haystack_offset = haystack_offsets[i];
prev_needles_offset = needles_offsets[i];
needles.clear();
}
return Status::OK();
}
};
using FunctionMultiMatchAny = FunctionsMultiStringSearch<
FunctionMultiMatchAnyImpl<Int8, MultiMatchTraits::Find::Any, /*WithEditDistance*/ false>>;
void register_function_multi_string_search(SimpleFunctionFactory& factory) {
factory.register_function<FunctionMultiMatchAny>();
}
} // namespace doris::vectorized

View File

@ -0,0 +1,256 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
// This file is copied from
// https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/Regexps.h
// and modified by Doris
#pragma once
#include <hs/hs.h>
#include <boost/container_hash/hash.hpp>
#include <map>
#include <memory>
#include <mutex>
#include <optional>
#include <string>
#include <utility>
#include <vector>
#include "vec/common/string_ref.h"
#include "vec/data_types/data_type_number.h"
#include "vec/data_types/data_type_string.h"
namespace doris::vectorized {
namespace multiregexps {
template <typename Deleter, Deleter deleter>
struct HyperscanDeleter {
template <typename T>
void operator()(T* ptr) const {
deleter(ptr);
}
};
/// Helper unique pointers to correctly delete the allocated space when hyperscan cannot compile something and we throw an exception.
using CompilerError =
std::unique_ptr<hs_compile_error_t,
HyperscanDeleter<decltype(&hs_free_compile_error), &hs_free_compile_error>>;
using ScratchPtr = std::unique_ptr<hs_scratch_t,
HyperscanDeleter<decltype(&hs_free_scratch), &hs_free_scratch>>;
using DataBasePtr =
std::unique_ptr<hs_database_t,
HyperscanDeleter<decltype(&hs_free_database), &hs_free_database>>;
/// Database is thread safe across multiple threads and Scratch is not but we can copy it whenever we use it in the searcher.
class Regexps {
public:
Regexps(hs_database_t* db_, hs_scratch_t* scratch_) : db {db_}, scratch {scratch_} {}
hs_database_t* getDB() const { return db.get(); }
hs_scratch_t* getScratch() const { return scratch.get(); }
private:
DataBasePtr db;
ScratchPtr scratch;
};
class DeferredConstructedRegexps {
public:
explicit DeferredConstructedRegexps(std::function<Regexps()> constructor_)
: constructor(std::move(constructor_)) {}
Regexps* get() {
std::lock_guard lock(mutex);
if (regexps) return &*regexps;
regexps = constructor();
return &*regexps;
}
private:
std::mutex mutex;
std::function<Regexps()> constructor;
std::optional<Regexps> regexps;
};
using DeferredConstructedRegexpsPtr = std::shared_ptr<DeferredConstructedRegexps>;
template <bool save_indices, bool WithEditDistance>
inline Regexps constructRegexps(const std::vector<String>& str_patterns,
[[maybe_unused]] std::optional<UInt32> edit_distance) {
/// Common pointers
std::vector<const char*> patterns;
std::vector<unsigned int> flags;
/// Pointer for external edit distance compilation
std::vector<hs_expr_ext> ext_exprs;
std::vector<const hs_expr_ext*> ext_exprs_ptrs;
patterns.reserve(str_patterns.size());
flags.reserve(str_patterns.size());
if constexpr (WithEditDistance) {
ext_exprs.reserve(str_patterns.size());
ext_exprs_ptrs.reserve(str_patterns.size());
}
for (const auto& ref : str_patterns) {
patterns.push_back(ref.data());
/* Flags below are the pattern matching flags.
* HS_FLAG_DOTALL is a compile flag where matching a . will not exclude newlines. This is a good
* performance practice according to Hyperscan API. https://intel.github.io/hyperscan/dev-reference/performance.html#dot-all-mode
* HS_FLAG_ALLOWEMPTY is a compile flag where empty strings are allowed to match.
* HS_FLAG_UTF8 is a flag where UTF8 literals are matched.
* HS_FLAG_SINGLEMATCH is a compile flag where each pattern match will be returned only once. it is a good performance practice
* as it is said in the Hyperscan documentation. https://intel.github.io/hyperscan/dev-reference/performance.html#single-match-flag
*/
flags.push_back(HS_FLAG_DOTALL | HS_FLAG_SINGLEMATCH | HS_FLAG_ALLOWEMPTY | HS_FLAG_UTF8);
if constexpr (WithEditDistance) {
/// Hyperscan currently does not support UTF8 matching with edit distance.
flags.back() &= ~HS_FLAG_UTF8;
ext_exprs.emplace_back();
/// HS_EXT_FLAG_EDIT_DISTANCE is a compile flag responsible for Levenstein distance.
ext_exprs.back().flags = HS_EXT_FLAG_EDIT_DISTANCE;
ext_exprs.back().edit_distance = edit_distance.value();
ext_exprs_ptrs.push_back(&ext_exprs.back());
}
}
hs_database_t* db = nullptr;
hs_compile_error_t* compile_error;
std::unique_ptr<unsigned int[]> ids;
/// We mark the patterns to provide the callback results.
if constexpr (save_indices) {
ids.reset(new unsigned int[patterns.size()]);
for (size_t i = 0; i < patterns.size(); ++i) ids[i] = static_cast<unsigned>(i + 1);
}
for (auto& pattern : patterns) {
LOG(INFO) << "pattern: " << pattern << "\n";
}
hs_error_t err;
if constexpr (!WithEditDistance)
err = hs_compile_multi(patterns.data(), flags.data(), ids.get(),
static_cast<unsigned>(patterns.size()), HS_MODE_BLOCK, nullptr, &db,
&compile_error);
else
err = hs_compile_ext_multi(patterns.data(), flags.data(), ids.get(), ext_exprs_ptrs.data(),
static_cast<unsigned>(patterns.size()), HS_MODE_BLOCK, nullptr,
&db, &compile_error);
if (err != HS_SUCCESS) {
/// CompilerError is a unique_ptr, so correct memory free after the exception is thrown.
CompilerError error(compile_error);
if (error->expression < 0)
LOG(FATAL) << "Logical error: " + String(error->message);
else
LOG(FATAL) << "Bad arguments: Pattern " + str_patterns[error->expression] +
"failed with error " + String(error->message);
}
/// We allocate the scratch space only once, then copy it across multiple threads with hs_clone_scratch
/// function which is faster than allocating scratch space each time in each thread.
hs_scratch_t* scratch = nullptr;
err = hs_alloc_scratch(db, &scratch);
/// If not HS_SUCCESS, it is guaranteed that the memory would not be allocated for scratch.
if (err != HS_SUCCESS) LOG(FATAL) << "Could not allocate scratch space for hyperscan";
return {db, scratch};
}
/// Maps string pattern vectors + edit distance to compiled vectorscan regexps. Uses the same eviction mechanism as the LocalCacheTable for
/// re2 patterns. Because vectorscan regexes are overall more heavy-weight (more expensive compilation, regexes can grow up to multiple
/// MBs, usage of scratch space), 1. GlobalCacheTable is a global singleton and, as a result, needs locking 2. the pattern compilation is
/// done outside GlobalCacheTable's lock, at the cost of another level of locking.
struct GlobalCacheTable {
constexpr static size_t CACHE_SIZE = 500; /// collision probability
struct Bucket {
std::vector<String> patterns; /// key
std::optional<UInt32> edit_distance; /// key
/// The compiled patterns and their state (vectorscan 'database' + scratch space) are wrapped in a shared_ptr. Refcounting guarantees
/// that eviction of a pattern does not affect parallel threads still using the pattern.
DeferredConstructedRegexpsPtr regexps; /// value
};
std::mutex mutex;
std::array<Bucket, CACHE_SIZE> known_regexps;
static size_t getBucketIndexFor(const std::vector<String> patterns,
std::optional<UInt32> edit_distance) {
size_t hash = 0;
for (const auto& pattern : patterns) boost::hash_combine(hash, pattern);
boost::hash_combine(hash, edit_distance);
return hash % CACHE_SIZE;
}
};
/// If WithEditDistance is False, edit_distance must be nullopt. Also, we use templates here because each instantiation of function template
/// has its own copy of local static variables which must not be the same for different hyperscan compilations.
template <bool save_indices, bool WithEditDistance>
inline DeferredConstructedRegexpsPtr getOrSet(const std::vector<StringRef>& patterns,
std::optional<UInt32> edit_distance) {
static GlobalCacheTable
pool; /// Different variables for different pattern parameters, thread-safe in C++11
std::vector<String> str_patterns;
str_patterns.reserve(patterns.size());
for (const auto& pattern : patterns) str_patterns.emplace_back(pattern.to_string());
size_t bucket_idx = GlobalCacheTable::getBucketIndexFor(str_patterns, edit_distance);
/// Lock cache to find compiled regexp for given pattern vector + edit distance.
std::lock_guard lock(pool.mutex);
GlobalCacheTable::Bucket& bucket = pool.known_regexps[bucket_idx];
/// Pattern compilation is expensive and we don't want to block other threads reading from / inserting into the cache while we hold the
/// cache lock during pattern compilation. Therefore, when a cache entry is created or replaced, only set the regexp constructor method
/// and compile outside the cache lock.
/// Note that the string patterns and the edit distance is passed into the constructor lambda by value, i.e. copied - it is not an
/// option to reference the corresponding string patterns / edit distance key in the cache table bucket because the cache entry may
/// already be evicted at the time the compilation starts.
if (bucket.regexps == nullptr) [[unlikely]] {
/// insert new entry
auto deferred_constructed_regexps =
std::make_shared<DeferredConstructedRegexps>([str_patterns, edit_distance]() {
return constructRegexps<save_indices, WithEditDistance>(str_patterns,
edit_distance);
});
bucket = {std::move(str_patterns), edit_distance, deferred_constructed_regexps};
} else if (bucket.patterns != str_patterns || bucket.edit_distance != edit_distance) {
/// replace existing entry
auto deferred_constructed_regexps =
std::make_shared<DeferredConstructedRegexps>([str_patterns, edit_distance]() {
return constructRegexps<save_indices, WithEditDistance>(str_patterns,
edit_distance);
});
bucket = {std::move(str_patterns), edit_distance, deferred_constructed_regexps};
}
return bucket.regexps;
}
} // namespace multiregexps
} // namespace doris::vectorized

View File

@ -77,6 +77,8 @@ void register_function_least_greast(SimpleFunctionFactory& factory);
void register_function_fake(SimpleFunctionFactory& factory);
void register_function_array(SimpleFunctionFactory& factory);
void register_function_geo(SimpleFunctionFactory& factory);
void register_function_multi_string_position(SimpleFunctionFactory& factory);
void register_function_multi_string_search(SimpleFunctionFactory& factory);
void register_function_encryption(SimpleFunctionFactory& factory);
void register_function_regexp_extract(SimpleFunctionFactory& factory);
@ -218,6 +220,8 @@ public:
register_function_array(instance);
register_function_geo(instance);
register_function_url(instance);
register_function_multi_string_position(instance);
register_function_multi_string_search(instance);
});
return instance;
}

View File

@ -0,0 +1,54 @@
---
{
"title": "multi_match_any",
"language": "en"
}
---
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->
## multi_match_any
### Description
#### Syntax
`TINYINT multi_match_any(VARCHAR haystack, ARRAY<VARCHAR> patterns)`
Checks whether the string `haystack` matches the regular expressions `patterns` in re2 syntax. returns 0 if none of the regular expressions are matched and 1 if any of the patterns matches.
### example
```
mysql> select multi_match_any('Hello, World!', ['hello', '!', 'world']);
+-----------------------------------------------------------+
| multi_match_any('Hello, World!', ['hello', '!', 'world']) |
+-----------------------------------------------------------+
| 1 |
+-----------------------------------------------------------+
mysql> select multi_match_any('abc', ['A', 'bcd']);
+--------------------------------------+
| multi_match_any('abc', ['A', 'bcd']) |
+--------------------------------------+
| 0 |
+--------------------------------------+
```
### keywords
MULTI_MATCH,MATCH,ANY

View File

@ -0,0 +1,54 @@
---
{
"title": "multi_search_all_positions",
"language": "en"
}
---
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->
## multi_search_all_positions
### Description
#### Syntax
`ARRAY<INT> multi_search_all_positions(VARCHAR haystack, ARRAY<VARCHAR> needles)`
Searches for the substrings `needles` in the string `haystack`, and returns array of positions of the found corresponding substrings in the string. Positions are indexed starting from 1.
### example
```
mysql> select multi_search_all_positions('Hello, World!', ['hello', '!', 'world']);
+----------------------------------------------------------------------+
| multi_search_all_positions('Hello, World!', ['hello', '!', 'world']) |
+----------------------------------------------------------------------+
| [0,13,0] |
+----------------------------------------------------------------------+
mysql> select multi_search_all_positions('abc', ['a', 'bc', 'd']);
+-----------------------------------------------------+
| multi_search_all_positions('abc', ['a', 'bc', 'd']) |
+-----------------------------------------------------+
| [1,2,0] |
+-----------------------------------------------------+
```
### keywords
MULTI_SEARCH,SEARCH,POSITIONS

View File

@ -0,0 +1,54 @@
---
{
"title": "multi_match_any",
"language": "zh-CN"
}
---
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->
## multi_match_any
### Description
#### Syntax
`TINYINT multi_match_any(VARCHAR haystack, ARRAY<VARCHAR> patterns)`
检查字符串 `haystack` 是否与 re2 语法中的正则表达式 `patterns` 相匹配。如果都没有匹配的正则表达式返回 0,否则返回 1。
### example
```
mysql> select multi_match_any('Hello, World!', ['hello', '!', 'world']);
+-----------------------------------------------------------+
| multi_match_any('Hello, World!', ['hello', '!', 'world']) |
+-----------------------------------------------------------+
| 1 |
+-----------------------------------------------------------+
mysql> select multi_match_any('abc', ['A', 'bcd']);
+--------------------------------------+
| multi_match_any('abc', ['A', 'bcd']) |
+--------------------------------------+
| 0 |
+--------------------------------------+
```
### keywords
MULTI_MATCH,MATCH,ANY

View File

@ -0,0 +1,54 @@
---
{
"title": "multi_search_all_positions",
"language": "zh-CN"
}
---
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->
## multi_search_all_positions
### Description
#### Syntax
`ARRAY<INT> multi_search_all_positions(VARCHAR haystack, ARRAY<VARCHAR> needles)`
搜索字符串 `haystack` 中的子字符串 `needles`,并返回在 `haystack` 中找到的相应 `needles` 的位置数组。位置的下标从 1 开始。
### example
```
mysql> select multi_search_all_positions('Hello, World!', ['hello', '!', 'world']);
+----------------------------------------------------------------------+
| multi_search_all_positions('Hello, World!', ['hello', '!', 'world']) |
+----------------------------------------------------------------------+
| [0,13,0] |
+----------------------------------------------------------------------+
mysql> select multi_search_all_positions('abc', ['a', 'bc', 'd']);
+-----------------------------------------------------+
| multi_search_all_positions('abc', ['a', 'bc', 'd']) |
+-----------------------------------------------------+
| [1,2,0] |
+-----------------------------------------------------+
```
### keywords
MULTI_SEARCH,SEARCH,POSITIONS

View File

@ -2806,6 +2806,10 @@ visible_functions = [
[['protocol'], 'STRING', ['STRING'],
'fake_symble_for_no_vec', '', '',
'vec', ''],
# search functions
[['multi_search_all_positions'], 'ARRAY_INT', ['STRING', 'ARRAY_STRING'], '', '', '', 'vec', 'ALWAYS_NOT_NULLABLE'],
[['multi_match_any'], 'TINYINT', ['STRING', 'ARRAY_STRING'], '', '', '', 'vec', 'ALWAYS_NOT_NULLABLE'],
]
# Except the following functions, other function will directly return

View File

@ -0,0 +1,25 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !select --
[4, 1, 1, 2, 6, 1, 1, 0, 4, 1, 14, 0, 10, 0, 16, 6]
-- !select --
[0, 0, 0, 2, 3, 0, 1, 0, 5, 0, 0, 0, 11, 10, 6, 7]
-- !select --
[1, 1, 0, 0, 0, 1, 1, 1, 4, 0, 6, 6, 0, 10, 1, 5]
-- !select --
[1, 0, 0, 8, 6, 0, 7, 1, 3, 0, 0, 0, 0, 12]
-- !select --
[7, 0, 0, 8, 0, 2, 0, 0, 6, 0, 2, 0, 3, 1]
-- !select --
[0, 0, 0, 19, 14, 22, 10, 0, 0, 13, 0, 8]
-- !select --
[6, 1, 1, 0, 0, 5, 1, 0, 8, 0, 5, 0, 2, 12, 0, 15, 0, 0]
-- !select --
[0, 0, 5, 1, 1, 0, 15, 1, 5, 10, 4, 0, 1, 0, 3, 0, 0, 0]
-- !select --
[0, 1, 11, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4, 1]
-- !select --
[0, 0, 0, 3, 0, 15, 0, 0, 12, 7, 0, 0, 0, 0, 5, 0]
-- !select --
[0, 0, 12, 4, 4, 0, 13, 23, 0, 1, 0, 2, 0, 0, 0, 3, 0, 0]
-- !select --
[6, 8, 1, 4, 0, 10, 0, 1, 14, 0, 1, 0, 5, 0, 0, 0, 0, 15, 0, 1]

View File

@ -0,0 +1,45 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !select --
0
-- !select --
0
-- !select --
0
-- !select --
1
-- !select --
0
-- !select --
0
-- !select --
0
-- !select --
1
-- !select --
1
-- !select --
1
-- !select --
0
-- !select --
1
-- !select --
0
-- !select --
1
-- !select --
1
-- !select --
1
-- !select --
0
-- !select --
0
-- !select --
0
-- !select --
1
-- !select --
1
-- !select --
1

View File

@ -0,0 +1,31 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
suite("test_multi_string_position") {
qt_select "select multi_search_all_positions('jmdqwjbrxlbatqeixknricfk', ['qwjbrxlba', 'jmd', '', 'mdqwjbrxlbatqe', 'jbrxlbatqeixknric', 'jmdqwjbrxlbatqeixknri', '', 'fdtmnwtts', 'qwjbrxlba', '', 'qeixknricfk', 'hzjjgrnoilfkvzxaemzhf', 'lb', 'kamz', 'ixknr', 'jbrxlbatq'])"
qt_select "select multi_search_all_positions('coxcctuehmzkbrsmodfvx', ['bkhnp', 'nlypjvriuk', 'rkslxwfqjjivcwdexrdtvjdtvuu', 'oxcctuehm', 'xcctuehmzkbrsm', 'kfrieuocovykjmkwxbdlkgwctwvcuh', 'coxc', 'lbwvetgxyndxjqqwthtkgasbafii', 'ctuehmzkbrsmodfvx', 'obzldxjldxowk', 'ngfikgigeyll', 'wdaejjukowgvzijnw', 'zkbr', 'mzkb', 'tuehm', 'ue'])"
qt_select "select multi_search_all_positions('mpswgtljbbrmivkcglamemayfn', ['', 'm', 'saejhpnfgfq', 'rzanrkdssmmkanqjpfi', 'oputeneprgoowg', 'mp', '', '', 'wgtljbbrmivkcglamemay', 'cbpthtrgrmgfypizi', 'tl', 'tlj', 'xuhs', 'brmivkcglamemayfn', '', 'gtljb'])"
qt_select "select multi_search_all_positions('arbphzbbecypbzsqsljurtddve', ['arbphzb', 'mnrboimjfijnti', 'cikcrd', 'becypbz', 'z', 'uocmqgnczhdcrvtqrnaxdxjjlhakoszuwc', 'bbe', '', 'bp', 'yhltnexlpdijkdzt', 'jkwjmrckvgmccmmrolqvy', 'vdxmicjmfbtsbqqmqcgtnrvdgaucsgspwg', 'witlfqwvhmmyjrnrzttrikhhsrd', 'pbzsqsljurt'])"
qt_select "select multi_search_all_positions('aizovxqpzcbbxuhwtiaaqhdqjdei', ['qpzcbbxuhw', 'jugrpglqbm', 'dspwhzpyjohhtizegrnswhjfpdz', 'pzcbbxuh', 'vayzeszlycke', 'i', 'gvrontcpqavsjxtjwzgwxugiyhkhmhq', 'gyzmeroxztgaurmrqwtmsxcqnxaezuoapatvu', 'xqpzc', 'mjiswsvlvlpqrhhptqq', 'iz', 'hmzjxxfjsvcvdpqwtrdrp', 'zovxqpzcbbxuhwtia', 'ai'])"
qt_select "select multi_search_all_positions('ydfgiluhyxwqdfiwtzobwzscyxhuov', ['srsoubrgghleyheujsbwwwykerzlqphgejpxvog', 'axchkyleddjwkvbuyhmekpbbbztxdlm', 'zqodzvlkmfe', 'obwz', 'fi', 'zsc', 'xwq', 'pvmurvrd', 'uulcdtexckmrsokmgdpkstlkoavyrmxeaacvydxf', 'dfi', 'mxcngttujzgtlssrmluaflmjuv', 'hyxwqdfiwtzobwzscyxhu'])"
qt_select "select multi_search_all_positions('pyepgwainvmwekwhhqxxvzdjw', ['w', '', '', 'gvvkllofjnxvcu', 'kmwwhboplctvzazcyfpxhwtaddfnhekei', 'gwainv', 'pyepgwain', 'ekpnogkzzmbpfynsunwqp', 'invmwe', 'hrxpiplfplqjsstuybksuteoz', 'gwa', 'akfpyduqrwosxcbdemtxrxvundrgse', 'yepgwainvmw', 'wekwhhqxxvzdjw', 'fyimzvedmyriubgoznmcav', 'whhq', 'ozxowbwdqfisuupyzaqynoprgsjhkwlum', 'vpoufrofekajksdp'])"
qt_select "select multi_search_all_positions('lqwahffxurkbhhzytequotkfk', ['rwjqudpuaiufle', 'livwgbnflvy', 'hffxurkbhh', '', '', 'xcajwbqbttzfzfowjubmmgnmssat', 'zytequ', 'lq', 'h', 'rkbhh', 'a', 'immejthwgdr', '', 'llhhnlhcvnxxorzzjt', 'w', 'cvjynqxcivmmmvc', 'wexjomdcmursppjtsweybheyxzleuz', 'fzronsnddfxwlkkzidiknhpjipyrcrzel'])"
qt_select "select multi_search_all_positions('nkddriylnakicwgdwrfxpodqea', ['izwdpgrgpmjlwkanjrffgela', '', 'kicw', 'hltmfymgmrjckdiylkzjlvvyuleksikdjrg', 'yigveskrbidknjxigwilmkgyizewikh', 'xyvzhsnqmuec', 'odcgzlavzrwesjks', 'oilvfgliktoujukpgzvhmokdgkssqgqot', 'llsfsurvimbahwqtbqbp', 'nxj', 'pimydixeobdxmdkvhcyzcgnbhzsydx', 'couzmvxedobuohibgxwoxvmpote', 'driylnakicwgdwrf', 'nkddr'])"
qt_select "select multi_search_all_positions('jnckhtjqwycyihuejibqmddrdxe', ['tajzx', 'vuddoylclxatcjvinusdwt', 'spxkhxvzsljkmnzpeubszjnhqczavgtqopxn', 'ckhtjqwycyi', 'xlbfzdxspldoes', 'u', 'czosfebeznt', 'gzhabdsuyreisxvyfrfrkq', 'yihuejibqmd', 'jqwycyihuejibqm', 'cfbvprgzx', 'hxu', 'vxbhrfpzacgd', 'afoaij', 'htjqwycyihu', 'httzbskqd'])"
qt_select "select multi_search_all_positions('dzejajvpoojdkqbnayahygidyrjmb', ['khwxxvtnqhobbvwgwkpusjlhlzifiuclycml', 'nzvuhtwdaivo', 'dkqbnayahygidyr', 'jajvpoo', 'j', 'wdtbvwmeqgyvetu', 'kqbn', 'idyrjmb', 'tsnxuxevsxrxpgpfdgrkhwqpkse', '', 'efsdgzuefhdzkmquxu', 'zejajvpoojdkqbnayahyg', 'ugwfuighbygrxyctop', 'fcbxzbdugc', 'dxmzzrcplob', 'ejaj', 'wmmupyxrylvawsyfccluiiene', 'ohzmsqhpzbafvbzqwzftbvftei'])"
qt_select "select multi_search_all_positions('ffaujlverosspbzaqefjzql', ['lvero', 'erossp', 'f', 'ujlverosspbz', 'btfimgklzzxlbkbuqyrmnud', 'osspb', 'muqexvtjuaar', 'f', 'bzaq', 'lprihswhwkdhqciqhfaowarn', 'ffaujlve', 'uhbbjrqjb', 'jlver', 'umucyhbbu', 'pjthtzmgxhvpbdphesnnztuu', 'xfqhfdfsbbazactpastzvzqudgk', 'lvovjfoatc', 'z', 'givejzhoqsd', ''])"
}

View File

@ -0,0 +1,41 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
suite("test_multi_string_search") {
qt_select "select multi_match_any('mpnsguhwsitzvuleiwebwjfitmsg', ['wbirxqoabpblrnvvmjizj', 'cfcxhuvrexyzyjsh', 'oldhtubemyuqlqbwvwwkwin', 'bumoozxdkjglzu', 'intxlfohlxmajjomw', 'dxkeghohv', 'arsvmwwkjeopnlwnan', 'ouugllgowpqtaxslcopkytbfhifaxbgt', 'hkedmjlbcrzvryaopjqdjjc', 'tbqkljywstuahzh', 'o', 'wowoclosyfcuwotmvjygzuzhrery', 'vpefjiffkhlggntcu', 'ytdixvasrorhripzfhjdmlhqksmctyycwp'])"
qt_select "select multi_match_any('qjjzqexjpgkglgxpzrbqbnskq', ['vaiatcjacmlffdzsejpdareqzy', 'xspcfzdufkmecud', 'bcvtbuqtctq', 'nkcopwbfytgemkqcfnnno', 'dylxnzuyhq', 'tno', 'scukuhufly', 'cdyquzuqlptv', 'ohluyfeksyxepezdhqmtfmgkvzsyph', 'ualzwtahvqvtijwp', 'jg', 'gwbawqlngzcknzgtmlj', 'qimvjcgbkkp', 'eaedbcgyrdvv', 'qcwrncjoewwedyyewcdkh', 'uqcvhngoqngmitjfxpznqomertqnqcveoqk', 'ydrgjiankgygpm', 'axepgap'])"
qt_select "select multi_match_any('fdkmtqmxnegwvnjhghjq', ['vynkybvdmhgeezybbdqfrukibisj', 'knazzamgjjpavwhvdkwigykh', 'peumnifrmdhhmrqqnemw', 'lmsnyvqoisinlaqobxojlwfbi', 'oqwfzs', 'dymudxxeodwjpgbibnkvr', 'vomtfsnizkplgzktqyoiw', 'yoyfuhlpgrzds', 'cefao', 'gi', 'srpgxfjwl', 'etsjusdeiwbfe', 'ikvtzdopxo', 'ljfkavrau', 'soqdhxtenfrkmeic', 'ktprjwfcelzbup', 'pcvuoddqwsaurcqdtjfnczekwni', 'agkqkqxkfbkfgyqliahsljim'])"
qt_select "select multi_match_any('khljxzxlpcrxpkrfybbfk', ['', 'lpc', 'rxpkrfybb', 'crxp', '', 'pkr', 'jxzxlpcrxpkrf', '', 'xzxlpcr', 'xpk', 'fyb', 'xzxlpcrxpkrfybbfk', 'k', 'lpcrxp', 'ljxzxlpcr', 'r', 'pkr', 'fk'])"
qt_select "select multi_match_any('rbrizgjbigvzfnpgmpkqxoqxvdj', ['ee', 'cohqnb', 'msol', 'yhlujcvhklnhuomy', 'ietn', 'vgmnlkcsybtokrepzrm', 'wspiryefojxysgrzsxyrluykxfnnbzdstcel', 'mxisnsivndbefqxwznimwgazuulupbaihavg', 'vpzdjvqqeizascxmzdhuq', 'pgvncohlxcqjhfkm', 'mbaypcnfapltsegquurahlsruqvipfhrhq', 'ioxjbcyyqujfveujfhnfdfokfcrlsincjbdt', 'cnvlujyowompdrqjwjx', 'wobwed', 'kdfhaoxiuifotmptcmdbk', 'leoamsnorcvtlmokdomkzuo', 'jjw', 'ogugysetxuqmvggneosbsfbonszepsatq'])"
qt_select "select multi_match_any('uymwxzyjbfegbhgswiqhinf', ['lizxzbzlwljkr', 'ukxygktlpzuyijcqeqktxenlaqi', 'onperabgbdiafsxwbvpjtyt', 'xfqgoqvhqph', 'aflmcwabtwgmajmmqelxwkaolyyhmdlc', 'yfz', 'meffuiaicvwed', 'hhzvgmifzamgftkifaeowayjrnnzw', 'nwewybtajv', 'ectiye', 'epjeiljegmqqjncubj', 'zsjgftqjrn', 'pssng', 'raqoarfhdoeujulvqmdo'])"
qt_select "select multi_match_any('omgghgnzjmecpzqmtcvw', ['fjhlzbszodmzavzg', 'gfofrnwrxprkfiokv', 'jmjiiqpgznlmyrxwewzqzbe', 'pkyrsqkltlmxr', 'crqgkgqkkyujcyoc', 'endagbcxwqhueczuasykmajfsvtcmh', 'xytmxtrnkdysuwltqomehddp', 'etmdxyyfotfyifwvbykghijvwv', 'mwqtgrncyhkfhjdg', 'iuvymofrqpp', 'pgllsdanlhzqhkstwsmzzftp', 'disjylcceufxtjdvhy'])"
qt_select "select multi_match_any('mznihnmshftvnmmhnrulizzpslq', ['nrul', 'mshftvnmmhnr', 'z', 'mhnrulizzps', 'hftvnmmhnrul', 'ihnmshftvnmmhnrulizzp', 'izz', '', 'uli', 'nihnmshftvnmmhnru', 'hnrulizzp', 'nrulizz'])"
qt_select "select multi_match_any('ruqmqrsxrbftvruvahonradau', ['uqmqrsxrbft', 'ftv', 'tvruvahonrad', 'mqrsxrbftvruvahon', 'rbftvruvah', 'qrsxrbftvru', 'o', 'ahonradau', 'a', 'ft', '', 'u', 'rsxrbftvruvahonradau', 'ruvahon', 'bftvruvahonradau', 'qrsxrbftvru', 't', 'vahonrada', 'vruvahonradau', 'onra'])"
qt_select "select multi_match_any('gpsevxtcoeexrltyzduyidmtzxf', ['exrltyzduyid', 'vxtcoeexrltyz', 'xr', 'ltyzduyidmt', 'yzduy', 'exr', 'coeexrltyzduy', 'coeexrltyzduy', 'rlty', 'rltyzduyidm', 'exrltyz', 'xtcoeexrlty', 'vxtcoeexrltyzduyidm', '', 'coeexrl', 'sevxtcoeexrltyzdu', 'dmt', ''])"
qt_select "select multi_match_any('dyhycfhzyewaikgursyxfkuv', ['sktnofpugrmyxmbizzrivmhn', 'fhlgadpoqcvktbfzncxbllvwutdawmw', 'eewzjpcgzrqmltbgmhafwlwqb', 'tpogbkyj', 'rtllntxjgkzs', 'mirbvsqexscnzglogigbujgdwjvcv', 'iktwpgjsakemewmahgqza', 'xgfvzkvqgiuoihjjnxwwpznxhz', 'nxaumpaknreklbwynvxdsmatjekdlxvklh', 'zadzwqhgfxqllihuudozxeixyokhny', 'tdqpgfpzexlkslodps', 'slztannufxaabqfcjyfquafgfhfb', 'xvjldhfuwurvkb', 'aecv', 'uycfsughpikqsbcmwvqygdyexkcykhbnau', 'jr'])"
qt_select "select multi_match_any('vbcsettndwuntnruiyclvvwoo', ['dwuntnru', '', 'ttndwuntnruiyclvv', 'ntnr', 'nruiyclvvw', 'wo', '', 'bcsettndwuntnruiycl', 'yc', 'untnruiyclvvw', 'csettndwuntnr', 'ntnruiyclvvwo'])"
qt_select "select multi_match_any('pqqnugshlczcuxhpjxjbcnro', ['dpeedqy', 'rtsc', 'jdgla', 'qkgudqjiyzvlvsj', 'xmfxawhijgxxtydbd', 'ebgzazqthb', 'wyrjhvhwzhmpybnylirrn', 'iviqbyuclayqketooztwegtkgwnsezfl', 'bhvidy', 'hijctxxweboq', 't', 'osnzfbziidteiaifgaanm'])"
qt_select "select multi_match_any('loqchlxspwuvvccucskuytr', ['', 'k', 'qchlxspwu', 'u', 'hlxspwuvv', 'wuvvccucsku', 'vcc', 'uyt', 'uvv', 'spwu', 'ytr', 'wuvvccucs', 'xspwuv', 'lxspwuvvccuc', 'spwuvvccu', 'oqchlxspwuvvccucskuy'])"
qt_select "select multi_match_any('pjjyzupzwllshlnatiujmwvaofr', ['lnatiujmwvao', '', 'zupzwllsh', 'nati', 'wllshl', 'hlnatiujmwv', 'mwvao', 'shlnat', 'ati', 'wllshlnatiujmwvao', 'wllshlnatiujmwvaofr', 'nat'])"
qt_select "select multi_match_any('iketunkleyaqaxdlocci', ['nkleyaqaxd', 'etunkleyaq', 'yaqaxdlocci', 'tunkleyaq', 'eyaqaxdlocc', 'leyaq', 'nkleyaqaxdl', 'tunkleya', 'kleyaqa', 'etunkleya', 'leyaqa', 'dlo', 'yaqa', 'leyaqaxd', 'etunkleyaq', ''])"
qt_select "select multi_match_any('drqianqtangmgbdwruvblkqd', ['wusajejyucamkyl', 'wsgibljugzrpkniliy', 'lhwqqiuafwffyersqjgjvvvfurx', 'jfokpzzxfdonelorqu', 'ccwkpcgac', 'jmyulqpndkmzbfztobwtm', 'rwrgfkccgxht', 'ggldjecrgbngkonphtcxrkcviujihidjx', 'spwweavbiokizv', 'lv', 'krb', 'vstnhvkbwlqbconaxgbfobqky', 'pvxwdc', 'thrl', 'ahsblffdveamceonqwrbeyxzccmux', 'yozji', 'oejtaxwmeovtqtz', 'zsnzznvqpxdvdxhznxrjn', 'hse', 'kcmkrccxmljzizracxwmpoaggywhdfpxkq'])"
qt_select "select multi_match_any('yasnpckniistxcejowfijjsvkdajz', ['slkpxhtsmrtvtm', 'crsbq', 'rdeshtxbfrlfwpsqojassxmvlfbzefldavmgme', 'ipetilcbpsfroefkjirquciwtxhrimbmwnlyv', 'knjpwkmdwbvdbapuyqbtsw', 'horueidziztxovqhsicnklmharuxhtgrsr', 'ofohrgpz', 'oneqnwyevbaqsonrcpmxcynflojmsnix', 'shg', 'nglqzczevgevwawdfperpeytuodjlf'])"
qt_select "select multi_match_any('ueptpscfgxhplwsueckkxs', ['ohhygchclbpcdwmftperprn', 'dvpjdqmqckekndvcerqrpkxen', 'lohhvarnmyi', 'zppd', 'qmqxgfewitsunbuhffozcpjtc', 'hsjbioisycsrawktqssjovkmltxodjgv', 'dbzuunwbkrtosyvctdujqtvaawfnvuq', 'gupbvpqthqxae', 'abjdmijaaiasnccgxttmqdsz', 'uccyumqoyqe', 'kxxliepyzlc', 'wbqcqtbyyjbqcgdbpkmzugksmcxhvr', 'piedxm', 'uncpphzoif', 'exkdankwck', 'qeitzozdrqopsergzr', 'hesgrhaftgesnzflrrtjdobxhbepjoas', 'wfpexx'])"
qt_select "select multi_match_any('ldrzgttlqaphekkkdukgngl', ['gttlqaphekkkdukgn', 'ekkkd', 'gttlqaphe', 'qaphek', 'h', 'kdu', 'he', 'phek', '', 'drzgttlqaphekkkd'])"
qt_select "select multi_match_any('ololo', ['ololo', 'ololo', 'ololo'])"
qt_select "select multi_match_any('khljxzxlpcrxpkrfybbfk', ['k'])"
}