ILIKE operator (#12125)

* Integrated CachingAllocator into MarkCache

* fixed build errors

* reset func hotfix

* upd: Fixing build

* updated submodules links

* fix 2

* updating grabber allocator proto

* updating lost work

* updating CMake to use concepts

* some other changes to get it building (integration into MarkCache)

* further integration into caches

* updated Async metrics, fixed some build errors

* and some other errors revealing

* added perfect forwarding to some functions

* fix: forward template

* fix: constexpr modifier

* fix: FakePODAllocator missing member func

* updated PODArray constructor taking alloc params

* fix: PODArray overload with n restored

* fix: FakePODAlloc duplicating alloc() func

* added constexpr variable for alloc_tag_t

* split cache values by allocators, provided updates

* fix: memcpy

* fix: constexpr modifier

* fix: noexcept modifier

* fix: alloc_tag_t for PODArray constructor

* fix: PODArray copy ctor with different alloc

* fix: resize() signature

* updating to lastest working master

* syncing with 273267

* first draft version

* fix: update Searcher to case-insensitive

* added ILIKE test

* fixed style errors, updated test, split like and ilike,  added notILike

* replaced inconsistent comments

* fixed show tables ilike

* updated missing test cases

* regenerated ya.make

* Update 01355_ilike.sql

Co-authored-by: myrrc <me-clickhouse@myrrec.space>
Co-authored-by: alexey-milovidov <milovidov@yandex-team.ru>
This commit is contained in:
myrrc 2020-07-05 18:57:59 +03:00 committed by GitHub
parent ec563e54dc
commit 8c3417fbf7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 232 additions and 32 deletions

View File

@ -79,7 +79,7 @@ Suggest::Suggest()
"IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE",
"PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE",
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED",
"INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP"};
"INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE"};
}
void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit)

View File

