Get out garbage from AST [#METR-2944].

This commit is contained in:
Alexey Milovidov 2017-01-03 10:37:29 +03:00
parent af5e2c9953
commit 4140e7e889
7 changed files with 84 additions and 348 deletions

View File

@ -1,319 +1,59 @@
#pragma once
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTTablesInSelectQuery.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Storages/IStorage.h>
#include <DB/Storages/StorageDistributed.h>
#include <DB/Interpreters/Context.h>
#include <string>
#include <DB/Core/Types.h>
#include <DB/Interpreters/SettingsCommon.h>
#include <deque>
#include <unordered_map>
#include <type_traits>
namespace DB
{
namespace ErrorCodes
{
extern const int DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED;
}
class IAST;
class IStorage;
class ASTSelectQuery;
class Context;
namespace
{
using NameToAttributes = std::unordered_map<std::string, IAST::Attributes>;
const NameToAttributes name_to_attributes =
struct Attributes
{
{ "in", IAST::IsIn },
{ "notIn", IAST::IsNotIn },
{ "globalIn", IAST::IsIn | IAST::IsGlobal },
{ "globalNotIn", IAST::IsNotIn | IAST::IsGlobal }
/// Указатель на начало секции [NOT]IN или JOIN в которой включен этот узел,
/// если имеется такая секция.
IAST * enclosing_in_or_join = nullptr;
/** Глубина одного узла N - это глубина того запроса SELECT, которому принадлежит N.
* Дальше глубина одного запроса SELECT определяется следующим образом:
* - если запрос Q корневой, то select_query_depth(Q) = 0
* - если запрос S является непосредственным подзапросом одного запроса R,
* то select_query_depth(S) = select_query_depth(R) + 1
*/
UInt32 select_query_depth = 0;
bool is_in = false;
bool is_join = false;
bool is_global = false;
};
/// Из названия секции IN получить соответствующие атрибуты.
IAST::Attributes getAttributesFromInSubqueryName(const std::string & name)
{
auto it = name_to_attributes.find(name);
if (it != name_to_attributes.end())
return it->second;
else
return 0;
}
using ASTProperties = std::unordered_map<void *, Attributes>;
/// Из атрибутов составить название секции IN.
std::string getNameFromInSubqueryAttributes(IAST::Attributes attributes)
{
std::string name;
if (attributes & IAST::IsIn)
{
if (attributes & IAST::IsGlobal)
name = "globalIn";
else
name = "in";
}
else if (attributes & IAST::IsNotIn)
{
if (attributes & IAST::IsGlobal)
name = "globalNotIn";
else
name = "notIn";
}
return name;
}
/// Проверить, указана ли таблица в секции FROM.
bool isQueryFromTable(const ASTSelectQuery & query)
{
auto query_table = query.table();
if (query_table)
{
if (typeid_cast<const ASTSelectQuery *>(query_table.get()))
return false;
else if (typeid_cast<const ASTFunction *>(query_table.get()))
return false;
else
return true;
}
return false;
}
/// Проверить, является ли движок распределённым с количеством шардов более одного.
template <typename TStorageDistributed>
bool isEligibleStorageForInJoinPreprocessing(const StoragePtr & storage)
{
if (!storage)
return false;
if (!storage->isRemote())
return false;
auto storage_distributed = static_cast<TStorageDistributed *>(storage.get());
if (storage_distributed->getShardCount() < 2)
return false;
return true;
}
}
/** Этот класс предоставляет контроль над выполнением распределённых запросов внутри секций IN или JOIN.
* Мы используем шаблон, потому что движок StorageDistributed слишком сложный, чтобы писать юнит-тесты,
* которые бы зависели от него.
*/
template <typename TStorageDistributed = StorageDistributed, typename Enable = void>
class InJoinSubqueriesPreprocessor;
template <typename TStorageDistributed>
class InJoinSubqueriesPreprocessor<TStorageDistributed,
typename std::enable_if<std::is_base_of<IStorage, TStorageDistributed>::value>::type> final
class InJoinSubqueriesPreprocessor
{
public:
InJoinSubqueriesPreprocessor(ASTSelectQuery * select_query_,
const Context & context_, const StoragePtr & storage_)
: select_query(select_query_), context(context_), settings(context.getSettingsRef()), storage(storage_)
{
}
InJoinSubqueriesPreprocessor(const Context & context) : context(context) {}
void process(ASTSelectQuery * query) const;
InJoinSubqueriesPreprocessor(const InJoinSubqueriesPreprocessor &) = delete;
InJoinSubqueriesPreprocessor & operator=(const InJoinSubqueriesPreprocessor &) = delete;
/** В зависимости от профиля пользователя проверить наличие прав на выполнение
* распределённых подзапросов внутри секций IN или JOIN и обработать эти подзапросы.
*/
void perform()
{
if (settings.distributed_product_mode == DistributedProductMode::ALLOW)
{
/// Согласно профиля пользователя распределённые подзапросы внутри секций IN и JOIN разрешены.
/// Ничего не делаем.
return;
}
if (select_query == nullptr)
return;
/// Проверить главный запрос. В секции FROM должна быть указана распределённая таблица
/// с количеством шардов более одного. Табличные функции пропускаем.
if (select_query->attributes & IAST::IsPreprocessedForInJoinSubqueries)
return;
if (!isQueryFromTable(*select_query))
{
select_query->setAttributes(IAST::IsPreprocessedForInJoinSubqueries);
return;
}
if (!isEligibleStorageForInJoinPreprocessing<TStorageDistributed>(storage))
{
select_query->setAttributes(IAST::IsPreprocessedForInJoinSubqueries);
return;
}
/// Собрать информацию про все подзапросы внутри секций IN или JOIN.
/// Обработать те подзапросы, которые распределённые.
std::deque<IAST *> to_preprocess;
to_preprocess.push_back(select_query);
while (!to_preprocess.empty())
{
auto node = to_preprocess.back();
to_preprocess.pop_back();
if (ASTFunction * function = typeid_cast<ASTFunction *>(node))
{
auto attributes = getAttributesFromInSubqueryName(function->name);
if (attributes != 0)
{
/// Найдена секция IN.
node->enclosing_in_or_join = node;
node->attributes |= attributes;
}
}
else if (ASTTablesInSelectQueryElement * join = typeid_cast<ASTTablesInSelectQueryElement *>(node))
{
if (join->table_join)
{
/// Найдена секция JOIN.
join->enclosing_in_or_join = join->table_join.get();
join->table_join->attributes |= IAST::IsJoin;
if (static_cast<const ASTTableJoin &>(*join->table_join).locality == ASTTableJoin::Locality::Global)
join->table_join->attributes |= IAST::IsGlobal;
}
}
else if (node != static_cast<IAST *>(select_query))
{
if (ASTSelectQuery * sub_select_query = typeid_cast<ASTSelectQuery *>(node))
{
++node->select_query_depth;
if (sub_select_query->enclosing_in_or_join)
{
/// Найден подзапрос внутри секции IN или JOIN.
preprocessSubquery(*sub_select_query);
}
}
}
if (!(node->attributes & IAST::IsPreprocessedForInJoinSubqueries))
{
for (auto & child : node->children)
{
if (!(child->attributes & IAST::IsPreprocessedForInJoinSubqueries))
{
auto n = child.get();
n->enclosing_in_or_join = node->enclosing_in_or_join;
n->select_query_depth = node->select_query_depth;
to_preprocess.push_back(n);
}
}
node->attributes |= IAST::IsPreprocessedForInJoinSubqueries;
}
}
}
/// These methods could be overriden for the need of the unit test.
virtual bool hasAtLeastTwoShards(const IStorage & table) const;
virtual std::pair<std::string, std::string> getRemoteDatabaseAndTableName(const IStorage & table) const;
virtual ~InJoinSubqueriesPreprocessor() {}
private:
void preprocessSubquery(ASTSelectQuery & sub_select_query)
{
auto & enclosing_in_or_join = *sub_select_query.enclosing_in_or_join;
bool is_global = enclosing_in_or_join.attributes & IAST::IsGlobal;
void preprocessSubquery(ASTSelectQuery & sub_select_query, ASTProperties & shadow_ast, SettingDistributedProductMode distributed_product_mode) const;
IStorage * getDistributedStorage(const ASTSelectQuery & select_query) const;
/// Если подзапрос внутри секции IN или JOIN является непосредственным потомком
/// главного запроса и указано ключевое слово GLOBAL, то подзапрос пропускается.
if ((sub_select_query.select_query_depth == 1) && is_global)
{
sub_select_query.attributes |= IAST::IsPreprocessedForInJoinSubqueries;
return;
}
auto subquery_table_storage = getDistributedSubqueryStorage(sub_select_query);
if (!subquery_table_storage)
return;
if (settings.distributed_product_mode == DistributedProductMode::DENY)
{
/// Согласно профиля пользователя распределённые подзапросы внутри секций IN и JOIN запрещены.
throw Exception("Double-distributed IN/JOIN subqueries is denied (distributed_product_mode = 'deny')."
" You may rewrite query to use local tables in subqueries, or use GLOBAL keyword, or set distributed_product_mode to suitable value.",
ErrorCodes::DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED);
}
else if (settings.distributed_product_mode == DistributedProductMode::GLOBAL)
{
/// Согласно профиля пользователя распределённые подзапросы внутри секций IN и JOIN разрешены.
/// Преобразовать [NOT] IN в GLOBAL [NOT] IN, и JOIN в GLOBAL JOIN.
if (!is_global)
{
if (enclosing_in_or_join.attributes & IAST::IsJoin)
{
auto & join = static_cast<ASTTableJoin &>(enclosing_in_or_join);
join.locality = ASTTableJoin::Locality::Global;
}
else if (enclosing_in_or_join.attributes & (IAST::IsIn | IAST::IsNotIn))
{
auto & function = static_cast<ASTFunction &>(enclosing_in_or_join);
function.name = getNameFromInSubqueryAttributes(function.attributes | IAST::IsGlobal);
}
else
throw Exception("InJoinSubqueriesPreprocessor: Internal error", ErrorCodes::LOGICAL_ERROR);
}
}
else if (settings.distributed_product_mode == DistributedProductMode::LOCAL)
{
/// Согласно профиля пользователя распределённые подзапросы внутри секций IN и JOIN разрешены.
/// Преобразовать распределённую таблицу в соответствующую удалённую таблицу.
auto & distributed_storage = static_cast<TStorageDistributed &>(*subquery_table_storage);
sub_select_query.replaceDatabaseAndTable(
distributed_storage.getRemoteDatabaseName(),
distributed_storage.getRemoteTableName());
}
else
throw Exception("InJoinSubqueriesPreprocessor: Internal error", ErrorCodes::LOGICAL_ERROR);
}
StoragePtr getDistributedSubqueryStorage(const ASTSelectQuery & sub_select_query) const
{
auto database = sub_select_query.database();
auto table = sub_select_query.table();
if (!table)
return {};
const auto identifier = typeid_cast<const ASTIdentifier *>(table.get());
if (identifier == nullptr)
return {};
const std::string & table_name = identifier->name;
std::string database_name;
if (database)
database_name = typeid_cast<const ASTIdentifier &>(*database).name;
else
database_name = "";
auto subquery_table_storage = context.tryGetTable(database_name, table_name);
if (!subquery_table_storage)
return {};
if (!isEligibleStorageForInJoinPreprocessing<TStorageDistributed>(subquery_table_storage))
return {};
return subquery_table_storage;
}
private:
ASTSelectQuery * select_query;
const Context & context;
const Settings & settings;
const StoragePtr & storage;
};
}

View File

@ -6,6 +6,8 @@
#include <vector>
#include <map>
#include <unordered_map>
#include <unordered_set>
namespace DB
{
@ -97,6 +99,8 @@ private:
FunctionParentMap or_parent_map;
/// Позиция каждого столбца.
ColumnToPosition column_to_position;
/// Set of nodes, that was visited.
std::unordered_set<void *> visited_nodes;
};
}

