[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:
@ -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
|
||||
|
||||
236
be/src/vec/functions/functions_multi_string_position.cpp
Normal file
236
be/src/vec/functions/functions_multi_string_position.cpp
Normal 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
|
||||
283
be/src/vec/functions/functions_multi_string_search.cpp
Normal file
283
be/src/vec/functions/functions_multi_string_search.cpp
Normal 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
|
||||
256
be/src/vec/functions/regexps.h
Normal file
256
be/src/vec/functions/regexps.h
Normal 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
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -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
|
||||
@ -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
|
||||
@ -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
|
||||
@ -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
|
||||
@ -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
|
||||
|
||||
@ -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]
|
||||
@ -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
|
||||
@ -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', ''])"
|
||||
}
|
||||
@ -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'])"
|
||||
}
|
||||
Reference in New Issue
Block a user