mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge
This commit is contained in:
commit
1c9aa586ed
@ -160,11 +160,16 @@ struct PositionUTF8Impl
|
||||
/// Переводит выражение LIKE в regexp re2. Например, abc%def -> ^abc.*def$
|
||||
inline String likePatternToRegexp(const String & pattern)
|
||||
{
|
||||
String res = "^";
|
||||
String res;
|
||||
res.reserve(pattern.size() * 2);
|
||||
const char * pos = pattern.data();
|
||||
const char * end = pos + pattern.size();
|
||||
|
||||
if (pos < end && *pos == '%')
|
||||
++pos;
|
||||
else
|
||||
res = "^";
|
||||
|
||||
while (pos < end)
|
||||
{
|
||||
switch (*pos)
|
||||
@ -174,7 +179,10 @@ inline String likePatternToRegexp(const String & pattern)
|
||||
res += *pos;
|
||||
break;
|
||||
case '%':
|
||||
res += ".*";
|
||||
if (pos + 1 != end)
|
||||
res += ".*";
|
||||
else
|
||||
return res;
|
||||
break;
|
||||
case '_':
|
||||
res += ".";
|
||||
@ -347,6 +355,7 @@ struct MatchImpl
|
||||
/// Текущий индекс в массиве строк.
|
||||
size_t i = 0;
|
||||
|
||||
/// TODO Надо сделать так, чтобы searcher был общим на все вызовы функции.
|
||||
Volnitsky searcher(strstr_pattern.data(), strstr_pattern.size(), end - pos);
|
||||
|
||||
/// Искать будем следующее вхождение сразу во всех строках.
|
||||
@ -369,14 +378,87 @@ struct MatchImpl
|
||||
++i;
|
||||
}
|
||||
|
||||
/// Хвостик, в котором не может быть подстрок.
|
||||
memset(&res[i], revert, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & regexp = Regexps::get<like, true>(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]) - 1);
|
||||
|
||||
const auto & regexp = Regexps::get<like, true>(pattern);
|
||||
|
||||
std::string required_substring;
|
||||
bool is_trivial;
|
||||
bool required_substring_is_prefix; /// для anchored выполнения регекспа.
|
||||
|
||||
regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
|
||||
|
||||
if (required_substring.empty())
|
||||
{
|
||||
size_t prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = revert ^ regexp->getRE2()->Match(
|
||||
re2_st::StringPiece(reinterpret_cast<const char *>(&data[prev_offset]), offsets[i] - prev_offset - 1),
|
||||
0, offsets[i] - prev_offset - 1, re2_st::RE2::UNANCHORED, nullptr, 0);
|
||||
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// NOTE Это почти совпадает со случаем likePatternIsStrstr.
|
||||
|
||||
const UInt8 * begin = &data[0];
|
||||
const UInt8 * pos = begin;
|
||||
const UInt8 * end = pos + data.size();
|
||||
|
||||
/// Текущий индекс в массиве строк.
|
||||
size_t i = 0;
|
||||
|
||||
Volnitsky searcher(required_substring.data(), required_substring.size(), end - pos);
|
||||
|
||||
/// Искать будем следующее вхождение сразу во всех строках.
|
||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||
{
|
||||
/// Определим, к какому индексу оно относится.
|
||||
while (begin + offsets[i] < pos)
|
||||
{
|
||||
res[i] = revert;
|
||||
++i;
|
||||
}
|
||||
|
||||
/// Проверяем, что вхождение не переходит через границы строк.
|
||||
if (pos + strstr_pattern.size() < begin + offsets[i])
|
||||
{
|
||||
/// И если не переходит - при необходимости, проверяем регекспом.
|
||||
|
||||
if (is_trivial)
|
||||
res[i] = !revert;
|
||||
else
|
||||
{
|
||||
const char * str_data = reinterpret_cast<const char *>(&data[i != 0 ? offsets[i - 1] : 0]);
|
||||
size_t str_size = (i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]) - 1;
|
||||
|
||||
if (required_substring_is_prefix)
|
||||
res[i] = revert ^ regexp->getRE2()->Match(
|
||||
re2_st::StringPiece(str_data, str_size),
|
||||
reinterpret_cast<const char *>(pos) - str_data, str_size, re2_st::RE2::ANCHOR_START, nullptr, 0);
|
||||
else
|
||||
res[i] = revert ^ regexp->getRE2()->Match(
|
||||
re2_st::StringPiece(str_data, str_size),
|
||||
0, str_size, re2_st::RE2::UNANCHORED, nullptr, 0);
|
||||
}
|
||||
}
|
||||
else
|
||||
res[i] = revert;
|
||||
|
||||
pos = begin + offsets[i];
|
||||
++i;
|
||||
}
|
||||
|
||||
memset(&res[i], revert, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -293,7 +293,7 @@ public:
|
||||
*/
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result, bool negative) const;
|
||||
|
||||
std::string describe()
|
||||
std::string describe() const
|
||||
{
|
||||
if (!ordered_set_elements)
|
||||
return "{}";
|
||||
@ -312,7 +312,7 @@ public:
|
||||
}
|
||||
|
||||
/// проверяет есть ли в Set элементы для заданного диапазона индекса
|
||||
BoolMask mayBeTrueInRange(const Range & range);
|
||||
BoolMask mayBeTrueInRange(const Range & range) const;
|
||||
|
||||
size_t getTotalRowCount() const { return data.getTotalRowCount(); }
|
||||
size_t getTotalByteCount() const { return data.getTotalByteCount(); }
|
||||
|
@ -124,7 +124,7 @@ struct Range
|
||||
private:
|
||||
static bool equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
|
||||
static bool less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
|
||||
|
||||
|
||||
public:
|
||||
Field left; /// левая граница, если есть
|
||||
Field right; /// правая граница, если есть
|
||||
@ -132,17 +132,17 @@ public:
|
||||
bool right_bounded; /// ограничен ли справа
|
||||
bool left_included; /// включает левую границу, если есть
|
||||
bool right_included; /// включает правую границу, если есть
|
||||
|
||||
|
||||
/// Всё множество.
|
||||
Range() : left(), right(), left_bounded(false), right_bounded(false), left_included(false), right_included(false) {}
|
||||
|
||||
|
||||
/// Одна точка.
|
||||
Range(const Field & point) : left(point), right(point), left_bounded(true), right_bounded(true), left_included(true), right_included(true) {}
|
||||
|
||||
|
||||
/// Ограниченный с двух сторон диапазон.
|
||||
Range(const Field & left_, bool left_included_, const Field & right_, bool right_included_)
|
||||
: left(left_), right(right_), left_bounded(true), right_bounded(true), left_included(left_included_), right_included(right_included_) {}
|
||||
|
||||
|
||||
static Range createRightBounded(const Field & right_point, bool right_included)
|
||||
{
|
||||
Range r;
|
||||
@ -151,7 +151,7 @@ public:
|
||||
r.right_included = right_included;
|
||||
return r;
|
||||
}
|
||||
|
||||
|
||||
static Range createLeftBounded(const Field & left_point, bool left_included)
|
||||
{
|
||||
Range r;
|
||||
@ -160,7 +160,7 @@ public:
|
||||
r.left_included = left_included;
|
||||
return r;
|
||||
}
|
||||
|
||||
|
||||
/// Установить левую границу.
|
||||
void setLeft(const Field & point, bool included)
|
||||
{
|
||||
@ -168,7 +168,7 @@ public:
|
||||
left_bounded = true;
|
||||
left_included = included;
|
||||
}
|
||||
|
||||
|
||||
/// Установить правую границу.
|
||||
void setRight(const Field & point, bool included)
|
||||
{
|
||||
@ -176,13 +176,13 @@ public:
|
||||
right_bounded = true;
|
||||
right_included = included;
|
||||
}
|
||||
|
||||
|
||||
/// x входит в range
|
||||
bool contains(const Field & x) const
|
||||
{
|
||||
return !leftThan(x) && !rightThan(x);
|
||||
}
|
||||
|
||||
|
||||
/// x находится левее
|
||||
bool rightThan(const Field & x) const
|
||||
{
|
||||
@ -190,7 +190,7 @@ public:
|
||||
? !(less(left, x) || (left_included && equals(x, left)))
|
||||
: false);
|
||||
}
|
||||
|
||||
|
||||
/// x находится правее
|
||||
bool leftThan(const Field & x) const
|
||||
{
|
||||
@ -198,7 +198,7 @@ public:
|
||||
? !(less(x, right) || (right_included && equals(x, right)))
|
||||
: false);
|
||||
}
|
||||
|
||||
|
||||
bool intersectsRange(const Range & r) const
|
||||
{
|
||||
/// r левее меня.
|
||||
@ -219,7 +219,7 @@ public:
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool containsRange(const Range & r) const
|
||||
{
|
||||
/// r начинается левее меня.
|
||||
@ -242,50 +242,58 @@ public:
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
String toString() const
|
||||
{
|
||||
std::stringstream str;
|
||||
|
||||
|
||||
if (!left_bounded)
|
||||
str << "(-inf, ";
|
||||
else
|
||||
str << (left_included ? '[' : '(') << apply_visitor(FieldVisitorToString(), left) << ", ";
|
||||
|
||||
|
||||
if (!right_bounded)
|
||||
str << "+inf)";
|
||||
else
|
||||
str << apply_visitor(FieldVisitorToString(), right) << (right_included ? ']' : ')');
|
||||
|
||||
|
||||
return str.str();
|
||||
}
|
||||
};
|
||||
|
||||
class ASTSet;
|
||||
|
||||
|
||||
/** Условие на индекс.
|
||||
*
|
||||
* Состоит из условий на принадлежность ключа всевозможным диапазонам или множествам,
|
||||
* а также логических связок AND/OR/NOT над этими условиями.
|
||||
*
|
||||
* Составляет reverse polish notation от этих условий
|
||||
* и умеет вычислять (интерпретировать) её выполнимость над диапазонами ключа.
|
||||
*/
|
||||
class PKCondition
|
||||
{
|
||||
public:
|
||||
/// Не учитывает секцию SAMPLE. all_columns - набор всех столбцов таблицы.
|
||||
PKCondition(ASTPtr query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr);
|
||||
|
||||
|
||||
/// Выполнимо ли условие в диапазоне ключей.
|
||||
/// left_pk и right_pk должны содержать все поля из sort_descr в соответствующем порядке.
|
||||
bool mayBeTrueInRange(const Field * left_pk, const Field * right_pk);
|
||||
|
||||
bool mayBeTrueInRange(const Field * left_pk, const Field * right_pk) const;
|
||||
|
||||
/// Выполнимо ли условие в полубесконечном (не ограниченном справа) диапазоне ключей.
|
||||
/// left_pk должен содержать все поля из sort_descr в соответствующем порядке.
|
||||
bool mayBeTrueAfter(const Field * left_pk);
|
||||
|
||||
bool alwaysTrue()
|
||||
{
|
||||
return rpn.size() == 1 && rpn[0].function == RPNElement::FUNCTION_UNKNOWN;
|
||||
}
|
||||
|
||||
bool mayBeTrueAfter(const Field * left_pk) const;
|
||||
|
||||
/// Проверяет, что индекс не может быть использован.
|
||||
bool alwaysUnknown() const;
|
||||
|
||||
/// Наложить дополнительное условие: значение в столбце column должно быть в диапазоне range.
|
||||
/// Возвращает, есть ли такой столбец в первичном ключе.
|
||||
bool addCondition(const String & column, const Range & range);
|
||||
|
||||
String toString();
|
||||
|
||||
String toString() const;
|
||||
private:
|
||||
/// Выражение хранится в виде обратной польской строки (Reverse Polish Notation).
|
||||
struct RPNElement
|
||||
@ -303,37 +311,37 @@ private:
|
||||
FUNCTION_AND,
|
||||
FUNCTION_OR,
|
||||
};
|
||||
|
||||
|
||||
RPNElement() {}
|
||||
RPNElement(Function function_) : function(function_) {}
|
||||
RPNElement(Function function_, size_t key_column_) : function(function_), key_column(key_column_) {}
|
||||
RPNElement(Function function_, size_t key_column_, const Range & range_)
|
||||
: function(function_), range(range_), key_column(key_column_) {}
|
||||
|
||||
String toString();
|
||||
|
||||
|
||||
String toString() const;
|
||||
|
||||
Function function;
|
||||
|
||||
|
||||
/// Для FUNCTION_IN_RANGE и FUNCTION_NOT_IN_RANGE.
|
||||
Range range;
|
||||
size_t key_column;
|
||||
/// Для FUNCTION_IN_SET
|
||||
ASTPtr in_function;
|
||||
|
||||
ASTSet * inFunctionToSet();
|
||||
|
||||
const ASTSet * inFunctionToSet() const;
|
||||
};
|
||||
|
||||
|
||||
typedef std::vector<RPNElement> RPN;
|
||||
typedef std::map<String, size_t> ColumnIndices;
|
||||
|
||||
bool mayBeTrueInRange(const Field * left_pk, const Field * right_pk, bool right_bounded);
|
||||
|
||||
|
||||
bool mayBeTrueInRange(const Field * left_pk, const Field * right_pk, bool right_bounded) const;
|
||||
|
||||
void traverseAST(ASTPtr & node, Block & block_with_constants);
|
||||
bool atomFromAST(ASTPtr & node, Block & block_with_constants, RPNElement & out);
|
||||
bool operatorFromAST(ASTFunction * func, RPNElement & out);
|
||||
|
||||
|
||||
RPN rpn;
|
||||
|
||||
|
||||
SortDescription sort_descr;
|
||||
ColumnIndices pk_columns;
|
||||
};
|
||||
|
@ -558,7 +558,7 @@ void Set::executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t
|
||||
}
|
||||
|
||||
|
||||
BoolMask Set::mayBeTrueInRange(const Range & range)
|
||||
BoolMask Set::mayBeTrueInRange(const Range & range) const
|
||||
{
|
||||
if (!ordered_set_elements)
|
||||
throw DB::Exception("Ordered set in not created.");
|
||||
@ -588,7 +588,10 @@ BoolMask Set::mayBeTrueInRange(const Range & range)
|
||||
}
|
||||
else
|
||||
{
|
||||
auto left_it = range.left_bounded ? std::lower_bound(ordered_set_elements->begin(), ordered_set_elements->end(), left) : ordered_set_elements->begin();
|
||||
auto left_it = range.left_bounded
|
||||
? std::lower_bound(ordered_set_elements->begin(), ordered_set_elements->end(), left)
|
||||
: ordered_set_elements->begin();
|
||||
|
||||
if (range.left_bounded && !range.left_included && left_it != ordered_set_elements->end() && *left_it == left)
|
||||
++left_it;
|
||||
|
||||
@ -599,7 +602,10 @@ BoolMask Set::mayBeTrueInRange(const Range & range)
|
||||
}
|
||||
else
|
||||
{
|
||||
auto right_it = range.right_bounded ? std::upper_bound(ordered_set_elements->begin(), ordered_set_elements->end(), right) : ordered_set_elements->end();
|
||||
auto right_it = range.right_bounded
|
||||
? std::upper_bound(ordered_set_elements->begin(), ordered_set_elements->end(), right)
|
||||
: ordered_set_elements->end();
|
||||
|
||||
if (range.right_bounded && !range.right_included && right_it != ordered_set_elements->begin() && *(right_it--) == right)
|
||||
--right_it;
|
||||
|
||||
@ -613,13 +619,9 @@ BoolMask Set::mayBeTrueInRange(const Range & range)
|
||||
--right_it;
|
||||
/// в диапазон не попадает ни одного ключа из in
|
||||
if (*right_it < *left_it)
|
||||
{
|
||||
can_be_true = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
can_be_true = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -59,10 +59,12 @@ int main(int argc, char ** argv)
|
||||
}
|
||||
|
||||
Context context;
|
||||
NamesAndTypesList columns;
|
||||
columns.emplace_back("x", new DataTypeInt16);
|
||||
columns.emplace_back("s1", new DataTypeString);
|
||||
columns.emplace_back("s2", new DataTypeString);
|
||||
NamesAndTypesList columns
|
||||
{
|
||||
{"x", new DataTypeInt16},
|
||||
{"s1", new DataTypeString},
|
||||
{"s2", new DataTypeString}
|
||||
};
|
||||
context.setColumns(columns);
|
||||
|
||||
ExpressionAnalyzer analyzer(ast, context, context.getColumns());
|
||||
|
@ -83,7 +83,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
|
||||
PKCondition key_condition(query, context, data.getColumnsList(), data.getSortDescription());
|
||||
PKCondition date_condition(query, context, data.getColumnsList(), SortDescription(1, SortColumnDescription(data.date_column_name, 1)));
|
||||
|
||||
if (settings.force_index_by_date && date_condition.alwaysTrue())
|
||||
if (settings.force_index_by_date && date_condition.alwaysUnknown())
|
||||
throw Exception("Index by date is not used and setting 'force_index_by_date' is set.", ErrorCodes::INDEX_NOT_USED);
|
||||
|
||||
/// Выберем куски, в которых могут быть данные, удовлетворяющие date_condition, и которые подходят под условие на _part.
|
||||
@ -556,7 +556,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPkRange(
|
||||
size_t marks_count = index.size() / key_size;
|
||||
|
||||
/// Если индекс не используется.
|
||||
if (key_condition.alwaysTrue())
|
||||
if (key_condition.alwaysUnknown())
|
||||
{
|
||||
res.push_back(MarkRange(0, marks_count));
|
||||
}
|
||||
|
@ -42,7 +42,7 @@ PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndT
|
||||
if (select.prewhere_expression)
|
||||
{
|
||||
traverseAST(select.prewhere_expression, block_with_constants);
|
||||
rpn.push_back(RPNElement(RPNElement::FUNCTION_AND));
|
||||
rpn.emplace_back(RPNElement::FUNCTION_AND);
|
||||
}
|
||||
}
|
||||
else if (select.prewhere_expression)
|
||||
@ -51,7 +51,7 @@ PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndT
|
||||
}
|
||||
else
|
||||
{
|
||||
rpn.push_back(RPNElement(RPNElement::FUNCTION_UNKNOWN));
|
||||
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
}
|
||||
}
|
||||
|
||||
@ -59,8 +59,8 @@ bool PKCondition::addCondition(const String & column, const Range & range)
|
||||
{
|
||||
if (!pk_columns.count(column))
|
||||
return false;
|
||||
rpn.push_back(RPNElement(RPNElement::FUNCTION_IN_RANGE, pk_columns[column], range));
|
||||
rpn.push_back(RPNElement(RPNElement::FUNCTION_AND));
|
||||
rpn.emplace_back(RPNElement::FUNCTION_IN_RANGE, pk_columns[column], range);
|
||||
rpn.emplace_back(RPNElement::FUNCTION_AND);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -224,7 +224,7 @@ bool PKCondition::operatorFromAST(ASTFunction * func, RPNElement & out)
|
||||
return true;
|
||||
}
|
||||
|
||||
String PKCondition::toString()
|
||||
String PKCondition::toString() const
|
||||
{
|
||||
String res;
|
||||
for (size_t i = 0; i < rpn.size(); ++i)
|
||||
@ -236,7 +236,7 @@ String PKCondition::toString()
|
||||
return res;
|
||||
}
|
||||
|
||||
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk, bool right_bounded)
|
||||
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk, bool right_bounded) const
|
||||
{
|
||||
/// Найдем диапазоны элементов ключа.
|
||||
std::vector<Range> key_ranges(sort_descr.size(), Range());
|
||||
@ -264,10 +264,10 @@ bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk
|
||||
std::vector<BoolMask> rpn_stack;
|
||||
for (size_t i = 0; i < rpn.size(); ++i)
|
||||
{
|
||||
RPNElement & element = rpn[i];
|
||||
const auto & element = rpn[i];
|
||||
if (element.function == RPNElement::FUNCTION_UNKNOWN)
|
||||
{
|
||||
rpn_stack.push_back(BoolMask(true, true));
|
||||
rpn_stack.emplace_back(true, true);
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE || element.function == RPNElement::FUNCTION_IN_RANGE)
|
||||
{
|
||||
@ -275,15 +275,15 @@ bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk
|
||||
bool intersects = element.range.intersectsRange(key_range);
|
||||
bool contains = element.range.containsRange(key_range);
|
||||
|
||||
rpn_stack.push_back(BoolMask(intersects, !contains));
|
||||
rpn_stack.emplace_back(intersects, !contains);
|
||||
if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
|
||||
rpn_stack.back() = !rpn_stack.back();
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_IN_SET || element.function == RPNElement::FUNCTION_NOT_IN_SET)
|
||||
{
|
||||
ASTFunction * in_func = typeid_cast<ASTFunction *>(element.in_function.get());
|
||||
ASTs & args = typeid_cast<ASTExpressionList &>(*in_func->arguments).children;
|
||||
ASTSet * ast_set = typeid_cast<ASTSet *>(args[1].get());
|
||||
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
|
||||
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
|
||||
auto ast_set = typeid_cast<const ASTSet *>(args[1].get());
|
||||
if (in_func && ast_set)
|
||||
{
|
||||
const Range & key_range = key_ranges[element.key_column];
|
||||
@ -294,7 +294,7 @@ bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk
|
||||
}
|
||||
else
|
||||
{
|
||||
throw DB::Exception("Set for IN is not created yet!");
|
||||
throw DB::Exception("Set for IN is not created yet!", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_NOT)
|
||||
@ -303,16 +303,16 @@ bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_AND)
|
||||
{
|
||||
BoolMask arg1 = rpn_stack.back();
|
||||
auto arg1 = rpn_stack.back();
|
||||
rpn_stack.pop_back();
|
||||
BoolMask arg2 = rpn_stack.back();
|
||||
auto arg2 = rpn_stack.back();
|
||||
rpn_stack.back() = arg1 & arg2;
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_OR)
|
||||
{
|
||||
BoolMask arg1 = rpn_stack.back();
|
||||
auto arg1 = rpn_stack.back();
|
||||
rpn_stack.pop_back();
|
||||
BoolMask arg2 = rpn_stack.back();
|
||||
auto arg2 = rpn_stack.back();
|
||||
rpn_stack.back() = arg1 | arg2;
|
||||
}
|
||||
else
|
||||
@ -325,27 +325,27 @@ bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk
|
||||
return rpn_stack[0].can_be_true;
|
||||
}
|
||||
|
||||
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk)
|
||||
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk) const
|
||||
{
|
||||
return mayBeTrueInRange(left_pk, right_pk, true);
|
||||
}
|
||||
|
||||
bool PKCondition::mayBeTrueAfter(const Field * left_pk)
|
||||
bool PKCondition::mayBeTrueAfter(const Field * left_pk) const
|
||||
{
|
||||
return mayBeTrueInRange(left_pk, nullptr, false);
|
||||
}
|
||||
|
||||
ASTSet * PKCondition::RPNElement::inFunctionToSet()
|
||||
const ASTSet * PKCondition::RPNElement::inFunctionToSet() const
|
||||
{
|
||||
ASTFunction * in_func = typeid_cast<ASTFunction *>(in_function.get());
|
||||
auto in_func = typeid_cast<const ASTFunction *>(in_function.get());
|
||||
if (!in_func)
|
||||
return nullptr;
|
||||
ASTs & args = typeid_cast<ASTExpressionList &>(*in_func->arguments).children;
|
||||
ASTSet * ast_set = typeid_cast<ASTSet *>(args[1].get());
|
||||
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
|
||||
auto ast_set = typeid_cast<const ASTSet *>(args[1].get());
|
||||
return ast_set;
|
||||
}
|
||||
|
||||
String PKCondition::RPNElement::toString()
|
||||
String PKCondition::RPNElement::toString() const
|
||||
{
|
||||
std::ostringstream ss;
|
||||
switch (function)
|
||||
@ -374,4 +374,50 @@ String PKCondition::RPNElement::toString()
|
||||
return "ERROR";
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool PKCondition::alwaysUnknown() const
|
||||
{
|
||||
std::vector<UInt8> rpn_stack;
|
||||
|
||||
for (size_t i = 0; i < rpn.size(); ++i)
|
||||
{
|
||||
const auto & element = rpn[i];
|
||||
|
||||
if (element.function == RPNElement::FUNCTION_UNKNOWN)
|
||||
{
|
||||
rpn_stack.push_back(true);
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE
|
||||
|| element.function == RPNElement::FUNCTION_IN_RANGE
|
||||
|| element.function == RPNElement::FUNCTION_IN_SET
|
||||
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
|
||||
{
|
||||
rpn_stack.push_back(false);
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_NOT)
|
||||
{
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_AND)
|
||||
{
|
||||
auto arg1 = rpn_stack.back();
|
||||
rpn_stack.pop_back();
|
||||
auto arg2 = rpn_stack.back();
|
||||
rpn_stack.back() = arg1 & arg2;
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_OR)
|
||||
{
|
||||
auto arg1 = rpn_stack.back();
|
||||
rpn_stack.pop_back();
|
||||
auto arg2 = rpn_stack.back();
|
||||
rpn_stack.back() = arg1 | arg2;
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected function type in PKCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
return rpn_stack[0];
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -1467,6 +1467,8 @@ void StorageReplicatedMergeTree::alterThread()
|
||||
/// Если описание столбцов изменилось, обновим структуру таблицы локально.
|
||||
if (changed_version)
|
||||
{
|
||||
LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock.");
|
||||
|
||||
auto table_lock = lockStructureForAlter();
|
||||
|
||||
const auto columns_changed = columns != data.getColumnsListNonMaterialized();
|
||||
|
@ -9,9 +9,8 @@ namespace DB
|
||||
|
||||
|
||||
StorageSystemDatabases::StorageSystemDatabases(const std::string & name_)
|
||||
: name(name_)
|
||||
: name(name_), columns{{"name", new DataTypeString}}
|
||||
{
|
||||
columns.emplace_back("name", new DataTypeString);
|
||||
}
|
||||
|
||||
StoragePtr StorageSystemDatabases::create(const std::string & name_)
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <DB/Dictionaries/IDictionary.h>
|
||||
#include <DB/Dictionaries/IDictionarySource.h>
|
||||
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||
#include <statdaemons/ext/map.hpp>
|
||||
#include <mutex>
|
||||
|
||||
namespace DB
|
||||
@ -87,15 +88,12 @@ BlockInputStreams StorageSystemDictionaries::read(
|
||||
col_origin.column->insert(dict_info.second.second);
|
||||
|
||||
const auto & dict_struct = dict_ptr->getStructure();
|
||||
Array attribute_names;
|
||||
Array attribute_types;
|
||||
for (const auto & attribute : dict_struct.attributes)
|
||||
{
|
||||
attribute_names.push_back(attribute.name);
|
||||
attribute_types.push_back(attribute.type->getName());
|
||||
}
|
||||
col_attribute_names.column->insert(attribute_names);
|
||||
col_attribute_types.column->insert(attribute_types);
|
||||
col_attribute_names.column->insert(ext::map<Array>(dict_struct.attributes, [] (auto & attr) -> decltype(auto) {
|
||||
return attr.name;
|
||||
}));
|
||||
col_attribute_types.column->insert(ext::map<Array>(dict_struct.attributes, [] (auto & attr) -> decltype(auto) {
|
||||
return attr.type->getName();
|
||||
}));
|
||||
col_has_hierarchy.column->insert(UInt64{dict_ptr->hasHierarchy()});
|
||||
col_bytes_allocated.column->insert(dict_ptr->getBytesAllocated());
|
||||
col_hit_rate.column->insert(dict_ptr->getHitRate());
|
||||
|
@ -11,10 +11,13 @@ namespace DB
|
||||
|
||||
|
||||
StorageSystemEvents::StorageSystemEvents(const std::string & name_)
|
||||
: name(name_)
|
||||
: name(name_),
|
||||
columns
|
||||
{
|
||||
{"event", new DataTypeString},
|
||||
{"value", new DataTypeUInt64},
|
||||
}
|
||||
{
|
||||
columns.emplace_back("event", new DataTypeString);
|
||||
columns.emplace_back("value", new DataTypeUInt64);
|
||||
}
|
||||
|
||||
StoragePtr StorageSystemEvents::create(const std::string & name_)
|
||||
|
@ -54,9 +54,8 @@ private:
|
||||
|
||||
|
||||
StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_)
|
||||
: name(name_), multithreaded(multithreaded_)
|
||||
: name(name_), columns{{"number", new DataTypeUInt64}}, multithreaded(multithreaded_)
|
||||
{
|
||||
columns.emplace_back("number", new DataTypeUInt64);
|
||||
}
|
||||
|
||||
StoragePtr StorageSystemNumbers::create(const std::string & name_, bool multithreaded_)
|
||||
|
@ -12,9 +12,8 @@ namespace DB
|
||||
|
||||
|
||||
StorageSystemOne::StorageSystemOne(const std::string & name_)
|
||||
: name(name_)
|
||||
: name(name_), columns{{"dummy", new DataTypeUInt8}}
|
||||
{
|
||||
columns.emplace_back("dummy", new DataTypeUInt8);
|
||||
}
|
||||
|
||||
StoragePtr StorageSystemOne::create(const std::string & name_)
|
||||
|
@ -14,21 +14,24 @@ namespace DB
|
||||
|
||||
|
||||
StorageSystemParts::StorageSystemParts(const std::string & name_)
|
||||
: name(name_)
|
||||
{
|
||||
columns.emplace_back("partition", new DataTypeString);
|
||||
columns.emplace_back("name", new DataTypeString);
|
||||
columns.emplace_back("replicated", new DataTypeUInt8);
|
||||
columns.emplace_back("active", new DataTypeUInt8);
|
||||
columns.emplace_back("marks", new DataTypeUInt64);
|
||||
columns.emplace_back("bytes", new DataTypeUInt64);
|
||||
columns.emplace_back("modification_time", new DataTypeDateTime);
|
||||
columns.emplace_back("remove_time", new DataTypeDateTime);
|
||||
columns.emplace_back("refcount", new DataTypeUInt32);
|
||||
: name(name_),
|
||||
columns
|
||||
{
|
||||
{"partition", new DataTypeString},
|
||||
{"name", new DataTypeString},
|
||||
{"replicated", new DataTypeUInt8},
|
||||
{"active", new DataTypeUInt8},
|
||||
{"marks", new DataTypeUInt64},
|
||||
{"bytes", new DataTypeUInt64},
|
||||
{"modification_time", new DataTypeDateTime},
|
||||
{"remove_time", new DataTypeDateTime},
|
||||
{"refcount", new DataTypeUInt32},
|
||||
|
||||
columns.emplace_back("database", new DataTypeString);
|
||||
columns.emplace_back("table", new DataTypeString);
|
||||
columns.emplace_back("engine", new DataTypeString);
|
||||
{"database", new DataTypeString},
|
||||
{"table", new DataTypeString},
|
||||
{"engine", new DataTypeString},
|
||||
}
|
||||
{
|
||||
}
|
||||
|
||||
StoragePtr StorageSystemParts::create(const std::string & name_)
|
||||
|
@ -10,11 +10,14 @@ namespace DB
|
||||
|
||||
|
||||
StorageSystemTables::StorageSystemTables(const std::string & name_)
|
||||
: name(name_)
|
||||
: name(name_),
|
||||
columns
|
||||
{
|
||||
{"database", new DataTypeString},
|
||||
{"name", new DataTypeString},
|
||||
{"engine", new DataTypeString},
|
||||
}
|
||||
{
|
||||
columns.emplace_back("database", new DataTypeString);
|
||||
columns.emplace_back("name", new DataTypeString);
|
||||
columns.emplace_back("engine", new DataTypeString);
|
||||
}
|
||||
|
||||
StoragePtr StorageSystemTables::create(const std::string & name_)
|
||||
@ -32,10 +35,9 @@ static ColumnWithNameAndType getFilteredDatabases(ASTPtr query, const Context &
|
||||
|
||||
Block block;
|
||||
block.insert(column);
|
||||
for (auto database_it = context.getDatabases().begin(); database_it != context.getDatabases().end(); ++database_it)
|
||||
{
|
||||
column.column->insert(database_it->first);
|
||||
}
|
||||
for (const auto db : context.getDatabases())
|
||||
column.column->insert(db.first);
|
||||
|
||||
VirtualColumnUtils::filterBlockWithQuery(query, block, context);
|
||||
|
||||
return block.getByPosition(0);
|
||||
|
@ -31,8 +31,7 @@ int main(int argc, const char ** argv)
|
||||
return 1;
|
||||
}
|
||||
Context context;
|
||||
NamesAndTypesList columns;
|
||||
columns.emplace_back("key", new DataTypeUInt64);
|
||||
NamesAndTypesList columns{{"key", new DataTypeUInt64}};
|
||||
SortDescription sort_descr;
|
||||
sort_descr.push_back(SortColumnDescription("key", 1));
|
||||
|
||||
|
3
dbms/tests/queries/0_stateless/00139_like.reference
Normal file
3
dbms/tests/queries/0_stateless/00139_like.reference
Normal file
@ -0,0 +1,3 @@
|
||||
79628
|
||||
79628
|
||||
102851
|
5
dbms/tests/queries/0_stateless/00139_like.sql
Normal file
5
dbms/tests/queries/0_stateless/00139_like.sql
Normal file
@ -0,0 +1,5 @@
|
||||
/* Заметим, что запросы написаны так, как будто пользователь не понимает смысл символа _ в LIKE выражении. */
|
||||
SELECT count() FROM test.hits WHERE URL LIKE '%/avtomobili_s_probegom/_%__%__%__%';
|
||||
SELECT count() FROM test.hits WHERE URL LIKE '/avtomobili_s_probegom/_%__%__%__%';
|
||||
SELECT count() FROM test.hits WHERE URL LIKE '%_/avtomobili_s_probegom/_%__%__%__%';
|
||||
SELECT count() FROM test.hits WHERE URL LIKE '%avtomobili%';
|
Loading…
Reference in New Issue
Block a user