View File

@ -35,36 +35,6 @@ public:
ASTs children;
StringRange range;
/// Указатель на начало секции [NOT]IN или JOIN в которой включен этот узел,
/// если имеется такая секция.
IAST * enclosing_in_or_join = nullptr;
/// Атрибуты, которые нужны для некоторых алгоритмов на синтаксических деревьях.
using Attributes = UInt32;
Attributes attributes = 0;
/// TODO Grabage. Need to throw away.
/// Был ли узел посещён? (см. класс LogicalExpressionsOptimizer)
static constexpr Attributes IsVisited = 1U;
/// Был ли узел обработан? (см. класс InJoinSubqueriesPreprocessor)
static constexpr Attributes IsPreprocessedForInJoinSubqueries = 1U << 1;
/// Является ли узел секцией IN?
static constexpr Attributes IsIn = 1U << 2;
/// Является ли узел секцией NOT IN?
static constexpr Attributes IsNotIn = 1U << 3;
/// Является ли узел секцией JOIN?
static constexpr Attributes IsJoin = 1U << 4;
/// Имеет ли секция IN/NOT IN/JOIN атрибут GLOBAL?
static constexpr Attributes IsGlobal = 1U << 5;
/** Глубина одного узла N - это глубина того запроса SELECT, которому принадлежит N.
* Дальше глубина одного запроса SELECT определяется следующим образом:
* - если запрос Q корневой, то select_query_depth(Q) = 0
* - если запрос S является непосредственным подзапросом одного запроса R,
* то select_query_depth(S) = select_query_depth(R) + 1
*/
UInt32 select_query_depth = 0;
/** Строка с полным запросом.
* Этот указатель не дает ее удалить, пока range в нее ссылается.
*/
@ -95,14 +65,6 @@ public:
/** Получить глубокую копию дерева. */
virtual ASTPtr clone() const = 0;
/// Рекурсивно установить атрибуты в поддереве, корнем которого является текущий узел.
void setAttributes(Attributes attributes_)
{
attributes |= attributes_;
for (auto it : children)
it->setAttributes(attributes_);
}
/** Получить текст, который идентифицирует этот элемент и всё поддерево.
* Обычно он содержит идентификатор элемента и getTreeID от всех детей.
*/