@ -1,5 +1,6 @@
#pragma once
#include <type_traits>
#include <common/types.h>
#include <Common/Volnitsky.h>
#include <Columns/ColumnString.h>
@ -27,7 +28,7 @@ namespace ErrorCodes
}
/// Is the LIKE expression reduced to finding a substring in a string?
/// Is the [I]LIKE expression reduced to finding a substring in a string?
static inline bool likePatternIsStrstr(const String & pattern, String & res)
{
res = "";
@ -67,17 +68,21 @@ static inline bool likePatternIsStrstr(const String & pattern, String & res)
return true;
}
/** 'like' - if true, treat pattern as SQL LIKE; if false - treat pattern as re2 regexp.
/** 'like' - if true, treat pattern as SQL LIKE or ILIKE; if false - treat pattern as re2 regexp.
* NOTE: We want to run regexp search for whole block by one call (as implemented in function 'position')
* but for that, regexp engine must support \0 bytes and their interpretation as string boundaries.
*/
template <bool like, bool revert = false>
template <bool like, bool revert = false, bool case_insensitive = false>
struct MatchImpl
{
static constexpr bool use_default_implementation_for_constants = true;
using ResultType = UInt8;
using Searcher = std::conditional_t<case_insensitive,
VolnitskyCaseInsensitiveUTF8,
VolnitskyUTF8>;
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
{
@ -85,7 +90,8 @@ struct MatchImpl
return;
String strstr_pattern;
/// A simple case where the LIKE expression reduces to finding a substring in a string
/// A simple case where the [I]LIKE expression reduces to finding a substring in a string
if (like && likePatternIsStrstr(pattern, strstr_pattern))
{
const UInt8 * begin = data.data();
@ -96,7 +102,7 @@ struct MatchImpl
size_t i = 0;
/// TODO You need to make that `searcher` is common to all the calls of the function.
Volnitsky searcher(strstr_pattern.data(), strstr_pattern.size(), end - pos);
Searcher searcher(strstr_pattern.data(), strstr_pattern.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
@ -126,7 +132,10 @@ struct MatchImpl
{
size_t size = offsets.size();
auto regexp = Regexps::get<like, true>(pattern);
constexpr int flags = case_insensitive ?
Regexps::Regexp::RE_CASELESS : 0;
auto regexp = Regexps::get<like, true>(pattern, flags);
std::string required_substring;
bool is_trivial;
@ -170,7 +179,7 @@ struct MatchImpl
/// The current index in the array of strings.
size_t i = 0;
Volnitsky searcher(required_substring.data(), required_substring.size(), end - pos);
Searcher searcher(required_substring.data(), required_substring.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
@ -248,7 +257,7 @@ struct MatchImpl
/// If pattern is larger than string size - it cannot be found.
if (strstr_pattern.size() <= n)
{
Volnitsky searcher(strstr_pattern.data(), strstr_pattern.size(), end - pos);
Searcher searcher(strstr_pattern.data(), strstr_pattern.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
@ -328,7 +337,7 @@ struct MatchImpl
/// If required substring is larger than string size - it cannot be found.
if (strstr_pattern.size() <= n)
{
Volnitsky searcher(required_substring.data(), required_substring.size(), end - pos);
Searcher searcher(required_substring.data(), required_substring.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))

View File

@ -59,19 +59,20 @@ namespace Regexps
* In destructor, it returns the object back to the Pool for further reuse.
*/
template <bool like, bool no_capture>
inline Pool::Pointer get(const std::string & pattern)
inline Pool::Pointer get(const std::string & pattern, int flags = 0)
{
/// C++11 has thread-safe function-local statics on most modern compilers.
static Pool known_regexps; /// Different variables for different pattern parameters.
return known_regexps.get(pattern, [&pattern]
return known_regexps.get(pattern, [flags, &pattern]
{
int flags = OptimizedRegularExpression::RE_DOT_NL;
int flags_final = flags | OptimizedRegularExpression::RE_DOT_NL;
if (no_capture)
flags |= OptimizedRegularExpression::RE_NO_CAPTURE;
flags_final |= OptimizedRegularExpression::RE_NO_CAPTURE;
ProfileEvents::increment(ProfileEvents::RegexpCreated);
return new Regexp{createRegexp<like>(pattern, flags)};
return new Regexp{createRegexp<like>(pattern, flags_final)};
});
}
}

24
src/Functions/ilike.cpp Normal file
View File

@ -0,0 +1,24 @@
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "MatchImpl.h"
namespace DB
{
struct NameILike
{
static constexpr auto name = "ilike";
};
namespace
{
using ILikeImpl = MatchImpl<true, false, /*case-insensitive*/true>;
}
using FunctionILike = FunctionsStringSearch<ILikeImpl, NameILike>;
void registerFunctionILike(FunctionFactory & factory)
{
factory.registerFunction<FunctionILike>();
}
}

View File

@ -11,11 +11,15 @@ struct NameLike
static constexpr auto name = "like";
};
using FunctionLike = FunctionsStringSearch<MatchImpl<true>, NameLike>;
namespace
{
using LikeImpl = MatchImpl</*SQL LIKE */ true, /*revert*/false>;
}
using FunctionLike = FunctionsStringSearch<LikeImpl, NameLike>;
void registerFunctionLike(FunctionFactory & factory)
{
factory.registerFunction<FunctionLike>();
}
}

View File

@ -4,7 +4,8 @@
namespace DB
{
/// Transforms the LIKE expression into regexp re2. For example, abc%def -> ^abc.*def$
/// Transforms the [I]LIKE expression into regexp re2. For example, abc%def -> ^abc.*def$
inline String likePatternToRegexp(const String & pattern)
{
String res;

View File

@ -0,0 +1,24 @@
#include "FunctionsStringSearch.h"
#include "FunctionFactory.h"
#include "MatchImpl.h"
namespace DB
{
struct NameNotILike
{
static constexpr auto name = "notILike";
};
namespace
{
using NotILikeImpl = MatchImpl<true, true, /*case-insensitive*/true>;
}
using FunctionNotILike = FunctionsStringSearch<NotILikeImpl, NameNotILike>;
void registerFunctionNotILike(FunctionFactory & factory)
{
factory.registerFunction<FunctionNotILike>();
}
}

View File

@ -4,7 +4,9 @@ namespace DB
class FunctionFactory;
void registerFunctionLike(FunctionFactory &);
void registerFunctionILike(FunctionFactory &);
void registerFunctionNotLike(FunctionFactory &);
void registerFunctionNotILike(FunctionFactory &);
void registerFunctionMatch(FunctionFactory &);
void registerFunctionExtract(FunctionFactory &);
void registerFunctionReplaceOne(FunctionFactory &);
@ -24,7 +26,9 @@ void registerFunctionExtractAllGroupsHorizontal(FunctionFactory &);
void registerFunctionsStringRegexp(FunctionFactory & factory)
{
registerFunctionLike(factory);
registerFunctionILike(factory);
registerFunctionNotLike(factory);
registerFunctionNotILike(factory);
registerFunctionMatch(factory);
registerFunctionExtract(factory);
registerFunctionReplaceOne(factory);
@ -41,6 +45,4 @@ void registerFunctionsStringRegexp(FunctionFactory & factory)
registerFunctionExtractAllGroupsVertical(factory);
registerFunctionExtractAllGroupsHorizontal(factory);
}
}

View File

@ -92,8 +92,8 @@ SRCS(
array/emptyArrayToSingle.cpp
array/hasAll.cpp
array/hasAny.cpp
array/hasSubstr.cpp
array/has.cpp
array/hasSubstr.cpp
array/indexOf.cpp
array/length.cpp
array/range.cpp
@ -228,6 +228,7 @@ SRCS(
ifNull.cpp
IFunction.cpp
ignore.cpp
ilike.cpp
in.cpp
intDiv.cpp
intDivOrZero.cpp
@ -288,6 +289,7 @@ SRCS(
neighbor.cpp
notEmpty.cpp
notEquals.cpp
notILike.cpp
notLike.cpp
now64.cpp
now.cpp

View File

@ -132,7 +132,7 @@ public:
{
/// leave other comparisons as is
}
else if (functionIsLikeOperator(node.name) || /// LIKE, NOT LIKE
else if (functionIsLikeOperator(node.name) || /// LIKE, NOT LIKE, ILIKE, NOT ILIKE
functionIsInOperator(node.name)) /// IN, NOT IN
{
/// leave as is. It's not possible to make push down here cause of unknown aliases and not implemented JOIN predicates.

View File

@ -17,7 +17,7 @@ inline bool functionIsInOrGlobalInOperator(const std::string & name)
inline bool functionIsLikeOperator(const std::string & name)
{
return name == "like" || name == "notLike";
return name == "like" || name == "ilike" || name == "notLike" || name == "notILike";
}
inline bool functionIsJoinGet(const std::string & name)

View File

@ -54,7 +54,7 @@ ASTPtr ASTFunction::clone() const
}
/** A special hack. If it's LIKE or NOT LIKE expression and the right hand side is a string literal,
/** A special hack. If it's [I]LIKE or NOT [I]LIKE expression and the right hand side is a string literal,
* we will highlight unescaped metacharacters % and _ in string literal for convenience.
* Motivation: most people are unaware that _ is a metacharacter and forgot to properly escape it with two backslashes.
* With highlighting we make it clearly obvious.
@ -168,7 +168,9 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
"greater", " > ",
"equals", " = ",
"like", " LIKE ",
"ilike", " ILIKE ",
"notLike", " NOT LIKE ",
"notILike", " NOT ILIKE ",
"in", " IN ",
"notIn", " NOT IN ",
"globalIn", " GLOBAL IN ",
@ -186,7 +188,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
bool special_hilite = settings.hilite
&& (name == "like" || name == "notLike")
&& (name == "like" || name == "notLike" || name == "ilike" || name == "notILike")
&& highlightStringLiteralWithMetacharacters(arguments->children[1], settings, "%_");
/// Format x IN 1 as x IN (1): put parens around rhs even if there is a single element in set.

View File

@ -22,8 +22,13 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format
else if (clusters)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW CLUSTERS" << (settings.hilite ? hilite_none : "");
if (!like.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << (not_like ? " NOT" : "") << " LIKE " << (settings.hilite ? hilite_none : "")
settings.ostr
<< (settings.hilite ? hilite_keyword : "")
<< (not_like ? " NOT" : "")
<< (case_insensitive_like ? " ILIKE " : " LIKE ")
<< (settings.hilite ? hilite_none : "")
<< std::quoted(like, '\'');
if (limit_length)
@ -47,8 +52,13 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format
<< backQuoteIfNeed(from);
if (!like.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << (not_like ? " NOT" : "") << " LIKE " << (settings.hilite ? hilite_none : "")
settings.ostr
<< (settings.hilite ? hilite_keyword : "")
<< (not_like ? " NOT" : "")
<< (case_insensitive_like ? " ILIKE " : " LIKE ")
<< (settings.hilite ? hilite_none : "")
<< std::quoted(like, '\'');
else if (where_expression)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");

View File

@ -19,10 +19,14 @@ public:
bool cluster{false};
bool dictionaries{false};
bool temporary{false};
String cluster_str;
String from;
String like;
bool not_like{false};
bool case_insensitive_like{false};
ASTPtr where_expression;
ASTPtr limit_length;

View File

@ -1115,6 +1115,7 @@ const char * ParserAlias::restricted_keywords[] =
"NOT",
"BETWEEN",
"LIKE",
"ILIKE",
nullptr
};

View File

@ -47,7 +47,9 @@ const char * ParserComparisonExpression::operators[] =
">", "greater",
"=", "equals",
"LIKE", "like",
"ILIKE", "ilike",
"NOT LIKE", "notLike",
"NOT ILIKE", "notILike",
"IN", "in",
"NOT IN", "notIn",
"GLOBAL IN", "globalIn",

View File

@ -28,6 +28,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ParserKeyword s_in("IN");
ParserKeyword s_not("NOT");
ParserKeyword s_like("LIKE");
ParserKeyword s_ilike("ILIKE");
ParserKeyword s_where("WHERE");
ParserKeyword s_limit("LIMIT");
ParserStringLiteral like_p;
@ -53,8 +54,11 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (s_not.ignore(pos, expected))
query->not_like = true;
if (s_like.ignore(pos, expected))
if (bool insensitive = s_ilike.ignore(pos, expected); insensitive || s_like.ignore(pos, expected))
{
if (insensitive)
query->case_insensitive_like = true;
if (!like_p.parse(pos, like, expected))
return false;
}
@ -98,8 +102,11 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (s_not.ignore(pos, expected))
query->not_like = true;
if (s_like.ignore(pos, expected))
if (bool insensitive = s_ilike.ignore(pos, expected); insensitive || s_like.ignore(pos, expected))
{
if (insensitive)
query->case_insensitive_like = true;
if (!like_p.parse(pos, like, expected))
return false;
}
@ -119,6 +126,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
tryGetIdentifierNameInto(database, query->from);
if (like)
query->like = safeGet<const String &>(like->as<ASTLiteral &>().value);

View File

@ -7,14 +7,14 @@ namespace DB
{
/** Query like this:
* SHOW TABLES [FROM db] [[NOT] LIKE 'str'] [LIMIT expr]
* SHOW TABLES [FROM db] [[NOT] [I]LIKE 'str'] [LIMIT expr]
* or
* SHOW DATABASES.
*/
class ParserShowTablesQuery : public IParserBase
{
protected:
const char * getName() const override { return "SHOW [TEMPORARY] TABLES|DATABASES|CLUSTERS|CLUSTER 'name' [[NOT] LIKE 'str'] [LIMIT expr]"; }
const char * getName() const override { return "SHOW [TEMPORARY] TABLES|DATABASES|CLUSTERS|CLUSTER 'name' [[NOT] [I]LIKE 'str'] [LIMIT expr]"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};

View File

@ -841,6 +841,7 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
func_name = "greaterOrEquals";
else if (func_name == "in" || func_name == "notIn" ||
func_name == "like" || func_name == "notLike" ||
func_name == "ilike" || func_name == "notIlike" ||
func_name == "startsWith")
{
/// "const IN data_column" doesn't make sense (unlike "data_column IN const")

View File

@ -20,7 +20,7 @@ using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
/** A field, that can be stored in two reperesenations:
/** A field, that can be stored in two representations:
* - A standalone field.
* - A field with reference to its position in a block.
* It's needed for execution of functions on ranges during

View File

@ -0,0 +1,44 @@
0
1
1
1
1
0
1
1
1
1
1
1
0
0
0
0
0
1
1
1
0
0
1
1
1
1
1
1
1
1
1
1
0
0
1
1
0
1
1
1
1
1
test1
test2

View File

@ -0,0 +1,61 @@
SELECT 'Hello' ILIKE '';
SELECT 'Hello' ILIKE '%';
SELECT 'Hello' ILIKE '%%';
SELECT 'Hello' ILIKE '%%%';
SELECT 'Hello' ILIKE '%_%';
SELECT 'Hello' ILIKE '_';
SELECT 'Hello' ILIKE '_%';
SELECT 'Hello' ILIKE '%_';
SELECT 'Hello' ILIKE 'H%o';
SELECT 'hello' ILIKE 'H%o';
SELECT 'hello' ILIKE 'h%o';
SELECT 'Hello' ILIKE 'h%o';
SELECT 'Hello' NOT ILIKE 'H%o';
SELECT 'hello' NOT ILIKE 'H%o';
SELECT 'hello' NOT ILIKE 'h%o';
SELECT 'Hello' NOT ILIKE 'h%o';
SELECT 'OHello' ILIKE '%lhell%';
SELECT 'Ohello' ILIKE '%hell%';
SELECT 'hEllo' ILIKE '%HEL%';
SELECT 'OHello' NOT ILIKE '%lhell%';
SELECT 'Ohello' NOT ILIKE '%hell%';
SELECT 'hEllo' NOT ILIKE '%HEL%';
SELECT materialize('prepre_f') ILIKE '%pre_f%';
SELECT 'abcdef' ILIKE '%aBc%def%';
SELECT 'ABCDDEF' ILIKE '%abc%def%';
SELECT 'Abc\nDef' ILIKE '%abc%def%';
SELECT 'abc\ntdef' ILIKE '%abc%def%';
SELECT 'abct\ndef' ILIKE '%abc%dEf%';
SELECT 'abc\n\ndeF' ILIKE '%abc%def%';
SELECT 'abc\n\ntdef' ILIKE '%abc%deF%';
SELECT 'Abc\nt\ndef' ILIKE '%abc%def%';
SELECT 'abct\n\ndef' ILIKE '%abc%def%';
SELECT 'ab\ndef' ILIKE '%Abc%def%';
SELECT 'aBc\nef' ILIKE '%ABC%DEF%';
SELECT CAST('hello' AS FixedString(5)) ILIKE '%he%o%';
SELECT 'ёЁё' ILIKE 'Ё%Ё';
SELECT 'ощщЁё' ILIKE 'Щ%Ё';
SELECT 'ощЩЁё' ILIKE '%Щ%Ё';
SELECT 'Щущпандер' ILIKE '%щп%е%';
SELECT 'Щущпандер' ILIKE '%щП%е%';
SELECT 'ощщЁё' ILIKE '%щ%';
SELECT 'ощЩЁё' ILIKE '%ё%';
SHOW TABLES NOT ILIKE '%';
DROP DATABASE IF EXISTS test_01355;
CREATE DATABASE test_01355;
USE test_01355;
CREATE TABLE test1 (x UInt8) ENGINE = Memory;
CREATE TABLE test2 (x UInt8) ENGINE = Memory;
SHOW TABLES ILIKE 'tES%';
SHOW TABLES NOT ILIKE 'TeS%';
DROP DATABASE test_01355;