dbms: development [#CONV-2944].

This commit is contained in:
Alexey Milovidov 2011-10-17 08:28:39 +00:00
parent 05bac9a719
commit db4ad9135e
5 changed files with 207 additions and 27 deletions

View File

@ -16,11 +16,16 @@ namespace DB
*
* position(haystack, needle) - обычный поиск подстроки в строке, возвращает позицию (в байтах) найденной подстроки, начиная с 1, или 0, если подстрока не найдена.
* positionUTF8(haystack, needle) - то же самое, но позиция вычисляется в кодовых точках, при условии, что строка в кодировке UTF-8.
* like(haystack, pattern) - поиск по регулярному выражению LIKE; возвращает 0 или 1.
*
* like(haystack, pattern) - поиск по регулярному выражению LIKE; возвращает 0 или 1. Регистронезависимое, но только для латиницы.
* notLike(haystack, pattern)
*
* match(haystack, pattern) - поиск по регулярному выражению re2; возвращает 0 или 1.
*
* extract(haystack, pattern) - вынимает первый subpattern, (или нулевой, если первого нет) согласно регулярному выражению re2;
* возвращает пустую строку, если не матчится.
* extract(haystack, pattern, n) - вынимает n-ый subpattern; возвращает пустую строку, если не матчится.
*
* replaceOne(haystack, pattern, replacement) - замена шаблона по заданным правилам, только первое вхождение.
* replaceAll(haystack, pattern, replacement) - замена шаблона по заданным правилам, все вхождения.
*
@ -30,6 +35,9 @@ namespace DB
struct PositionImpl
{
typedef UInt64 ResultType;
/// Предполагается, что res нужного размера и инициализирован нулями.
static void vector(const std::vector<UInt8> & data, const std::vector<size_t> & offsets,
const std::string & needle,
std::vector<UInt64> & res)
@ -46,23 +54,15 @@ struct PositionImpl
{
/// Определим, к какому индексу оно относится.
while (begin + offsets[i] < pos)
{
res[i] = 0;
++i;
}
/// Проверяем, что вхождение не переходит через границы строк.
if (pos + needle.size() < begin + offsets[i])
res[i] = (i != 0) ? pos - begin - offsets[i - 1] + 1 : (pos - begin + 1);
else
res[i] = 0;
pos = begin + offsets[i];
++i;
}
for (size_t size = offsets.size(); i < size; ++i)
res[i] = 0;
}
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
@ -76,13 +76,173 @@ struct PositionImpl
};
class FunctionPosition : public IFunction
struct PositionUTF8Impl
{
typedef UInt64 ResultType;
static void vector(const std::vector<UInt8> & data, const std::vector<size_t> & offsets,
const std::string & needle,
std::vector<UInt64> & res)
{
const UInt8 * begin = &data[0];
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// Текущий индекс в массиве строк.
size_t i = 0;
/// Искать будем следующее вхождение сразу во всех строках.
while (pos < end && NULL != (pos = reinterpret_cast<UInt8 *>(memmem(pos, end - pos, needle.data(), needle.size()))))
{
/// Определим, к какому индексу оно относится.
while (begin + offsets[i] < pos)
++i;
/// Проверяем, что вхождение не переходит через границы строк.
if (pos + needle.size() < begin + offsets[i])
{
/// А теперь надо найти, сколько кодовых точек находится перед pos.
res[i] = 1;
for (const UInt8 * c = begin + (i != 0 ? offsets[i - 1] : 0); c < pos; ++c)
if (*c <= 0x7F || *c >= 0xC0)
++res[i];
}
pos = begin + offsets[i];
++i;
}
}
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
{
res = data.find(needle);
if (res == std::string::npos)
res = 0;
else
++res;
}
};
inline String likePatternToRegexp(const String & pattern)
{
String res = "^";
res.reserve(pattern.size() * 2);
const char * pos = pattern.data();
const char * end = pos + pattern.size();
while (pos < end)
{
switch (*pos)
{
case '^': case '$': case '.': case '[': case '|': case '(': case ')': case '?': case '*': case '+': case '{':
res += '\\';
res += *pos;
break;
case '%':
res += ".*";
break;
case '_':
res += ".";
break;
case '\\':
++pos;
if (pos == end)
res += "\\\\";
else
{
if (*pos == '%' || *pos == '_')
res += *pos;
else
{
res += '\\';
res += *pos;
}
}
break;
default:
res += *pos;
break;
}
++pos;
}
res += '$';
return res;
}
struct Regexps
{
typedef std::map<String, OptimizedRegularExpression> KnownRegexps;
static const OptimizedRegularExpression & get(const std::string & pattern)
{
/// В GCC thread safe statics.
static KnownRegexps known_regexps;
static Poco::FastMutex mutex;
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
KnownRegexps::const_iterator it = known_regexps.find(pattern);
if (known_regexps.end() == it)
it = known_regexps.insert(std::make_pair(pattern, OptimizedRegularExpression(pattern))).first;
return it->second;
}
static const OptimizedRegularExpression & getLike(const std::string & pattern)
{
/// В GCC thread safe statics.
static KnownRegexps known_regexps;
static Poco::FastMutex mutex;
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
KnownRegexps::const_iterator it = known_regexps.find(pattern);
if (known_regexps.end() == it)
it = known_regexps.insert(std::make_pair(pattern, OptimizedRegularExpression(likePatternToRegexp(pattern), OptimizedRegularExpression::RE_CASELESS))).first;
return it->second;
}
};
/** like - использовать выражения LIKE, если true; использовать выражения re2, если false.
* Замечание: хотелось бы запускать регексп сразу над всем массивом, аналогично функции position,
* но для этого пришлось бы сделать поддержку символов \0 в движке регулярных выражений,
* и их интерпретацию как начал и концов строк.
*/
template <bool like, bool revert = false>
struct MatchImpl
{
typedef UInt8 ResultType;
static void vector(const std::vector<UInt8> & data, const std::vector<size_t> & offsets,
const std::string & pattern,
std::vector<UInt8> & res)
{
const OptimizedRegularExpression & regexp = like ? Regexps::getLike(pattern) : Regexps::get(pattern);
size_t size = offsets.size();
for (size_t i = 0; i < size; ++i)
res[i] = revert ^ regexp.match(reinterpret_cast<const char *>(&data[i != 0 ? offsets[i - 1] : 0]), i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]);
}
static void constant(const std::string & data, const std::string & pattern, UInt8 & res)
{
const OptimizedRegularExpression & regexp = like ? Regexps::getLike(pattern) : Regexps::get(pattern);
res = revert ^ regexp.match(data);
}
};
template <typename Impl, typename Name>
class FunctionsStringSearch : public IFunction
{
public:
/// Получить имя функции.
String getName() const
{
return "position";
return Name::get();
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
@ -101,13 +261,13 @@ public:
throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return new DataTypeUInt64;
return new typename DataTypeFromFieldType<typename Impl::ResultType>::Type;
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
typedef UInt64 ResultType;
typedef typename Impl::ResultType ResultType;
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
@ -121,14 +281,14 @@ public:
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
block.getByPosition(result).column = col_res;
ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
vec_res.resize(col->size());
PositionImpl::vector(dynamic_cast<const ColumnUInt8 &>(col->getData()).getData(), col->getOffsets(), col_needle->getData(), vec_res);
Impl::vector(dynamic_cast<const ColumnUInt8 &>(col->getData()).getData(), col->getOffsets(), col_needle->getData(), vec_res);
}
else if (const ColumnConstString * col = dynamic_cast<const ColumnConstString *>(&*column))
{
ResultType res = 0;
PositionImpl::constant(col->getData(), col_needle->getData(), res);
Impl::constant(col->getData(), col_needle->getData(), res);
ColumnConst<ResultType> * col_res = new ColumnConst<ResultType>(col->size(), res);
block.getByPosition(result).column = col_res;
@ -140,4 +300,17 @@ public:
}
};
struct NamePosition { static const char * get() { return "position"; } };
struct NamePositionUTF8 { static const char * get() { return "positionUTF8"; } };
struct NameMatch { static const char * get() { return "match"; } };
struct NameLike { static const char * get() { return "like"; } };
struct NameNotLike { static const char * get() { return "notLike"; } };
typedef FunctionsStringSearch<PositionImpl, NamePosition> FunctionPosition;
typedef FunctionsStringSearch<PositionUTF8Impl, NamePositionUTF8> FunctionPositionUTF8;
typedef FunctionsStringSearch<MatchImpl<false>, NameMatch> FunctionMatch;
typedef FunctionsStringSearch<MatchImpl<true>, NameLike> FunctionLike;
typedef FunctionsStringSearch<MatchImpl<true, true>, NameNotLike> FunctionNotLike;
}

