This commit is contained in:
Alexey Arno 2015-03-27 13:51:33 +03:00
commit 1c9aa586ed
18 changed files with 280 additions and 128 deletions

View File

@ -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]));
}
}
}

View File

@ -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(); }

View File

@ -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;
};

View File

@ -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;
}
}
}
}

View File

@ -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());

View File

@ -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));
}

View File

@ -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];
}
}

View File

@ -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();

View File

@ -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_)

View File

@ -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());

View File

@ -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_)

View File

@ -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_)

View File

@ -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_)

View File

@ -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_)

View File

@ -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);

View File

@ -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));

View File

@ -0,0 +1,3 @@
79628
79628
102851

View 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%';