View File

@ -150,7 +150,7 @@ void ExpressionAnalyzer::init()
/// В зависимости от профиля пользователя проверить наличие прав на выполнение
/// распределённых подзапросов внутри секций IN или JOIN и обработать эти подзапросы.
InJoinSubqueriesPreprocessor<>(select_query, context, storage).perform();
InJoinSubqueriesPreprocessor(context).process(select_query);
/// Оптимизирует логические выражения.
LogicalExpressionsOptimizer(select_query, settings).perform();

View File

@ -39,7 +39,7 @@ void LogicalExpressionsOptimizer::perform()
{
if (select_query == nullptr)
return;
if (select_query->attributes & IAST::IsVisited)
if (visited_nodes.count(select_query))
return;
size_t position = 0;
@ -90,7 +90,7 @@ void LogicalExpressionsOptimizer::reorderColumns()
void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
{
if (select_query->attributes & IAST::IsVisited)
if (visited_nodes.count(select_query))
return;
using Edge = std::pair<IAST *, IAST *>;
@ -137,7 +137,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
}
}
to_node->attributes |= IAST::IsVisited;
visited_nodes.insert(to_node);
if (found_chain)
{
@ -153,9 +153,9 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
{
for (auto & child : to_node->children)
{
if (typeid_cast<ASTSelectQuery *>(&*child) == nullptr)
if (typeid_cast<ASTSelectQuery *>(child.get()) == nullptr)
{
if (!(child->attributes & IAST::IsVisited))
if (!visited_nodes.count(child.get()))
to_visit.push_back(Edge(to_node, &*child));
else
{

View File

@ -16,6 +16,16 @@
#include <utility>
#include <string>
namespace DB
{
namespace ErrorCodes
{
extern const int DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED;
}
}
/// Упрощённый вариант класса StorageDistributed.
class StorageDistributedFake : private ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
{
@ -49,6 +59,33 @@ private:
DB::NamesAndTypesList names_and_types;
};
class InJoinSubqueriesPreprocessorMock : public DB::InJoinSubqueriesPreprocessor
{
public:
using DB::InJoinSubqueriesPreprocessor::InJoinSubqueriesPreprocessor;
bool hasAtLeastTwoShards(const DB::IStorage & table) const override
{
if (!table.isRemote())
return false;
const StorageDistributedFake * distributed = typeid_cast<const StorageDistributedFake *>(&table);
if (!distributed)
return false;
return distributed->getShardCount() >= 2;
}
std::pair<std::string, std::string>
getRemoteDatabaseAndTableName(const DB::IStorage & table) const override
{
const StorageDistributedFake & distributed = typeid_cast<const StorageDistributedFake &>(table);
return { distributed.getRemoteDatabaseName(), distributed.getRemoteTableName() };
}
};
struct TestEntry
{
unsigned int line_num;
@ -67,6 +104,7 @@ bool parse(DB::ASTPtr & ast, const std::string & query);
bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs);
void reorder(DB::IAST * ast);
TestEntries entries =
{
/// Тривиальный запрос.
@ -1196,8 +1234,7 @@ TestResult check(const TestEntry & entry)
try
{
DB::InJoinSubqueriesPreprocessor<StorageDistributedFake> preprocessor(select_query, context, storage_distributed_visits);
preprocessor.perform();
InJoinSubqueriesPreprocessorMock(context).process(select_query);
}
catch (const DB::Exception & ex)
{

View File

@ -7,13 +7,6 @@
namespace DB
{
constexpr IAST::Attributes IAST::IsVisited;
constexpr IAST::Attributes IAST::IsPreprocessedForInJoinSubqueries;
constexpr IAST::Attributes IAST::IsIn;
constexpr IAST::Attributes IAST::IsNotIn;
constexpr IAST::Attributes IAST::IsJoin;
constexpr IAST::Attributes IAST::IsGlobal;
const char * IAST::hilite_keyword = "\033[1m";
const char * IAST::hilite_identifier = "\033[0;36m";
const char * IAST::hilite_function = "\033[0;33m";