View File

@ -15,7 +15,8 @@ namespace DB
* В отличие от std::istream, предоставляет доступ к внутреннему буферу,
* а также позволяет вручную управлять позицией внутри буфера.
*
* Замечание! Используется char *, а не const char * (для того, чтобы можно было вынести общий код в BufferBase).
* Замечание! Используется char *, а не const char *
* (для того, чтобы можно было вынести общий код в BufferBase, а также для того, чтобы можно было заполнять буфер новыми данными).
* Это вызывает неудобства - например, при использовании ReadBuffer для чтения из куска памяти const char *,
* приходится использовать const_cast.
*

View File

@ -186,14 +186,16 @@ public:
ParserComparisonExpression()
: elem_parser(new ParserAdditiveExpression),
operator_parser(boost::assign::map_list_of
("==", "equals")
("!=", "notEquals")
("<>", "notEquals")
("<=", "lessOrEquals")
(">=", "greaterOrEquals")
("<", "less")
(">", "greater")
("=", "equals"),
("==", "equals")
("!=", "notEquals")
("<>", "notEquals")
("<=", "lessOrEquals")
(">=", "greaterOrEquals")
("<", "less")
(">", "greater")
("=", "equals")
("LIKE", "like")
("NOT LIKE", "notLike"),
elem_parser)
{
}

View File

@ -178,6 +178,10 @@ int main(int argc, char ** argv)
("toTime", new DB::FunctionToTime)
("position", new DB::FunctionPosition)
("positionUTF8", new DB::FunctionPositionUTF8)
("match", new DB::FunctionMatch)
("like", new DB::FunctionLike)
("notLike", new DB::FunctionNotLike)
;
context.aggregate_function_factory = new DB::AggregateFunctionFactory;

View File

@ -78,7 +78,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP
Operators_t::const_iterator it;
for (it = operators.begin(); it != operators.end(); ++it)
{
ParserString op(it->first, true);
ParserString op(it->first, true, true);
if (op.ignore(pos, end, expected))
break;
}
@ -144,7 +144,7 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr &
Operators_t::const_iterator it;
for (it = operators.begin(); it != operators.end(); ++it)
{
ParserString op(it->first, true);
ParserString op(it->first, true, true);
if (op.ignore(pos, end, expected))
break;
}