mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Support 'start_pos' argument in 'position' function
This commit is contained in:
parent
dd475fe82c
commit
9ed7df64cd
@ -11,7 +11,6 @@
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Search and replace functions in strings:
|
||||
@ -51,15 +50,26 @@ public:
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
bool isVariadic() const override {
|
||||
return Impl::supports_start_pos;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override {
|
||||
if (Impl::supports_start_pos) {
|
||||
return 0;
|
||||
}
|
||||
return 2;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return Impl::use_default_implementation_for_constants; }
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override
|
||||
{
|
||||
return Impl::use_default_implementation_for_constants
|
||||
? ColumnNumbers{1, 2}
|
||||
: ColumnNumbers{};
|
||||
if (!Impl::use_default_implementation_for_constants)
|
||||
return ColumnNumbers{};
|
||||
if (!Impl::supports_start_pos)
|
||||
return ColumnNumbers{1, 2};
|
||||
return ColumnNumbers{1, 2, 3};
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
@ -72,6 +82,13 @@ public:
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() >= 3) {
|
||||
if (!isUnsignedInteger(arguments[2])) {
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
|
||||
}
|
||||
|
||||
@ -82,17 +99,36 @@ public:
|
||||
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
|
||||
const ColumnPtr & column_needle = block.getByPosition(arguments[1]).column;
|
||||
|
||||
ColumnPtr column_start_pos = nullptr;
|
||||
if (arguments.size() >= 3) {
|
||||
column_start_pos = block.getByPosition(arguments[2]).column;
|
||||
}
|
||||
|
||||
const ColumnConst * col_haystack_const = typeid_cast<const ColumnConst *>(&*column_haystack);
|
||||
const ColumnConst * col_needle_const = typeid_cast<const ColumnConst *>(&*column_needle);
|
||||
|
||||
if constexpr (!Impl::use_default_implementation_for_constants)
|
||||
{
|
||||
bool is_col_start_pos_const = column_start_pos == nullptr || isColumnConst(*column_start_pos);
|
||||
if (col_haystack_const && col_needle_const)
|
||||
{
|
||||
ResultType res{};
|
||||
Impl::constantConstant(col_haystack_const->getValue<String>(), col_needle_const->getValue<String>(), res);
|
||||
block.getByPosition(result).column
|
||||
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(is_col_start_pos_const ? 1 : column_start_pos->size());
|
||||
|
||||
Impl::constantConstant(
|
||||
col_haystack_const->getValue<String>(),
|
||||
col_needle_const->getValue<String>(),
|
||||
column_start_pos,
|
||||
vec_res);
|
||||
|
||||
if (is_col_start_pos_const) {
|
||||
block.getByPosition(result).column
|
||||
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(vec_res[0]));
|
||||
} else {
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -112,16 +148,28 @@ public:
|
||||
col_haystack_vector->getOffsets(),
|
||||
col_needle_vector->getChars(),
|
||||
col_needle_vector->getOffsets(),
|
||||
column_start_pos,
|
||||
vec_res);
|
||||
else if (col_haystack_vector && col_needle_const)
|
||||
Impl::vectorConstant(
|
||||
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needle_const->getValue<String>(), vec_res);
|
||||
col_haystack_vector->getChars(),
|
||||
col_haystack_vector->getOffsets(),
|
||||
col_needle_const->getValue<String>(),
|
||||
column_start_pos,
|
||||
vec_res);
|
||||
else if (col_haystack_vector_fixed && col_needle_const)
|
||||
Impl::vectorFixedConstant(
|
||||
col_haystack_vector_fixed->getChars(), col_haystack_vector_fixed->getN(), col_needle_const->getValue<String>(), vec_res);
|
||||
col_haystack_vector_fixed->getChars(),
|
||||
col_haystack_vector_fixed->getN(),
|
||||
col_needle_const->getValue<String>(),
|
||||
vec_res);
|
||||
else if (col_haystack_const && col_needle_vector)
|
||||
Impl::constantVector(
|
||||
col_haystack_const->getValue<String>(), col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res);
|
||||
col_haystack_const->getValue<String>(),
|
||||
col_needle_vector->getChars(),
|
||||
col_needle_vector->getOffsets(),
|
||||
column_start_pos,
|
||||
vec_res);
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal columns " + block.getByPosition(arguments[0]).column->getName() + " and "
|
||||
|
@ -79,12 +79,19 @@ struct ExtractParamImpl
|
||||
using ResultType = typename ParamExtractor::ResultType;
|
||||
|
||||
static constexpr bool use_default_implementation_for_constants = true;
|
||||
static constexpr bool supports_start_pos = false;
|
||||
|
||||
/// It is assumed that `res` is the correct size and initialized with zeros.
|
||||
static void vectorConstant(const ColumnString::Chars & data, const ColumnString::Offsets & offsets,
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
std::string needle,
|
||||
const ColumnPtr & start_pos,
|
||||
PaddedPODArray<ResultType> & res)
|
||||
{
|
||||
if (start_pos != nullptr) {
|
||||
throw Exception("Functions 'visitParamHas' and 'visitParamExtract*' doesn't support start_pos argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
/// We are looking for a parameter simply as a substring of the form "name"
|
||||
needle = "\"" + needle + "\":";
|
||||
|
||||
|
@ -19,10 +19,19 @@ struct HasTokenImpl
|
||||
using ResultType = UInt8;
|
||||
|
||||
static constexpr bool use_default_implementation_for_constants = true;
|
||||
static constexpr bool supports_start_pos = false;
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
const std::string & pattern,
|
||||
const ColumnPtr & start_pos,
|
||||
PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
if (start_pos != nullptr) {
|
||||
throw Exception("Function 'hasToken' does not support start_pos argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
if (offsets.empty())
|
||||
return;
|
||||
|
||||
|
@ -76,6 +76,7 @@ template <bool like, bool revert = false, bool case_insensitive = false>
|
||||
struct MatchImpl
|
||||
{
|
||||
static constexpr bool use_default_implementation_for_constants = true;
|
||||
static constexpr bool supports_start_pos = false;
|
||||
|
||||
using ResultType = UInt8;
|
||||
|
||||
@ -84,8 +85,16 @@ struct MatchImpl
|
||||
VolnitskyUTF8>;
|
||||
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
const std::string & pattern,
|
||||
const ColumnPtr & start_pos,
|
||||
PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
if (start_pos != nullptr) {
|
||||
throw Exception("Functions 'like' and 'match' don't support start_pos argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
if (offsets.empty())
|
||||
return;
|
||||
|
||||
@ -238,7 +247,8 @@ struct MatchImpl
|
||||
|
||||
/// Very carefully crafted copy-paste.
|
||||
static void vectorFixedConstant(
|
||||
const ColumnString::Chars & data, size_t n, const std::string & pattern, PaddedPODArray<UInt8> & res)
|
||||
const ColumnString::Chars & data, size_t n, const std::string & pattern,
|
||||
PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
if (data.empty())
|
||||
return;
|
||||
|
@ -1,11 +1,11 @@
|
||||
#include "FunctionsStringSearch.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <Poco/UTF8String.h>
|
||||
#include <Common/Volnitsky.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -42,6 +42,10 @@ struct PositionCaseSensitiveASCII
|
||||
return MultiSearcherInBigHaystack(needles);
|
||||
}
|
||||
|
||||
static const char * advancePos(const char * pos, const char * end, size_t n) {
|
||||
return std::min(pos + n, end);
|
||||
}
|
||||
|
||||
/// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8).
|
||||
static size_t countChars(const char * begin, const char * end) { return end - begin; }
|
||||
|
||||
@ -73,6 +77,10 @@ struct PositionCaseInsensitiveASCII
|
||||
return MultiSearcherInBigHaystack(needles);
|
||||
}
|
||||
|
||||
static const char * advancePos(const char * pos, const char * end, size_t n) {
|
||||
return std::min(pos + n, end);
|
||||
}
|
||||
|
||||
static size_t countChars(const char * begin, const char * end) { return end - begin; }
|
||||
|
||||
static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); }
|
||||
@ -100,6 +108,19 @@ struct PositionCaseSensitiveUTF8
|
||||
return MultiSearcherInBigHaystack(needles);
|
||||
}
|
||||
|
||||
static const char * advancePos(const char * pos, const char * end, size_t n)
|
||||
{
|
||||
for (auto it = pos; it != end; ++it) {
|
||||
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it))) {
|
||||
if (n == 0) {
|
||||
return it;
|
||||
}
|
||||
n--;
|
||||
}
|
||||
}
|
||||
return end;
|
||||
}
|
||||
|
||||
static size_t countChars(const char * begin, const char * end)
|
||||
{
|
||||
size_t res = 0;
|
||||
@ -134,13 +155,16 @@ struct PositionCaseInsensitiveUTF8
|
||||
return MultiSearcherInBigHaystack(needles);
|
||||
}
|
||||
|
||||
static const char * advancePos(const char * pos, const char * end, size_t n)
|
||||
{
|
||||
// reuse implementation that doesn't depend on case
|
||||
return PositionCaseSensitiveUTF8::advancePos(pos, end, n);
|
||||
}
|
||||
|
||||
static size_t countChars(const char * begin, const char * end)
|
||||
{
|
||||
size_t res = 0;
|
||||
for (auto it = begin; it != end; ++it)
|
||||
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
|
||||
++res;
|
||||
return res;
|
||||
// reuse implementation that doesn't depend on case
|
||||
return PositionCaseSensitiveUTF8::countChars(begin, end);
|
||||
}
|
||||
|
||||
static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); }
|
||||
@ -151,12 +175,17 @@ template <typename Impl>
|
||||
struct PositionImpl
|
||||
{
|
||||
static constexpr bool use_default_implementation_for_constants = false;
|
||||
static constexpr bool supports_start_pos = true;
|
||||
|
||||
using ResultType = UInt64;
|
||||
|
||||
/// Find one substring in many strings.
|
||||
static void vectorConstant(
|
||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
const std::string & needle,
|
||||
const ColumnPtr & start_pos,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
const UInt8 * begin = data.data();
|
||||
const UInt8 * pos = begin;
|
||||
@ -176,13 +205,23 @@ struct PositionImpl
|
||||
res[i] = 0;
|
||||
++i;
|
||||
}
|
||||
auto start = start_pos != nullptr ? start_pos->getUInt(i) : 0;
|
||||
|
||||
/// We check that the entry does not pass through the boundaries of strings.
|
||||
if (pos + needle.size() < begin + offsets[i])
|
||||
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
|
||||
else
|
||||
if (pos + needle.size() < begin + offsets[i]) {
|
||||
auto res_pos = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
|
||||
if (res_pos < start) {
|
||||
pos = reinterpret_cast<const UInt8 *>(Impl::advancePos(
|
||||
reinterpret_cast<const char *>(pos),
|
||||
reinterpret_cast<const char *>(begin + offsets[i]),
|
||||
start - res_pos));
|
||||
continue;
|
||||
}
|
||||
res[i] = res_pos;
|
||||
}
|
||||
else {
|
||||
res[i] = 0;
|
||||
|
||||
}
|
||||
pos = begin + offsets[i];
|
||||
++i;
|
||||
}
|
||||
@ -192,24 +231,64 @@ struct PositionImpl
|
||||
}
|
||||
|
||||
/// Search for substring in string.
|
||||
static void constantConstant(std::string data, std::string needle, UInt64 & res)
|
||||
static void constantConstantScalar(
|
||||
std::string data,
|
||||
std::string needle,
|
||||
UInt64 start_pos,
|
||||
UInt64 & res)
|
||||
{
|
||||
Impl::toLowerIfNeed(data);
|
||||
Impl::toLowerIfNeed(needle);
|
||||
auto start = std::max(start_pos, 1ul);
|
||||
|
||||
res = data.find(needle);
|
||||
if (needle.size() == 0) {
|
||||
size_t haystack_size = Impl::countChars(data.data(), data.data() + data.size());
|
||||
res = start <= haystack_size + 1 ? start : 0;
|
||||
return;
|
||||
}
|
||||
|
||||
size_t start_byte = Impl::advancePos(data.data(), data.data() + data.size(), start - 1) - data.data();
|
||||
res = data.find(needle, start_byte);
|
||||
if (res == std::string::npos)
|
||||
res = 0;
|
||||
else
|
||||
res = 1 + Impl::countChars(data.data(), data.data() + res);
|
||||
}
|
||||
|
||||
/// Search for substring in string starting from different positions.
|
||||
static void constantConstant(
|
||||
std::string data,
|
||||
std::string needle,
|
||||
const ColumnPtr & start_pos,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
Impl::toLowerIfNeed(data);
|
||||
Impl::toLowerIfNeed(needle);
|
||||
|
||||
if (start_pos == nullptr) {
|
||||
constantConstantScalar(data, needle, 0, res[0]);
|
||||
return;
|
||||
}
|
||||
|
||||
size_t haystack_size = Impl::countChars(data.data(), data.data() + data.size());
|
||||
|
||||
size_t size = start_pos != nullptr ? start_pos->size() : 0;
|
||||
for (size_t i = 0; i < size; ++i) {
|
||||
auto start = start_pos->getUInt(i);
|
||||
|
||||
if (start > haystack_size + 1) {
|
||||
res[i] = 0;
|
||||
continue;
|
||||
}
|
||||
constantConstantScalar(data, needle, start, res[i]);
|
||||
}
|
||||
}
|
||||
|
||||
/// Search each time for a different single substring inside each time different string.
|
||||
static void vectorVector(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
const ColumnString::Offsets & haystack_offsets,
|
||||
const ColumnString::Chars & needle_data,
|
||||
const ColumnString::Offsets & needle_offsets,
|
||||
const ColumnPtr & start_pos,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
ColumnString::Offset prev_haystack_offset = 0;
|
||||
@ -222,10 +301,15 @@ struct PositionImpl
|
||||
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
||||
size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1;
|
||||
|
||||
if (0 == needle_size)
|
||||
auto start = start_pos != nullptr ? std::max(start_pos->getUInt(i), 1ul) : 1ul;
|
||||
|
||||
if (start > haystack_size + 1) {
|
||||
res[i] = 0;
|
||||
}
|
||||
else if (0 == needle_size)
|
||||
{
|
||||
/// An empty string is always at the very beginning of `haystack`.
|
||||
res[i] = 1;
|
||||
/// An empty string is always at any position in `haystack`.
|
||||
res[i] = start;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -234,8 +318,12 @@ struct PositionImpl
|
||||
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]),
|
||||
needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end
|
||||
|
||||
const char * beg = Impl::advancePos(
|
||||
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
|
||||
reinterpret_cast<const char *>(&haystack_data[haystack_offsets[i] - 1]),
|
||||
start - 1);
|
||||
/// searcher returns a pointer to the found substring or to the end of `haystack`.
|
||||
size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1])
|
||||
size_t pos = searcher.search(reinterpret_cast<const UInt8 *>(beg), &haystack_data[haystack_offsets[i] - 1])
|
||||
- &haystack_data[prev_haystack_offset];
|
||||
|
||||
if (pos != haystack_size)
|
||||
@ -259,6 +347,7 @@ struct PositionImpl
|
||||
const String & haystack,
|
||||
const ColumnString::Chars & needle_data,
|
||||
const ColumnString::Offsets & needle_offsets,
|
||||
const ColumnPtr & start_pos,
|
||||
PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
// NOTE You could use haystack indexing. But this is a rare case.
|
||||
@ -271,17 +360,23 @@ struct PositionImpl
|
||||
{
|
||||
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
||||
|
||||
if (0 == needle_size)
|
||||
auto start = start_pos != nullptr ? std::max(start_pos->getUInt(i), 1ul) : 1ul;
|
||||
|
||||
if (start > haystack.size() + 1) {
|
||||
res[i] = 0;
|
||||
}
|
||||
else if (0 == needle_size)
|
||||
{
|
||||
res[i] = 1;
|
||||
res[i] = start;
|
||||
}
|
||||
else
|
||||
{
|
||||
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
||||
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1);
|
||||
|
||||
const char * beg = Impl::advancePos(haystack.data(), haystack.data() + haystack.size(), start - 1);
|
||||
size_t pos = searcher.search(
|
||||
reinterpret_cast<const UInt8 *>(haystack.data()),
|
||||
reinterpret_cast<const UInt8 *>(beg),
|
||||
reinterpret_cast<const UInt8 *>(haystack.data()) + haystack.size())
|
||||
- reinterpret_cast<const UInt8 *>(haystack.data());
|
||||
|
||||
|
@ -23441,3 +23441,316 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
|
@ -6,6 +6,23 @@ select 1 = position('abc', 'abc');
|
||||
select 2 = position('abc', 'bc');
|
||||
select 3 = position('abc', 'c');
|
||||
|
||||
select 1 = position('', '', 0);
|
||||
select 1 = position('', '', 1);
|
||||
select 0 = position('', '', 2);
|
||||
select 1 = position('a', '', 1);
|
||||
select 2 = position('a', '', 2);
|
||||
select 0 = position('a', '', 3);
|
||||
|
||||
select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position('aaaa', '', number)) from numbers(10);
|
||||
select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position(materialize('aaaa'), '', number)) from numbers(10);
|
||||
select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position('aaaa', materialize(''), number)) from numbers(10);
|
||||
select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position(materialize('aaaa'), materialize(''), number)) from numbers(10);
|
||||
|
||||
select [1, 1, 2, 3, 4, 0, 0, 0, 0, 0] = groupArray(position('aaaa', 'a', number)) from numbers(10);
|
||||
select [1, 1, 2, 3, 4, 0, 0, 0, 0, 0] = groupArray(position(materialize('aaaa'), 'a', number)) from numbers(10);
|
||||
select [1, 1, 2, 3, 4, 0, 0, 0, 0, 0] = groupArray(position('aaaa', materialize('a'), number)) from numbers(10);
|
||||
select [1, 1, 2, 3, 4, 0, 0, 0, 0, 0] = groupArray(position(materialize('aaaa'), materialize('a'), number)) from numbers(10);
|
||||
|
||||
select 1 = position(materialize(''), '');
|
||||
select 1 = position(materialize('abc'), '');
|
||||
select 0 = position(materialize(''), 'abc');
|
||||
@ -27,6 +44,16 @@ select 1 = position('абв', 'абв');
|
||||
select 3 = position('абв', 'бв');
|
||||
select 5 = position('абв', 'в');
|
||||
|
||||
select 2 = position('abcabc', 'b', 0);
|
||||
select 2 = position('abcabc', 'b', 1);
|
||||
select 2 = position('abcabc', 'b', 2);
|
||||
select 5 = position('abcabc', 'b', 3);
|
||||
select 5 = position('abcabc', 'b', 4);
|
||||
select 5 = position('abcabc', 'b', 5);
|
||||
select 0 = position('abcabc', 'b', 6);
|
||||
select 2 = position('abcabc', 'bca', 0);
|
||||
select 0 = position('abcabc', 'bca', 3);
|
||||
|
||||
select 1 = position(materialize(''), '');
|
||||
select 1 = position(materialize('абв'), '');
|
||||
select 0 = position(materialize(''), 'абв');
|
||||
@ -48,6 +75,14 @@ select 1 = positionUTF8('абв', 'абв');
|
||||
select 2 = positionUTF8('абв', 'бв');
|
||||
select 3 = positionUTF8('абв', 'в');
|
||||
|
||||
select 3 = position('абвабв', 'б', 2);
|
||||
select 3 = position('абвабв', 'б', 3);
|
||||
select 3 = position('абвабв', 'бва', 2);
|
||||
select 9 = position('абвабв', 'б', 4);
|
||||
select 0 = position('абвабв', 'бва', 4);
|
||||
select 5 = position('абвабв', 'в', 0);
|
||||
select 11 = position('абвабв', 'в', 6);
|
||||
|
||||
select 1 = positionUTF8(materialize(''), '');
|
||||
select 1 = positionUTF8(materialize('абв'), '');
|
||||
select 0 = positionUTF8(materialize(''), 'абв');
|
||||
@ -62,6 +97,51 @@ select 1 = positionUTF8(materialize('абв'), 'абв') from system.numbers lim
|
||||
select 2 = positionUTF8(materialize('абв'), 'бв') from system.numbers limit 10;
|
||||
select 3 = positionUTF8(materialize('абв'), 'в') from system.numbers limit 10;
|
||||
|
||||
select 2 = positionUTF8('абвабв', 'б', 0);
|
||||
select 2 = positionUTF8('абвабв', 'б', 1);
|
||||
select 2 = positionUTF8('абвабв', 'б', 2);
|
||||
select 5 = positionUTF8('абвабв', 'б', 3);
|
||||
select 5 = positionUTF8('абвабв', 'б', 4);
|
||||
select 5 = positionUTF8('абвабв', 'б', 5);
|
||||
select 0 = positionUTF8('абвабв', 'б', 6);
|
||||
select 2 = positionUTF8('абвабв', 'бва', 0);
|
||||
select 0 = positionUTF8('абвабв', 'бва', 3);
|
||||
|
||||
select 2 = positionUTF8(materialize('абвабв'), 'б', 0) from system.numbers limit 10;
|
||||
select 2 = positionUTF8(materialize('абвабв'), 'б', 1) from system.numbers limit 10;
|
||||
select 2 = positionUTF8(materialize('абвабв'), 'б', 2) from system.numbers limit 10;
|
||||
select 5 = positionUTF8(materialize('абвабв'), 'б', 3) from system.numbers limit 10;
|
||||
select 5 = positionUTF8(materialize('абвабв'), 'б', 4) from system.numbers limit 10;
|
||||
select 5 = positionUTF8(materialize('абвабв'), 'б', 5) from system.numbers limit 10;
|
||||
select 0 = positionUTF8(materialize('абвабв'), 'б', 6) from system.numbers limit 10;
|
||||
select 2 = positionUTF8(materialize('абвабв'), 'бва', 0) from system.numbers limit 10;
|
||||
select 0 = positionUTF8(materialize('абвабв'), 'бва', 3) from system.numbers limit 10;
|
||||
|
||||
select 2 = positionUTF8('абвабв', materialize('б'), 0) from system.numbers limit 10;
|
||||
select 2 = positionUTF8('абвабв', materialize('б'), 1) from system.numbers limit 10;
|
||||
select 2 = positionUTF8('абвабв', materialize('б'), 2) from system.numbers limit 10;
|
||||
select 5 = positionUTF8('абвабв', materialize('б'), 3) from system.numbers limit 10;
|
||||
select 5 = positionUTF8('абвабв', materialize('б'), 4) from system.numbers limit 10;
|
||||
select 5 = positionUTF8('абвабв', materialize('б'), 5) from system.numbers limit 10;
|
||||
select 0 = positionUTF8('абвабв', materialize('б'), 6) from system.numbers limit 10;
|
||||
select 2 = positionUTF8('абвабв', materialize('бва'), 0) from system.numbers limit 10;
|
||||
select 0 = positionUTF8('абвабв', materialize('бва'), 3) from system.numbers limit 10;
|
||||
|
||||
select 2 = positionUTF8(materialize('абвабв'), materialize('б'), 0) from system.numbers limit 10;
|
||||
select 2 = positionUTF8(materialize('абвабв'), materialize('б'), 1) from system.numbers limit 10;
|
||||
select 2 = positionUTF8(materialize('абвабв'), materialize('б'), 2) from system.numbers limit 10;
|
||||
select 5 = positionUTF8(materialize('абвабв'), materialize('б'), 3) from system.numbers limit 10;
|
||||
select 5 = positionUTF8(materialize('абвабв'), materialize('б'), 4) from system.numbers limit 10;
|
||||
select 5 = positionUTF8(materialize('абвабв'), materialize('б'), 5) from system.numbers limit 10;
|
||||
select 0 = positionUTF8(materialize('абвабв'), materialize('б'), 6) from system.numbers limit 10;
|
||||
select 2 = positionUTF8(materialize('абвабв'), materialize('бва'), 0) from system.numbers limit 10;
|
||||
select 0 = positionUTF8(materialize('абвабв'), materialize('бва'), 3) from system.numbers limit 10;
|
||||
|
||||
select [2, 2, 2, 5, 5, 5, 0, 0, 0, 0] = groupArray(positionUTF8(materialize('абвабв'), materialize('б'), number)) from numbers(10);
|
||||
select [2, 2, 2, 5, 5, 5, 0, 0, 0, 0] = groupArray(positionUTF8('абвабв', materialize('б'), number)) from numbers(10);
|
||||
select [2, 2, 2, 5, 5, 5, 0, 0, 0, 0] = groupArray(positionUTF8('абвабв', 'б', number)) from numbers(10);
|
||||
select [2, 2, 2, 5, 5, 5, 0, 0, 0, 0] = groupArray(positionUTF8(materialize('абвабв'), 'б', number)) from numbers(10);
|
||||
|
||||
select 1 = positionCaseInsensitive('', '');
|
||||
select 1 = positionCaseInsensitive('abc', '');
|
||||
select 0 = positionCaseInsensitive('', 'aBc');
|
||||
|
Loading…
Reference in New Issue
Block a user