mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 04:12:19 +00:00
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:
parent
ec563e54dc
commit
8c3417fbf7
@ -79,7 +79,7 @@ Suggest::Suggest()
|
|||||||
"IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE",
|
"IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE",
|
||||||
"PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE",
|
"PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE",
|
||||||
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED",
|
"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)
|
void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit)
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <type_traits>
|
||||||
#include <common/types.h>
|
#include <common/types.h>
|
||||||
#include <Common/Volnitsky.h>
|
#include <Common/Volnitsky.h>
|
||||||
#include <Columns/ColumnString.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)
|
static inline bool likePatternIsStrstr(const String & pattern, String & res)
|
||||||
{
|
{
|
||||||
res = "";
|
res = "";
|
||||||
@ -67,17 +68,21 @@ static inline bool likePatternIsStrstr(const String & pattern, String & res)
|
|||||||
return true;
|
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')
|
* 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.
|
* 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
|
struct MatchImpl
|
||||||
{
|
{
|
||||||
static constexpr bool use_default_implementation_for_constants = true;
|
static constexpr bool use_default_implementation_for_constants = true;
|
||||||
|
|
||||||
using ResultType = UInt8;
|
using ResultType = UInt8;
|
||||||
|
|
||||||
|
using Searcher = std::conditional_t<case_insensitive,
|
||||||
|
VolnitskyCaseInsensitiveUTF8,
|
||||||
|
VolnitskyUTF8>;
|
||||||
|
|
||||||
static void vectorConstant(
|
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, PaddedPODArray<UInt8> & res)
|
||||||
{
|
{
|
||||||
@ -85,7 +90,8 @@ struct MatchImpl
|
|||||||
return;
|
return;
|
||||||
|
|
||||||
String strstr_pattern;
|
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))
|
if (like && likePatternIsStrstr(pattern, strstr_pattern))
|
||||||
{
|
{
|
||||||
const UInt8 * begin = data.data();
|
const UInt8 * begin = data.data();
|
||||||
@ -96,7 +102,7 @@ struct MatchImpl
|
|||||||
size_t i = 0;
|
size_t i = 0;
|
||||||
|
|
||||||
/// TODO You need to make that `searcher` is common to all the calls of the function.
|
/// 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.
|
/// We will search for the next occurrence in all rows at once.
|
||||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||||
@ -126,7 +132,10 @@ struct MatchImpl
|
|||||||
{
|
{
|
||||||
size_t size = offsets.size();
|
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;
|
std::string required_substring;
|
||||||
bool is_trivial;
|
bool is_trivial;
|
||||||
@ -170,7 +179,7 @@ struct MatchImpl
|
|||||||
/// The current index in the array of strings.
|
/// The current index in the array of strings.
|
||||||
size_t i = 0;
|
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.
|
/// We will search for the next occurrence in all rows at once.
|
||||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
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 pattern is larger than string size - it cannot be found.
|
||||||
if (strstr_pattern.size() <= n)
|
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.
|
/// We will search for the next occurrence in all rows at once.
|
||||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
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 required substring is larger than string size - it cannot be found.
|
||||||
if (strstr_pattern.size() <= n)
|
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.
|
/// We will search for the next occurrence in all rows at once.
|
||||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||||
|
@ -59,19 +59,20 @@ namespace Regexps
|
|||||||
* In destructor, it returns the object back to the Pool for further reuse.
|
* In destructor, it returns the object back to the Pool for further reuse.
|
||||||
*/
|
*/
|
||||||
template <bool like, bool no_capture>
|
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.
|
/// C++11 has thread-safe function-local statics on most modern compilers.
|
||||||
static Pool known_regexps; /// Different variables for different pattern parameters.
|
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)
|
if (no_capture)
|
||||||
flags |= OptimizedRegularExpression::RE_NO_CAPTURE;
|
flags_final |= OptimizedRegularExpression::RE_NO_CAPTURE;
|
||||||
|
|
||||||
ProfileEvents::increment(ProfileEvents::RegexpCreated);
|
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
24
src/Functions/ilike.cpp
Normal 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>();
|
||||||
|
}
|
||||||
|
}
|
@ -11,11 +11,15 @@ struct NameLike
|
|||||||
static constexpr auto name = "like";
|
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)
|
void registerFunctionLike(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerFunction<FunctionLike>();
|
factory.registerFunction<FunctionLike>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -4,7 +4,8 @@
|
|||||||
|
|
||||||
namespace DB
|
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)
|
inline String likePatternToRegexp(const String & pattern)
|
||||||
{
|
{
|
||||||
String res;
|
String res;
|
||||||
|
24
src/Functions/notILike.cpp
Normal file
24
src/Functions/notILike.cpp
Normal 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>();
|
||||||
|
}
|
||||||
|
}
|
@ -4,7 +4,9 @@ namespace DB
|
|||||||
class FunctionFactory;
|
class FunctionFactory;
|
||||||
|
|
||||||
void registerFunctionLike(FunctionFactory &);
|
void registerFunctionLike(FunctionFactory &);
|
||||||
|
void registerFunctionILike(FunctionFactory &);
|
||||||
void registerFunctionNotLike(FunctionFactory &);
|
void registerFunctionNotLike(FunctionFactory &);
|
||||||
|
void registerFunctionNotILike(FunctionFactory &);
|
||||||
void registerFunctionMatch(FunctionFactory &);
|
void registerFunctionMatch(FunctionFactory &);
|
||||||
void registerFunctionExtract(FunctionFactory &);
|
void registerFunctionExtract(FunctionFactory &);
|
||||||
void registerFunctionReplaceOne(FunctionFactory &);
|
void registerFunctionReplaceOne(FunctionFactory &);
|
||||||
@ -24,7 +26,9 @@ void registerFunctionExtractAllGroupsHorizontal(FunctionFactory &);
|
|||||||
void registerFunctionsStringRegexp(FunctionFactory & factory)
|
void registerFunctionsStringRegexp(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
registerFunctionLike(factory);
|
registerFunctionLike(factory);
|
||||||
|
registerFunctionILike(factory);
|
||||||
registerFunctionNotLike(factory);
|
registerFunctionNotLike(factory);
|
||||||
|
registerFunctionNotILike(factory);
|
||||||
registerFunctionMatch(factory);
|
registerFunctionMatch(factory);
|
||||||
registerFunctionExtract(factory);
|
registerFunctionExtract(factory);
|
||||||
registerFunctionReplaceOne(factory);
|
registerFunctionReplaceOne(factory);
|
||||||
@ -41,6 +45,4 @@ void registerFunctionsStringRegexp(FunctionFactory & factory)
|
|||||||
registerFunctionExtractAllGroupsVertical(factory);
|
registerFunctionExtractAllGroupsVertical(factory);
|
||||||
registerFunctionExtractAllGroupsHorizontal(factory);
|
registerFunctionExtractAllGroupsHorizontal(factory);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -92,8 +92,8 @@ SRCS(
|
|||||||
array/emptyArrayToSingle.cpp
|
array/emptyArrayToSingle.cpp
|
||||||
array/hasAll.cpp
|
array/hasAll.cpp
|
||||||
array/hasAny.cpp
|
array/hasAny.cpp
|
||||||
array/hasSubstr.cpp
|
|
||||||
array/has.cpp
|
array/has.cpp
|
||||||
|
array/hasSubstr.cpp
|
||||||
array/indexOf.cpp
|
array/indexOf.cpp
|
||||||
array/length.cpp
|
array/length.cpp
|
||||||
array/range.cpp
|
array/range.cpp
|
||||||
@ -228,6 +228,7 @@ SRCS(
|
|||||||
ifNull.cpp
|
ifNull.cpp
|
||||||
IFunction.cpp
|
IFunction.cpp
|
||||||
ignore.cpp
|
ignore.cpp
|
||||||
|
ilike.cpp
|
||||||
in.cpp
|
in.cpp
|
||||||
intDiv.cpp
|
intDiv.cpp
|
||||||
intDivOrZero.cpp
|
intDivOrZero.cpp
|
||||||
@ -288,6 +289,7 @@ SRCS(
|
|||||||
neighbor.cpp
|
neighbor.cpp
|
||||||
notEmpty.cpp
|
notEmpty.cpp
|
||||||
notEquals.cpp
|
notEquals.cpp
|
||||||
|
notILike.cpp
|
||||||
notLike.cpp
|
notLike.cpp
|
||||||
now64.cpp
|
now64.cpp
|
||||||
now.cpp
|
now.cpp
|
||||||
|
@ -132,7 +132,7 @@ public:
|
|||||||
{
|
{
|
||||||
/// leave other comparisons as is
|
/// 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
|
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.
|
/// leave as is. It's not possible to make push down here cause of unknown aliases and not implemented JOIN predicates.
|
||||||
|
@ -17,7 +17,7 @@ inline bool functionIsInOrGlobalInOperator(const std::string & name)
|
|||||||
|
|
||||||
inline bool functionIsLikeOperator(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)
|
inline bool functionIsJoinGet(const std::string & name)
|
||||||
|
@ -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.
|
* 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.
|
* 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.
|
* With highlighting we make it clearly obvious.
|
||||||
@ -168,7 +168,9 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
|||||||
"greater", " > ",
|
"greater", " > ",
|
||||||
"equals", " = ",
|
"equals", " = ",
|
||||||
"like", " LIKE ",
|
"like", " LIKE ",
|
||||||
|
"ilike", " ILIKE ",
|
||||||
"notLike", " NOT LIKE ",
|
"notLike", " NOT LIKE ",
|
||||||
|
"notILike", " NOT ILIKE ",
|
||||||
"in", " IN ",
|
"in", " IN ",
|
||||||
"notIn", " NOT IN ",
|
"notIn", " NOT IN ",
|
||||||
"globalIn", " GLOBAL 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 : "");
|
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
|
||||||
|
|
||||||
bool special_hilite = settings.hilite
|
bool special_hilite = settings.hilite
|
||||||
&& (name == "like" || name == "notLike")
|
&& (name == "like" || name == "notLike" || name == "ilike" || name == "notILike")
|
||||||
&& highlightStringLiteralWithMetacharacters(arguments->children[1], settings, "%_");
|
&& 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.
|
/// Format x IN 1 as x IN (1): put parens around rhs even if there is a single element in set.
|
||||||
|
@ -22,8 +22,13 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format
|
|||||||
else if (clusters)
|
else if (clusters)
|
||||||
{
|
{
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW CLUSTERS" << (settings.hilite ? hilite_none : "");
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW CLUSTERS" << (settings.hilite ? hilite_none : "");
|
||||||
|
|
||||||
if (!like.empty())
|
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, '\'');
|
<< std::quoted(like, '\'');
|
||||||
|
|
||||||
if (limit_length)
|
if (limit_length)
|
||||||
@ -47,8 +52,13 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format
|
|||||||
<< backQuoteIfNeed(from);
|
<< backQuoteIfNeed(from);
|
||||||
|
|
||||||
if (!like.empty())
|
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, '\'');
|
<< std::quoted(like, '\'');
|
||||||
|
|
||||||
else if (where_expression)
|
else if (where_expression)
|
||||||
{
|
{
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
|
||||||
|
@ -19,10 +19,14 @@ public:
|
|||||||
bool cluster{false};
|
bool cluster{false};
|
||||||
bool dictionaries{false};
|
bool dictionaries{false};
|
||||||
bool temporary{false};
|
bool temporary{false};
|
||||||
|
|
||||||
String cluster_str;
|
String cluster_str;
|
||||||
String from;
|
String from;
|
||||||
String like;
|
String like;
|
||||||
|
|
||||||
bool not_like{false};
|
bool not_like{false};
|
||||||
|
bool case_insensitive_like{false};
|
||||||
|
|
||||||
ASTPtr where_expression;
|
ASTPtr where_expression;
|
||||||
ASTPtr limit_length;
|
ASTPtr limit_length;
|
||||||
|
|
||||||
|
@ -1115,6 +1115,7 @@ const char * ParserAlias::restricted_keywords[] =
|
|||||||
"NOT",
|
"NOT",
|
||||||
"BETWEEN",
|
"BETWEEN",
|
||||||
"LIKE",
|
"LIKE",
|
||||||
|
"ILIKE",
|
||||||
nullptr
|
nullptr
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -47,7 +47,9 @@ const char * ParserComparisonExpression::operators[] =
|
|||||||
">", "greater",
|
">", "greater",
|
||||||
"=", "equals",
|
"=", "equals",
|
||||||
"LIKE", "like",
|
"LIKE", "like",
|
||||||
|
"ILIKE", "ilike",
|
||||||
"NOT LIKE", "notLike",
|
"NOT LIKE", "notLike",
|
||||||
|
"NOT ILIKE", "notILike",
|
||||||
"IN", "in",
|
"IN", "in",
|
||||||
"NOT IN", "notIn",
|
"NOT IN", "notIn",
|
||||||
"GLOBAL IN", "globalIn",
|
"GLOBAL IN", "globalIn",
|
||||||
|
@ -28,6 +28,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
ParserKeyword s_in("IN");
|
ParserKeyword s_in("IN");
|
||||||
ParserKeyword s_not("NOT");
|
ParserKeyword s_not("NOT");
|
||||||
ParserKeyword s_like("LIKE");
|
ParserKeyword s_like("LIKE");
|
||||||
|
ParserKeyword s_ilike("ILIKE");
|
||||||
ParserKeyword s_where("WHERE");
|
ParserKeyword s_where("WHERE");
|
||||||
ParserKeyword s_limit("LIMIT");
|
ParserKeyword s_limit("LIMIT");
|
||||||
ParserStringLiteral like_p;
|
ParserStringLiteral like_p;
|
||||||
@ -53,8 +54,11 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
if (s_not.ignore(pos, expected))
|
if (s_not.ignore(pos, expected))
|
||||||
query->not_like = true;
|
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))
|
if (!like_p.parse(pos, like, expected))
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -98,8 +102,11 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
if (s_not.ignore(pos, expected))
|
if (s_not.ignore(pos, expected))
|
||||||
query->not_like = true;
|
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))
|
if (!like_p.parse(pos, like, expected))
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -119,6 +126,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
}
|
}
|
||||||
|
|
||||||
tryGetIdentifierNameInto(database, query->from);
|
tryGetIdentifierNameInto(database, query->from);
|
||||||
|
|
||||||
if (like)
|
if (like)
|
||||||
query->like = safeGet<const String &>(like->as<ASTLiteral &>().value);
|
query->like = safeGet<const String &>(like->as<ASTLiteral &>().value);
|
||||||
|
|
||||||
|
@ -7,14 +7,14 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
/** Query like this:
|
/** Query like this:
|
||||||
* SHOW TABLES [FROM db] [[NOT] LIKE 'str'] [LIMIT expr]
|
* SHOW TABLES [FROM db] [[NOT] [I]LIKE 'str'] [LIMIT expr]
|
||||||
* or
|
* or
|
||||||
* SHOW DATABASES.
|
* SHOW DATABASES.
|
||||||
*/
|
*/
|
||||||
class ParserShowTablesQuery : public IParserBase
|
class ParserShowTablesQuery : public IParserBase
|
||||||
{
|
{
|
||||||
protected:
|
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;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -841,6 +841,7 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
|
|||||||
func_name = "greaterOrEquals";
|
func_name = "greaterOrEquals";
|
||||||
else if (func_name == "in" || func_name == "notIn" ||
|
else if (func_name == "in" || func_name == "notIn" ||
|
||||||
func_name == "like" || func_name == "notLike" ||
|
func_name == "like" || func_name == "notLike" ||
|
||||||
|
func_name == "ilike" || func_name == "notIlike" ||
|
||||||
func_name == "startsWith")
|
func_name == "startsWith")
|
||||||
{
|
{
|
||||||
/// "const IN data_column" doesn't make sense (unlike "data_column IN const")
|
/// "const IN data_column" doesn't make sense (unlike "data_column IN const")
|
||||||
|
@ -20,7 +20,7 @@ using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
|||||||
class ExpressionActions;
|
class ExpressionActions;
|
||||||
using ExpressionActionsPtr = std::shared_ptr<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 standalone field.
|
||||||
* - A field with reference to its position in a block.
|
* - A field with reference to its position in a block.
|
||||||
* It's needed for execution of functions on ranges during
|
* It's needed for execution of functions on ranges during
|
||||||
|
44
tests/queries/0_stateless/01355_ilike.reference
Normal file
44
tests/queries/0_stateless/01355_ilike.reference
Normal 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
|
61
tests/queries/0_stateless/01355_ilike.sql
Normal file
61
tests/queries/0_stateless/01355_ilike.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user