mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Fix usage of max_parser_depth setting; remove harmful default function arguments
This commit is contained in:
parent
d97121d0ee
commit
cdeda4ab91
@ -684,7 +684,7 @@ private:
|
|||||||
if (ignore_error)
|
if (ignore_error)
|
||||||
{
|
{
|
||||||
Tokens tokens(begin, end);
|
Tokens tokens(begin, end);
|
||||||
IParser::Pos token_iterator(tokens);
|
IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth);
|
||||||
while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid())
|
while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid())
|
||||||
++token_iterator;
|
++token_iterator;
|
||||||
begin = token_iterator->end;
|
begin = token_iterator->end;
|
||||||
@ -958,10 +958,12 @@ private:
|
|||||||
ParserQuery parser(end, true);
|
ParserQuery parser(end, true);
|
||||||
ASTPtr res;
|
ASTPtr res;
|
||||||
|
|
||||||
|
const auto & settings = context.getSettingsRef();
|
||||||
|
|
||||||
if (is_interactive || ignore_error)
|
if (is_interactive || ignore_error)
|
||||||
{
|
{
|
||||||
String message;
|
String message;
|
||||||
res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, 0);
|
res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, settings.max_query_size, settings.max_parser_depth);
|
||||||
|
|
||||||
if (!res)
|
if (!res)
|
||||||
{
|
{
|
||||||
@ -970,7 +972,7 @@ private:
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, 0);
|
res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, settings.max_query_size, settings.max_parser_depth);
|
||||||
|
|
||||||
if (is_interactive)
|
if (is_interactive)
|
||||||
{
|
{
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Compression/CompressionFactory.h>
|
#include <Compression/CompressionFactory.h>
|
||||||
#include <Common/TerminalSize.h>
|
#include <Common/TerminalSize.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -123,7 +124,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
|
|||||||
DB::ParserCodec codec_parser;
|
DB::ParserCodec codec_parser;
|
||||||
|
|
||||||
std::string codecs_line = boost::algorithm::join(codecs, ",");
|
std::string codecs_line = boost::algorithm::join(codecs, ",");
|
||||||
auto ast = DB::parseQuery(codec_parser, "(" + codecs_line + ")", 0);
|
auto ast = DB::parseQuery(codec_parser, "(" + codecs_line + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
codec = DB::CompressionCodecFactory::instance().get(ast, nullptr);
|
codec = DB::CompressionCodecFactory::instance().get(ast, nullptr);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -1197,7 +1197,9 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
|||||||
query += " LIMIT " + limit;
|
query += " LIMIT " + limit;
|
||||||
|
|
||||||
ParserQuery p_query(query.data() + query.size());
|
ParserQuery p_query(query.data() + query.size());
|
||||||
return parseQuery(p_query, query, 0);
|
|
||||||
|
const auto & settings = context.getSettingsRef();
|
||||||
|
return parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Load balancing
|
/// Load balancing
|
||||||
@ -1409,7 +1411,8 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
|||||||
query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES ";
|
query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES ";
|
||||||
|
|
||||||
ParserQuery p_query(query.data() + query.size());
|
ParserQuery p_query(query.data() + query.size());
|
||||||
query_insert_ast = parseQuery(p_query, query, 0);
|
const auto & settings = context.getSettingsRef();
|
||||||
|
query_insert_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||||
|
|
||||||
LOG_DEBUG(log, "Executing INSERT query: " << query);
|
LOG_DEBUG(log, "Executing INSERT query: " << query);
|
||||||
}
|
}
|
||||||
@ -1634,7 +1637,8 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time
|
|||||||
&task_cluster->settings_pull);
|
&task_cluster->settings_pull);
|
||||||
|
|
||||||
ParserCreateQuery parser_create_query;
|
ParserCreateQuery parser_create_query;
|
||||||
return parseQuery(parser_create_query, create_query_pull_str, 0);
|
const auto & settings = context.getSettingsRef();
|
||||||
|
return parseQuery(parser_create_query, create_query_pull_str, settings.max_query_size, settings.max_parser_depth);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it.
|
/// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it.
|
||||||
@ -1712,7 +1716,8 @@ std::set<String> ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti
|
|||||||
}
|
}
|
||||||
|
|
||||||
ParserQuery parser_query(query.data() + query.size());
|
ParserQuery parser_query(query.data() + query.size());
|
||||||
ASTPtr query_ast = parseQuery(parser_query, query, 0);
|
const auto & settings = context.getSettingsRef();
|
||||||
|
ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||||
|
|
||||||
LOG_DEBUG(log, "Computing destination partition set, executing query: " << query);
|
LOG_DEBUG(log, "Computing destination partition set, executing query: " << query);
|
||||||
|
|
||||||
@ -1759,7 +1764,8 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts,
|
|||||||
<< partition_quoted_name << " existence, executing query: " << query);
|
<< partition_quoted_name << " existence, executing query: " << query);
|
||||||
|
|
||||||
ParserQuery parser_query(query.data() + query.size());
|
ParserQuery parser_query(query.data() + query.size());
|
||||||
ASTPtr query_ast = parseQuery(parser_query, query, 0);
|
const auto & settings = context.getSettingsRef();
|
||||||
|
ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||||
|
|
||||||
Context local_context = context;
|
Context local_context = context;
|
||||||
local_context.setSettings(task_cluster->settings_pull);
|
local_context.setSettings(task_cluster->settings_pull);
|
||||||
@ -1793,7 +1799,8 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi
|
|||||||
<< "existence, executing query: " << query);
|
<< "existence, executing query: " << query);
|
||||||
|
|
||||||
ParserQuery parser_query(query.data() + query.size());
|
ParserQuery parser_query(query.data() + query.size());
|
||||||
ASTPtr query_ast = parseQuery(parser_query, query, 0);
|
const auto & settings = context.getSettingsRef();
|
||||||
|
ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||||
|
|
||||||
Context local_context = context;
|
Context local_context = context;
|
||||||
local_context.setSettings(task_cluster->settings_pull);
|
local_context.setSettings(task_cluster->settings_pull);
|
||||||
@ -1826,7 +1833,8 @@ UInt64 ClusterCopier::executeQueryOnCluster(
|
|||||||
if (query_ast_ == nullptr)
|
if (query_ast_ == nullptr)
|
||||||
{
|
{
|
||||||
ParserQuery p_query(query.data() + query.size());
|
ParserQuery p_query(query.data() + query.size());
|
||||||
query_ast = parseQuery(p_query, query, 0);
|
const auto & settings = context.getSettingsRef();
|
||||||
|
query_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
query_ast = query_ast_;
|
query_ast = query_ast_;
|
||||||
|
@ -4,6 +4,9 @@
|
|||||||
#include "Internals.h"
|
#include "Internals.h"
|
||||||
#include "ClusterPartition.h"
|
#include "ClusterPartition.h"
|
||||||
|
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
@ -260,9 +263,10 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
|
|||||||
+ "." + escapeForFileName(table_push.second);
|
+ "." + escapeForFileName(table_push.second);
|
||||||
|
|
||||||
engine_push_str = config.getString(table_prefix + "engine");
|
engine_push_str = config.getString(table_prefix + "engine");
|
||||||
|
|
||||||
{
|
{
|
||||||
ParserStorage parser_storage;
|
ParserStorage parser_storage;
|
||||||
engine_push_ast = parseQuery(parser_storage, engine_push_str, 0);
|
engine_push_ast = parseQuery(parser_storage, engine_push_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
engine_push_partition_key_ast = extractPartitionKey(engine_push_ast);
|
engine_push_partition_key_ast = extractPartitionKey(engine_push_ast);
|
||||||
primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast));
|
primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast));
|
||||||
engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast);
|
engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast);
|
||||||
@ -273,7 +277,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
|
|||||||
auxiliary_engine_split_asts.reserve(number_of_splits);
|
auxiliary_engine_split_asts.reserve(number_of_splits);
|
||||||
{
|
{
|
||||||
ParserExpressionWithOptionalAlias parser_expression(false);
|
ParserExpressionWithOptionalAlias parser_expression(false);
|
||||||
sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0);
|
sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second,
|
main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second,
|
||||||
sharding_key_ast);
|
sharding_key_ast);
|
||||||
|
|
||||||
@ -291,7 +295,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
|
|||||||
if (!where_condition_str.empty())
|
if (!where_condition_str.empty())
|
||||||
{
|
{
|
||||||
ParserExpressionWithOptionalAlias parser_expression(false);
|
ParserExpressionWithOptionalAlias parser_expression(false);
|
||||||
where_condition_ast = parseQuery(parser_expression, where_condition_str, 0);
|
where_condition_ast = parseQuery(parser_expression, where_condition_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
|
|
||||||
// Will use canonical expression form
|
// Will use canonical expression form
|
||||||
where_condition_str = queryToString(where_condition_ast);
|
where_condition_str = queryToString(where_condition_ast);
|
||||||
|
@ -53,7 +53,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
|||||||
const char * end = pos + query.size();
|
const char * end = pos + query.size();
|
||||||
|
|
||||||
ParserQuery parser(end);
|
ParserQuery parser(end);
|
||||||
ASTPtr res = parseQuery(parser, pos, end, "query", 0);
|
ASTPtr res = parseQuery(parser, pos, end, "query", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
|
|
||||||
if (!quiet)
|
if (!quiet)
|
||||||
{
|
{
|
||||||
|
@ -267,8 +267,10 @@ void LocalServer::processQueries()
|
|||||||
String initial_create_query = getInitialCreateTableQuery();
|
String initial_create_query = getInitialCreateTableQuery();
|
||||||
String queries_str = initial_create_query + config().getRawString("query");
|
String queries_str = initial_create_query + config().getRawString("query");
|
||||||
|
|
||||||
|
const auto & settings = context->getSettingsRef();
|
||||||
|
|
||||||
std::vector<String> queries;
|
std::vector<String> queries;
|
||||||
auto parse_res = splitMultipartQuery(queries_str, queries);
|
auto parse_res = splitMultipartQuery(queries_str, queries, settings.max_query_size, settings.max_parser_depth);
|
||||||
|
|
||||||
if (!parse_res.second)
|
if (!parse_res.second)
|
||||||
throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR);
|
throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR);
|
||||||
|
@ -120,12 +120,14 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
|
|||||||
|
|
||||||
SCOPE_EXIT(SQLFreeStmt(hstmt, SQL_DROP));
|
SCOPE_EXIT(SQLFreeStmt(hstmt, SQL_DROP));
|
||||||
|
|
||||||
|
const auto & context_settings = context->getSettingsRef();
|
||||||
|
|
||||||
/// TODO Why not do SQLColumns instead?
|
/// TODO Why not do SQLColumns instead?
|
||||||
std::string name = schema_name.empty() ? table_name : schema_name + "." + table_name;
|
std::string name = schema_name.empty() ? table_name : schema_name + "." + table_name;
|
||||||
std::stringstream ss;
|
std::stringstream ss;
|
||||||
std::string input = "SELECT * FROM " + name + " WHERE 1 = 0";
|
std::string input = "SELECT * FROM " + name + " WHERE 1 = 0";
|
||||||
ParserQueryWithOutput parser;
|
ParserQueryWithOutput parser;
|
||||||
ASTPtr select = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr select = parseQuery(parser, input.data(), input.data() + input.size(), "", context_settings.max_query_size, context_settings.max_parser_depth);
|
||||||
|
|
||||||
IAST::FormatSettings settings(ss, true);
|
IAST::FormatSettings settings(ss, true);
|
||||||
settings.always_quote_identifiers = true;
|
settings.always_quote_identifiers = true;
|
||||||
|
@ -32,6 +32,7 @@
|
|||||||
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
|
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
|
||||||
#include <Interpreters/InterpreterShowGrantsQuery.h>
|
#include <Interpreters/InterpreterShowGrantsQuery.h>
|
||||||
#include <Common/quoteString.h>
|
#include <Common/quoteString.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
#include <boost/range/adaptor/map.hpp>
|
#include <boost/range/adaptor/map.hpp>
|
||||||
#include <boost/range/algorithm/copy.hpp>
|
#include <boost/range/algorithm/copy.hpp>
|
||||||
#include <boost/range/algorithm_ext/push_back.hpp>
|
#include <boost/range/algorithm_ext/push_back.hpp>
|
||||||
@ -93,7 +94,7 @@ namespace
|
|||||||
const char * end = begin + file_contents.size();
|
const char * end = begin + file_contents.size();
|
||||||
while (pos < end)
|
while (pos < end)
|
||||||
{
|
{
|
||||||
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0));
|
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
|
||||||
while (isWhitespaceASCII(*pos) || *pos == ';')
|
while (isWhitespaceASCII(*pos) || *pos == ';')
|
||||||
++pos;
|
++pos;
|
||||||
}
|
}
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
#include <Common/quoteString.h>
|
#include <Common/quoteString.h>
|
||||||
#include <ext/range.h>
|
#include <ext/range.h>
|
||||||
#include <boost/smart_ptr/make_shared.hpp>
|
#include <boost/smart_ptr/make_shared.hpp>
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -77,7 +78,7 @@ void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_)
|
|||||||
try
|
try
|
||||||
{
|
{
|
||||||
ParserExpression parser;
|
ParserExpression parser;
|
||||||
parsed_conditions[type] = parseQuery(parser, condition, 0);
|
parsed_conditions[type] = parseQuery(parser, condition, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -65,7 +66,7 @@ void getAggregateFunctionNameAndParametersArray(
|
|||||||
ParserExpressionList params_parser(false);
|
ParserExpressionList params_parser(false);
|
||||||
ASTPtr args_ast = parseQuery(params_parser,
|
ASTPtr args_ast = parseQuery(params_parser,
|
||||||
parameters_str.data(), parameters_str.data() + parameters_str.size(),
|
parameters_str.data(), parameters_str.data() + parameters_str.size(),
|
||||||
"parameters of aggregate function in " + error_context, 0);
|
"parameters of aggregate function in " + error_context, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
|
|
||||||
if (args_ast->children.empty())
|
if (args_ast->children.empty())
|
||||||
throw Exception("Incorrect list of parameters to aggregate function "
|
throw Exception("Incorrect list of parameters to aggregate function "
|
||||||
|
@ -462,7 +462,7 @@ CompressionCodecPtr makeCodec(const std::string & codec_string, const DataTypePt
|
|||||||
{
|
{
|
||||||
const std::string codec_statement = "(" + codec_string + ")";
|
const std::string codec_statement = "(" + codec_string + ")";
|
||||||
Tokens tokens(codec_statement.begin().base(), codec_statement.end().base());
|
Tokens tokens(codec_statement.begin().base(), codec_statement.end().base());
|
||||||
IParser::Pos token_iterator(tokens);
|
IParser::Pos token_iterator(tokens, 0);
|
||||||
|
|
||||||
Expected expected;
|
Expected expected;
|
||||||
ASTPtr codec_ast;
|
ASTPtr codec_ast;
|
||||||
|
@ -95,3 +95,6 @@
|
|||||||
/// Actually, there may be multiple acquisitions of different locks for a given table within one query.
|
/// Actually, there may be multiple acquisitions of different locks for a given table within one query.
|
||||||
/// Check with IStorage class for the list of possible locks
|
/// Check with IStorage class for the list of possible locks
|
||||||
#define DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC 120
|
#define DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC 120
|
||||||
|
|
||||||
|
/// Default limit on recursion depth of recursive descend parser.
|
||||||
|
#define DBMS_DEFAULT_MAX_PARSER_DEPTH 1000
|
||||||
|
@ -404,7 +404,7 @@ struct Settings : public SettingsCollection<Settings>
|
|||||||
M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \
|
M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \
|
||||||
M(SettingUInt64, multiple_joins_rewriter_version, 1, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \
|
M(SettingUInt64, multiple_joins_rewriter_version, 1, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \
|
||||||
M(SettingBool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
|
M(SettingBool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
|
||||||
M(SettingUInt64, max_parser_depth, 1000, "Maximum parser depth.", 0) \
|
M(SettingUInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
|
||||||
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
|
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
|
||||||
M(SettingBool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
|
M(SettingBool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
|
||||||
M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \
|
M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \
|
||||||
|
@ -33,7 +33,7 @@ try
|
|||||||
std::string input = "SELECT number, number / 3, number * number";
|
std::string input = "SELECT number, number / 3, number * number";
|
||||||
|
|
||||||
ParserSelectQuery parser;
|
ParserSelectQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
|
|
||||||
Context context = Context::createGlobal();
|
Context context = Context::createGlobal();
|
||||||
context.makeGlobalContext();
|
context.makeGlobalContext();
|
||||||
|
@ -35,7 +35,7 @@ try
|
|||||||
std::string input = "SELECT number, number % 3 == 1";
|
std::string input = "SELECT number, number % 3 == 1";
|
||||||
|
|
||||||
ParserSelectQuery parser;
|
ParserSelectQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
|
|
||||||
formatAST(*ast, std::cerr);
|
formatAST(*ast, std::cerr);
|
||||||
std::cerr << std::endl;
|
std::cerr << std::endl;
|
||||||
|
@ -9,6 +9,8 @@
|
|||||||
#include <Poco/String.h>
|
#include <Poco/String.h>
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -26,7 +28,7 @@ namespace ErrorCodes
|
|||||||
DataTypePtr DataTypeFactory::get(const String & full_name) const
|
DataTypePtr DataTypeFactory::get(const String & full_name) const
|
||||||
{
|
{
|
||||||
ParserIdentifierWithOptionalParameters parser;
|
ParserIdentifierWithOptionalParameters parser;
|
||||||
ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0);
|
ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
return get(ast);
|
return get(ast);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -234,7 +234,8 @@ void DatabaseOrdinary::alterTable(
|
|||||||
}
|
}
|
||||||
|
|
||||||
ParserCreateQuery parser;
|
ParserCreateQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, statement.data(), statement.data() + statement.size(), "in file " + table_metadata_path, 0);
|
ASTPtr ast = parseQuery(parser, statement.data(), statement.data() + statement.size(), "in file " + table_metadata_path,
|
||||||
|
0, context.getSettingsRef().max_parser_depth);
|
||||||
|
|
||||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||||
|
|
||||||
|
@ -52,7 +52,7 @@ TEST(ConvertDictionaryAST, SimpleDictConfiguration)
|
|||||||
" RANGE(MIN second_column MAX third_column)";
|
" RANGE(MIN second_column MAX third_column)";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||||
|
|
||||||
@ -120,7 +120,7 @@ TEST(ConvertDictionaryAST, TrickyAttributes)
|
|||||||
" SOURCE(CLICKHOUSE(HOST 'localhost'))";
|
" SOURCE(CLICKHOUSE(HOST 'localhost'))";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||||
|
|
||||||
@ -165,7 +165,7 @@ TEST(ConvertDictionaryAST, ComplexKeyAndLayoutWithParams)
|
|||||||
" LIFETIME(MIN 1 MAX 10)";
|
" LIFETIME(MIN 1 MAX 10)";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||||
|
|
||||||
@ -216,7 +216,7 @@ TEST(ConvertDictionaryAST, ComplexSource)
|
|||||||
" RANGE(MIN second_column MAX third_column)";
|
" RANGE(MIN second_column MAX third_column)";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||||
/// source
|
/// source
|
||||||
|
@ -463,7 +463,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
|
|||||||
|
|
||||||
ParserQuery parser_query(end);
|
ParserQuery parser_query(end);
|
||||||
String description;
|
String description;
|
||||||
task.query = parseQuery(parser_query, begin, end, description, 0);
|
task.query = parseQuery(parser_query, begin, end, description, 0, context.getSettingsRef().max_parser_depth);
|
||||||
}
|
}
|
||||||
|
|
||||||
// XXX: serious design flaw since `ASTQueryWithOnCluster` is not inherited from `IAST`!
|
// XXX: serious design flaw since `ASTQueryWithOnCluster` is not inherited from `IAST`!
|
||||||
|
@ -6,6 +6,8 @@
|
|||||||
#include <Common/escapeForFileName.h>
|
#include <Common/escapeForFileName.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
#include <IO/WriteBufferFromFile.h>
|
#include <IO/WriteBufferFromFile.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
@ -181,7 +183,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
|
|||||||
String type_name = column.type->getName();
|
String type_name = column.type->getName();
|
||||||
auto pos = type_name.data();
|
auto pos = type_name.data();
|
||||||
const auto end = pos + type_name.size();
|
const auto end = pos + type_name.size();
|
||||||
column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0);
|
column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
columns_list->children.emplace_back(column_declaration);
|
columns_list->children.emplace_back(column_declaration);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -207,7 +209,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
|
|||||||
String type_name = column.type->getName();
|
String type_name = column.type->getName();
|
||||||
auto type_name_pos = type_name.data();
|
auto type_name_pos = type_name.data();
|
||||||
const auto type_name_end = type_name_pos + type_name.size();
|
const auto type_name_end = type_name_pos + type_name.size();
|
||||||
column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0);
|
column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
|
|
||||||
if (column.default_desc.expression)
|
if (column.default_desc.expression)
|
||||||
{
|
{
|
||||||
@ -227,7 +229,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
|
|||||||
auto codec_desc_pos = codec_desc.data();
|
auto codec_desc_pos = codec_desc.data();
|
||||||
const auto codec_desc_end = codec_desc_pos + codec_desc.size();
|
const auto codec_desc_end = codec_desc_pos + codec_desc.size();
|
||||||
ParserIdentifierWithParameters codec_p;
|
ParserIdentifierWithParameters codec_p;
|
||||||
column_declaration->codec = parseQuery(codec_p, codec_desc_pos, codec_desc_end, "column codec", 0);
|
column_declaration->codec = parseQuery(codec_p, codec_desc_pos, codec_desc_end, "column codec", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (column.ttl)
|
if (column.ttl)
|
||||||
|
@ -267,7 +267,7 @@ BlockIO InterpreterKillQueryQuery::execute()
|
|||||||
else
|
else
|
||||||
{
|
{
|
||||||
ParserAlterCommand parser;
|
ParserAlterCommand parser;
|
||||||
auto command_ast = parseQuery(parser, command_col.getDataAt(i).toString(), 0);
|
auto command_ast = parseQuery(parser, command_col.getDataAt(i).toString(), 0, context.getSettingsRef().max_parser_depth);
|
||||||
required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand(command_ast->as<const ASTAlterCommand &>(), table_id.database_name, table_id.table_name);
|
required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand(command_ast->as<const ASTAlterCommand &>(), table_id.database_name, table_id.table_name);
|
||||||
if (!access->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), required_access_rights))
|
if (!access->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), required_access_rights))
|
||||||
{
|
{
|
||||||
|
@ -137,7 +137,7 @@ String InterpreterSelectQuery::generateFilterActions(ExpressionActionsPtr & acti
|
|||||||
for (const auto & column_str : prerequisite_columns)
|
for (const auto & column_str : prerequisite_columns)
|
||||||
{
|
{
|
||||||
ParserExpression expr_parser;
|
ParserExpression expr_parser;
|
||||||
expr_list->children.push_back(parseQuery(expr_parser, column_str, 0));
|
expr_list->children.push_back(parseQuery(expr_parser, column_str, 0, context->getSettingsRef().max_parser_depth));
|
||||||
}
|
}
|
||||||
|
|
||||||
select_ast->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared<ASTTablesInSelectQuery>());
|
select_ast->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared<ASTTablesInSelectQuery>());
|
||||||
|
@ -21,6 +21,7 @@
|
|||||||
#include <DataStreams/OneBlockInputStream.h>
|
#include <DataStreams/OneBlockInputStream.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
#include <ext/range.h>
|
#include <ext/range.h>
|
||||||
#include <sstream>
|
#include <sstream>
|
||||||
|
|
||||||
@ -37,7 +38,7 @@ namespace
|
|||||||
ASTPtr getCreateQueryImpl(
|
ASTPtr getCreateQueryImpl(
|
||||||
const User & user,
|
const User & user,
|
||||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||||
bool attach_mode = false)
|
bool attach_mode)
|
||||||
{
|
{
|
||||||
auto query = std::make_shared<ASTCreateUserQuery>();
|
auto query = std::make_shared<ASTCreateUserQuery>();
|
||||||
query->name = user.getName();
|
query->name = user.getName();
|
||||||
@ -72,7 +73,7 @@ namespace
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager * manager, bool attach_mode = false)
|
ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager * manager, bool attach_mode)
|
||||||
{
|
{
|
||||||
auto query = std::make_shared<ASTCreateRoleQuery>();
|
auto query = std::make_shared<ASTCreateRoleQuery>();
|
||||||
query->name = role.getName();
|
query->name = role.getName();
|
||||||
@ -90,7 +91,7 @@ namespace
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControlManager * manager, bool attach_mode = false)
|
ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControlManager * manager, bool attach_mode)
|
||||||
{
|
{
|
||||||
auto query = std::make_shared<ASTCreateSettingsProfileQuery>();
|
auto query = std::make_shared<ASTCreateSettingsProfileQuery>();
|
||||||
query->name = profile.getName();
|
query->name = profile.getName();
|
||||||
@ -121,7 +122,7 @@ namespace
|
|||||||
ASTPtr getCreateQueryImpl(
|
ASTPtr getCreateQueryImpl(
|
||||||
const Quota & quota,
|
const Quota & quota,
|
||||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||||
bool attach_mode = false)
|
bool attach_mode)
|
||||||
{
|
{
|
||||||
auto query = std::make_shared<ASTCreateQuotaQuery>();
|
auto query = std::make_shared<ASTCreateQuotaQuery>();
|
||||||
query->name = quota.getName();
|
query->name = quota.getName();
|
||||||
@ -156,7 +157,7 @@ namespace
|
|||||||
ASTPtr getCreateQueryImpl(
|
ASTPtr getCreateQueryImpl(
|
||||||
const RowPolicy & policy,
|
const RowPolicy & policy,
|
||||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||||
bool attach_mode = false)
|
bool attach_mode)
|
||||||
{
|
{
|
||||||
auto query = std::make_shared<ASTCreateRowPolicyQuery>();
|
auto query = std::make_shared<ASTCreateRowPolicyQuery>();
|
||||||
query->name_parts = RowPolicy::FullNameParts{policy.getDatabase(), policy.getTableName(), policy.getName()};
|
query->name_parts = RowPolicy::FullNameParts{policy.getDatabase(), policy.getTableName(), policy.getName()};
|
||||||
@ -171,7 +172,7 @@ namespace
|
|||||||
if (!condition.empty())
|
if (!condition.empty())
|
||||||
{
|
{
|
||||||
ParserExpression parser;
|
ParserExpression parser;
|
||||||
ASTPtr expr = parseQuery(parser, condition, 0);
|
ASTPtr expr = parseQuery(parser, condition, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
query->conditions.push_back(std::pair{index, expr});
|
query->conditions.push_back(std::pair{index, expr});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -190,7 +191,7 @@ namespace
|
|||||||
ASTPtr getCreateQueryImpl(
|
ASTPtr getCreateQueryImpl(
|
||||||
const IAccessEntity & entity,
|
const IAccessEntity & entity,
|
||||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||||
bool attach_mode = false)
|
bool attach_mode)
|
||||||
{
|
{
|
||||||
if (const User * user = typeid_cast<const User *>(&entity))
|
if (const User * user = typeid_cast<const User *>(&entity))
|
||||||
return getCreateQueryImpl(*user, manager, attach_mode);
|
return getCreateQueryImpl(*user, manager, attach_mode);
|
||||||
@ -264,24 +265,24 @@ ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreat
|
|||||||
if (show_query.current_user)
|
if (show_query.current_user)
|
||||||
{
|
{
|
||||||
auto user = context.getUser();
|
auto user = context.getUser();
|
||||||
return getCreateQueryImpl(*user, &access_control);
|
return getCreateQueryImpl(*user, &access_control, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (show_query.current_quota)
|
if (show_query.current_quota)
|
||||||
{
|
{
|
||||||
auto quota = access_control.read<Quota>(context.getQuota()->getUsageInfo().quota_id);
|
auto quota = access_control.read<Quota>(context.getQuota()->getUsageInfo().quota_id);
|
||||||
return getCreateQueryImpl(*quota, &access_control);
|
return getCreateQueryImpl(*quota, &access_control, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
auto type = getType(show_query.kind);
|
auto type = getType(show_query.kind);
|
||||||
if (show_query.kind == Kind::ROW_POLICY)
|
if (show_query.kind == Kind::ROW_POLICY)
|
||||||
{
|
{
|
||||||
RowPolicyPtr policy = access_control.read<RowPolicy>(show_query.row_policy_name.getFullName(context));
|
RowPolicyPtr policy = access_control.read<RowPolicy>(show_query.row_policy_name.getFullName(context));
|
||||||
return getCreateQueryImpl(*policy, &access_control);
|
return getCreateQueryImpl(*policy, &access_control, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
auto entity = access_control.read(access_control.getID(type, show_query.name));
|
auto entity = access_control.read(access_control.getID(type, show_query.name));
|
||||||
return getCreateQueryImpl(*entity, &access_control);
|
return getCreateQueryImpl(*entity, &access_control, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -35,7 +36,7 @@ namespace
|
|||||||
ASTPtr makeSubqueryTemplate()
|
ASTPtr makeSubqueryTemplate()
|
||||||
{
|
{
|
||||||
ParserTablesInSelectQueryElement parser(true);
|
ParserTablesInSelectQueryElement parser(true);
|
||||||
ASTPtr subquery_template = parseQuery(parser, "(select * from _t) as `--.s`", 0);
|
ASTPtr subquery_template = parseQuery(parser, "(select * from _t) as `--.s`", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
if (!subquery_template)
|
if (!subquery_template)
|
||||||
throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR);
|
||||||
return subquery_template;
|
return subquery_template;
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Core/Settings.h>
|
#include <Core/Settings.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
#include <Core/NamesAndTypes.h>
|
#include <Core/NamesAndTypes.h>
|
||||||
|
|
||||||
#include <Interpreters/SyntaxAnalyzer.h>
|
#include <Interpreters/SyntaxAnalyzer.h>
|
||||||
@ -587,7 +588,7 @@ void replaceJoinedTable(const ASTSelectQuery & select_query)
|
|||||||
if (table_id.alias.empty() && table_id.isShort())
|
if (table_id.alias.empty() && table_id.isShort())
|
||||||
{
|
{
|
||||||
ParserTableExpression parser;
|
ParserTableExpression parser;
|
||||||
table_expr = parseQuery(parser, expr, 0)->as<ASTTableExpression &>();
|
table_expr = parseQuery(parser, expr, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH)->as<ASTTableExpression &>();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
#include <boost/noncopyable.hpp>
|
#include <boost/noncopyable.hpp>
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
#include <Core/Types.h>
|
#include <Core/Types.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Common/Stopwatch.h>
|
#include <Common/Stopwatch.h>
|
||||||
@ -460,7 +461,7 @@ ASTPtr SystemLog<LogElement>::getCreateTableQuery()
|
|||||||
ParserStorage storage_parser;
|
ParserStorage storage_parser;
|
||||||
ASTPtr storage_ast = parseQuery(
|
ASTPtr storage_ast = parseQuery(
|
||||||
storage_parser, storage_def.data(), storage_def.data() + storage_def.size(),
|
storage_parser, storage_def.data(), storage_def.data() + storage_def.size(),
|
||||||
"Storage to create table for " + LogElement::name(), 0);
|
"Storage to create table for " + LogElement::name(), 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
create->set(create->storage, storage_ast);
|
create->set(create->storage, storage_ast);
|
||||||
|
|
||||||
return create;
|
return create;
|
||||||
|
@ -36,7 +36,7 @@ static void executeCreateQuery(
|
|||||||
bool has_force_restore_data_flag)
|
bool has_force_restore_data_flag)
|
||||||
{
|
{
|
||||||
ParserCreateQuery parser;
|
ParserCreateQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0);
|
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0, context.getSettingsRef().max_parser_depth);
|
||||||
|
|
||||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||||
ast_create_query.attach = true;
|
ast_create_query.attach = true;
|
||||||
|
@ -47,9 +47,6 @@ add_executable (two_level_hash_map two_level_hash_map.cpp)
|
|||||||
target_include_directories (two_level_hash_map SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR})
|
target_include_directories (two_level_hash_map SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR})
|
||||||
target_link_libraries (two_level_hash_map PRIVATE dbms)
|
target_link_libraries (two_level_hash_map PRIVATE dbms)
|
||||||
|
|
||||||
add_executable (logical_expressions_optimizer logical_expressions_optimizer.cpp)
|
|
||||||
target_link_libraries (logical_expressions_optimizer PRIVATE dbms clickhouse_parsers)
|
|
||||||
|
|
||||||
add_executable (in_join_subqueries_preprocessor in_join_subqueries_preprocessor.cpp)
|
add_executable (in_join_subqueries_preprocessor in_join_subqueries_preprocessor.cpp)
|
||||||
target_link_libraries (in_join_subqueries_preprocessor PRIVATE dbms clickhouse_parsers)
|
target_link_libraries (in_join_subqueries_preprocessor PRIVATE dbms clickhouse_parsers)
|
||||||
add_check(in_join_subqueries_preprocessor)
|
add_check(in_join_subqueries_preprocessor)
|
||||||
|
@ -76,7 +76,7 @@ try
|
|||||||
") ENGINE = Log";
|
") ENGINE = Log";
|
||||||
|
|
||||||
ParserCreateQuery parser;
|
ParserCreateQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
|
|
||||||
Context context = Context::createGlobal();
|
Context context = Context::createGlobal();
|
||||||
context.makeGlobalContext();
|
context.makeGlobalContext();
|
||||||
|
@ -41,7 +41,7 @@ int main(int argc, char ** argv)
|
|||||||
"s1 < s2 AND x % 3 < x % 5";
|
"s1 < s2 AND x % 3 < x % 5";
|
||||||
|
|
||||||
ParserSelectQuery parser;
|
ParserSelectQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
|
|
||||||
formatAST(*ast, std::cerr);
|
formatAST(*ast, std::cerr);
|
||||||
std::cerr << std::endl;
|
std::cerr << std::endl;
|
||||||
|
@ -72,7 +72,7 @@ private:
|
|||||||
ParserSelectQuery parser;
|
ParserSelectQuery parser;
|
||||||
std::string message;
|
std::string message;
|
||||||
auto text = query.data();
|
auto text = query.data();
|
||||||
if (ASTPtr ast = tryParseQuery(parser, text, text + query.size(), message, false, "", false, 0))
|
if (ASTPtr ast = tryParseQuery(parser, text, text + query.size(), message, false, "", false, 0, 0))
|
||||||
return ast;
|
return ast;
|
||||||
throw Exception(message, ErrorCodes::SYNTAX_ERROR);
|
throw Exception(message, ErrorCodes::SYNTAX_ERROR);
|
||||||
}
|
}
|
||||||
|
@ -12,11 +12,11 @@ TEST(QueryNormalizer, SimpleCycleAlias)
|
|||||||
{
|
{
|
||||||
String query = "a as b, b as a";
|
String query = "a as b, b as a";
|
||||||
ParserExpressionList parser(false);
|
ParserExpressionList parser(false);
|
||||||
ASTPtr ast = parseQuery(parser, query, 0);
|
ASTPtr ast = parseQuery(parser, query, 0, 0);
|
||||||
|
|
||||||
Aliases aliases;
|
Aliases aliases;
|
||||||
aliases["a"] = parseQuery(parser, "b as a", 0)->children[0];
|
aliases["a"] = parseQuery(parser, "b as a", 0, 0)->children[0];
|
||||||
aliases["b"] = parseQuery(parser, "a as b", 0)->children[0];
|
aliases["b"] = parseQuery(parser, "a as b", 0, 0)->children[0];
|
||||||
|
|
||||||
Settings settings;
|
Settings settings;
|
||||||
QueryNormalizer::Data normalizer_data(aliases, settings);
|
QueryNormalizer::Data normalizer_data(aliases, settings);
|
||||||
|
@ -1224,7 +1224,7 @@ bool parse(DB::ASTPtr & ast, const std::string & query)
|
|||||||
std::string message;
|
std::string message;
|
||||||
auto begin = query.data();
|
auto begin = query.data();
|
||||||
auto end = begin + query.size();
|
auto end = begin + query.size();
|
||||||
ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0);
|
ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0, 0);
|
||||||
return ast != nullptr;
|
return ast != nullptr;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,295 +0,0 @@
|
|||||||
#include <Parsers/ASTSelectQuery.h>
|
|
||||||
#include <Parsers/ParserSelectQuery.h>
|
|
||||||
#include <Parsers/parseQuery.h>
|
|
||||||
#include <Parsers/queryToString.h>
|
|
||||||
#include <Interpreters/LogicalExpressionsOptimizer.h>
|
|
||||||
#include <Core/Settings.h>
|
|
||||||
#include <Common/typeid_cast.h>
|
|
||||||
|
|
||||||
#include <iostream>
|
|
||||||
#include <vector>
|
|
||||||
#include <utility>
|
|
||||||
#include <string>
|
|
||||||
#include <algorithm>
|
|
||||||
|
|
||||||
|
|
||||||
namespace
|
|
||||||
{
|
|
||||||
|
|
||||||
struct TestEntry
|
|
||||||
{
|
|
||||||
std::string input;
|
|
||||||
std::string expected_output;
|
|
||||||
UInt64 limit;
|
|
||||||
};
|
|
||||||
|
|
||||||
using TestEntries = std::vector<TestEntry>;
|
|
||||||
using TestResult = std::pair<bool, std::string>;
|
|
||||||
|
|
||||||
void run();
|
|
||||||
void performTests(const TestEntries & entries);
|
|
||||||
TestResult check(const TestEntry & entry);
|
|
||||||
bool parse(DB::ASTPtr & ast, const std::string & query);
|
|
||||||
bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs);
|
|
||||||
void reorder(DB::IAST * ast);
|
|
||||||
|
|
||||||
|
|
||||||
void run()
|
|
||||||
{
|
|
||||||
/// NOTE: Queries are not always realistic, but we are only interested in the syntax.
|
|
||||||
TestEntries entries =
|
|
||||||
{
|
|
||||||
{
|
|
||||||
"SELECT 1",
|
|
||||||
"SELECT 1",
|
|
||||||
3
|
|
||||||
},
|
|
||||||
|
|
||||||
// WHERE
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
"SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
4
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
"SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol')",
|
|
||||||
3
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
"SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol')",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report WHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
"SELECT name, value FROM report WHERE (value = 1000) OR name IN ('Alice', 'Bob', 'Carol')",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report WHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol') OR (value = 2000)",
|
|
||||||
"SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol') OR value IN (1000, 2000)",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT value FROM report WHERE ((value + 1) = 1000) OR ((2 * value) = 2000) OR ((2 * value) = 4000) OR ((value + 1) = 3000)",
|
|
||||||
"SELECT value FROM report WHERE ((value + 1) IN (1000, 3000)) OR ((2 * value) IN (2000, 4000))",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report WHERE ((name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')) AND ((value = 1000) OR (value = 2000))",
|
|
||||||
"SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol') AND ((value = 1000) OR (value = 2000))",
|
|
||||||
3
|
|
||||||
},
|
|
||||||
|
|
||||||
// PREWHERE
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
"SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
4
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
"SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol')",
|
|
||||||
3
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
"SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol')",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report PREWHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol')",
|
|
||||||
"SELECT name, value FROM report PREWHERE (value = 1000) OR name IN ('Alice', 'Bob', 'Carol')",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT name, value FROM report PREWHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol') OR (value = 2000)",
|
|
||||||
"SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol') OR value IN (1000, 2000)",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT value FROM report PREWHERE ((value + 1) = 1000) OR ((2 * value) = 2000) OR ((2 * value) = 4000) OR ((value + 1) = 3000)",
|
|
||||||
"SELECT value FROM report PREWHERE (value + 1) IN (1000, 3000) OR (2 * value) IN (2000, 4000)",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
// HAVING
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number = 1",
|
|
||||||
"SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number = 1",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)",
|
|
||||||
"SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number IN (1, 2)",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)",
|
|
||||||
"SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)",
|
|
||||||
3
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING ((number + 1) = 1) OR ((number + 1) = 2) OR ((number + 3) = 7)",
|
|
||||||
"SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING ((number + 3) = 7) OR (number + 1) IN (1, 2)",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
// PREWHERE + WHERE + HAVING
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE (U = 1) OR (U = 2) "
|
|
||||||
"WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)",
|
|
||||||
"SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE U IN (1, 2) "
|
|
||||||
"WHERE T IN (1, 2) GROUP BY number HAVING number IN (1, 2)",
|
|
||||||
2
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE (U = 1) OR (U = 2) OR (U = 3) "
|
|
||||||
"WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)",
|
|
||||||
"SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE U IN (1, 2, 3) "
|
|
||||||
"WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)",
|
|
||||||
3
|
|
||||||
},
|
|
||||||
|
|
||||||
{
|
|
||||||
"SELECT x = 1 OR x=2 OR (x = 3 AS x3) AS y, 4 AS x",
|
|
||||||
"SELECT x IN (1, 2, 3) AS y, 4 AS x",
|
|
||||||
2
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
performTests(entries);
|
|
||||||
}
|
|
||||||
|
|
||||||
void performTests(const TestEntries & entries)
|
|
||||||
{
|
|
||||||
unsigned int count = 0;
|
|
||||||
unsigned int i = 1;
|
|
||||||
|
|
||||||
for (const auto & entry : entries)
|
|
||||||
{
|
|
||||||
auto res = check(entry);
|
|
||||||
if (res.first)
|
|
||||||
{
|
|
||||||
++count;
|
|
||||||
std::cout << "Test " << i << " passed.\n";
|
|
||||||
}
|
|
||||||
else
|
|
||||||
std::cout << "Test " << i << " failed. Expected: " << entry.expected_output << ". Received: " << res.second << "\n";
|
|
||||||
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
std::cout << count << " out of " << entries.size() << " test(s) passed.\n";
|
|
||||||
}
|
|
||||||
|
|
||||||
TestResult check(const TestEntry & entry)
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
/// Parse and optimize the incoming query.
|
|
||||||
DB::ASTPtr ast_input;
|
|
||||||
if (!parse(ast_input, entry.input))
|
|
||||||
return TestResult(false, "parse error");
|
|
||||||
|
|
||||||
auto select_query = typeid_cast<DB::ASTSelectQuery *>(&*ast_input);
|
|
||||||
|
|
||||||
DB::LogicalExpressionsOptimizer optimizer(select_query, entry.limit);
|
|
||||||
optimizer.perform();
|
|
||||||
|
|
||||||
/// Parse the expected result.
|
|
||||||
DB::ASTPtr ast_expected;
|
|
||||||
if (!parse(ast_expected, entry.expected_output))
|
|
||||||
return TestResult(false, "parse error");
|
|
||||||
|
|
||||||
/// Compare the optimized query and the expected result.
|
|
||||||
bool res = equals(ast_input, ast_expected);
|
|
||||||
std::string output = DB::queryToString(ast_input);
|
|
||||||
|
|
||||||
return TestResult(res, output);
|
|
||||||
}
|
|
||||||
catch (DB::Exception & e)
|
|
||||||
{
|
|
||||||
return TestResult(false, e.displayText());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
bool parse(DB::ASTPtr & ast, const std::string & query)
|
|
||||||
{
|
|
||||||
DB::ParserSelectQuery parser;
|
|
||||||
std::string message;
|
|
||||||
auto begin = query.data();
|
|
||||||
auto end = begin + query.size();
|
|
||||||
ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0);
|
|
||||||
return ast != nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)
|
|
||||||
{
|
|
||||||
DB::ASTPtr lhs_reordered = lhs->clone();
|
|
||||||
reorder(&*lhs_reordered);
|
|
||||||
|
|
||||||
DB::ASTPtr rhs_reordered = rhs->clone();
|
|
||||||
reorder(&*rhs_reordered);
|
|
||||||
|
|
||||||
return lhs_reordered->getTreeHash() == rhs_reordered->getTreeHash();
|
|
||||||
}
|
|
||||||
|
|
||||||
void reorderImpl(DB::IAST * ast)
|
|
||||||
{
|
|
||||||
if (ast == nullptr)
|
|
||||||
return;
|
|
||||||
|
|
||||||
auto & children = ast->children;
|
|
||||||
if (children.empty())
|
|
||||||
return;
|
|
||||||
|
|
||||||
for (auto & child : children)
|
|
||||||
reorderImpl(&*child);
|
|
||||||
|
|
||||||
std::sort(children.begin(), children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)
|
|
||||||
{
|
|
||||||
return lhs->getTreeHash() < rhs->getTreeHash();
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
void reorder(DB::IAST * ast)
|
|
||||||
{
|
|
||||||
if (ast == nullptr)
|
|
||||||
return;
|
|
||||||
|
|
||||||
auto select_query = typeid_cast<DB::ASTSelectQuery *>(ast);
|
|
||||||
if (select_query == nullptr)
|
|
||||||
return;
|
|
||||||
|
|
||||||
reorderImpl(select_query->where().get());
|
|
||||||
reorderImpl(select_query->prewhere().get());
|
|
||||||
reorderImpl(select_query->having().get());
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
int main()
|
|
||||||
{
|
|
||||||
run();
|
|
||||||
return 0;
|
|
||||||
}
|
|
@ -56,8 +56,6 @@ public:
|
|||||||
/// Token iterator augmented with depth information. This allows to control recursion depth.
|
/// Token iterator augmented with depth information. This allows to control recursion depth.
|
||||||
struct Pos : TokenIterator
|
struct Pos : TokenIterator
|
||||||
{
|
{
|
||||||
using TokenIterator::TokenIterator;
|
|
||||||
|
|
||||||
uint32_t depth = 0;
|
uint32_t depth = 0;
|
||||||
uint32_t max_depth = 0;
|
uint32_t max_depth = 0;
|
||||||
|
|
||||||
|
@ -328,19 +328,28 @@ ASTPtr parseQuery(
|
|||||||
IParser & parser,
|
IParser & parser,
|
||||||
const std::string & query,
|
const std::string & query,
|
||||||
const std::string & query_description,
|
const std::string & query_description,
|
||||||
size_t max_query_size)
|
size_t max_query_size,
|
||||||
|
size_t max_parser_depth)
|
||||||
{
|
{
|
||||||
return parseQuery(parser, query.data(), query.data() + query.size(), query_description, max_query_size);
|
return parseQuery(parser, query.data(), query.data() + query.size(), query_description, max_query_size, max_parser_depth);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
ASTPtr parseQuery(IParser & parser, const std::string & query, size_t max_query_size)
|
ASTPtr parseQuery(
|
||||||
|
IParser & parser,
|
||||||
|
const std::string & query,
|
||||||
|
size_t max_query_size,
|
||||||
|
size_t max_parser_depth)
|
||||||
{
|
{
|
||||||
return parseQuery(parser, query.data(), query.data() + query.size(), parser.getName(), max_query_size);
|
return parseQuery(parser, query.data(), query.data() + query.size(), parser.getName(), max_query_size, max_parser_depth);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
std::pair<const char *, bool> splitMultipartQuery(const std::string & queries, std::vector<std::string> & queries_list)
|
std::pair<const char *, bool> splitMultipartQuery(
|
||||||
|
const std::string & queries,
|
||||||
|
std::vector<std::string> & queries_list,
|
||||||
|
size_t max_query_size,
|
||||||
|
size_t max_parser_depth)
|
||||||
{
|
{
|
||||||
ASTPtr ast;
|
ASTPtr ast;
|
||||||
|
|
||||||
@ -356,7 +365,7 @@ std::pair<const char *, bool> splitMultipartQuery(const std::string & queries, s
|
|||||||
{
|
{
|
||||||
begin = pos;
|
begin = pos;
|
||||||
|
|
||||||
ast = parseQueryAndMovePosition(parser, pos, end, "", true, 0);
|
ast = parseQueryAndMovePosition(parser, pos, end, "", true, max_query_size, max_parser_depth);
|
||||||
|
|
||||||
auto * insert = ast->as<ASTInsertQuery>();
|
auto * insert = ast->as<ASTInsertQuery>();
|
||||||
|
|
||||||
|
@ -17,7 +17,7 @@ ASTPtr tryParseQuery(
|
|||||||
bool allow_multi_statements, /// If false, check for non-space characters after semicolon and set error message if any.
|
bool allow_multi_statements, /// If false, check for non-space characters after semicolon and set error message if any.
|
||||||
size_t max_query_size, /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded".
|
size_t max_query_size, /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded".
|
||||||
/// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query.
|
/// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query.
|
||||||
size_t max_parser_depth = 0);
|
size_t max_parser_depth);
|
||||||
|
|
||||||
|
|
||||||
/// Parse query or throw an exception with error message.
|
/// Parse query or throw an exception with error message.
|
||||||
@ -27,8 +27,8 @@ ASTPtr parseQueryAndMovePosition(
|
|||||||
const char * end,
|
const char * end,
|
||||||
const std::string & description,
|
const std::string & description,
|
||||||
bool allow_multi_statements,
|
bool allow_multi_statements,
|
||||||
size_t max_query_size = 0,
|
size_t max_query_size,
|
||||||
size_t max_parser_depth = 0);
|
size_t max_parser_depth);
|
||||||
|
|
||||||
ASTPtr parseQuery(
|
ASTPtr parseQuery(
|
||||||
IParser & parser,
|
IParser & parser,
|
||||||
@ -36,24 +36,30 @@ ASTPtr parseQuery(
|
|||||||
const char * end,
|
const char * end,
|
||||||
const std::string & description,
|
const std::string & description,
|
||||||
size_t max_query_size,
|
size_t max_query_size,
|
||||||
size_t max_parser_depth = 0);
|
size_t max_parser_depth);
|
||||||
|
|
||||||
ASTPtr parseQuery(
|
ASTPtr parseQuery(
|
||||||
IParser & parser,
|
IParser & parser,
|
||||||
const std::string & query,
|
const std::string & query,
|
||||||
const std::string & query_description,
|
const std::string & query_description,
|
||||||
size_t max_query_size);
|
size_t max_query_size,
|
||||||
|
size_t max_parser_depth);
|
||||||
|
|
||||||
ASTPtr parseQuery(
|
ASTPtr parseQuery(
|
||||||
IParser & parser,
|
IParser & parser,
|
||||||
const std::string & query,
|
const std::string & query,
|
||||||
size_t max_query_size);
|
size_t max_query_size,
|
||||||
|
size_t max_parser_depth);
|
||||||
|
|
||||||
|
|
||||||
/** Split queries separated by ; on to list of single queries
|
/** Split queries separated by ; on to list of single queries
|
||||||
* Returns pointer to the end of last successfully parsed query (first), and true if all queries are successfully parsed (second)
|
* Returns pointer to the end of last successfully parsed query (first), and true if all queries are successfully parsed (second)
|
||||||
* NOTE: INSERT's data should be placed in single line.
|
* NOTE: INSERT's data should be placed in single line.
|
||||||
*/
|
*/
|
||||||
std::pair<const char *, bool> splitMultipartQuery(const std::string & queries, std::vector<std::string> & queries_list);
|
std::pair<const char *, bool> splitMultipartQuery(
|
||||||
|
const std::string & queries,
|
||||||
|
std::vector<std::string> & queries_list,
|
||||||
|
size_t max_query_size,
|
||||||
|
size_t max_parser_depth);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -12,7 +12,7 @@ int main(int, char **)
|
|||||||
|
|
||||||
std::string input = "CREATE TABLE hits (URL String, UserAgentMinor2 FixedString(2), EventTime DateTime) ENGINE = Log";
|
std::string input = "CREATE TABLE hits (URL String, UserAgentMinor2 FixedString(2), EventTime DateTime) ENGINE = Log";
|
||||||
ParserCreateQuery parser;
|
ParserCreateQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
|
|
||||||
formatAST(*ast, std::cerr);
|
formatAST(*ast, std::cerr);
|
||||||
std::cerr << std::endl;
|
std::cerr << std::endl;
|
||||||
|
@ -43,7 +43,7 @@ TEST(ParserDictionaryDDL, SimpleDictionary)
|
|||||||
" RANGE(MIN second_column MAX third_column)";
|
" RANGE(MIN second_column MAX third_column)";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
EXPECT_EQ(create->table, "dict1");
|
EXPECT_EQ(create->table, "dict1");
|
||||||
EXPECT_EQ(create->database, "test");
|
EXPECT_EQ(create->database, "test");
|
||||||
@ -139,7 +139,7 @@ TEST(ParserDictionaryDDL, AttributesWithMultipleProperties)
|
|||||||
" SOURCE(CLICKHOUSE(HOST 'localhost'))";
|
" SOURCE(CLICKHOUSE(HOST 'localhost'))";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
EXPECT_EQ(create->table, "dict2");
|
EXPECT_EQ(create->table, "dict2");
|
||||||
EXPECT_EQ(create->database, "");
|
EXPECT_EQ(create->database, "");
|
||||||
@ -186,7 +186,7 @@ TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder)
|
|||||||
" LIFETIME(300)";
|
" LIFETIME(300)";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
|
|
||||||
/// test attributes
|
/// test attributes
|
||||||
@ -241,7 +241,7 @@ TEST(ParserDictionaryDDL, NestedSource)
|
|||||||
" RANGE(MIN second_column MAX third_column)";
|
" RANGE(MIN second_column MAX third_column)";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
EXPECT_EQ(create->table, "dict4");
|
EXPECT_EQ(create->table, "dict4");
|
||||||
EXPECT_EQ(create->database, "");
|
EXPECT_EQ(create->database, "");
|
||||||
@ -289,7 +289,7 @@ TEST(ParserDictionaryDDL, Formatting)
|
|||||||
" RANGE(MIN second_column MAX third_column)";
|
" RANGE(MIN second_column MAX third_column)";
|
||||||
|
|
||||||
ParserCreateDictionaryQuery parser;
|
ParserCreateDictionaryQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||||
auto str = serializeAST(*create, true);
|
auto str = serializeAST(*create, true);
|
||||||
EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)");
|
EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)");
|
||||||
@ -300,7 +300,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery)
|
|||||||
String input1 = "DROP DICTIONARY test.dict1";
|
String input1 = "DROP DICTIONARY test.dict1";
|
||||||
|
|
||||||
ParserDropQuery parser;
|
ParserDropQuery parser;
|
||||||
ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0);
|
ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0, 0);
|
||||||
ASTDropQuery * drop1 = ast1->as<ASTDropQuery>();
|
ASTDropQuery * drop1 = ast1->as<ASTDropQuery>();
|
||||||
|
|
||||||
EXPECT_TRUE(drop1->is_dictionary);
|
EXPECT_TRUE(drop1->is_dictionary);
|
||||||
@ -311,7 +311,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery)
|
|||||||
|
|
||||||
String input2 = "DROP DICTIONARY IF EXISTS dict2";
|
String input2 = "DROP DICTIONARY IF EXISTS dict2";
|
||||||
|
|
||||||
ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0);
|
ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0, 0);
|
||||||
ASTDropQuery * drop2 = ast2->as<ASTDropQuery>();
|
ASTDropQuery * drop2 = ast2->as<ASTDropQuery>();
|
||||||
|
|
||||||
EXPECT_TRUE(drop2->is_dictionary);
|
EXPECT_TRUE(drop2->is_dictionary);
|
||||||
@ -326,7 +326,7 @@ TEST(ParserDictionaryDDL, ParsePropertiesQueries)
|
|||||||
String input1 = "SHOW CREATE DICTIONARY test.dict1";
|
String input1 = "SHOW CREATE DICTIONARY test.dict1";
|
||||||
|
|
||||||
ParserTablePropertiesQuery parser;
|
ParserTablePropertiesQuery parser;
|
||||||
ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0);
|
ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0, 0);
|
||||||
ASTShowCreateDictionaryQuery * show1 = ast1->as<ASTShowCreateDictionaryQuery>();
|
ASTShowCreateDictionaryQuery * show1 = ast1->as<ASTShowCreateDictionaryQuery>();
|
||||||
|
|
||||||
EXPECT_EQ(show1->table, "dict1");
|
EXPECT_EQ(show1->table, "dict1");
|
||||||
@ -335,7 +335,7 @@ TEST(ParserDictionaryDDL, ParsePropertiesQueries)
|
|||||||
|
|
||||||
String input2 = "EXISTS DICTIONARY dict2";
|
String input2 = "EXISTS DICTIONARY dict2";
|
||||||
|
|
||||||
ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0);
|
ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0, 0);
|
||||||
ASTExistsDictionaryQuery * show2 = ast2->as<ASTExistsDictionaryQuery>();
|
ASTExistsDictionaryQuery * show2 = ast2->as<ASTExistsDictionaryQuery>();
|
||||||
|
|
||||||
EXPECT_EQ(show2->table, "dict2");
|
EXPECT_EQ(show2->table, "dict2");
|
||||||
|
@ -22,7 +22,7 @@ try
|
|||||||
" FORMAT TabSeparated";
|
" FORMAT TabSeparated";
|
||||||
|
|
||||||
ParserQueryWithOutput parser;
|
ParserQueryWithOutput parser;
|
||||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||||
|
|
||||||
std::cout << "Success." << std::endl;
|
std::cout << "Success." << std::endl;
|
||||||
formatAST(*ast, std::cerr);
|
formatAST(*ast, std::cerr);
|
||||||
|
@ -23,11 +23,13 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
#include <Compression/CompressionFactory.h>
|
#include <Compression/CompressionFactory.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/SyntaxAnalyzer.h>
|
#include <Interpreters/SyntaxAnalyzer.h>
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -102,7 +104,7 @@ void ColumnDescription::readText(ReadBuffer & buf)
|
|||||||
ParserColumnDeclaration column_parser(/* require type */ true);
|
ParserColumnDeclaration column_parser(/* require type */ true);
|
||||||
String column_line;
|
String column_line;
|
||||||
readEscapedStringUntilEOL(column_line, buf);
|
readEscapedStringUntilEOL(column_line, buf);
|
||||||
ASTPtr ast = parseQuery(column_parser, column_line, "column parser", 0);
|
ASTPtr ast = parseQuery(column_parser, column_line, "column parser", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
if (const auto * col_ast = ast->as<ASTColumnDeclaration>())
|
if (const auto * col_ast = ast->as<ASTColumnDeclaration>())
|
||||||
{
|
{
|
||||||
name = col_ast->name;
|
name = col_ast->name;
|
||||||
|
@ -5,6 +5,8 @@
|
|||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
|
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -28,7 +30,7 @@ ConstraintsDescription ConstraintsDescription::parse(const String & str)
|
|||||||
|
|
||||||
ConstraintsDescription res;
|
ConstraintsDescription res;
|
||||||
ParserConstraintDeclarationList parser;
|
ParserConstraintDeclarationList parser;
|
||||||
ASTPtr list = parseQuery(parser, str, 0);
|
ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
|
|
||||||
for (const auto & constraint : list->children)
|
for (const auto & constraint : list->children)
|
||||||
res.constraints.push_back(std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint));
|
res.constraints.push_back(std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint));
|
||||||
|
@ -5,6 +5,9 @@
|
|||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
|
|
||||||
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -42,7 +45,7 @@ IndicesDescription IndicesDescription::parse(const String & str)
|
|||||||
|
|
||||||
IndicesDescription res;
|
IndicesDescription res;
|
||||||
ParserIndexDeclarationList parser;
|
ParserIndexDeclarationList parser;
|
||||||
ASTPtr list = parseQuery(parser, str, 0);
|
ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
|
|
||||||
for (const auto & index : list->children)
|
for (const auto & index : list->children)
|
||||||
res.indices.push_back(std::dynamic_pointer_cast<ASTIndexDeclaration>(index));
|
res.indices.push_back(std::dynamic_pointer_cast<ASTIndexDeclaration>(index));
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/quoteString.h>
|
#include <Common/quoteString.h>
|
||||||
|
#include <Core/Defines.h>
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
@ -137,7 +138,7 @@ void MutationCommands::readText(ReadBuffer & in)
|
|||||||
|
|
||||||
ParserAlterCommandList p_alter_commands;
|
ParserAlterCommandList p_alter_commands;
|
||||||
auto commands_ast = parseQuery(
|
auto commands_ast = parseQuery(
|
||||||
p_alter_commands, commands_str.data(), commands_str.data() + commands_str.length(), "mutation commands list", 0);
|
p_alter_commands, commands_str.data(), commands_str.data() + commands_str.length(), "mutation commands list", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
for (ASTAlterCommand * command_ast : commands_ast->as<ASTAlterCommandList &>().commands)
|
for (ASTAlterCommand * command_ast : commands_ast->as<ASTAlterCommandList &>().commands)
|
||||||
{
|
{
|
||||||
auto command = MutationCommand::parse(command_ast, true);
|
auto command = MutationCommand::parse(command_ast, true);
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Disks/DiskSpaceMonitor.h>
|
#include <Core/Defines.h>
|
||||||
|
|
||||||
#include <Common/FieldVisitors.h>
|
#include <Common/FieldVisitors.h>
|
||||||
#include <Common/Macros.h>
|
#include <Common/Macros.h>
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
@ -25,6 +26,8 @@
|
|||||||
#include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
|
#include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
|
||||||
#include <Storages/VirtualColumnUtils.h>
|
#include <Storages/VirtualColumnUtils.h>
|
||||||
|
|
||||||
|
#include <Disks/DiskSpaceMonitor.h>
|
||||||
|
|
||||||
#include <Databases/IDatabase.h>
|
#include <Databases/IDatabase.h>
|
||||||
|
|
||||||
#include <Parsers/formatAST.h>
|
#include <Parsers/formatAST.h>
|
||||||
@ -479,7 +482,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
|||||||
if (metadata_diff.sorting_key_changed)
|
if (metadata_diff.sorting_key_changed)
|
||||||
{
|
{
|
||||||
ParserNotEmptyExpressionList parser(false);
|
ParserNotEmptyExpressionList parser(false);
|
||||||
auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0);
|
auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
|
|
||||||
if (new_sorting_key_expr_list->children.size() == 1)
|
if (new_sorting_key_expr_list->children.size() == 1)
|
||||||
metadata.order_by_ast = new_sorting_key_expr_list->children[0];
|
metadata.order_by_ast = new_sorting_key_expr_list->children[0];
|
||||||
@ -507,7 +510,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
|||||||
if (metadata_diff.ttl_table_changed)
|
if (metadata_diff.ttl_table_changed)
|
||||||
{
|
{
|
||||||
ParserTTLExpressionList parser;
|
ParserTTLExpressionList parser;
|
||||||
metadata.ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0);
|
metadata.ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -137,7 +137,7 @@ ColumnsDescription getStructureOfRemoteTableInShard(
|
|||||||
column.default_desc.kind = columnDefaultKindFromString(kind_name);
|
column.default_desc.kind = columnDefaultKindFromString(kind_name);
|
||||||
String expr_str = (*default_expr)[i].get<const String &>();
|
String expr_str = (*default_expr)[i].get<const String &>();
|
||||||
column.default_desc.expression = parseQuery(
|
column.default_desc.expression = parseQuery(
|
||||||
expr_parser, expr_str.data(), expr_str.data() + expr_str.size(), "default expression", 0);
|
expr_parser, expr_str.data(), expr_str.data() + expr_str.size(), "default expression", 0, context.getSettingsRef().max_parser_depth);
|
||||||
}
|
}
|
||||||
|
|
||||||
res.add(column);
|
res.add(column);
|
||||||
|
@ -48,7 +48,7 @@ static State & state()
|
|||||||
static void check(const std::string & query, const std::string & expected, const Context & context, const NamesAndTypesList & columns)
|
static void check(const std::string & query, const std::string & expected, const Context & context, const NamesAndTypesList & columns)
|
||||||
{
|
{
|
||||||
ParserSelectQuery parser;
|
ParserSelectQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, query, 1000);
|
ASTPtr ast = parseQuery(parser, query, 1000, 1000);
|
||||||
SelectQueryInfo query_info;
|
SelectQueryInfo query_info;
|
||||||
query_info.syntax_analyzer_result = SyntaxAnalyzer(context).analyzeSelect(ast, columns);
|
query_info.syntax_analyzer_result = SyntaxAnalyzer(context).analyzeSelect(ast, columns);
|
||||||
query_info.query = ast;
|
query_info.query = ast;
|
||||||
|
@ -19,9 +19,7 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, con
|
|||||||
Expected expected;
|
Expected expected;
|
||||||
|
|
||||||
Tokens tokens(structure.c_str(), structure.c_str() + structure.size());
|
Tokens tokens(structure.c_str(), structure.c_str() + structure.size());
|
||||||
IParser::Pos token_iterator(tokens);
|
IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth);
|
||||||
const Settings & settings = context.getSettingsRef();
|
|
||||||
token_iterator.max_depth = settings.max_parser_depth;
|
|
||||||
|
|
||||||
ParserColumnDeclarationList parser;
|
ParserColumnDeclarationList parser;
|
||||||
ASTPtr columns_list_raw;
|
ASTPtr columns_list_raw;
|
||||||
|
Loading…
Reference in New Issue
Block a user