mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #4649 from abyss7/iast-as
Implement and use method IAST::as<Derived>()
This commit is contained in:
commit
8ac3e89628
@ -704,7 +704,7 @@ private:
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(ast.get());
|
auto * insert = ast->as<ASTInsertQuery>();
|
||||||
|
|
||||||
if (insert && insert->data)
|
if (insert && insert->data)
|
||||||
{
|
{
|
||||||
@ -799,14 +799,11 @@ private:
|
|||||||
written_progress_chars = 0;
|
written_progress_chars = 0;
|
||||||
written_first_block = false;
|
written_first_block = false;
|
||||||
|
|
||||||
const ASTSetQuery * set_query = typeid_cast<const ASTSetQuery *>(&*parsed_query);
|
|
||||||
const ASTUseQuery * use_query = typeid_cast<const ASTUseQuery *>(&*parsed_query);
|
|
||||||
/// INSERT query for which data transfer is needed (not an INSERT SELECT) is processed separately.
|
|
||||||
const ASTInsertQuery * insert = typeid_cast<const ASTInsertQuery *>(&*parsed_query);
|
|
||||||
|
|
||||||
connection->forceConnected();
|
connection->forceConnected();
|
||||||
|
|
||||||
if (insert && !insert->select)
|
/// INSERT query for which data transfer is needed (not an INSERT SELECT) is processed separately.
|
||||||
|
const auto * insert_query = parsed_query->as<ASTInsertQuery>();
|
||||||
|
if (insert_query && !insert_query->select)
|
||||||
processInsertQuery();
|
processInsertQuery();
|
||||||
else
|
else
|
||||||
processOrdinaryQuery();
|
processOrdinaryQuery();
|
||||||
@ -814,7 +811,7 @@ private:
|
|||||||
/// Do not change context (current DB, settings) in case of an exception.
|
/// Do not change context (current DB, settings) in case of an exception.
|
||||||
if (!got_exception)
|
if (!got_exception)
|
||||||
{
|
{
|
||||||
if (set_query)
|
if (const auto * set_query = parsed_query->as<ASTSetQuery>())
|
||||||
{
|
{
|
||||||
/// Save all changes in settings to avoid losing them if the connection is lost.
|
/// Save all changes in settings to avoid losing them if the connection is lost.
|
||||||
for (const auto & change : set_query->changes)
|
for (const auto & change : set_query->changes)
|
||||||
@ -826,7 +823,7 @@ private:
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (use_query)
|
if (const auto * use_query = parsed_query->as<ASTUseQuery>())
|
||||||
{
|
{
|
||||||
const String & new_database = use_query->database;
|
const String & new_database = use_query->database;
|
||||||
/// If the client initiates the reconnection, it takes the settings from the config.
|
/// If the client initiates the reconnection, it takes the settings from the config.
|
||||||
@ -858,7 +855,7 @@ private:
|
|||||||
/// Convert external tables to ExternalTableData and send them using the connection.
|
/// Convert external tables to ExternalTableData and send them using the connection.
|
||||||
void sendExternalTables()
|
void sendExternalTables()
|
||||||
{
|
{
|
||||||
auto * select = typeid_cast<const ASTSelectWithUnionQuery *>(&*parsed_query);
|
const auto * select = parsed_query->as<ASTSelectWithUnionQuery>();
|
||||||
if (!select && !external_tables.empty())
|
if (!select && !external_tables.empty())
|
||||||
throw Exception("External tables could be sent only with select query", ErrorCodes::BAD_ARGUMENTS);
|
throw Exception("External tables could be sent only with select query", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
@ -883,7 +880,7 @@ private:
|
|||||||
void processInsertQuery()
|
void processInsertQuery()
|
||||||
{
|
{
|
||||||
/// Send part of query without data, because data will be sent separately.
|
/// Send part of query without data, because data will be sent separately.
|
||||||
const ASTInsertQuery & parsed_insert_query = typeid_cast<const ASTInsertQuery &>(*parsed_query);
|
const auto & parsed_insert_query = parsed_query->as<ASTInsertQuery &>();
|
||||||
String query_without_data = parsed_insert_query.data
|
String query_without_data = parsed_insert_query.data
|
||||||
? query.substr(0, parsed_insert_query.data - query.data())
|
? query.substr(0, parsed_insert_query.data - query.data())
|
||||||
: query;
|
: query;
|
||||||
@ -940,7 +937,7 @@ private:
|
|||||||
void sendData(Block & sample, const ColumnsDescription & columns_description)
|
void sendData(Block & sample, const ColumnsDescription & columns_description)
|
||||||
{
|
{
|
||||||
/// If INSERT data must be sent.
|
/// If INSERT data must be sent.
|
||||||
const ASTInsertQuery * parsed_insert_query = typeid_cast<const ASTInsertQuery *>(&*parsed_query);
|
const auto * parsed_insert_query = parsed_query->as<ASTInsertQuery>();
|
||||||
if (!parsed_insert_query)
|
if (!parsed_insert_query)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
@ -965,7 +962,7 @@ private:
|
|||||||
String current_format = insert_format;
|
String current_format = insert_format;
|
||||||
|
|
||||||
/// Data format can be specified in the INSERT query.
|
/// Data format can be specified in the INSERT query.
|
||||||
if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*parsed_query))
|
if (const auto * insert = parsed_query->as<ASTInsertQuery>())
|
||||||
{
|
{
|
||||||
if (!insert->format.empty())
|
if (!insert->format.empty())
|
||||||
current_format = insert->format;
|
current_format = insert->format;
|
||||||
@ -1231,12 +1228,14 @@ private:
|
|||||||
String current_format = format;
|
String current_format = format;
|
||||||
|
|
||||||
/// The query can specify output format or output file.
|
/// The query can specify output format or output file.
|
||||||
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
|
/// FIXME: try to prettify this cast using `as<>()`
|
||||||
|
if (const auto * query_with_output = dynamic_cast<const ASTQueryWithOutput *>(parsed_query.get()))
|
||||||
{
|
{
|
||||||
if (query_with_output->out_file != nullptr)
|
if (query_with_output->out_file)
|
||||||
{
|
{
|
||||||
const auto & out_file_node = typeid_cast<const ASTLiteral &>(*query_with_output->out_file);
|
const auto & out_file_node = query_with_output->out_file->as<ASTLiteral &>();
|
||||||
const auto & out_file = out_file_node.value.safeGet<std::string>();
|
const auto & out_file = out_file_node.value.safeGet<std::string>();
|
||||||
|
|
||||||
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
|
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
|
||||||
out_buf = &*out_file_buf;
|
out_buf = &*out_file_buf;
|
||||||
|
|
||||||
@ -1248,7 +1247,7 @@ private:
|
|||||||
{
|
{
|
||||||
if (has_vertical_output_suffix)
|
if (has_vertical_output_suffix)
|
||||||
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
|
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
|
||||||
const auto & id = typeid_cast<const ASTIdentifier &>(*query_with_output->format);
|
const auto & id = query_with_output->format->as<ASTIdentifier &>();
|
||||||
current_format = id.name;
|
current_format = id.name;
|
||||||
}
|
}
|
||||||
if (query_with_output->settings_ast)
|
if (query_with_output->settings_ast)
|
||||||
|
@ -483,7 +483,7 @@ String DB::TaskShard::getHostNameExample() const
|
|||||||
|
|
||||||
static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast)
|
static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast)
|
||||||
{
|
{
|
||||||
const ASTStorage & storage = typeid_cast<const ASTStorage &>(*storage_ast);
|
const auto & storage = storage_ast->as<ASTStorage &>();
|
||||||
return storage.partition_by || storage.order_by || storage.sample_by;
|
return storage.partition_by || storage.order_by || storage.sample_by;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -491,8 +491,8 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
|
|||||||
{
|
{
|
||||||
String storage_str = queryToString(storage_ast);
|
String storage_str = queryToString(storage_ast);
|
||||||
|
|
||||||
const ASTStorage & storage = typeid_cast<const ASTStorage &>(*storage_ast);
|
const auto & storage = storage_ast->as<ASTStorage &>();
|
||||||
const ASTFunction & engine = typeid_cast<const ASTFunction &>(*storage.engine);
|
const auto & engine = storage.engine->as<ASTFunction &>();
|
||||||
|
|
||||||
if (!endsWith(engine.name, "MergeTree"))
|
if (!endsWith(engine.name, "MergeTree"))
|
||||||
{
|
{
|
||||||
@ -501,7 +501,7 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
|
|||||||
}
|
}
|
||||||
|
|
||||||
ASTPtr arguments_ast = engine.arguments->clone();
|
ASTPtr arguments_ast = engine.arguments->clone();
|
||||||
ASTs & arguments = typeid_cast<ASTExpressionList &>(*arguments_ast).children;
|
ASTs & arguments = arguments_ast->children;
|
||||||
|
|
||||||
if (isExtendedDefinitionStorage(storage_ast))
|
if (isExtendedDefinitionStorage(storage_ast))
|
||||||
{
|
{
|
||||||
@ -1179,12 +1179,12 @@ protected:
|
|||||||
/// Removes MATERIALIZED and ALIAS columns from create table query
|
/// Removes MATERIALIZED and ALIAS columns from create table query
|
||||||
static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast)
|
static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast)
|
||||||
{
|
{
|
||||||
const ASTs & column_asts = typeid_cast<ASTCreateQuery &>(*query_ast).columns_list->columns->children;
|
const ASTs & column_asts = query_ast->as<ASTCreateQuery &>().columns_list->columns->children;
|
||||||
auto new_columns = std::make_shared<ASTExpressionList>();
|
auto new_columns = std::make_shared<ASTExpressionList>();
|
||||||
|
|
||||||
for (const ASTPtr & column_ast : column_asts)
|
for (const ASTPtr & column_ast : column_asts)
|
||||||
{
|
{
|
||||||
const ASTColumnDeclaration & column = typeid_cast<const ASTColumnDeclaration &>(*column_ast);
|
const auto & column = column_ast->as<ASTColumnDeclaration &>();
|
||||||
|
|
||||||
if (!column.default_specifier.empty())
|
if (!column.default_specifier.empty())
|
||||||
{
|
{
|
||||||
@ -1197,12 +1197,11 @@ protected:
|
|||||||
}
|
}
|
||||||
|
|
||||||
ASTPtr new_query_ast = query_ast->clone();
|
ASTPtr new_query_ast = query_ast->clone();
|
||||||
ASTCreateQuery & new_query = typeid_cast<ASTCreateQuery &>(*new_query_ast);
|
auto & new_query = new_query_ast->as<ASTCreateQuery &>();
|
||||||
|
|
||||||
auto new_columns_list = std::make_shared<ASTColumns>();
|
auto new_columns_list = std::make_shared<ASTColumns>();
|
||||||
new_columns_list->set(new_columns_list->columns, new_columns);
|
new_columns_list->set(new_columns_list->columns, new_columns);
|
||||||
new_columns_list->set(
|
new_columns_list->set(new_columns_list->indices, query_ast->as<ASTCreateQuery>()->columns_list->indices->clone());
|
||||||
new_columns_list->indices, typeid_cast<ASTCreateQuery &>(*query_ast).columns_list->indices->clone());
|
|
||||||
|
|
||||||
new_query.replace(new_query.columns_list, new_columns_list);
|
new_query.replace(new_query.columns_list, new_columns_list);
|
||||||
|
|
||||||
@ -1212,7 +1211,7 @@ protected:
|
|||||||
/// Replaces ENGINE and table name in a create query
|
/// Replaces ENGINE and table name in a create query
|
||||||
std::shared_ptr<ASTCreateQuery> rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast)
|
std::shared_ptr<ASTCreateQuery> rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast)
|
||||||
{
|
{
|
||||||
ASTCreateQuery & create = typeid_cast<ASTCreateQuery &>(*create_query_ast);
|
const auto & create = create_query_ast->as<ASTCreateQuery &>();
|
||||||
auto res = std::make_shared<ASTCreateQuery>(create);
|
auto res = std::make_shared<ASTCreateQuery>(create);
|
||||||
|
|
||||||
if (create.storage == nullptr || new_storage_ast == nullptr)
|
if (create.storage == nullptr || new_storage_ast == nullptr)
|
||||||
@ -1646,7 +1645,7 @@ protected:
|
|||||||
/// Try create table (if not exists) on each shard
|
/// Try create table (if not exists) on each shard
|
||||||
{
|
{
|
||||||
auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast);
|
auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast);
|
||||||
typeid_cast<ASTCreateQuery &>(*create_query_push_ast).if_not_exists = true;
|
create_query_push_ast->as<ASTCreateQuery &>().if_not_exists = true;
|
||||||
String query = queryToString(create_query_push_ast);
|
String query = queryToString(create_query_push_ast);
|
||||||
|
|
||||||
LOG_DEBUG(log, "Create destination tables. Query: " << query);
|
LOG_DEBUG(log, "Create destination tables. Query: " << query);
|
||||||
@ -1779,7 +1778,7 @@ protected:
|
|||||||
|
|
||||||
void dropAndCreateLocalTable(const ASTPtr & create_ast)
|
void dropAndCreateLocalTable(const ASTPtr & create_ast)
|
||||||
{
|
{
|
||||||
auto & create = typeid_cast<ASTCreateQuery &>(*create_ast);
|
const auto & create = create_ast->as<ASTCreateQuery &>();
|
||||||
dropLocalTableIfExists({create.database, create.table});
|
dropLocalTableIfExists({create.database, create.table});
|
||||||
|
|
||||||
InterpreterCreateQuery interpreter(create_ast, context);
|
InterpreterCreateQuery interpreter(create_ast, context);
|
||||||
|
@ -15,7 +15,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
Array getAggregateFunctionParametersArray(const ASTPtr & expression_list, const std::string & error_context)
|
Array getAggregateFunctionParametersArray(const ASTPtr & expression_list, const std::string & error_context)
|
||||||
{
|
{
|
||||||
const ASTs & parameters = typeid_cast<const ASTExpressionList &>(*expression_list).children;
|
const ASTs & parameters = expression_list->children;
|
||||||
if (parameters.empty())
|
if (parameters.empty())
|
||||||
throw Exception("Parameters list to aggregate functions cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
throw Exception("Parameters list to aggregate functions cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
@ -23,14 +23,14 @@ Array getAggregateFunctionParametersArray(const ASTPtr & expression_list, const
|
|||||||
|
|
||||||
for (size_t i = 0; i < parameters.size(); ++i)
|
for (size_t i = 0; i < parameters.size(); ++i)
|
||||||
{
|
{
|
||||||
const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(parameters[i].get());
|
const auto * literal = parameters[i]->as<ASTLiteral>();
|
||||||
if (!lit)
|
if (!literal)
|
||||||
{
|
{
|
||||||
throw Exception("Parameters to aggregate functions must be literals" + (error_context.empty() ? "" : " (in " + error_context +")"),
|
throw Exception("Parameters to aggregate functions must be literals" + (error_context.empty() ? "" : " (in " + error_context +")"),
|
||||||
ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS);
|
ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS);
|
||||||
}
|
}
|
||||||
|
|
||||||
params_row[i] = lit->value;
|
params_row[i] = literal->value;
|
||||||
}
|
}
|
||||||
|
|
||||||
return params_row;
|
return params_row;
|
||||||
@ -67,8 +67,7 @@ void getAggregateFunctionNameAndParametersArray(
|
|||||||
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);
|
||||||
|
|
||||||
ASTExpressionList & args_list = typeid_cast<ASTExpressionList &>(*args_ast);
|
if (args_ast->children.empty())
|
||||||
if (args_list.children.empty())
|
|
||||||
throw Exception("Incorrect list of parameters to aggregate function "
|
throw Exception("Incorrect list of parameters to aggregate function "
|
||||||
+ aggregate_function_name, ErrorCodes::BAD_ARGUMENTS);
|
+ aggregate_function_name, ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
|
63
dbms/src/Common/TypePromotion.h
Normal file
63
dbms/src/Common/TypePromotion.h
Normal file
@ -0,0 +1,63 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/* This base class adds public methods:
|
||||||
|
* - Derived * as<Derived>()
|
||||||
|
* - const Derived * as<Derived>() const
|
||||||
|
* - Derived & as<Derived &>()
|
||||||
|
* - const Derived & as<Derived &>() const
|
||||||
|
*/
|
||||||
|
|
||||||
|
template <class Base>
|
||||||
|
class TypePromotion
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
/// Need a helper-struct to fight the lack of the function-template partial specialization.
|
||||||
|
template <class T, bool is_const, bool is_ref = std::is_reference_v<T>>
|
||||||
|
struct CastHelper;
|
||||||
|
|
||||||
|
template <class T>
|
||||||
|
struct CastHelper<T, false, true>
|
||||||
|
{
|
||||||
|
auto & value(Base * ptr) { return typeid_cast<T>(*ptr); }
|
||||||
|
};
|
||||||
|
|
||||||
|
template <class T>
|
||||||
|
struct CastHelper<T, true, true>
|
||||||
|
{
|
||||||
|
auto & value(const Base * ptr) { return typeid_cast<std::add_lvalue_reference_t<std::add_const_t<std::remove_reference_t<T>>>>(*ptr); }
|
||||||
|
};
|
||||||
|
|
||||||
|
template <class T>
|
||||||
|
struct CastHelper<T, false, false>
|
||||||
|
{
|
||||||
|
auto * value(Base * ptr) { return typeid_cast<T *>(ptr); }
|
||||||
|
};
|
||||||
|
|
||||||
|
template <class T>
|
||||||
|
struct CastHelper<T, true, false>
|
||||||
|
{
|
||||||
|
auto * value(const Base * ptr) { return typeid_cast<std::add_const_t<T> *>(ptr); }
|
||||||
|
};
|
||||||
|
|
||||||
|
public:
|
||||||
|
template <class Derived>
|
||||||
|
auto as() -> std::invoke_result_t<decltype(&CastHelper<Derived, false>::value), CastHelper<Derived, false>, Base *>
|
||||||
|
{
|
||||||
|
// TODO: if we do downcast to base type, then just return |this|.
|
||||||
|
return CastHelper<Derived, false>().value(static_cast<Base *>(this));
|
||||||
|
}
|
||||||
|
|
||||||
|
template <class Derived>
|
||||||
|
auto as() const -> std::invoke_result_t<decltype(&CastHelper<Derived, true>::value), CastHelper<Derived, true>, const Base *>
|
||||||
|
{
|
||||||
|
// TODO: if we do downcast to base type, then just return |this|.
|
||||||
|
return CastHelper<Derived, true>().value(static_cast<const Base *>(this));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
} // namespace DB
|
@ -25,18 +25,32 @@ namespace DB
|
|||||||
template <typename To, typename From>
|
template <typename To, typename From>
|
||||||
std::enable_if_t<std::is_reference_v<To>, To> typeid_cast(From & from)
|
std::enable_if_t<std::is_reference_v<To>, To> typeid_cast(From & from)
|
||||||
{
|
{
|
||||||
if (typeid(from) == typeid(To))
|
try
|
||||||
return static_cast<To>(from);
|
{
|
||||||
else
|
if (typeid(from) == typeid(To))
|
||||||
throw DB::Exception("Bad cast from type " + demangle(typeid(from).name()) + " to " + demangle(typeid(To).name()),
|
return static_cast<To>(from);
|
||||||
DB::ErrorCodes::BAD_CAST);
|
}
|
||||||
|
catch (const std::exception & e)
|
||||||
|
{
|
||||||
|
throw DB::Exception(e.what(), DB::ErrorCodes::BAD_CAST);
|
||||||
|
}
|
||||||
|
|
||||||
|
throw DB::Exception("Bad cast from type " + demangle(typeid(from).name()) + " to " + demangle(typeid(To).name()),
|
||||||
|
DB::ErrorCodes::BAD_CAST);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename To, typename From>
|
template <typename To, typename From>
|
||||||
To typeid_cast(From * from)
|
To typeid_cast(From * from)
|
||||||
{
|
{
|
||||||
if (typeid(*from) == typeid(std::remove_pointer_t<To>))
|
try
|
||||||
return static_cast<To>(from);
|
{
|
||||||
else
|
if (typeid(*from) == typeid(std::remove_pointer_t<To>))
|
||||||
return nullptr;
|
return static_cast<To>(from);
|
||||||
|
else
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
catch (const std::exception & e)
|
||||||
|
{
|
||||||
|
throw DB::Exception(e.what(), DB::ErrorCodes::BAD_CAST);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -144,7 +144,7 @@ void registerCodecDelta(CompressionCodecFactory & factory)
|
|||||||
throw Exception("Delta codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
|
throw Exception("Delta codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
|
||||||
|
|
||||||
const auto children = arguments->children;
|
const auto children = arguments->children;
|
||||||
const ASTLiteral * literal = static_cast<const ASTLiteral *>(children[0].get());
|
const auto * literal = children[0]->as<ASTLiteral>();
|
||||||
size_t user_bytes_size = literal->value.safeGet<UInt64>();
|
size_t user_bytes_size = literal->value.safeGet<UInt64>();
|
||||||
if (user_bytes_size != 1 && user_bytes_size != 2 && user_bytes_size != 4 && user_bytes_size != 8)
|
if (user_bytes_size != 1 && user_bytes_size != 2 && user_bytes_size != 4 && user_bytes_size != 8)
|
||||||
throw Exception("Delta value for delta codec can be 1, 2, 4 or 8, given " + toString(user_bytes_size), ErrorCodes::ILLEGAL_CODEC_PARAMETER);
|
throw Exception("Delta value for delta codec can be 1, 2, 4 or 8, given " + toString(user_bytes_size), ErrorCodes::ILLEGAL_CODEC_PARAMETER);
|
||||||
|
@ -86,7 +86,7 @@ void registerCodecLZ4HC(CompressionCodecFactory & factory)
|
|||||||
throw Exception("LZ4HC codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
|
throw Exception("LZ4HC codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
|
||||||
|
|
||||||
const auto children = arguments->children;
|
const auto children = arguments->children;
|
||||||
const ASTLiteral * literal = static_cast<const ASTLiteral *>(children[0].get());
|
const auto * literal = children[0]->as<ASTLiteral>();
|
||||||
level = literal->value.safeGet<UInt64>();
|
level = literal->value.safeGet<UInt64>();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -100,4 +100,3 @@ CompressionCodecLZ4HC::CompressionCodecLZ4HC(int level_)
|
|||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -73,7 +73,7 @@ void registerCodecZSTD(CompressionCodecFactory & factory)
|
|||||||
throw Exception("ZSTD codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
|
throw Exception("ZSTD codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
|
||||||
|
|
||||||
const auto children = arguments->children;
|
const auto children = arguments->children;
|
||||||
const ASTLiteral * literal = static_cast<const ASTLiteral *>(children[0].get());
|
const auto * literal = children[0]->as<ASTLiteral>();
|
||||||
level = literal->value.safeGet<UInt64>();
|
level = literal->value.safeGet<UInt64>();
|
||||||
if (level > ZSTD_maxCLevel())
|
if (level > ZSTD_maxCLevel())
|
||||||
throw Exception("ZSTD codec can't have level more that " + toString(ZSTD_maxCLevel()) + ", given " + toString(level), ErrorCodes::ILLEGAL_CODEC_PARAMETER);
|
throw Exception("ZSTD codec can't have level more that " + toString(ZSTD_maxCLevel()) + ", given " + toString(level), ErrorCodes::ILLEGAL_CODEC_PARAMETER);
|
||||||
|
@ -56,15 +56,15 @@ CompressionCodecPtr CompressionCodecFactory::get(const std::vector<CodecNameWith
|
|||||||
|
|
||||||
CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, DataTypePtr column_type) const
|
CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, DataTypePtr column_type) const
|
||||||
{
|
{
|
||||||
if (const auto * func = typeid_cast<const ASTFunction *>(ast.get()))
|
if (const auto * func = ast->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
Codecs codecs;
|
Codecs codecs;
|
||||||
codecs.reserve(func->arguments->children.size());
|
codecs.reserve(func->arguments->children.size());
|
||||||
for (const auto & inner_codec_ast : func->arguments->children)
|
for (const auto & inner_codec_ast : func->arguments->children)
|
||||||
{
|
{
|
||||||
if (const auto * family_name = typeid_cast<const ASTIdentifier *>(inner_codec_ast.get()))
|
if (const auto * family_name = inner_codec_ast->as<ASTIdentifier>())
|
||||||
codecs.emplace_back(getImpl(family_name->name, {}, column_type));
|
codecs.emplace_back(getImpl(family_name->name, {}, column_type));
|
||||||
else if (const auto * ast_func = typeid_cast<const ASTFunction *>(inner_codec_ast.get()))
|
else if (const auto * ast_func = inner_codec_ast->as<ASTFunction>())
|
||||||
codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments, column_type));
|
codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments, column_type));
|
||||||
else
|
else
|
||||||
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||||
|
@ -1,14 +1,17 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <memory>
|
#include <Compression/CompressionInfo.h>
|
||||||
|
#include <Compression/ICompressionCodec.h>
|
||||||
|
#include <DataTypes/IDataType.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
#include <Common/IFactoryWithAliases.h>
|
||||||
|
|
||||||
|
#include <ext/singleton.h>
|
||||||
|
|
||||||
#include <functional>
|
#include <functional>
|
||||||
|
#include <memory>
|
||||||
#include <optional>
|
#include <optional>
|
||||||
#include <unordered_map>
|
#include <unordered_map>
|
||||||
#include <ext/singleton.h>
|
|
||||||
#include <DataTypes/IDataType.h>
|
|
||||||
#include <Common/IFactoryWithAliases.h>
|
|
||||||
#include <Compression/ICompressionCodec.h>
|
|
||||||
#include <Compression/CompressionInfo.h>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -19,10 +22,6 @@ using CompressionCodecPtr = std::shared_ptr<ICompressionCodec>;
|
|||||||
|
|
||||||
using CodecNameWithLevel = std::pair<String, std::optional<int>>;
|
using CodecNameWithLevel = std::pair<String, std::optional<int>>;
|
||||||
|
|
||||||
class IAST;
|
|
||||||
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
/** Creates a codec object by name of compression algorithm family and parameters.
|
/** Creates a codec object by name of compression algorithm family and parameters.
|
||||||
*/
|
*/
|
||||||
class CompressionCodecFactory final : public ext::singleton<CompressionCodecFactory>
|
class CompressionCodecFactory final : public ext::singleton<CompressionCodecFactory>
|
||||||
|
@ -20,7 +20,7 @@ namespace ErrorCodes
|
|||||||
InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
|
InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
|
||||||
const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, Context & context)
|
const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, Context & context)
|
||||||
{
|
{
|
||||||
const ASTInsertQuery * ast_insert_query = dynamic_cast<const ASTInsertQuery *>(ast.get());
|
const auto * ast_insert_query = ast->as<ASTInsertQuery>();
|
||||||
|
|
||||||
if (!ast_insert_query)
|
if (!ast_insert_query)
|
||||||
throw Exception("Logical error: query requires data to insert, but it is not INSERT query", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: query requires data to insert, but it is not INSERT query", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
@ -340,30 +340,30 @@ static DataTypePtr create(const ASTPtr & arguments)
|
|||||||
throw Exception("Data type AggregateFunction requires parameters: "
|
throw Exception("Data type AggregateFunction requires parameters: "
|
||||||
"name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
"name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
if (const ASTFunction * parametric = typeid_cast<const ASTFunction *>(arguments->children[0].get()))
|
if (const auto * parametric = arguments->children[0]->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
if (parametric->parameters)
|
if (parametric->parameters)
|
||||||
throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR);
|
throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR);
|
||||||
function_name = parametric->name;
|
function_name = parametric->name;
|
||||||
|
|
||||||
const ASTs & parameters = typeid_cast<const ASTExpressionList &>(*parametric->arguments).children;
|
const ASTs & parameters = parametric->arguments->children;
|
||||||
params_row.resize(parameters.size());
|
params_row.resize(parameters.size());
|
||||||
|
|
||||||
for (size_t i = 0; i < parameters.size(); ++i)
|
for (size_t i = 0; i < parameters.size(); ++i)
|
||||||
{
|
{
|
||||||
const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(parameters[i].get());
|
const auto * literal = parameters[i]->as<ASTLiteral>();
|
||||||
if (!lit)
|
if (!literal)
|
||||||
throw Exception("Parameters to aggregate functions must be literals",
|
throw Exception("Parameters to aggregate functions must be literals",
|
||||||
ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS);
|
ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS);
|
||||||
|
|
||||||
params_row[i] = lit->value;
|
params_row[i] = literal->value;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (auto opt_name = getIdentifierName(arguments->children[0]))
|
else if (auto opt_name = getIdentifierName(arguments->children[0]))
|
||||||
{
|
{
|
||||||
function_name = *opt_name;
|
function_name = *opt_name;
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTLiteral *>(arguments->children[0].get()))
|
else if (arguments->children[0]->as<ASTLiteral>())
|
||||||
{
|
{
|
||||||
throw Exception("Aggregate function name for data type AggregateFunction must be passed as identifier (without quotes) or function",
|
throw Exception("Aggregate function name for data type AggregateFunction must be passed as identifier (without quotes) or function",
|
||||||
ErrorCodes::BAD_ARGUMENTS);
|
ErrorCodes::BAD_ARGUMENTS);
|
||||||
@ -389,4 +389,3 @@ void registerDataTypeAggregateFunction(DataTypeFactory & factory)
|
|||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -186,7 +186,7 @@ static DataTypePtr create(const ASTPtr & arguments)
|
|||||||
if (arguments->children.size() != 1)
|
if (arguments->children.size() != 1)
|
||||||
throw Exception("DateTime data type can optionally have only one argument - time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
throw Exception("DateTime data type can optionally have only one argument - time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
const ASTLiteral * arg = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
|
const auto * arg = arguments->children[0]->as<ASTLiteral>();
|
||||||
if (!arg || arg->value.getType() != Field::Types::String)
|
if (!arg || arg->value.getType() != Field::Types::String)
|
||||||
throw Exception("Parameter for DateTime data type must be string literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Parameter for DateTime data type must be string literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
@ -357,7 +357,7 @@ static DataTypePtr create(const ASTPtr & arguments)
|
|||||||
/// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument.
|
/// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument.
|
||||||
for (const ASTPtr & child : arguments->children)
|
for (const ASTPtr & child : arguments->children)
|
||||||
{
|
{
|
||||||
const ASTFunction * func = typeid_cast<const ASTFunction *>(child.get());
|
const auto * func = child->as<ASTFunction>();
|
||||||
if (!func
|
if (!func
|
||||||
|| func->name != "equals"
|
|| func->name != "equals"
|
||||||
|| func->parameters
|
|| func->parameters
|
||||||
@ -366,8 +366,8 @@ static DataTypePtr create(const ASTPtr & arguments)
|
|||||||
throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer",
|
throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer",
|
||||||
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||||
|
|
||||||
const ASTLiteral * name_literal = typeid_cast<const ASTLiteral *>(func->arguments->children[0].get());
|
const auto * name_literal = func->arguments->children[0]->as<ASTLiteral>();
|
||||||
const ASTLiteral * value_literal = typeid_cast<const ASTLiteral *>(func->arguments->children[1].get());
|
const auto * value_literal = func->arguments->children[1]->as<ASTLiteral>();
|
||||||
|
|
||||||
if (!name_literal
|
if (!name_literal
|
||||||
|| !value_literal
|
|| !value_literal
|
||||||
|
@ -32,19 +32,19 @@ DataTypePtr DataTypeFactory::get(const String & full_name) const
|
|||||||
|
|
||||||
DataTypePtr DataTypeFactory::get(const ASTPtr & ast) const
|
DataTypePtr DataTypeFactory::get(const ASTPtr & ast) const
|
||||||
{
|
{
|
||||||
if (const ASTFunction * func = typeid_cast<const ASTFunction *>(ast.get()))
|
if (const auto * func = ast->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
if (func->parameters)
|
if (func->parameters)
|
||||||
throw Exception("Data type cannot have multiple parenthesed parameters.", ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE);
|
throw Exception("Data type cannot have multiple parenthesed parameters.", ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE);
|
||||||
return get(func->name, func->arguments);
|
return get(func->name, func->arguments);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (const ASTIdentifier * ident = typeid_cast<const ASTIdentifier *>(ast.get()))
|
if (const auto * ident = ast->as<ASTIdentifier>())
|
||||||
{
|
{
|
||||||
return get(ident->name, {});
|
return get(ident->name, {});
|
||||||
}
|
}
|
||||||
|
|
||||||
if (const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(ast.get()))
|
if (const auto * lit = ast->as<ASTLiteral>())
|
||||||
{
|
{
|
||||||
if (lit->value.isNull())
|
if (lit->value.isNull())
|
||||||
return get("Null", {});
|
return get("Null", {});
|
||||||
|
@ -1,12 +1,15 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <memory>
|
|
||||||
#include <functional>
|
|
||||||
#include <unordered_map>
|
|
||||||
#include <Common/IFactoryWithAliases.h>
|
|
||||||
#include <DataTypes/IDataType.h>
|
#include <DataTypes/IDataType.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
#include <Common/IFactoryWithAliases.h>
|
||||||
|
|
||||||
#include <ext/singleton.h>
|
#include <ext/singleton.h>
|
||||||
|
|
||||||
|
#include <functional>
|
||||||
|
#include <memory>
|
||||||
|
#include <unordered_map>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -17,9 +20,6 @@ using DataTypePtr = std::shared_ptr<const IDataType>;
|
|||||||
class IDataTypeDomain;
|
class IDataTypeDomain;
|
||||||
using DataTypeDomainPtr = std::unique_ptr<const IDataTypeDomain>;
|
using DataTypeDomainPtr = std::unique_ptr<const IDataTypeDomain>;
|
||||||
|
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Creates a data type by name of data type family and parameters.
|
/** Creates a data type by name of data type family and parameters.
|
||||||
*/
|
*/
|
||||||
|
@ -273,7 +273,7 @@ static DataTypePtr create(const ASTPtr & arguments)
|
|||||||
if (!arguments || arguments->children.size() != 1)
|
if (!arguments || arguments->children.size() != 1)
|
||||||
throw Exception("FixedString data type family must have exactly one argument - size in bytes", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
throw Exception("FixedString data type family must have exactly one argument - size in bytes", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
const ASTLiteral * argument = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
|
const auto * argument = arguments->children[0]->as<ASTLiteral>();
|
||||||
if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.get<UInt64>() == 0)
|
if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.get<UInt64>() == 0)
|
||||||
throw Exception("FixedString data type family must have a number (positive integer) as its argument", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
throw Exception("FixedString data type family must have a number (positive integer) as its argument", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||||
|
|
||||||
|
@ -531,7 +531,7 @@ static DataTypePtr create(const ASTPtr & arguments)
|
|||||||
|
|
||||||
for (const ASTPtr & child : arguments->children)
|
for (const ASTPtr & child : arguments->children)
|
||||||
{
|
{
|
||||||
if (const ASTNameTypePair * name_and_type_pair = typeid_cast<const ASTNameTypePair *>(child.get()))
|
if (const auto * name_and_type_pair = child->as<ASTNameTypePair>())
|
||||||
{
|
{
|
||||||
nested_types.emplace_back(DataTypeFactory::instance().get(name_and_type_pair->type));
|
nested_types.emplace_back(DataTypeFactory::instance().get(name_and_type_pair->type));
|
||||||
names.emplace_back(name_and_type_pair->name);
|
names.emplace_back(name_and_type_pair->name);
|
||||||
|
@ -208,8 +208,8 @@ static DataTypePtr create(const ASTPtr & arguments)
|
|||||||
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
|
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
const ASTLiteral * precision = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
|
const auto * precision = arguments->children[0]->as<ASTLiteral>();
|
||||||
const ASTLiteral * scale = typeid_cast<const ASTLiteral *>(arguments->children[1].get());
|
const auto * scale = arguments->children[1]->as<ASTLiteral>();
|
||||||
|
|
||||||
if (!precision || precision->value.getType() != Field::Types::UInt64 ||
|
if (!precision || precision->value.getType() != Field::Types::UInt64 ||
|
||||||
!scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64))
|
!scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64))
|
||||||
@ -228,7 +228,7 @@ static DataTypePtr createExect(const ASTPtr & arguments)
|
|||||||
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
|
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
const ASTLiteral * scale_arg = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
|
const auto * scale_arg = arguments->children[0]->as<ASTLiteral>();
|
||||||
|
|
||||||
if (!scale_arg || !(scale_arg->value.getType() == Field::Types::Int64 || scale_arg->value.getType() == Field::Types::UInt64))
|
if (!scale_arg || !(scale_arg->value.getType() == Field::Types::Int64 || scale_arg->value.getType() == Field::Types::UInt64))
|
||||||
throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
@ -370,7 +370,7 @@ static ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const Str
|
|||||||
|
|
||||||
if (ast)
|
if (ast)
|
||||||
{
|
{
|
||||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||||
ast_create_query.attach = false;
|
ast_create_query.attach = false;
|
||||||
ast_create_query.database = database;
|
ast_create_query.database = database;
|
||||||
}
|
}
|
||||||
@ -415,8 +415,7 @@ void DatabaseOrdinary::renameTable(
|
|||||||
ASTPtr ast = getQueryFromMetadata(detail::getTableMetadataPath(metadata_path, table_name));
|
ASTPtr ast = getQueryFromMetadata(detail::getTableMetadataPath(metadata_path, table_name));
|
||||||
if (!ast)
|
if (!ast)
|
||||||
throw Exception("There is no metadata file for table " + table_name, ErrorCodes::FILE_DOESNT_EXIST);
|
throw Exception("There is no metadata file for table " + table_name, ErrorCodes::FILE_DOESNT_EXIST);
|
||||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
ast->as<ASTCreateQuery &>().table = to_table_name;
|
||||||
ast_create_query.table = to_table_name;
|
|
||||||
|
|
||||||
/// NOTE Non-atomic.
|
/// NOTE Non-atomic.
|
||||||
to_database_concrete->createTable(context, to_table_name, table, ast);
|
to_database_concrete->createTable(context, to_table_name, table, ast);
|
||||||
@ -534,7 +533,7 @@ 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);
|
||||||
|
|
||||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
const auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||||
|
|
||||||
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns);
|
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns);
|
||||||
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(indices);
|
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(indices);
|
||||||
|
@ -26,7 +26,7 @@ namespace ErrorCodes
|
|||||||
String getTableDefinitionFromCreateQuery(const ASTPtr & query)
|
String getTableDefinitionFromCreateQuery(const ASTPtr & query)
|
||||||
{
|
{
|
||||||
ASTPtr query_clone = query->clone();
|
ASTPtr query_clone = query->clone();
|
||||||
ASTCreateQuery & create = typeid_cast<ASTCreateQuery &>(*query_clone.get());
|
auto & create = query_clone->as<ASTCreateQuery &>();
|
||||||
|
|
||||||
/// We remove everything that is not needed for ATTACH from the query.
|
/// We remove everything that is not needed for ATTACH from the query.
|
||||||
create.attach = true;
|
create.attach = true;
|
||||||
@ -62,7 +62,7 @@ std::pair<String, StoragePtr> createTableFromDefinition(
|
|||||||
ParserCreateQuery parser;
|
ParserCreateQuery parser;
|
||||||
ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), description_for_error_message, 0);
|
ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), description_for_error_message, 0);
|
||||||
|
|
||||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||||
ast_create_query.attach = true;
|
ast_create_query.attach = true;
|
||||||
ast_create_query.database = database_name;
|
ast_create_query.database = database_name;
|
||||||
|
|
||||||
|
@ -1,16 +1,18 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Core/Types.h>
|
|
||||||
#include <Core/NamesAndTypes.h>
|
#include <Core/NamesAndTypes.h>
|
||||||
|
#include <Core/Types.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
#include <Storages/ColumnsDescription.h>
|
#include <Storages/ColumnsDescription.h>
|
||||||
#include <Storages/IndicesDescription.h>
|
#include <Storages/IndicesDescription.h>
|
||||||
#include <ctime>
|
|
||||||
#include <memory>
|
|
||||||
#include <functional>
|
|
||||||
#include <Poco/File.h>
|
#include <Poco/File.h>
|
||||||
#include <Common/escapeForFileName.h>
|
|
||||||
#include <Common/ThreadPool.h>
|
#include <Common/ThreadPool.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Common/escapeForFileName.h>
|
||||||
|
|
||||||
|
#include <ctime>
|
||||||
|
#include <functional>
|
||||||
|
#include <memory>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -21,9 +23,6 @@ class Context;
|
|||||||
class IStorage;
|
class IStorage;
|
||||||
using StoragePtr = std::shared_ptr<IStorage>;
|
using StoragePtr = std::shared_ptr<IStorage>;
|
||||||
|
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
struct Settings;
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
@ -157,4 +156,3 @@ using DatabasePtr = std::shared_ptr<IDatabase>;
|
|||||||
using Databases = std::map<String, DatabasePtr>;
|
using Databases = std::map<String, DatabasePtr>;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -84,11 +84,11 @@ SetPtr makeExplicitSet(
|
|||||||
|
|
||||||
auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr
|
auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr
|
||||||
{
|
{
|
||||||
auto ast_function = typeid_cast<const ASTFunction *>(tuple_ast.get());
|
const auto * func = tuple_ast->as<ASTFunction>();
|
||||||
if (ast_function && ast_function->name == "tuple" && !ast_function->arguments->children.empty())
|
if (func && func->name == "tuple" && !func->arguments->children.empty())
|
||||||
{
|
{
|
||||||
/// Won't parse all values of outer tuple.
|
/// Won't parse all values of outer tuple.
|
||||||
auto element = ast_function->arguments->children.at(0);
|
auto element = func->arguments->children.at(0);
|
||||||
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(element, context);
|
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(element, context);
|
||||||
return std::make_shared<DataTypeTuple>(DataTypes({value_raw.second}));
|
return std::make_shared<DataTypeTuple>(DataTypes({value_raw.second}));
|
||||||
}
|
}
|
||||||
@ -122,7 +122,7 @@ SetPtr makeExplicitSet(
|
|||||||
/// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc.
|
/// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc.
|
||||||
else if (left_tuple_depth + 1 == right_tuple_depth)
|
else if (left_tuple_depth + 1 == right_tuple_depth)
|
||||||
{
|
{
|
||||||
ASTFunction * set_func = typeid_cast<ASTFunction *>(right_arg.get());
|
const auto * set_func = right_arg->as<ASTFunction>();
|
||||||
|
|
||||||
if (!set_func || set_func->name != "tuple")
|
if (!set_func || set_func->name != "tuple")
|
||||||
throw Exception("Incorrect type of 2nd argument for function " + node->name
|
throw Exception("Incorrect type of 2nd argument for function " + node->name
|
||||||
@ -263,11 +263,10 @@ void ActionsVisitor::visit(const ASTPtr & ast)
|
|||||||
};
|
};
|
||||||
|
|
||||||
/// If the result of the calculation already exists in the block.
|
/// If the result of the calculation already exists in the block.
|
||||||
if ((typeid_cast<ASTFunction *>(ast.get()) || typeid_cast<ASTLiteral *>(ast.get()))
|
if ((ast->as<ASTFunction>() || ast->as<ASTLiteral>()) && actions_stack.getSampleBlock().has(getColumnName()))
|
||||||
&& actions_stack.getSampleBlock().has(getColumnName()))
|
|
||||||
return;
|
return;
|
||||||
|
|
||||||
if (auto * identifier = typeid_cast<ASTIdentifier *>(ast.get()))
|
if (const auto * identifier = ast->as<ASTIdentifier>())
|
||||||
{
|
{
|
||||||
if (!only_consts && !actions_stack.getSampleBlock().has(getColumnName()))
|
if (!only_consts && !actions_stack.getSampleBlock().has(getColumnName()))
|
||||||
{
|
{
|
||||||
@ -288,7 +287,7 @@ void ActionsVisitor::visit(const ASTPtr & ast)
|
|||||||
actions_stack.addAction(ExpressionAction::addAliases({{identifier->name, identifier->alias}}));
|
actions_stack.addAction(ExpressionAction::addAliases({{identifier->name, identifier->alias}}));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
|
else if (const auto * node = ast->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
if (node->name == "lambda")
|
if (node->name == "lambda")
|
||||||
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||||
@ -383,14 +382,14 @@ void ActionsVisitor::visit(const ASTPtr & ast)
|
|||||||
auto & child = node->arguments->children[arg];
|
auto & child = node->arguments->children[arg];
|
||||||
auto child_column_name = child->getColumnName();
|
auto child_column_name = child->getColumnName();
|
||||||
|
|
||||||
ASTFunction * lambda = typeid_cast<ASTFunction *>(child.get());
|
const auto * lambda = child->as<ASTFunction>();
|
||||||
if (lambda && lambda->name == "lambda")
|
if (lambda && lambda->name == "lambda")
|
||||||
{
|
{
|
||||||
/// If the argument is a lambda expression, just remember its approximate type.
|
/// If the argument is a lambda expression, just remember its approximate type.
|
||||||
if (lambda->arguments->children.size() != 2)
|
if (lambda->arguments->children.size() != 2)
|
||||||
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(lambda->arguments->children.at(0).get());
|
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||||
|
|
||||||
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
||||||
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
||||||
@ -454,12 +453,12 @@ void ActionsVisitor::visit(const ASTPtr & ast)
|
|||||||
{
|
{
|
||||||
ASTPtr child = node->arguments->children[i];
|
ASTPtr child = node->arguments->children[i];
|
||||||
|
|
||||||
ASTFunction * lambda = typeid_cast<ASTFunction *>(child.get());
|
const auto * lambda = child->as<ASTFunction>();
|
||||||
if (lambda && lambda->name == "lambda")
|
if (lambda && lambda->name == "lambda")
|
||||||
{
|
{
|
||||||
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
|
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
|
||||||
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(lambda->arguments->children.at(0).get());
|
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||||
ASTs lambda_arg_asts = lambda_args_tuple->arguments->children;
|
const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children;
|
||||||
NamesAndTypesList lambda_arguments;
|
NamesAndTypesList lambda_arguments;
|
||||||
|
|
||||||
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
||||||
@ -517,7 +516,7 @@ void ActionsVisitor::visit(const ASTPtr & ast)
|
|||||||
ExpressionAction::applyFunction(function_builder, argument_names, getColumnName()));
|
ExpressionAction::applyFunction(function_builder, argument_names, getColumnName()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (ASTLiteral * literal = typeid_cast<ASTLiteral *>(ast.get()))
|
else if (const auto * literal = ast->as<ASTLiteral>())
|
||||||
{
|
{
|
||||||
DataTypePtr type = applyVisitor(FieldToDataType(), literal->value);
|
DataTypePtr type = applyVisitor(FieldToDataType(), literal->value);
|
||||||
|
|
||||||
@ -533,8 +532,7 @@ void ActionsVisitor::visit(const ASTPtr & ast)
|
|||||||
for (auto & child : ast->children)
|
for (auto & child : ast->children)
|
||||||
{
|
{
|
||||||
/// Do not go to FROM, JOIN, UNION.
|
/// Do not go to FROM, JOIN, UNION.
|
||||||
if (!typeid_cast<const ASTTableExpression *>(child.get())
|
if (!child->as<ASTTableExpression>() && !child->as<ASTSelectQuery>())
|
||||||
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
|
||||||
visit(child);
|
visit(child);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -550,8 +548,8 @@ SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bl
|
|||||||
const ASTPtr & arg = args.children.at(1);
|
const ASTPtr & arg = args.children.at(1);
|
||||||
|
|
||||||
/// If the subquery or table name for SELECT.
|
/// If the subquery or table name for SELECT.
|
||||||
const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(arg.get());
|
const auto * identifier = arg->as<ASTIdentifier>();
|
||||||
if (typeid_cast<const ASTSubquery *>(arg.get()) || identifier)
|
if (arg->as<ASTSubquery>() || identifier)
|
||||||
{
|
{
|
||||||
auto set_key = PreparedSetKey::forSubquery(*arg);
|
auto set_key = PreparedSetKey::forSubquery(*arg);
|
||||||
if (prepared_sets.count(set_key))
|
if (prepared_sets.count(set_key))
|
||||||
|
@ -1,15 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <unordered_map>
|
|
||||||
#include <memory>
|
|
||||||
#include <Core/Types.h>
|
#include <Core/Types.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
#include <unordered_map>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
using Aliases = std::unordered_map<String, ASTPtr>;
|
using Aliases = std::unordered_map<String, ASTPtr>;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -45,7 +45,7 @@ ExpressionActionsPtr AnalyzedJoin::createJoinedBlockActions(
|
|||||||
if (!join)
|
if (!join)
|
||||||
return nullptr;
|
return nullptr;
|
||||||
|
|
||||||
const auto & join_params = static_cast<const ASTTableJoin &>(*join->table_join);
|
const auto & join_params = join->table_join->as<ASTTableJoin &>();
|
||||||
|
|
||||||
/// Create custom expression list with join keys from right table.
|
/// Create custom expression list with join keys from right table.
|
||||||
auto expression_list = std::make_shared<ASTExpressionList>();
|
auto expression_list = std::make_shared<ASTExpressionList>();
|
||||||
|
@ -40,11 +40,10 @@ public:
|
|||||||
|
|
||||||
static bool needChildVisit(ASTPtr & node, const ASTPtr & child)
|
static bool needChildVisit(ASTPtr & node, const ASTPtr & child)
|
||||||
{
|
{
|
||||||
if (typeid_cast<ASTTablesInSelectQuery *>(node.get()))
|
if (node->as<ASTTablesInSelectQuery>())
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (typeid_cast<ASTSubquery *>(child.get()) ||
|
if (child->as<ASTSubquery>() || child->as<ASTSelectQuery>())
|
||||||
typeid_cast<ASTSelectQuery *>(child.get()))
|
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
@ -52,9 +51,9 @@ public:
|
|||||||
|
|
||||||
static void visit(ASTPtr & ast, Data & data)
|
static void visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * t = typeid_cast<ASTIdentifier *>(ast.get()))
|
if (const auto * t = ast->as<ASTIdentifier>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
if (auto * t = typeid_cast<ASTSelectQuery *>(ast.get()))
|
if (const auto * t = ast->as<ASTSelectQuery>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -73,7 +72,7 @@ private:
|
|||||||
const String nested_table_name = ast->getColumnName();
|
const String nested_table_name = ast->getColumnName();
|
||||||
const String nested_table_alias = ast->getAliasOrColumnName();
|
const String nested_table_alias = ast->getAliasOrColumnName();
|
||||||
|
|
||||||
if (nested_table_alias == nested_table_name && !isIdentifier(ast))
|
if (nested_table_alias == nested_table_name && !ast->as<ASTIdentifier>())
|
||||||
throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, ErrorCodes::ALIAS_REQUIRED);
|
throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, ErrorCodes::ALIAS_REQUIRED);
|
||||||
|
|
||||||
if (data.array_join_alias_to_name.count(nested_table_alias) || data.aliases.count(nested_table_alias))
|
if (data.array_join_alias_to_name.count(nested_table_alias) || data.aliases.count(nested_table_alias))
|
||||||
|
@ -98,7 +98,7 @@ void SelectStreamFactory::createForShard(
|
|||||||
|
|
||||||
if (table_func_ptr)
|
if (table_func_ptr)
|
||||||
{
|
{
|
||||||
auto table_function = static_cast<const ASTFunction *>(table_func_ptr.get());
|
const auto * table_function = table_func_ptr->as<ASTFunction>();
|
||||||
main_table_storage = TableFunctionFactory::instance().get(table_function->name, context)->execute(table_func_ptr, context);
|
main_table_storage = TableFunctionFactory::instance().get(table_function->name, context)->execute(table_func_ptr, context);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -892,8 +892,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
|
|||||||
|
|
||||||
if (!res)
|
if (!res)
|
||||||
{
|
{
|
||||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(
|
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression->as<ASTFunction>()->name, *this);
|
||||||
typeid_cast<const ASTFunction *>(table_expression.get())->name, *this);
|
|
||||||
|
|
||||||
/// Run it and remember the result
|
/// Run it and remember the result
|
||||||
res = table_function_ptr->execute(table_expression, *this);
|
res = table_function_ptr->execute(table_expression, *this);
|
||||||
|
@ -1,23 +1,24 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Core/Block.h>
|
||||||
|
#include <Core/NamesAndTypes.h>
|
||||||
|
#include <Core/Types.h>
|
||||||
|
#include <Interpreters/ClientInfo.h>
|
||||||
|
#include <Interpreters/Settings.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
#include <Common/LRUCache.h>
|
||||||
|
#include <Common/MultiVersion.h>
|
||||||
|
#include <Common/ThreadPool.h>
|
||||||
|
#include <Common/config.h>
|
||||||
|
|
||||||
|
#include <atomic>
|
||||||
#include <chrono>
|
#include <chrono>
|
||||||
#include <condition_variable>
|
#include <condition_variable>
|
||||||
#include <functional>
|
#include <functional>
|
||||||
#include <memory>
|
#include <memory>
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
#include <thread>
|
|
||||||
#include <atomic>
|
|
||||||
#include <optional>
|
#include <optional>
|
||||||
|
#include <thread>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Common/MultiVersion.h>
|
|
||||||
#include <Common/LRUCache.h>
|
|
||||||
#include <Common/ThreadPool.h>
|
|
||||||
#include <Core/Types.h>
|
|
||||||
#include <Core/NamesAndTypes.h>
|
|
||||||
#include <Core/Block.h>
|
|
||||||
#include <Interpreters/Settings.h>
|
|
||||||
#include <Interpreters/ClientInfo.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace Poco
|
namespace Poco
|
||||||
@ -68,8 +69,6 @@ class IStorage;
|
|||||||
class ITableFunction;
|
class ITableFunction;
|
||||||
using StoragePtr = std::shared_ptr<IStorage>;
|
using StoragePtr = std::shared_ptr<IStorage>;
|
||||||
using Tables = std::map<String, StoragePtr>;
|
using Tables = std::map<String, StoragePtr>;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
class IBlockInputStream;
|
class IBlockInputStream;
|
||||||
class IBlockOutputStream;
|
class IBlockOutputStream;
|
||||||
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
||||||
|
@ -36,13 +36,13 @@ struct JoinedTable
|
|||||||
|
|
||||||
JoinedTable(ASTPtr table_element)
|
JoinedTable(ASTPtr table_element)
|
||||||
{
|
{
|
||||||
element = typeid_cast<ASTTablesInSelectQueryElement *>(table_element.get());
|
element = table_element->as<ASTTablesInSelectQueryElement>();
|
||||||
if (!element)
|
if (!element)
|
||||||
throw Exception("Logical error: TablesInSelectQueryElement expected", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: TablesInSelectQueryElement expected", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
if (element->table_join)
|
if (element->table_join)
|
||||||
{
|
{
|
||||||
join = typeid_cast<ASTTableJoin *>(element->table_join.get());
|
join = element->table_join->as<ASTTableJoin>();
|
||||||
if (join->kind == ASTTableJoin::Kind::Cross ||
|
if (join->kind == ASTTableJoin::Kind::Cross ||
|
||||||
join->kind == ASTTableJoin::Kind::Comma)
|
join->kind == ASTTableJoin::Kind::Comma)
|
||||||
{
|
{
|
||||||
@ -56,7 +56,7 @@ struct JoinedTable
|
|||||||
|
|
||||||
if (element->table_expression)
|
if (element->table_expression)
|
||||||
{
|
{
|
||||||
auto & expr = typeid_cast<const ASTTableExpression &>(*element->table_expression);
|
const auto & expr = element->table_expression->as<ASTTableExpression &>();
|
||||||
table = DatabaseAndTableWithAlias(expr);
|
table = DatabaseAndTableWithAlias(expr);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -105,7 +105,7 @@ public:
|
|||||||
|
|
||||||
for (auto & child : node.arguments->children)
|
for (auto & child : node.arguments->children)
|
||||||
{
|
{
|
||||||
if (auto func = typeid_cast<const ASTFunction *>(child.get()))
|
if (const auto * func = child->as<ASTFunction>())
|
||||||
visit(*func, child);
|
visit(*func, child);
|
||||||
else
|
else
|
||||||
ands_only = false;
|
ands_only = false;
|
||||||
@ -160,8 +160,8 @@ private:
|
|||||||
if (node.arguments->children.size() != 2)
|
if (node.arguments->children.size() != 2)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
auto left = typeid_cast<const ASTIdentifier *>(node.arguments->children[0].get());
|
const auto * left = node.arguments->children[0]->as<ASTIdentifier>();
|
||||||
auto right = typeid_cast<const ASTIdentifier *>(node.arguments->children[1].get());
|
const auto * right = node.arguments->children[1]->as<ASTIdentifier>();
|
||||||
if (!left || !right)
|
if (!left || !right)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
@ -213,7 +213,7 @@ bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables
|
|||||||
if (!select.tables)
|
if (!select.tables)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
auto tables = typeid_cast<const ASTTablesInSelectQuery *>(select.tables.get());
|
const auto * tables = select.tables->as<ASTTablesInSelectQuery>();
|
||||||
if (!tables)
|
if (!tables)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
@ -232,7 +232,7 @@ bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables
|
|||||||
if (num_tables > 2 && t.has_using)
|
if (num_tables > 2 && t.has_using)
|
||||||
throw Exception("Multiple CROSS/COMMA JOIN do not support USING", ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Multiple CROSS/COMMA JOIN do not support USING", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
|
||||||
if (ASTTableJoin * join = t.join)
|
if (auto * join = t.join)
|
||||||
if (join->kind == ASTTableJoin::Kind::Comma)
|
if (join->kind == ASTTableJoin::Kind::Comma)
|
||||||
++num_comma;
|
++num_comma;
|
||||||
}
|
}
|
||||||
@ -244,7 +244,7 @@ bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables
|
|||||||
|
|
||||||
void CrossToInnerJoinMatcher::visit(ASTPtr & ast, Data & data)
|
void CrossToInnerJoinMatcher::visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * t = typeid_cast<ASTSelectQuery *>(ast.get()))
|
if (auto * t = ast->as<ASTSelectQuery>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -449,6 +449,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
|
|||||||
task.query = parseQuery(parser_query, begin, end, description, 0);
|
task.query = parseQuery(parser_query, begin, end, description, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// XXX: serious design flaw since `ASTQueryWithOnCluster` is not inherited from `IAST`!
|
||||||
if (!task.query || !(task.query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(task.query.get())))
|
if (!task.query || !(task.query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(task.query.get())))
|
||||||
throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
|
throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
|
||||||
|
|
||||||
@ -612,7 +613,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper)
|
|||||||
String rewritten_query = queryToString(rewritten_ast);
|
String rewritten_query = queryToString(rewritten_ast);
|
||||||
LOG_DEBUG(log, "Executing query: " << rewritten_query);
|
LOG_DEBUG(log, "Executing query: " << rewritten_query);
|
||||||
|
|
||||||
if (auto ast_alter = dynamic_cast<const ASTAlterQuery *>(rewritten_ast.get()))
|
if (const auto * ast_alter = rewritten_ast->as<ASTAlterQuery>())
|
||||||
{
|
{
|
||||||
processTaskAlter(task, ast_alter, rewritten_query, task.entry_path, zookeeper);
|
processTaskAlter(task, ast_alter, rewritten_query, task.entry_path, zookeeper);
|
||||||
}
|
}
|
||||||
@ -1211,7 +1212,8 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont
|
|||||||
ASTPtr query_ptr = query_ptr_->clone();
|
ASTPtr query_ptr = query_ptr_->clone();
|
||||||
ASTQueryWithOutput::resetOutputASTIfExist(*query_ptr);
|
ASTQueryWithOutput::resetOutputASTIfExist(*query_ptr);
|
||||||
|
|
||||||
auto query = dynamic_cast<ASTQueryWithOnCluster *>(query_ptr.get());
|
// XXX: serious design flaw since `ASTQueryWithOnCluster` is not inherited from `IAST`!
|
||||||
|
auto * query = dynamic_cast<ASTQueryWithOnCluster *>(query_ptr.get());
|
||||||
if (!query)
|
if (!query)
|
||||||
{
|
{
|
||||||
throw Exception("Distributed execution is not supported for such DDL queries", ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Distributed execution is not supported for such DDL queries", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
@ -1220,7 +1222,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont
|
|||||||
if (!context.getSettingsRef().allow_distributed_ddl)
|
if (!context.getSettingsRef().allow_distributed_ddl)
|
||||||
throw Exception("Distributed DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
|
throw Exception("Distributed DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
|
||||||
|
|
||||||
if (auto query_alter = dynamic_cast<const ASTAlterQuery *>(query_ptr.get()))
|
if (const auto * query_alter = query_ptr->as<ASTAlterQuery>())
|
||||||
{
|
{
|
||||||
for (const auto & command : query_alter->command_list->commands)
|
for (const auto & command : query_alter->command_list->commands)
|
||||||
{
|
{
|
||||||
|
@ -27,7 +27,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident
|
|||||||
|
|
||||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database)
|
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database)
|
||||||
{
|
{
|
||||||
const auto * identifier = typeid_cast<const ASTIdentifier *>(node.get());
|
const auto * identifier = node->as<ASTIdentifier>();
|
||||||
if (!identifier)
|
if (!identifier)
|
||||||
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
@ -78,10 +78,10 @@ std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelec
|
|||||||
|
|
||||||
for (const auto & child : select_query.tables->children)
|
for (const auto & child : select_query.tables->children)
|
||||||
{
|
{
|
||||||
ASTTablesInSelectQueryElement * tables_element = static_cast<ASTTablesInSelectQueryElement *>(child.get());
|
const auto * tables_element = child->as<ASTTablesInSelectQueryElement>();
|
||||||
|
|
||||||
if (tables_element->table_expression)
|
if (tables_element->table_expression)
|
||||||
tables_expression.emplace_back(static_cast<const ASTTableExpression *>(tables_element->table_expression.get()));
|
tables_expression.emplace_back(tables_element->table_expression->as<ASTTableExpression>());
|
||||||
}
|
}
|
||||||
|
|
||||||
return tables_expression;
|
return tables_expression;
|
||||||
@ -92,17 +92,16 @@ static const ASTTableExpression * getTableExpression(const ASTSelectQuery & sele
|
|||||||
if (!select.tables)
|
if (!select.tables)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
ASTTablesInSelectQuery & tables_in_select_query = static_cast<ASTTablesInSelectQuery &>(*select.tables);
|
const auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
|
||||||
if (tables_in_select_query.children.size() <= table_number)
|
if (tables_in_select_query.children.size() <= table_number)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
ASTTablesInSelectQueryElement & tables_element =
|
const auto & tables_element = tables_in_select_query.children[table_number]->as<ASTTablesInSelectQueryElement &>();
|
||||||
static_cast<ASTTablesInSelectQueryElement &>(*tables_in_select_query.children[table_number]);
|
|
||||||
|
|
||||||
if (!tables_element.table_expression)
|
if (!tables_element.table_expression)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
return static_cast<const ASTTableExpression *>(tables_element.table_expression.get());
|
return tables_element.table_expression->as<ASTTableExpression>();
|
||||||
}
|
}
|
||||||
|
|
||||||
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database)
|
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database)
|
||||||
@ -125,7 +124,7 @@ std::optional<DatabaseAndTableWithAlias> getDatabaseAndTable(const ASTSelectQuer
|
|||||||
return {};
|
return {};
|
||||||
|
|
||||||
ASTPtr database_and_table_name = table_expression->database_and_table_name;
|
ASTPtr database_and_table_name = table_expression->database_and_table_name;
|
||||||
if (!database_and_table_name || !isIdentifier(database_and_table_name))
|
if (!database_and_table_name || !database_and_table_name->as<ASTIdentifier>())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
return DatabaseAndTableWithAlias(database_and_table_name);
|
return DatabaseAndTableWithAlias(database_and_table_name);
|
||||||
@ -142,7 +141,7 @@ ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number
|
|||||||
return table_expression->table_function;
|
return table_expression->table_function;
|
||||||
|
|
||||||
if (table_expression->subquery)
|
if (table_expression->subquery)
|
||||||
return static_cast<const ASTSubquery *>(table_expression->subquery.get())->children[0];
|
return table_expression->subquery->children[0];
|
||||||
}
|
}
|
||||||
|
|
||||||
return nullptr;
|
return nullptr;
|
||||||
|
@ -1,18 +1,16 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Core/Names.h>
|
||||||
|
#include <Core/Types.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
#include <memory>
|
#include <memory>
|
||||||
#include <optional>
|
#include <optional>
|
||||||
|
|
||||||
#include <Core/Types.h>
|
|
||||||
#include <Core/Names.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
class ASTSelectQuery;
|
class ASTSelectQuery;
|
||||||
class ASTIdentifier;
|
class ASTIdentifier;
|
||||||
struct ASTTableExpression;
|
struct ASTTableExpression;
|
||||||
|
@ -41,19 +41,17 @@ static ASTPtr addTypeConversion(std::unique_ptr<ASTLiteral> && ast, const String
|
|||||||
bool ExecuteScalarSubqueriesMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
|
bool ExecuteScalarSubqueriesMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
|
||||||
{
|
{
|
||||||
/// Processed
|
/// Processed
|
||||||
if (typeid_cast<ASTSubquery *>(node.get()) ||
|
if (node->as<ASTSubquery>() || node->as<ASTFunction>())
|
||||||
typeid_cast<ASTFunction *>(node.get()))
|
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
/// Don't descend into subqueries in FROM section
|
/// Don't descend into subqueries in FROM section
|
||||||
if (typeid_cast<ASTTableExpression *>(node.get()))
|
if (node->as<ASTTableExpression>())
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (typeid_cast<ASTSelectQuery *>(node.get()))
|
if (node->as<ASTSelectQuery>())
|
||||||
{
|
{
|
||||||
/// Do not go to FROM, JOIN, UNION.
|
/// Do not go to FROM, JOIN, UNION.
|
||||||
if (typeid_cast<ASTTableExpression *>(child.get()) ||
|
if (child->as<ASTTableExpression>() || child->as<ASTSelectQuery>())
|
||||||
typeid_cast<ASTSelectQuery *>(child.get()))
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -62,9 +60,9 @@ bool ExecuteScalarSubqueriesMatcher::needChildVisit(ASTPtr & node, const ASTPtr
|
|||||||
|
|
||||||
void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data)
|
void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * t = typeid_cast<ASTSubquery *>(ast.get()))
|
if (const auto * t = ast->as<ASTSubquery>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
if (auto * t = typeid_cast<ASTFunction *>(ast.get()))
|
if (const auto * t = ast->as<ASTFunction>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -147,7 +145,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTFunction & func, ASTPtr & as
|
|||||||
out.push_back(&child);
|
out.push_back(&child);
|
||||||
else
|
else
|
||||||
for (size_t i = 0, size = func.arguments->children.size(); i < size; ++i)
|
for (size_t i = 0, size = func.arguments->children.size(); i < size; ++i)
|
||||||
if (i != 1 || !typeid_cast<ASTSubquery *>(func.arguments->children[i].get()))
|
if (i != 1 || !func.arguments->children[i]->as<ASTSubquery>())
|
||||||
out.push_back(&func.arguments->children[i]);
|
out.push_back(&func.arguments->children[i]);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -90,8 +90,6 @@ ExpressionAnalyzer::ExpressionAnalyzer(
|
|||||||
storage = syntax->storage;
|
storage = syntax->storage;
|
||||||
rewrite_subqueries = syntax->rewrite_subqueries;
|
rewrite_subqueries = syntax->rewrite_subqueries;
|
||||||
|
|
||||||
select_query = typeid_cast<ASTSelectQuery *>(query.get());
|
|
||||||
|
|
||||||
if (!additional_source_columns.empty())
|
if (!additional_source_columns.empty())
|
||||||
{
|
{
|
||||||
source_columns.insert(source_columns.end(), additional_source_columns.begin(), additional_source_columns.end());
|
source_columns.insert(source_columns.end(), additional_source_columns.begin(), additional_source_columns.end());
|
||||||
@ -130,6 +128,8 @@ void ExpressionAnalyzer::analyzeAggregation()
|
|||||||
* Everything below (compiling temporary ExpressionActions) - only for the purpose of query analysis (type output).
|
* Everything below (compiling temporary ExpressionActions) - only for the purpose of query analysis (type output).
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
if (select_query && (select_query->group_expression_list || select_query->having_expression))
|
if (select_query && (select_query->group_expression_list || select_query->having_expression))
|
||||||
has_aggregation = true;
|
has_aggregation = true;
|
||||||
|
|
||||||
@ -149,7 +149,7 @@ void ExpressionAnalyzer::analyzeAggregation()
|
|||||||
const ASTTablesInSelectQueryElement * join = select_query->join();
|
const ASTTablesInSelectQueryElement * join = select_query->join();
|
||||||
if (join)
|
if (join)
|
||||||
{
|
{
|
||||||
const auto table_join = static_cast<const ASTTableJoin &>(*join->table_join);
|
const auto & table_join = join->table_join->as<ASTTableJoin &>();
|
||||||
if (table_join.using_expression_list)
|
if (table_join.using_expression_list)
|
||||||
getRootActions(table_join.using_expression_list, true, temp_actions);
|
getRootActions(table_join.using_expression_list, true, temp_actions);
|
||||||
if (table_join.on_expression)
|
if (table_join.on_expression)
|
||||||
@ -250,6 +250,8 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables()
|
|||||||
|
|
||||||
void ExpressionAnalyzer::makeSetsForIndex()
|
void ExpressionAnalyzer::makeSetsForIndex()
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
if (storage && select_query && storage->supportsIndexForIn())
|
if (storage && select_query && storage->supportsIndexForIn())
|
||||||
{
|
{
|
||||||
if (select_query->where_expression)
|
if (select_query->where_expression)
|
||||||
@ -288,18 +290,18 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
|
|||||||
for (auto & child : node->children)
|
for (auto & child : node->children)
|
||||||
{
|
{
|
||||||
/// Don't descend into subqueries.
|
/// Don't descend into subqueries.
|
||||||
if (typeid_cast<ASTSubquery *>(child.get()))
|
if (child->as<ASTSubquery>())
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
/// Don't descend into lambda functions
|
/// Don't descend into lambda functions
|
||||||
const ASTFunction * func = typeid_cast<const ASTFunction *>(child.get());
|
const auto * func = child->as<ASTFunction>();
|
||||||
if (func && func->name == "lambda")
|
if (func && func->name == "lambda")
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
makeSetsForIndexImpl(child);
|
makeSetsForIndexImpl(child);
|
||||||
}
|
}
|
||||||
|
|
||||||
const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get());
|
const auto * func = node->as<ASTFunction>();
|
||||||
if (func && functionIsInOperator(func->name))
|
if (func && functionIsInOperator(func->name))
|
||||||
{
|
{
|
||||||
const IAST & args = *func->arguments;
|
const IAST & args = *func->arguments;
|
||||||
@ -307,7 +309,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
|
|||||||
if (storage && storage->mayBenefitFromIndexForIn(args.children.at(0), context))
|
if (storage && storage->mayBenefitFromIndexForIn(args.children.at(0), context))
|
||||||
{
|
{
|
||||||
const ASTPtr & arg = args.children.at(1);
|
const ASTPtr & arg = args.children.at(1);
|
||||||
if (typeid_cast<ASTSubquery *>(arg.get()) || isIdentifier(arg))
|
if (arg->as<ASTSubquery>() || arg->as<ASTIdentifier>())
|
||||||
{
|
{
|
||||||
if (settings.use_index_for_in_with_subqueries)
|
if (settings.use_index_for_in_with_subqueries)
|
||||||
tryMakeSetForIndexFromSubquery(arg);
|
tryMakeSetForIndexFromSubquery(arg);
|
||||||
@ -365,6 +367,8 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join,
|
|||||||
|
|
||||||
void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions)
|
void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
/// There can not be aggregate functions inside the WHERE and PREWHERE.
|
/// There can not be aggregate functions inside the WHERE and PREWHERE.
|
||||||
if (select_query && (ast.get() == select_query->where_expression.get() || ast.get() == select_query->prewhere_expression.get()))
|
if (select_query && (ast.get() == select_query->where_expression.get() || ast.get() == select_query->prewhere_expression.get()))
|
||||||
{
|
{
|
||||||
@ -379,7 +383,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
const ASTFunction * node = typeid_cast<const ASTFunction *>(ast.get());
|
const auto * node = ast->as<ASTFunction>();
|
||||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||||
{
|
{
|
||||||
has_aggregation = true;
|
has_aggregation = true;
|
||||||
@ -414,8 +418,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
|
|||||||
else
|
else
|
||||||
{
|
{
|
||||||
for (const auto & child : ast->children)
|
for (const auto & child : ast->children)
|
||||||
if (!typeid_cast<const ASTSubquery *>(child.get())
|
if (!child->as<ASTSubquery>() && !child->as<ASTSelectQuery>())
|
||||||
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
|
||||||
getAggregates(child, actions);
|
getAggregates(child, actions);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -423,21 +426,22 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
|
|||||||
|
|
||||||
void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * description)
|
void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * description)
|
||||||
{
|
{
|
||||||
const ASTFunction * node = typeid_cast<const ASTFunction *>(ast.get());
|
const auto * node = ast->as<ASTFunction>();
|
||||||
|
|
||||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||||
throw Exception("Aggregate function " + node->getColumnName()
|
throw Exception("Aggregate function " + node->getColumnName()
|
||||||
+ " is found " + String(description) + " in query", ErrorCodes::ILLEGAL_AGGREGATION);
|
+ " is found " + String(description) + " in query", ErrorCodes::ILLEGAL_AGGREGATION);
|
||||||
|
|
||||||
for (const auto & child : ast->children)
|
for (const auto & child : ast->children)
|
||||||
if (!typeid_cast<const ASTSubquery *>(child.get())
|
if (!child->as<ASTSubquery>() && !child->as<ASTSelectQuery>())
|
||||||
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
|
||||||
assertNoAggregates(child, description);
|
assertNoAggregates(child, description);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void ExpressionAnalyzer::assertSelect() const
|
void ExpressionAnalyzer::assertSelect() const
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
if (!select_query)
|
if (!select_query)
|
||||||
throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
@ -475,6 +479,8 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio
|
|||||||
|
|
||||||
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
|
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertSelect();
|
assertSelect();
|
||||||
|
|
||||||
bool is_array_join_left;
|
bool is_array_join_left;
|
||||||
@ -520,6 +526,8 @@ static void appendRequiredColumns(NameSet & required_columns, const Block & samp
|
|||||||
|
|
||||||
bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
|
bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertSelect();
|
assertSelect();
|
||||||
|
|
||||||
if (!select_query->join())
|
if (!select_query->join())
|
||||||
@ -528,8 +536,8 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
|||||||
initChain(chain, source_columns);
|
initChain(chain, source_columns);
|
||||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||||
|
|
||||||
const auto & join_element = static_cast<const ASTTablesInSelectQueryElement &>(*select_query->join());
|
const auto & join_element = select_query->join()->as<ASTTablesInSelectQueryElement &>();
|
||||||
auto & join_params = static_cast<ASTTableJoin &>(*join_element.table_join);
|
auto & join_params = join_element.table_join->as<ASTTableJoin &>();
|
||||||
|
|
||||||
if (join_params.strictness == ASTTableJoin::Strictness::Unspecified && join_params.kind != ASTTableJoin::Kind::Cross)
|
if (join_params.strictness == ASTTableJoin::Strictness::Unspecified && join_params.kind != ASTTableJoin::Kind::Cross)
|
||||||
{
|
{
|
||||||
@ -541,7 +549,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
|||||||
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
|
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto & table_to_join = static_cast<const ASTTableExpression &>(*join_element.table_expression);
|
const auto & table_to_join = join_element.table_expression->as<ASTTableExpression &>();
|
||||||
|
|
||||||
getActionsFromJoinKeys(join_params, only_types, step.actions);
|
getActionsFromJoinKeys(join_params, only_types, step.actions);
|
||||||
|
|
||||||
@ -559,7 +567,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
|||||||
|
|
||||||
if (table)
|
if (table)
|
||||||
{
|
{
|
||||||
StorageJoin * storage_join = dynamic_cast<StorageJoin *>(table.get());
|
auto * storage_join = dynamic_cast<StorageJoin *>(table.get());
|
||||||
|
|
||||||
if (storage_join)
|
if (storage_join)
|
||||||
{
|
{
|
||||||
@ -624,6 +632,8 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
|||||||
bool ExpressionAnalyzer::appendPrewhere(
|
bool ExpressionAnalyzer::appendPrewhere(
|
||||||
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
|
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertSelect();
|
assertSelect();
|
||||||
|
|
||||||
if (!select_query->prewhere_expression)
|
if (!select_query->prewhere_expression)
|
||||||
@ -697,6 +707,8 @@ bool ExpressionAnalyzer::appendPrewhere(
|
|||||||
|
|
||||||
bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
|
bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertSelect();
|
assertSelect();
|
||||||
|
|
||||||
if (!select_query->where_expression)
|
if (!select_query->where_expression)
|
||||||
@ -715,6 +727,8 @@ bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_t
|
|||||||
|
|
||||||
bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types)
|
bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertAggregation();
|
assertAggregation();
|
||||||
|
|
||||||
if (!select_query->group_expression_list)
|
if (!select_query->group_expression_list)
|
||||||
@ -735,6 +749,8 @@ bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only
|
|||||||
|
|
||||||
void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
|
void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertAggregation();
|
assertAggregation();
|
||||||
|
|
||||||
initChain(chain, source_columns);
|
initChain(chain, source_columns);
|
||||||
@ -759,6 +775,8 @@ void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChai
|
|||||||
|
|
||||||
bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
|
bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertAggregation();
|
assertAggregation();
|
||||||
|
|
||||||
if (!select_query->having_expression)
|
if (!select_query->having_expression)
|
||||||
@ -775,6 +793,8 @@ bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_
|
|||||||
|
|
||||||
void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
|
void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertSelect();
|
assertSelect();
|
||||||
|
|
||||||
initChain(chain, aggregated_columns);
|
initChain(chain, aggregated_columns);
|
||||||
@ -788,6 +808,8 @@ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_
|
|||||||
|
|
||||||
bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types)
|
bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertSelect();
|
assertSelect();
|
||||||
|
|
||||||
if (!select_query->order_expression_list)
|
if (!select_query->order_expression_list)
|
||||||
@ -801,7 +823,7 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only
|
|||||||
ASTs asts = select_query->order_expression_list->children;
|
ASTs asts = select_query->order_expression_list->children;
|
||||||
for (size_t i = 0; i < asts.size(); ++i)
|
for (size_t i = 0; i < asts.size(); ++i)
|
||||||
{
|
{
|
||||||
ASTOrderByElement * ast = typeid_cast<ASTOrderByElement *>(asts[i].get());
|
const auto * ast = asts[i]->as<ASTOrderByElement>();
|
||||||
if (!ast || ast->children.size() < 1)
|
if (!ast || ast->children.size() < 1)
|
||||||
throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
|
throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
|
||||||
ASTPtr order_expression = ast->children.at(0);
|
ASTPtr order_expression = ast->children.at(0);
|
||||||
@ -813,6 +835,8 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only
|
|||||||
|
|
||||||
bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
|
bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertSelect();
|
assertSelect();
|
||||||
|
|
||||||
if (!select_query->limit_by_expression_list)
|
if (!select_query->limit_by_expression_list)
|
||||||
@ -831,6 +855,8 @@ bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only
|
|||||||
|
|
||||||
void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
|
void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
|
||||||
{
|
{
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
assertSelect();
|
assertSelect();
|
||||||
|
|
||||||
initChain(chain, aggregated_columns);
|
initChain(chain, aggregated_columns);
|
||||||
@ -864,7 +890,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const
|
|||||||
|
|
||||||
void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries)
|
void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries)
|
||||||
{
|
{
|
||||||
ASTFunction * node = typeid_cast<ASTFunction *>(ast.get());
|
const auto * node = ast->as<ASTFunction>();
|
||||||
|
|
||||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||||
for (auto & argument : node->arguments->children)
|
for (auto & argument : node->arguments->children)
|
||||||
@ -883,7 +909,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje
|
|||||||
|
|
||||||
ASTs asts;
|
ASTs asts;
|
||||||
|
|
||||||
if (auto node = typeid_cast<const ASTExpressionList *>(query.get()))
|
if (const auto * node = query->as<ASTExpressionList>())
|
||||||
asts = node->children;
|
asts = node->children;
|
||||||
else
|
else
|
||||||
asts = ASTs(1, query);
|
asts = ASTs(1, query);
|
||||||
@ -965,21 +991,6 @@ void ExpressionAnalyzer::collectUsedColumns()
|
|||||||
if (columns_context.has_table_join)
|
if (columns_context.has_table_join)
|
||||||
{
|
{
|
||||||
const AnalyzedJoin & analyzed_join = analyzedJoin();
|
const AnalyzedJoin & analyzed_join = analyzedJoin();
|
||||||
#if 0
|
|
||||||
std::cerr << "key_names_left: ";
|
|
||||||
for (const auto & name : analyzed_join.key_names_left)
|
|
||||||
std::cerr << "'" << name << "' ";
|
|
||||||
std::cerr << "key_names_right: ";
|
|
||||||
for (const auto & name : analyzed_join.key_names_right)
|
|
||||||
std::cerr << "'" << name << "' ";
|
|
||||||
std::cerr << "columns_from_joined_table: ";
|
|
||||||
for (const auto & column : analyzed_join.columns_from_joined_table)
|
|
||||||
std::cerr << "'" << column.name_and_type.name << '/' << column.original_name << "' ";
|
|
||||||
std::cerr << "available_joined_columns: ";
|
|
||||||
for (const auto & column : analyzed_join.available_joined_columns)
|
|
||||||
std::cerr << "'" << column.name_and_type.name << '/' << column.original_name << "' ";
|
|
||||||
std::cerr << std::endl;
|
|
||||||
#endif
|
|
||||||
NameSet avaliable_columns;
|
NameSet avaliable_columns;
|
||||||
for (const auto & name : source_columns)
|
for (const auto & name : source_columns)
|
||||||
avaliable_columns.insert(name.name);
|
avaliable_columns.insert(name.name);
|
||||||
@ -1014,6 +1025,8 @@ void ExpressionAnalyzer::collectUsedColumns()
|
|||||||
required.insert(column_name_type.name);
|
required.insert(column_name_type.name);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
|
|
||||||
/// You need to read at least one column to find the number of rows.
|
/// You need to read at least one column to find the number of rows.
|
||||||
if (select_query && required.empty())
|
if (select_query && required.empty())
|
||||||
required.insert(ExpressionActions::getSmallestColumn(source_columns));
|
required.insert(ExpressionActions::getSmallestColumn(source_columns));
|
||||||
|
@ -1,9 +1,11 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/ActionsVisitor.h>
|
||||||
#include <Interpreters/AggregateDescription.h>
|
#include <Interpreters/AggregateDescription.h>
|
||||||
#include <Interpreters/Settings.h>
|
#include <Interpreters/Settings.h>
|
||||||
#include <Interpreters/ActionsVisitor.h>
|
|
||||||
#include <Interpreters/SyntaxAnalyzer.h>
|
#include <Interpreters/SyntaxAnalyzer.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -15,9 +17,6 @@ struct ExpressionActionsChain;
|
|||||||
class ExpressionActions;
|
class ExpressionActions;
|
||||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||||
|
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
using ASTs = std::vector<ASTPtr>;
|
|
||||||
struct ASTTableJoin;
|
struct ASTTableJoin;
|
||||||
|
|
||||||
class IBlockInputStream;
|
class IBlockInputStream;
|
||||||
@ -211,7 +210,6 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
ASTPtr query;
|
ASTPtr query;
|
||||||
ASTSelectQuery * select_query;
|
|
||||||
const Context & context;
|
const Context & context;
|
||||||
const ExtractedSettings settings;
|
const ExtractedSettings settings;
|
||||||
StoragePtr storage; /// The main table in FROM clause, if exists.
|
StoragePtr storage; /// The main table in FROM clause, if exists.
|
||||||
|
@ -22,7 +22,7 @@ public:
|
|||||||
|
|
||||||
static void visit(ASTPtr & ast, Data & data)
|
static void visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * t = typeid_cast<ASTIdentifier *>(ast.get()))
|
if (const auto * t = ast->as<ASTIdentifier>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -56,12 +56,12 @@ public:
|
|||||||
ASTPtr table_name;
|
ASTPtr table_name;
|
||||||
ASTPtr subquery_or_table_name;
|
ASTPtr subquery_or_table_name;
|
||||||
|
|
||||||
if (isIdentifier(subquery_or_table_name_or_table_expression))
|
if (subquery_or_table_name_or_table_expression->as<ASTIdentifier>())
|
||||||
{
|
{
|
||||||
table_name = subquery_or_table_name_or_table_expression;
|
table_name = subquery_or_table_name_or_table_expression;
|
||||||
subquery_or_table_name = table_name;
|
subquery_or_table_name = table_name;
|
||||||
}
|
}
|
||||||
else if (auto ast_table_expr = typeid_cast<const ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
|
else if (const auto * ast_table_expr = subquery_or_table_name_or_table_expression->as<ASTTableExpression>())
|
||||||
{
|
{
|
||||||
if (ast_table_expr->database_and_table_name)
|
if (ast_table_expr->database_and_table_name)
|
||||||
{
|
{
|
||||||
@ -74,7 +74,7 @@ public:
|
|||||||
subquery_or_table_name = subquery;
|
subquery_or_table_name = subquery;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (typeid_cast<const ASTSubquery *>(subquery_or_table_name_or_table_expression.get()))
|
else if (subquery_or_table_name_or_table_expression->as<ASTSubquery>())
|
||||||
{
|
{
|
||||||
subquery = subquery_or_table_name_or_table_expression;
|
subquery = subquery_or_table_name_or_table_expression;
|
||||||
subquery_or_table_name = subquery;
|
subquery_or_table_name = subquery;
|
||||||
@ -115,7 +115,7 @@ public:
|
|||||||
|
|
||||||
auto database_and_table_name = createTableIdentifier("", external_table_name);
|
auto database_and_table_name = createTableIdentifier("", external_table_name);
|
||||||
|
|
||||||
if (auto ast_table_expr = typeid_cast<ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
|
if (auto * ast_table_expr = subquery_or_table_name_or_table_expression->as<ASTTableExpression>())
|
||||||
{
|
{
|
||||||
ast_table_expr->subquery.reset();
|
ast_table_expr->subquery.reset();
|
||||||
ast_table_expr->database_and_table_name = database_and_table_name;
|
ast_table_expr->database_and_table_name = database_and_table_name;
|
||||||
@ -140,16 +140,16 @@ public:
|
|||||||
|
|
||||||
static void visit(ASTPtr & ast, Data & data)
|
static void visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * t = typeid_cast<ASTFunction *>(ast.get()))
|
if (auto * t = ast->as<ASTFunction>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
if (auto * t = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
|
if (auto * t = ast->as<ASTTablesInSelectQueryElement>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
}
|
}
|
||||||
|
|
||||||
static bool needChildVisit(ASTPtr &, const ASTPtr & child)
|
static bool needChildVisit(ASTPtr &, const ASTPtr & child)
|
||||||
{
|
{
|
||||||
/// We do not go into subqueries.
|
/// We do not go into subqueries.
|
||||||
if (typeid_cast<ASTSelectQuery *>(child.get()))
|
if (child->as<ASTSelectQuery>())
|
||||||
return false;
|
return false;
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -168,8 +168,7 @@ private:
|
|||||||
/// GLOBAL JOIN
|
/// GLOBAL JOIN
|
||||||
static void visit(ASTTablesInSelectQueryElement & table_elem, ASTPtr &, Data & data)
|
static void visit(ASTTablesInSelectQueryElement & table_elem, ASTPtr &, Data & data)
|
||||||
{
|
{
|
||||||
if (table_elem.table_join
|
if (table_elem.table_join && table_elem.table_join->as<ASTTableJoin &>().locality == ASTTableJoin::Locality::Global)
|
||||||
&& static_cast<const ASTTableJoin &>(*table_elem.table_join).locality == ASTTableJoin::Locality::Global)
|
|
||||||
{
|
{
|
||||||
data.addExternalStorage(table_elem.table_expression);
|
data.addExternalStorage(table_elem.table_expression);
|
||||||
data.has_global_subqueries = true;
|
data.has_global_subqueries = true;
|
||||||
|
@ -15,7 +15,7 @@ std::optional<String> IdentifierSemantic::getColumnName(const ASTIdentifier & no
|
|||||||
std::optional<String> IdentifierSemantic::getColumnName(const ASTPtr & ast)
|
std::optional<String> IdentifierSemantic::getColumnName(const ASTPtr & ast)
|
||||||
{
|
{
|
||||||
if (ast)
|
if (ast)
|
||||||
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
if (const auto * id = ast->as<ASTIdentifier>())
|
||||||
if (!id->semantic->special)
|
if (!id->semantic->special)
|
||||||
return id->name;
|
return id->name;
|
||||||
return {};
|
return {};
|
||||||
@ -31,7 +31,7 @@ std::optional<String> IdentifierSemantic::getTableName(const ASTIdentifier & nod
|
|||||||
std::optional<String> IdentifierSemantic::getTableName(const ASTPtr & ast)
|
std::optional<String> IdentifierSemantic::getTableName(const ASTPtr & ast)
|
||||||
{
|
{
|
||||||
if (ast)
|
if (ast)
|
||||||
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
if (const auto * id = ast->as<ASTIdentifier>())
|
||||||
if (id->semantic->special)
|
if (id->semantic->special)
|
||||||
return id->name;
|
return id->name;
|
||||||
return {};
|
return {};
|
||||||
@ -144,7 +144,7 @@ void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const Dat
|
|||||||
String IdentifierSemantic::columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
String IdentifierSemantic::columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
|
||||||
{
|
{
|
||||||
ASTPtr copy = identifier.clone();
|
ASTPtr copy = identifier.clone();
|
||||||
setColumnNormalName(typeid_cast<ASTIdentifier &>(*copy), db_and_table);
|
setColumnNormalName(copy->as<ASTIdentifier &>(), db_and_table);
|
||||||
return copy->getAliasOrColumnName();
|
return copy->getAliasOrColumnName();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -30,7 +30,7 @@ namespace
|
|||||||
template <typename F>
|
template <typename F>
|
||||||
void forEachNonGlobalSubquery(IAST * node, F && f)
|
void forEachNonGlobalSubquery(IAST * node, F && f)
|
||||||
{
|
{
|
||||||
if (ASTFunction * function = typeid_cast<ASTFunction *>(node))
|
if (auto * function = node->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
if (function->name == "in" || function->name == "notIn")
|
if (function->name == "in" || function->name == "notIn")
|
||||||
{
|
{
|
||||||
@ -40,14 +40,14 @@ void forEachNonGlobalSubquery(IAST * node, F && f)
|
|||||||
|
|
||||||
/// Pass into other functions, as subquery could be in aggregate or in lambda functions.
|
/// Pass into other functions, as subquery could be in aggregate or in lambda functions.
|
||||||
}
|
}
|
||||||
else if (ASTTablesInSelectQueryElement * join = typeid_cast<ASTTablesInSelectQueryElement *>(node))
|
else if (const auto * join = node->as<ASTTablesInSelectQueryElement>())
|
||||||
{
|
{
|
||||||
if (join->table_join && join->table_expression)
|
if (join->table_join && join->table_expression)
|
||||||
{
|
{
|
||||||
auto & table_join = static_cast<ASTTableJoin &>(*join->table_join);
|
auto & table_join = join->table_join->as<ASTTableJoin &>();
|
||||||
if (table_join.locality != ASTTableJoin::Locality::Global)
|
if (table_join.locality != ASTTableJoin::Locality::Global)
|
||||||
{
|
{
|
||||||
auto & subquery = static_cast<ASTTableExpression &>(*join->table_expression).subquery;
|
auto & subquery = join->table_expression->as<ASTTableExpression>()->subquery;
|
||||||
if (subquery)
|
if (subquery)
|
||||||
f(subquery.get(), nullptr, &table_join);
|
f(subquery.get(), nullptr, &table_join);
|
||||||
}
|
}
|
||||||
@ -59,7 +59,7 @@ void forEachNonGlobalSubquery(IAST * node, F && f)
|
|||||||
|
|
||||||
/// Descent into all children, but not into subqueries of other kind (scalar subqueries), that are irrelevant to us.
|
/// Descent into all children, but not into subqueries of other kind (scalar subqueries), that are irrelevant to us.
|
||||||
for (auto & child : node->children)
|
for (auto & child : node->children)
|
||||||
if (!typeid_cast<ASTSelectQuery *>(child.get()))
|
if (!child->as<ASTSelectQuery>())
|
||||||
forEachNonGlobalSubquery(child.get(), f);
|
forEachNonGlobalSubquery(child.get(), f);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -69,7 +69,7 @@ void forEachNonGlobalSubquery(IAST * node, F && f)
|
|||||||
template <typename F>
|
template <typename F>
|
||||||
void forEachTable(IAST * node, F && f)
|
void forEachTable(IAST * node, F && f)
|
||||||
{
|
{
|
||||||
if (auto table_expression = typeid_cast<ASTTableExpression *>(node))
|
if (auto * table_expression = node->as<ASTTableExpression>())
|
||||||
{
|
{
|
||||||
auto & database_and_table = table_expression->database_and_table_name;
|
auto & database_and_table = table_expression->database_and_table_name;
|
||||||
if (database_and_table)
|
if (database_and_table)
|
||||||
@ -103,15 +103,15 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
|
|||||||
if (!query->tables)
|
if (!query->tables)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
ASTTablesInSelectQuery & tables_in_select_query = static_cast<ASTTablesInSelectQuery &>(*query->tables);
|
const auto & tables_in_select_query = query->tables->as<ASTTablesInSelectQuery &>();
|
||||||
if (tables_in_select_query.children.empty())
|
if (tables_in_select_query.children.empty())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
ASTTablesInSelectQueryElement & tables_element = static_cast<ASTTablesInSelectQueryElement &>(*tables_in_select_query.children[0]);
|
const auto & tables_element = tables_in_select_query.children[0]->as<ASTTablesInSelectQueryElement &>();
|
||||||
if (!tables_element.table_expression)
|
if (!tables_element.table_expression)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
ASTTableExpression * table_expression = static_cast<ASTTableExpression *>(tables_element.table_expression.get());
|
const auto * table_expression = tables_element.table_expression->as<ASTTableExpression>();
|
||||||
|
|
||||||
/// If not ordinary table, skip it.
|
/// If not ordinary table, skip it.
|
||||||
if (!table_expression->database_and_table_name)
|
if (!table_expression->database_and_table_name)
|
||||||
@ -143,7 +143,7 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
|
|||||||
{
|
{
|
||||||
if (function)
|
if (function)
|
||||||
{
|
{
|
||||||
ASTFunction * concrete = static_cast<ASTFunction *>(function);
|
auto * concrete = function->as<ASTFunction>();
|
||||||
|
|
||||||
if (concrete->name == "in")
|
if (concrete->name == "in")
|
||||||
concrete->name = "globalIn";
|
concrete->name = "globalIn";
|
||||||
@ -157,7 +157,7 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
|
|||||||
throw Exception("Logical error: unexpected function name " + concrete->name, ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: unexpected function name " + concrete->name, ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
else if (table_join)
|
else if (table_join)
|
||||||
static_cast<ASTTableJoin &>(*table_join).locality = ASTTableJoin::Locality::Global;
|
table_join->as<ASTTableJoin &>().locality = ASTTableJoin::Locality::Global;
|
||||||
else
|
else
|
||||||
throw Exception("Logical error: unexpected AST node", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: unexpected AST node", ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
@ -30,7 +30,7 @@ InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, const Co
|
|||||||
|
|
||||||
BlockIO InterpreterAlterQuery::execute()
|
BlockIO InterpreterAlterQuery::execute()
|
||||||
{
|
{
|
||||||
auto & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
|
const auto & alter = query_ptr->as<ASTAlterQuery &>();
|
||||||
|
|
||||||
if (!alter.cluster.empty())
|
if (!alter.cluster.empty())
|
||||||
return executeDDLQueryOnCluster(query_ptr, context, {alter.database});
|
return executeDDLQueryOnCluster(query_ptr, context, {alter.database});
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
/** Allows you add or remove a column in the table.
|
/** Allows you add or remove a column in the table.
|
||||||
* It also allows you to manipulate the partitions of the MergeTree family tables.
|
* It also allows you to manipulate the partitions of the MergeTree family tables.
|
||||||
|
@ -19,8 +19,8 @@ InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, const Co
|
|||||||
|
|
||||||
BlockIO InterpreterCheckQuery::execute()
|
BlockIO InterpreterCheckQuery::execute()
|
||||||
{
|
{
|
||||||
ASTCheckQuery & alter = typeid_cast<ASTCheckQuery &>(*query_ptr);
|
const auto & alter = query_ptr->as<ASTCheckQuery &>();
|
||||||
String & table_name = alter.table;
|
const String & table_name = alter.table;
|
||||||
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;
|
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;
|
||||||
|
|
||||||
StoragePtr table = context.getTable(database_name, table_name);
|
StoragePtr table = context.getTable(database_name, table_name);
|
||||||
|
@ -197,7 +197,7 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col
|
|||||||
|
|
||||||
for (const auto & ast : column_list_ast.children)
|
for (const auto & ast : column_list_ast.children)
|
||||||
{
|
{
|
||||||
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast);
|
auto & col_decl = ast->as<ASTColumnDeclaration &>();
|
||||||
|
|
||||||
DataTypePtr column_type = nullptr;
|
DataTypePtr column_type = nullptr;
|
||||||
if (col_decl.type)
|
if (col_decl.type)
|
||||||
@ -240,7 +240,7 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col
|
|||||||
|
|
||||||
if (col_decl.comment)
|
if (col_decl.comment)
|
||||||
{
|
{
|
||||||
if (auto comment_str = typeid_cast<ASTLiteral &>(*col_decl.comment).value.get<String>(); !comment_str.empty())
|
if (auto comment_str = col_decl.comment->as<ASTLiteral &>().value.get<String>(); !comment_str.empty())
|
||||||
comments.emplace(col_decl.name, comment_str);
|
comments.emplace(col_decl.name, comment_str);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -526,7 +526,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
|
|||||||
String as_table_name = create.as_table;
|
String as_table_name = create.as_table;
|
||||||
|
|
||||||
ASTPtr as_create_ptr = context.getCreateTableQuery(as_database_name, as_table_name);
|
ASTPtr as_create_ptr = context.getCreateTableQuery(as_database_name, as_table_name);
|
||||||
const auto & as_create = typeid_cast<const ASTCreateQuery &>(*as_create_ptr);
|
const auto & as_create = as_create_ptr->as<ASTCreateQuery &>();
|
||||||
|
|
||||||
if (as_create.is_view)
|
if (as_create.is_view)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
@ -566,8 +566,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
|||||||
{
|
{
|
||||||
// Table SQL definition is available even if the table is detached
|
// Table SQL definition is available even if the table is detached
|
||||||
auto query = context.getCreateTableQuery(database_name, table_name);
|
auto query = context.getCreateTableQuery(database_name, table_name);
|
||||||
auto & as_create = typeid_cast<const ASTCreateQuery &>(*query);
|
create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE
|
||||||
create = as_create; // Copy the saved create query, but use ATTACH instead of CREATE
|
|
||||||
create.attach = true;
|
create.attach = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -695,7 +694,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
|||||||
|
|
||||||
BlockIO InterpreterCreateQuery::execute()
|
BlockIO InterpreterCreateQuery::execute()
|
||||||
{
|
{
|
||||||
ASTCreateQuery & create = typeid_cast<ASTCreateQuery &>(*query_ptr);
|
auto & create = query_ptr->as<ASTCreateQuery &>();
|
||||||
checkAccess(create);
|
checkAccess(create);
|
||||||
ASTQueryWithOutput::resetOutputASTIfExist(create);
|
ASTQueryWithOutput::resetOutputASTIfExist(create);
|
||||||
|
|
||||||
|
@ -58,7 +58,7 @@ Block InterpreterDescribeQuery::getSampleBlock()
|
|||||||
|
|
||||||
BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
|
BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
|
||||||
{
|
{
|
||||||
const ASTDescribeQuery & ast = typeid_cast<const ASTDescribeQuery &>(*query_ptr);
|
const auto & ast = query_ptr->as<ASTDescribeQuery &>();
|
||||||
|
|
||||||
NamesAndTypesList columns;
|
NamesAndTypesList columns;
|
||||||
ColumnDefaults column_defaults;
|
ColumnDefaults column_defaults;
|
||||||
@ -66,7 +66,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
|
|||||||
ColumnCodecs column_codecs;
|
ColumnCodecs column_codecs;
|
||||||
StoragePtr table;
|
StoragePtr table;
|
||||||
|
|
||||||
auto table_expression = typeid_cast<const ASTTableExpression *>(ast.table_expression.get());
|
const auto * table_expression = ast.table_expression->as<ASTTableExpression>();
|
||||||
|
|
||||||
if (table_expression->subquery)
|
if (table_expression->subquery)
|
||||||
{
|
{
|
||||||
@ -76,7 +76,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
|
|||||||
{
|
{
|
||||||
if (table_expression->table_function)
|
if (table_expression->table_function)
|
||||||
{
|
{
|
||||||
auto table_function = typeid_cast<const ASTFunction *>(table_expression->table_function.get());
|
const auto * table_function = table_expression->table_function->as<ASTFunction>();
|
||||||
/// Get the table function
|
/// Get the table function
|
||||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context);
|
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context);
|
||||||
/// Run it and remember the result
|
/// Run it and remember the result
|
||||||
@ -84,7 +84,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
auto identifier = typeid_cast<const ASTIdentifier *>(table_expression->database_and_table_name.get());
|
const auto * identifier = table_expression->database_and_table_name->as<ASTIdentifier>();
|
||||||
|
|
||||||
String database_name;
|
String database_name;
|
||||||
String table_name;
|
String table_name;
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Return names, types and other information about columns in specified table.
|
/** Return names, types and other information about columns in specified table.
|
||||||
|
@ -31,7 +31,7 @@ InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, Context &
|
|||||||
|
|
||||||
BlockIO InterpreterDropQuery::execute()
|
BlockIO InterpreterDropQuery::execute()
|
||||||
{
|
{
|
||||||
ASTDropQuery & drop = typeid_cast<ASTDropQuery &>(*query_ptr);
|
auto & drop = query_ptr->as<ASTDropQuery &>();
|
||||||
|
|
||||||
checkAccess(drop);
|
checkAccess(drop);
|
||||||
|
|
||||||
|
@ -1,16 +1,14 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Databases/IDatabase.h>
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
#include <Parsers/ASTDropQuery.h>
|
#include <Parsers/ASTDropQuery.h>
|
||||||
#include <Databases/IDatabase.h>
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
using DatabaseAndTable = std::pair<DatabasePtr, StoragePtr>;
|
using DatabaseAndTable = std::pair<DatabasePtr, StoragePtr>;
|
||||||
|
|
||||||
/** Allow to either drop table with all its data (DROP),
|
/** Allow to either drop table with all its data (DROP),
|
||||||
|
@ -32,7 +32,7 @@ Block InterpreterExistsQuery::getSampleBlock()
|
|||||||
|
|
||||||
BlockInputStreamPtr InterpreterExistsQuery::executeImpl()
|
BlockInputStreamPtr InterpreterExistsQuery::executeImpl()
|
||||||
{
|
{
|
||||||
const ASTExistsQuery & ast = typeid_cast<const ASTExistsQuery &>(*query_ptr);
|
const auto & ast = query_ptr->as<ASTExistsQuery &>();
|
||||||
bool res = ast.temporary ? context.isExternalTableExist(ast.table) : context.isTableExist(ast.database, ast.table);
|
bool res = ast.temporary ? context.isExternalTableExist(ast.table) : context.isTableExist(ast.database, ast.table);
|
||||||
|
|
||||||
return std::make_shared<OneBlockInputStream>(Block{{
|
return std::make_shared<OneBlockInputStream>(Block{{
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Check that table exists. Return single row with single column "result" of type UInt8 and value 0 or 1.
|
/** Check that table exists. Return single row with single column "result" of type UInt8 and value 0 or 1.
|
||||||
|
@ -39,7 +39,7 @@ Block InterpreterExplainQuery::getSampleBlock()
|
|||||||
|
|
||||||
BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||||
{
|
{
|
||||||
const ASTExplainQuery & ast = typeid_cast<const ASTExplainQuery &>(*query);
|
const auto & ast = query->as<ASTExplainQuery &>();
|
||||||
Block sample_block = getSampleBlock();
|
Block sample_block = getSampleBlock();
|
||||||
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
||||||
|
|
||||||
|
@ -2,15 +2,12 @@
|
|||||||
|
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/// Returns single row with explain results
|
/// Returns single row with explain results
|
||||||
class InterpreterExplainQuery : public IInterpreter
|
class InterpreterExplainQuery : public IInterpreter
|
||||||
{
|
{
|
||||||
|
@ -80,95 +80,95 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
|
|||||||
{
|
{
|
||||||
ProfileEvents::increment(ProfileEvents::Query);
|
ProfileEvents::increment(ProfileEvents::Query);
|
||||||
|
|
||||||
if (typeid_cast<ASTSelectQuery *>(query.get()))
|
if (query->as<ASTSelectQuery>())
|
||||||
{
|
{
|
||||||
/// This is internal part of ASTSelectWithUnionQuery.
|
/// This is internal part of ASTSelectWithUnionQuery.
|
||||||
/// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child.
|
/// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child.
|
||||||
return std::make_unique<InterpreterSelectQuery>(query, context, Names{}, stage);
|
return std::make_unique<InterpreterSelectQuery>(query, context, Names{}, stage);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTSelectWithUnionQuery *>(query.get()))
|
else if (query->as<ASTSelectWithUnionQuery>())
|
||||||
{
|
{
|
||||||
ProfileEvents::increment(ProfileEvents::SelectQuery);
|
ProfileEvents::increment(ProfileEvents::SelectQuery);
|
||||||
return std::make_unique<InterpreterSelectWithUnionQuery>(query, context, Names{}, stage);
|
return std::make_unique<InterpreterSelectWithUnionQuery>(query, context, Names{}, stage);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTInsertQuery *>(query.get()))
|
else if (query->as<ASTInsertQuery>())
|
||||||
{
|
{
|
||||||
ProfileEvents::increment(ProfileEvents::InsertQuery);
|
ProfileEvents::increment(ProfileEvents::InsertQuery);
|
||||||
/// readonly is checked inside InterpreterInsertQuery
|
/// readonly is checked inside InterpreterInsertQuery
|
||||||
bool allow_materialized = static_cast<bool>(context.getSettingsRef().insert_allow_materialized_columns);
|
bool allow_materialized = static_cast<bool>(context.getSettingsRef().insert_allow_materialized_columns);
|
||||||
return std::make_unique<InterpreterInsertQuery>(query, context, allow_materialized);
|
return std::make_unique<InterpreterInsertQuery>(query, context, allow_materialized);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTCreateQuery *>(query.get()))
|
else if (query->as<ASTCreateQuery>())
|
||||||
{
|
{
|
||||||
/// readonly and allow_ddl are checked inside InterpreterCreateQuery
|
/// readonly and allow_ddl are checked inside InterpreterCreateQuery
|
||||||
return std::make_unique<InterpreterCreateQuery>(query, context);
|
return std::make_unique<InterpreterCreateQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTDropQuery *>(query.get()))
|
else if (query->as<ASTDropQuery>())
|
||||||
{
|
{
|
||||||
/// readonly and allow_ddl are checked inside InterpreterDropQuery
|
/// readonly and allow_ddl are checked inside InterpreterDropQuery
|
||||||
return std::make_unique<InterpreterDropQuery>(query, context);
|
return std::make_unique<InterpreterDropQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTRenameQuery *>(query.get()))
|
else if (query->as<ASTRenameQuery>())
|
||||||
{
|
{
|
||||||
throwIfNoAccess(context);
|
throwIfNoAccess(context);
|
||||||
return std::make_unique<InterpreterRenameQuery>(query, context);
|
return std::make_unique<InterpreterRenameQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTShowTablesQuery *>(query.get()))
|
else if (query->as<ASTShowTablesQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterShowTablesQuery>(query, context);
|
return std::make_unique<InterpreterShowTablesQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTUseQuery *>(query.get()))
|
else if (query->as<ASTUseQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterUseQuery>(query, context);
|
return std::make_unique<InterpreterUseQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTSetQuery *>(query.get()))
|
else if (query->as<ASTSetQuery>())
|
||||||
{
|
{
|
||||||
/// readonly is checked inside InterpreterSetQuery
|
/// readonly is checked inside InterpreterSetQuery
|
||||||
return std::make_unique<InterpreterSetQuery>(query, context);
|
return std::make_unique<InterpreterSetQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTOptimizeQuery *>(query.get()))
|
else if (query->as<ASTOptimizeQuery>())
|
||||||
{
|
{
|
||||||
throwIfNoAccess(context);
|
throwIfNoAccess(context);
|
||||||
return std::make_unique<InterpreterOptimizeQuery>(query, context);
|
return std::make_unique<InterpreterOptimizeQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTExistsQuery *>(query.get()))
|
else if (query->as<ASTExistsQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterExistsQuery>(query, context);
|
return std::make_unique<InterpreterExistsQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTShowCreateTableQuery *>(query.get()))
|
else if (query->as<ASTShowCreateTableQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTShowCreateDatabaseQuery *>(query.get()))
|
else if (query->as<ASTShowCreateDatabaseQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTDescribeQuery *>(query.get()))
|
else if (query->as<ASTDescribeQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterDescribeQuery>(query, context);
|
return std::make_unique<InterpreterDescribeQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTExplainQuery *>(query.get()))
|
else if (query->as<ASTExplainQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterExplainQuery>(query, context);
|
return std::make_unique<InterpreterExplainQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTShowProcesslistQuery *>(query.get()))
|
else if (query->as<ASTShowProcesslistQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterShowProcesslistQuery>(query, context);
|
return std::make_unique<InterpreterShowProcesslistQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTAlterQuery *>(query.get()))
|
else if (query->as<ASTAlterQuery>())
|
||||||
{
|
{
|
||||||
throwIfNoAccess(context);
|
throwIfNoAccess(context);
|
||||||
return std::make_unique<InterpreterAlterQuery>(query, context);
|
return std::make_unique<InterpreterAlterQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTCheckQuery *>(query.get()))
|
else if (query->as<ASTCheckQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterCheckQuery>(query, context);
|
return std::make_unique<InterpreterCheckQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTKillQueryQuery *>(query.get()))
|
else if (query->as<ASTKillQueryQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterKillQueryQuery>(query, context);
|
return std::make_unique<InterpreterKillQueryQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTSystemQuery *>(query.get()))
|
else if (query->as<ASTSystemQuery>())
|
||||||
{
|
{
|
||||||
throwIfNoAccess(context);
|
throwIfNoAccess(context);
|
||||||
return std::make_unique<InterpreterSystemQuery>(query, context);
|
return std::make_unique<InterpreterSystemQuery>(query, context);
|
||||||
|
@ -2,14 +2,13 @@
|
|||||||
|
|
||||||
#include <Core/QueryProcessingStage.h>
|
#include <Core/QueryProcessingStage.h>
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
class InterpreterFactory
|
class InterpreterFactory
|
||||||
|
@ -46,7 +46,7 @@ StoragePtr InterpreterInsertQuery::getTable(const ASTInsertQuery & query)
|
|||||||
{
|
{
|
||||||
if (query.table_function)
|
if (query.table_function)
|
||||||
{
|
{
|
||||||
auto table_function = typeid_cast<const ASTFunction *>(query.table_function.get());
|
const auto * table_function = query.table_function->as<ASTFunction>();
|
||||||
const auto & factory = TableFunctionFactory::instance();
|
const auto & factory = TableFunctionFactory::instance();
|
||||||
return factory.get(table_function->name, context)->execute(query.table_function, context);
|
return factory.get(table_function->name, context)->execute(query.table_function, context);
|
||||||
}
|
}
|
||||||
@ -92,7 +92,7 @@ Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const
|
|||||||
|
|
||||||
BlockIO InterpreterInsertQuery::execute()
|
BlockIO InterpreterInsertQuery::execute()
|
||||||
{
|
{
|
||||||
ASTInsertQuery & query = typeid_cast<ASTInsertQuery &>(*query_ptr);
|
const auto & query = query_ptr->as<ASTInsertQuery &>();
|
||||||
checkAccess(query);
|
checkAccess(query);
|
||||||
StoragePtr table = getTable(query);
|
StoragePtr table = getTable(query);
|
||||||
|
|
||||||
@ -171,7 +171,7 @@ void InterpreterInsertQuery::checkAccess(const ASTInsertQuery & query)
|
|||||||
|
|
||||||
std::pair<String, String> InterpreterInsertQuery::getDatabaseTable() const
|
std::pair<String, String> InterpreterInsertQuery::getDatabaseTable() const
|
||||||
{
|
{
|
||||||
ASTInsertQuery & query = typeid_cast<ASTInsertQuery &>(*query_ptr);
|
const auto & query = query_ptr->as<ASTInsertQuery &>();
|
||||||
return {query.database, query.table};
|
return {query.database, query.table};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -172,7 +172,7 @@ public:
|
|||||||
|
|
||||||
BlockIO InterpreterKillQueryQuery::execute()
|
BlockIO InterpreterKillQueryQuery::execute()
|
||||||
{
|
{
|
||||||
ASTKillQueryQuery & query = typeid_cast<ASTKillQueryQuery &>(*query_ptr);
|
const auto & query = query_ptr->as<ASTKillQueryQuery &>();
|
||||||
|
|
||||||
if (!query.cluster.empty())
|
if (!query.cluster.empty())
|
||||||
return executeDDLQueryOnCluster(query_ptr, context, {"system"});
|
return executeDDLQueryOnCluster(query_ptr, context, {"system"});
|
||||||
@ -261,7 +261,7 @@ BlockIO InterpreterKillQueryQuery::execute()
|
|||||||
Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const String & table)
|
Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const String & table)
|
||||||
{
|
{
|
||||||
String select_query = "SELECT " + columns + " FROM " + table;
|
String select_query = "SELECT " + columns + " FROM " + table;
|
||||||
auto & where_expression = static_cast<ASTKillQueryQuery &>(*query_ptr).where_expression;
|
auto & where_expression = query_ptr->as<ASTKillQueryQuery>()->where_expression;
|
||||||
if (where_expression)
|
if (where_expression)
|
||||||
select_query += " WHERE " + queryToString(where_expression);
|
select_query += " WHERE " + queryToString(where_expression);
|
||||||
|
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
class InterpreterKillQueryQuery : public IInterpreter
|
class InterpreterKillQueryQuery : public IInterpreter
|
||||||
@ -28,4 +27,3 @@ private:
|
|||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -17,7 +17,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
BlockIO InterpreterOptimizeQuery::execute()
|
BlockIO InterpreterOptimizeQuery::execute()
|
||||||
{
|
{
|
||||||
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
|
const auto & ast = query_ptr->as<ASTOptimizeQuery &>();
|
||||||
|
|
||||||
if (!ast.cluster.empty())
|
if (!ast.cluster.empty())
|
||||||
return executeDDLQueryOnCluster(query_ptr, context, {ast.database});
|
return executeDDLQueryOnCluster(query_ptr, context, {ast.database});
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Just call method "optimize" for table.
|
/** Just call method "optimize" for table.
|
||||||
|
@ -36,7 +36,7 @@ struct RenameDescription
|
|||||||
|
|
||||||
BlockIO InterpreterRenameQuery::execute()
|
BlockIO InterpreterRenameQuery::execute()
|
||||||
{
|
{
|
||||||
ASTRenameQuery & rename = typeid_cast<ASTRenameQuery &>(*query_ptr);
|
const auto & rename = query_ptr->as<ASTRenameQuery &>();
|
||||||
|
|
||||||
if (!rename.cluster.empty())
|
if (!rename.cluster.empty())
|
||||||
{
|
{
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Rename one table
|
/** Rename one table
|
||||||
|
@ -23,7 +23,6 @@
|
|||||||
#include <DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h>
|
#include <DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h>
|
||||||
#include <DataStreams/ConvertingBlockInputStream.h>
|
#include <DataStreams/ConvertingBlockInputStream.h>
|
||||||
|
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
|
||||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
@ -169,7 +168,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
}
|
}
|
||||||
|
|
||||||
max_streams = settings.max_threads;
|
max_streams = settings.max_threads;
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
|
|
||||||
ASTPtr table_expression = extractTableExpression(query, 0);
|
ASTPtr table_expression = extractTableExpression(query, 0);
|
||||||
|
|
||||||
@ -177,8 +176,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
bool is_subquery = false;
|
bool is_subquery = false;
|
||||||
if (table_expression)
|
if (table_expression)
|
||||||
{
|
{
|
||||||
is_table_func = typeid_cast<const ASTFunction *>(table_expression.get());
|
is_table_func = table_expression->as<ASTFunction>();
|
||||||
is_subquery = typeid_cast<const ASTSelectWithUnionQuery *>(table_expression.get());
|
is_subquery = table_expression->as<ASTSelectWithUnionQuery>();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (input)
|
if (input)
|
||||||
@ -277,15 +276,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
ASTSelectQuery & InterpreterSelectQuery::selectQuery()
|
|
||||||
{
|
|
||||||
return typeid_cast<ASTSelectQuery &>(*query_ptr);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, String & table_name)
|
void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, String & table_name)
|
||||||
{
|
{
|
||||||
if (auto db_and_table = getDatabaseAndTable(selectQuery(), 0))
|
if (auto db_and_table = getDatabaseAndTable(getSelectQuery(), 0))
|
||||||
{
|
{
|
||||||
table_name = db_and_table->table;
|
table_name = db_and_table->table;
|
||||||
database_name = db_and_table->database;
|
database_name = db_and_table->database;
|
||||||
@ -384,7 +377,7 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
|
|||||||
|
|
||||||
{
|
{
|
||||||
ExpressionActionsChain chain(context);
|
ExpressionActionsChain chain(context);
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
|
|
||||||
Names additional_required_columns_after_prewhere;
|
Names additional_required_columns_after_prewhere;
|
||||||
|
|
||||||
@ -508,7 +501,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
|
|||||||
* then perform the remaining operations with one resulting stream.
|
* then perform the remaining operations with one resulting stream.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
ASTSelectQuery & query = selectQuery();
|
/// Now we will compose block streams that perform the necessary actions.
|
||||||
|
auto & query = getSelectQuery();
|
||||||
const Settings & settings = context.getSettingsRef();
|
const Settings & settings = context.getSettingsRef();
|
||||||
|
|
||||||
QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns;
|
QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns;
|
||||||
@ -570,8 +564,6 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
|
|||||||
|
|
||||||
if (to_stage > QueryProcessingStage::FetchColumns)
|
if (to_stage > QueryProcessingStage::FetchColumns)
|
||||||
{
|
{
|
||||||
/// Now we will compose block streams that perform the necessary actions.
|
|
||||||
|
|
||||||
/// Do I need to aggregate in a separate row rows that have not passed max_rows_to_group_by.
|
/// Do I need to aggregate in a separate row rows that have not passed max_rows_to_group_by.
|
||||||
bool aggregate_overflow_row =
|
bool aggregate_overflow_row =
|
||||||
expressions.need_aggregate &&
|
expressions.need_aggregate &&
|
||||||
@ -590,7 +582,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
|
|||||||
{
|
{
|
||||||
if (expressions.hasJoin())
|
if (expressions.hasJoin())
|
||||||
{
|
{
|
||||||
const ASTTableJoin & join = static_cast<const ASTTableJoin &>(*query.join()->table_join);
|
const auto & join = query.join()->table_join->as<ASTTableJoin &>();
|
||||||
if (isRightOrFull(join.kind))
|
if (isRightOrFull(join.kind))
|
||||||
pipeline.stream_with_non_joined_data = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin(
|
pipeline.stream_with_non_joined_data = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin(
|
||||||
pipeline.firstStream()->getHeader(), settings.max_block_size);
|
pipeline.firstStream()->getHeader(), settings.max_block_size);
|
||||||
@ -786,7 +778,7 @@ static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery &
|
|||||||
return {length, offset};
|
return {length, offset};
|
||||||
}
|
}
|
||||||
|
|
||||||
static UInt64 getLimitForSorting(ASTSelectQuery & query, const Context & context)
|
static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & context)
|
||||||
{
|
{
|
||||||
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY.
|
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY.
|
||||||
if (!query.distinct && !query.limit_by_expression_list)
|
if (!query.distinct && !query.limit_by_expression_list)
|
||||||
@ -802,7 +794,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
|||||||
QueryProcessingStage::Enum processing_stage, Pipeline & pipeline,
|
QueryProcessingStage::Enum processing_stage, Pipeline & pipeline,
|
||||||
const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere)
|
const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere)
|
||||||
{
|
{
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
const Settings & settings = context.getSettingsRef();
|
const Settings & settings = context.getSettingsRef();
|
||||||
|
|
||||||
/// Actions to calculate ALIAS if required.
|
/// Actions to calculate ALIAS if required.
|
||||||
@ -1097,7 +1089,7 @@ void InterpreterSelectQuery::executeWhere(Pipeline & pipeline, const ExpressionA
|
|||||||
{
|
{
|
||||||
pipeline.transform([&](auto & stream)
|
pipeline.transform([&](auto & stream)
|
||||||
{
|
{
|
||||||
stream = std::make_shared<FilterBlockInputStream>(stream, expression, selectQuery().where_expression->getColumnName(), remove_fiter);
|
stream = std::make_shared<FilterBlockInputStream>(stream, expression, getSelectQuery().where_expression->getColumnName(), remove_fiter);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1225,7 +1217,7 @@ void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const Expression
|
|||||||
{
|
{
|
||||||
pipeline.transform([&](auto & stream)
|
pipeline.transform([&](auto & stream)
|
||||||
{
|
{
|
||||||
stream = std::make_shared<FilterBlockInputStream>(stream, expression, selectQuery().having_expression->getColumnName());
|
stream = std::make_shared<FilterBlockInputStream>(stream, expression, getSelectQuery().having_expression->getColumnName());
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1237,8 +1229,13 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha
|
|||||||
const Settings & settings = context.getSettingsRef();
|
const Settings & settings = context.getSettingsRef();
|
||||||
|
|
||||||
pipeline.firstStream() = std::make_shared<TotalsHavingBlockInputStream>(
|
pipeline.firstStream() = std::make_shared<TotalsHavingBlockInputStream>(
|
||||||
pipeline.firstStream(), overflow_row, expression,
|
pipeline.firstStream(),
|
||||||
has_having ? selectQuery().having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold, final);
|
overflow_row,
|
||||||
|
expression,
|
||||||
|
has_having ? getSelectQuery().having_expression->getColumnName() : "",
|
||||||
|
settings.totals_mode,
|
||||||
|
settings.totals_auto_threshold,
|
||||||
|
final);
|
||||||
}
|
}
|
||||||
|
|
||||||
void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator)
|
void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator)
|
||||||
@ -1281,18 +1278,18 @@ void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const Expres
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static SortDescription getSortDescription(ASTSelectQuery & query)
|
static SortDescription getSortDescription(const ASTSelectQuery & query)
|
||||||
{
|
{
|
||||||
SortDescription order_descr;
|
SortDescription order_descr;
|
||||||
order_descr.reserve(query.order_expression_list->children.size());
|
order_descr.reserve(query.order_expression_list->children.size());
|
||||||
for (const auto & elem : query.order_expression_list->children)
|
for (const auto & elem : query.order_expression_list->children)
|
||||||
{
|
{
|
||||||
String name = elem->children.front()->getColumnName();
|
String name = elem->children.front()->getColumnName();
|
||||||
const ASTOrderByElement & order_by_elem = typeid_cast<const ASTOrderByElement &>(*elem);
|
const auto & order_by_elem = elem->as<ASTOrderByElement &>();
|
||||||
|
|
||||||
std::shared_ptr<Collator> collator;
|
std::shared_ptr<Collator> collator;
|
||||||
if (order_by_elem.collation)
|
if (order_by_elem.collation)
|
||||||
collator = std::make_shared<Collator>(typeid_cast<const ASTLiteral &>(*order_by_elem.collation).value.get<String>());
|
collator = std::make_shared<Collator>(order_by_elem.collation->as<ASTLiteral &>().value.get<String>());
|
||||||
|
|
||||||
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator);
|
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator);
|
||||||
}
|
}
|
||||||
@ -1303,7 +1300,7 @@ static SortDescription getSortDescription(ASTSelectQuery & query)
|
|||||||
|
|
||||||
void InterpreterSelectQuery::executeOrder(Pipeline & pipeline)
|
void InterpreterSelectQuery::executeOrder(Pipeline & pipeline)
|
||||||
{
|
{
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
SortDescription order_descr = getSortDescription(query);
|
SortDescription order_descr = getSortDescription(query);
|
||||||
UInt64 limit = getLimitForSorting(query, context);
|
UInt64 limit = getLimitForSorting(query, context);
|
||||||
|
|
||||||
@ -1335,7 +1332,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline)
|
|||||||
|
|
||||||
void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline)
|
void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline)
|
||||||
{
|
{
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
SortDescription order_descr = getSortDescription(query);
|
SortDescription order_descr = getSortDescription(query);
|
||||||
UInt64 limit = getLimitForSorting(query, context);
|
UInt64 limit = getLimitForSorting(query, context);
|
||||||
|
|
||||||
@ -1372,7 +1369,7 @@ void InterpreterSelectQuery::executeProjection(Pipeline & pipeline, const Expres
|
|||||||
|
|
||||||
void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_order, Names columns)
|
void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_order, Names columns)
|
||||||
{
|
{
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
if (query.distinct)
|
if (query.distinct)
|
||||||
{
|
{
|
||||||
const Settings & settings = context.getSettingsRef();
|
const Settings & settings = context.getSettingsRef();
|
||||||
@ -1415,7 +1412,7 @@ void InterpreterSelectQuery::executeUnion(Pipeline & pipeline)
|
|||||||
/// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined.
|
/// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined.
|
||||||
void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline)
|
void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline)
|
||||||
{
|
{
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
/// If there is LIMIT
|
/// If there is LIMIT
|
||||||
if (query.limit_length)
|
if (query.limit_length)
|
||||||
{
|
{
|
||||||
@ -1430,7 +1427,7 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline)
|
|||||||
|
|
||||||
void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline)
|
void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline)
|
||||||
{
|
{
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
if (!query.limit_by_value || !query.limit_by_expression_list)
|
if (!query.limit_by_value || !query.limit_by_expression_list)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
@ -1458,10 +1455,10 @@ bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query)
|
|||||||
|
|
||||||
if (auto query_table = extractTableExpression(query, 0))
|
if (auto query_table = extractTableExpression(query, 0))
|
||||||
{
|
{
|
||||||
if (auto ast_union = typeid_cast<const ASTSelectWithUnionQuery *>(query_table.get()))
|
if (const auto * ast_union = query_table->as<ASTSelectWithUnionQuery>())
|
||||||
{
|
{
|
||||||
for (const auto & elem : ast_union->list_of_selects->children)
|
for (const auto & elem : ast_union->list_of_selects->children)
|
||||||
if (hasWithTotalsInAnySubqueryInFromClause(typeid_cast<const ASTSelectQuery &>(*elem)))
|
if (hasWithTotalsInAnySubqueryInFromClause(elem->as<ASTSelectQuery &>()))
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1472,7 +1469,7 @@ bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query)
|
|||||||
|
|
||||||
void InterpreterSelectQuery::executeLimit(Pipeline & pipeline)
|
void InterpreterSelectQuery::executeLimit(Pipeline & pipeline)
|
||||||
{
|
{
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
/// If there is LIMIT
|
/// If there is LIMIT
|
||||||
if (query.limit_length)
|
if (query.limit_length)
|
||||||
{
|
{
|
||||||
@ -1544,13 +1541,13 @@ void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline)
|
|||||||
|
|
||||||
void InterpreterSelectQuery::ignoreWithTotals()
|
void InterpreterSelectQuery::ignoreWithTotals()
|
||||||
{
|
{
|
||||||
selectQuery().group_by_with_totals = false;
|
getSelectQuery().group_by_with_totals = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void InterpreterSelectQuery::initSettings()
|
void InterpreterSelectQuery::initSettings()
|
||||||
{
|
{
|
||||||
ASTSelectQuery & query = selectQuery();
|
auto & query = getSelectQuery();
|
||||||
if (query.settings)
|
if (query.settings)
|
||||||
InterpreterSetQuery(query.settings, context).executeForCurrentContext();
|
InterpreterSetQuery(query.settings, context).executeForCurrentContext();
|
||||||
}
|
}
|
||||||
|
@ -3,11 +3,12 @@
|
|||||||
#include <memory>
|
#include <memory>
|
||||||
|
|
||||||
#include <Core/QueryProcessingStage.h>
|
#include <Core/QueryProcessingStage.h>
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
|
||||||
#include <Interpreters/ExpressionActions.h>
|
|
||||||
#include <DataStreams/IBlockInputStream.h>
|
#include <DataStreams/IBlockInputStream.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Interpreters/ExpressionActions.h>
|
||||||
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Storages/SelectQueryInfo.h>
|
#include <Storages/SelectQueryInfo.h>
|
||||||
|
|
||||||
|
|
||||||
@ -16,7 +17,6 @@ namespace Poco { class Logger; }
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class ASTSelectQuery;
|
|
||||||
struct SubqueryForSet;
|
struct SubqueryForSet;
|
||||||
class InterpreterSelectWithUnionQuery;
|
class InterpreterSelectWithUnionQuery;
|
||||||
|
|
||||||
@ -99,6 +99,8 @@ private:
|
|||||||
bool only_analyze_,
|
bool only_analyze_,
|
||||||
bool modify_inplace);
|
bool modify_inplace);
|
||||||
|
|
||||||
|
ASTSelectQuery & getSelectQuery() { return query_ptr->as<ASTSelectQuery &>(); }
|
||||||
|
|
||||||
|
|
||||||
struct Pipeline
|
struct Pipeline
|
||||||
{
|
{
|
||||||
@ -133,7 +135,6 @@ private:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
ASTSelectQuery & selectQuery();
|
|
||||||
void executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run);
|
void executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run);
|
||||||
|
|
||||||
|
|
||||||
|
@ -36,7 +36,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
|
|||||||
to_stage(to_stage_),
|
to_stage(to_stage_),
|
||||||
subquery_depth(subquery_depth_)
|
subquery_depth(subquery_depth_)
|
||||||
{
|
{
|
||||||
const ASTSelectWithUnionQuery & ast = typeid_cast<const ASTSelectWithUnionQuery &>(*query_ptr);
|
const auto & ast = query_ptr->as<ASTSelectWithUnionQuery &>();
|
||||||
|
|
||||||
size_t num_selects = ast.list_of_selects->children.size();
|
size_t num_selects = ast.list_of_selects->children.size();
|
||||||
|
|
||||||
|
@ -16,7 +16,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
BlockIO InterpreterSetQuery::execute()
|
BlockIO InterpreterSetQuery::execute()
|
||||||
{
|
{
|
||||||
const ASTSetQuery & ast = typeid_cast<const ASTSetQuery &>(*query_ptr);
|
const auto & ast = query_ptr->as<ASTSetQuery &>();
|
||||||
|
|
||||||
checkAccess(ast);
|
checkAccess(ast);
|
||||||
|
|
||||||
@ -61,7 +61,7 @@ void InterpreterSetQuery::checkAccess(const ASTSetQuery & ast)
|
|||||||
|
|
||||||
void InterpreterSetQuery::executeForCurrentContext()
|
void InterpreterSetQuery::executeForCurrentContext()
|
||||||
{
|
{
|
||||||
const ASTSetQuery & ast = typeid_cast<const ASTSetQuery &>(*query_ptr);
|
const auto & ast = query_ptr->as<ASTSetQuery &>();
|
||||||
|
|
||||||
checkAccess(ast);
|
checkAccess(ast);
|
||||||
|
|
||||||
|
@ -1,15 +1,14 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
class ASTSetQuery;
|
class ASTSetQuery;
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Change one or several settings for the session or just for the current context.
|
/** Change one or several settings for the session or just for the current context.
|
||||||
|
@ -42,6 +42,7 @@ Block InterpreterShowCreateQuery::getSampleBlock()
|
|||||||
|
|
||||||
BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||||
{
|
{
|
||||||
|
/// FIXME: try to prettify this cast using `as<>()`
|
||||||
const auto & ast = dynamic_cast<const ASTQueryWithTableAndOutput &>(*query_ptr);
|
const auto & ast = dynamic_cast<const ASTQueryWithTableAndOutput &>(*query_ptr);
|
||||||
|
|
||||||
if (ast.temporary && !ast.database.empty())
|
if (ast.temporary && !ast.database.empty())
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Return single row with single column "statement" of type String with text of query to CREATE specified table.
|
/** Return single row with single column "statement" of type String with text of query to CREATE specified table.
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Return list of currently executing queries.
|
/** Return list of currently executing queries.
|
||||||
|
@ -25,7 +25,7 @@ InterpreterShowTablesQuery::InterpreterShowTablesQuery(const ASTPtr & query_ptr_
|
|||||||
|
|
||||||
String InterpreterShowTablesQuery::getRewrittenQuery()
|
String InterpreterShowTablesQuery::getRewrittenQuery()
|
||||||
{
|
{
|
||||||
const ASTShowTablesQuery & query = typeid_cast<const ASTShowTablesQuery &>(*query_ptr);
|
const auto & query = query_ptr->as<ASTShowTablesQuery &>();
|
||||||
|
|
||||||
/// SHOW DATABASES
|
/// SHOW DATABASES
|
||||||
if (query.databases)
|
if (query.databases)
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Return a list of tables or databases meets specified conditions.
|
/** Return a list of tables or databases meets specified conditions.
|
||||||
|
@ -117,7 +117,7 @@ InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, Contex
|
|||||||
|
|
||||||
BlockIO InterpreterSystemQuery::execute()
|
BlockIO InterpreterSystemQuery::execute()
|
||||||
{
|
{
|
||||||
auto & query = typeid_cast<ASTSystemQuery &>(*query_ptr);
|
auto & query = query_ptr->as<ASTSystemQuery &>();
|
||||||
|
|
||||||
using Type = ASTSystemQuery::Type;
|
using Type = ASTSystemQuery::Type;
|
||||||
|
|
||||||
@ -248,7 +248,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam
|
|||||||
/// Attach actions
|
/// Attach actions
|
||||||
{
|
{
|
||||||
/// getCreateTableQuery must return canonical CREATE query representation, there are no need for AST postprocessing
|
/// getCreateTableQuery must return canonical CREATE query representation, there are no need for AST postprocessing
|
||||||
auto & create = typeid_cast<ASTCreateQuery &>(*create_ast);
|
auto & create = create_ast->as<ASTCreateQuery &>();
|
||||||
create.attach = true;
|
create.attach = true;
|
||||||
|
|
||||||
std::string data_path = database->getDataPath();
|
std::string data_path = database->getDataPath();
|
||||||
|
@ -1,15 +1,15 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
class ASTSystemQuery;
|
class ASTSystemQuery;
|
||||||
class IStorage;
|
class IStorage;
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
using StoragePtr = std::shared_ptr<IStorage>;
|
using StoragePtr = std::shared_ptr<IStorage>;
|
||||||
|
|
||||||
|
|
||||||
|
@ -9,7 +9,7 @@ namespace DB
|
|||||||
|
|
||||||
BlockIO InterpreterUseQuery::execute()
|
BlockIO InterpreterUseQuery::execute()
|
||||||
{
|
{
|
||||||
const String & new_database = typeid_cast<const ASTUseQuery &>(*query_ptr).database;
|
const String & new_database = query_ptr->as<ASTUseQuery &>().database;
|
||||||
context.getSessionContext().setCurrentDatabase(new_database);
|
context.getSessionContext().setCurrentDatabase(new_database);
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
@ -1,14 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
|
|
||||||
/** Change default database for session.
|
/** Change default database for session.
|
||||||
|
@ -90,18 +90,17 @@ struct ColumnAliasesMatcher
|
|||||||
|
|
||||||
static bool needChildVisit(ASTPtr & node, const ASTPtr &)
|
static bool needChildVisit(ASTPtr & node, const ASTPtr &)
|
||||||
{
|
{
|
||||||
if (typeid_cast<const ASTQualifiedAsterisk *>(node.get()))
|
if (node->as<ASTQualifiedAsterisk>())
|
||||||
return false;
|
return false;
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
static void visit(ASTPtr & ast, Data & data)
|
static void visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * t = typeid_cast<ASTIdentifier *>(ast.get()))
|
if (auto * t = ast->as<ASTIdentifier>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
|
|
||||||
if (typeid_cast<ASTAsterisk *>(ast.get()) ||
|
if (ast->as<ASTAsterisk>() || ast->as<ASTQualifiedAsterisk>())
|
||||||
typeid_cast<ASTQualifiedAsterisk *>(ast.get()))
|
|
||||||
throw Exception("Multiple JOIN do not support asterisks yet", ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Multiple JOIN do not support asterisks yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -160,9 +159,9 @@ struct AppendSemanticVisitorData
|
|||||||
|
|
||||||
for (auto & child : select.select_expression_list->children)
|
for (auto & child : select.select_expression_list->children)
|
||||||
{
|
{
|
||||||
if (auto * node = typeid_cast<ASTAsterisk *>(child.get()))
|
if (auto * node = child->as<ASTAsterisk>())
|
||||||
AsteriskSemantic::setAliases(*node, rev_aliases);
|
AsteriskSemantic::setAliases(*node, rev_aliases);
|
||||||
if (auto * node = typeid_cast<ASTQualifiedAsterisk *>(child.get()))
|
if (auto * node = child->as<ASTQualifiedAsterisk>())
|
||||||
AsteriskSemantic::setAliases(*node, rev_aliases);
|
AsteriskSemantic::setAliases(*node, rev_aliases);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -196,7 +195,7 @@ bool needRewrite(ASTSelectQuery & select)
|
|||||||
if (!select.tables)
|
if (!select.tables)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
auto tables = typeid_cast<const ASTTablesInSelectQuery *>(select.tables.get());
|
const auto * tables = select.tables->as<ASTTablesInSelectQuery>();
|
||||||
if (!tables)
|
if (!tables)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
@ -206,11 +205,11 @@ bool needRewrite(ASTSelectQuery & select)
|
|||||||
|
|
||||||
for (size_t i = 1; i < tables->children.size(); ++i)
|
for (size_t i = 1; i < tables->children.size(); ++i)
|
||||||
{
|
{
|
||||||
auto table = typeid_cast<const ASTTablesInSelectQueryElement *>(tables->children[i].get());
|
const auto * table = tables->children[i]->as<ASTTablesInSelectQueryElement>();
|
||||||
if (!table || !table->table_join)
|
if (!table || !table->table_join)
|
||||||
throw Exception("Multiple JOIN expects joined tables", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Multiple JOIN expects joined tables", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
auto join = typeid_cast<const ASTTableJoin &>(*table->table_join);
|
const auto & join = table->table_join->as<ASTTableJoin &>();
|
||||||
if (isComma(join.kind))
|
if (isComma(join.kind))
|
||||||
throw Exception("COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query", ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
|
||||||
@ -233,7 +232,7 @@ using AppendSemanticVisitor = InDepthNodeVisitor<AppendSemanticMatcher, true>;
|
|||||||
|
|
||||||
void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
|
void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * t = typeid_cast<ASTSelectQuery *>(ast.get()))
|
if (auto * t = ast->as<ASTSelectQuery>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -261,10 +260,10 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Da
|
|||||||
/// JOIN sections
|
/// JOIN sections
|
||||||
for (auto & child : select.tables->children)
|
for (auto & child : select.tables->children)
|
||||||
{
|
{
|
||||||
auto table = typeid_cast<ASTTablesInSelectQueryElement *>(child.get());
|
auto * table = child->as<ASTTablesInSelectQueryElement>();
|
||||||
if (table->table_join)
|
if (table->table_join)
|
||||||
{
|
{
|
||||||
auto & join = typeid_cast<ASTTableJoin &>(*table->table_join);
|
auto & join = table->table_join->as<ASTTableJoin &>();
|
||||||
if (join.on_expression)
|
if (join.on_expression)
|
||||||
ColumnAliasesVisitor(aliases_data).visit(join.on_expression);
|
ColumnAliasesVisitor(aliases_data).visit(join.on_expression);
|
||||||
}
|
}
|
||||||
@ -307,8 +306,8 @@ static ASTPtr makeSubqueryTemplate()
|
|||||||
|
|
||||||
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right)
|
ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right)
|
||||||
{
|
{
|
||||||
auto left = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_left.get());
|
const auto * left = ast_left->as<ASTTablesInSelectQueryElement>();
|
||||||
auto right = typeid_cast<const ASTTablesInSelectQueryElement *>(ast_right.get());
|
const auto * right = ast_right->as<ASTTablesInSelectQueryElement>();
|
||||||
if (!left || !right)
|
if (!left || !right)
|
||||||
throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
@ -19,7 +19,7 @@ namespace ErrorCodes
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
LogicalExpressionsOptimizer::OrWithExpression::OrWithExpression(ASTFunction * or_function_,
|
LogicalExpressionsOptimizer::OrWithExpression::OrWithExpression(const ASTFunction * or_function_,
|
||||||
const IAST::Hash & expression_, const std::string & alias_)
|
const IAST::Hash & expression_, const std::string & alias_)
|
||||||
: or_function(or_function_), expression(expression_), alias(alias_)
|
: or_function(or_function_), expression(expression_), alias(alias_)
|
||||||
{
|
{
|
||||||
@ -111,24 +111,24 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
|||||||
|
|
||||||
bool found_chain = false;
|
bool found_chain = false;
|
||||||
|
|
||||||
auto function = typeid_cast<ASTFunction *>(to_node);
|
auto * function = to_node->as<ASTFunction>();
|
||||||
if ((function != nullptr) && (function->name == "or") && (function->children.size() == 1))
|
if (function && function->name == "or" && function->children.size() == 1)
|
||||||
{
|
{
|
||||||
auto expression_list = typeid_cast<ASTExpressionList *>(&*(function->children[0]));
|
const auto * expression_list = function->children[0]->as<ASTExpressionList>();
|
||||||
if (expression_list != nullptr)
|
if (expression_list)
|
||||||
{
|
{
|
||||||
/// The chain of elements of the OR expression.
|
/// The chain of elements of the OR expression.
|
||||||
for (auto & child : expression_list->children)
|
for (auto & child : expression_list->children)
|
||||||
{
|
{
|
||||||
auto equals = typeid_cast<ASTFunction *>(&*child);
|
auto * equals = child->as<ASTFunction>();
|
||||||
if ((equals != nullptr) && (equals->name == "equals") && (equals->children.size() == 1))
|
if (equals && equals->name == "equals" && equals->children.size() == 1)
|
||||||
{
|
{
|
||||||
auto equals_expression_list = typeid_cast<ASTExpressionList *>(&*(equals->children[0]));
|
const auto * equals_expression_list = equals->children[0]->as<ASTExpressionList>();
|
||||||
if ((equals_expression_list != nullptr) && (equals_expression_list->children.size() == 2))
|
if (equals_expression_list && equals_expression_list->children.size() == 2)
|
||||||
{
|
{
|
||||||
/// Equality expr = xN.
|
/// Equality expr = xN.
|
||||||
auto literal = typeid_cast<ASTLiteral *>(&*(equals_expression_list->children[1]));
|
const auto * literal = equals_expression_list->children[1]->as<ASTLiteral>();
|
||||||
if (literal != nullptr)
|
if (literal)
|
||||||
{
|
{
|
||||||
auto expr_lhs = equals_expression_list->children[0]->getTreeHash();
|
auto expr_lhs = equals_expression_list->children[0]->getTreeHash();
|
||||||
OrWithExpression or_with_expression{function, expr_lhs, function->tryGetAlias()};
|
OrWithExpression or_with_expression{function, expr_lhs, function->tryGetAlias()};
|
||||||
@ -157,7 +157,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
|||||||
{
|
{
|
||||||
for (auto & child : to_node->children)
|
for (auto & child : to_node->children)
|
||||||
{
|
{
|
||||||
if (typeid_cast<ASTSelectQuery *>(child.get()) == nullptr)
|
if (!child->as<ASTSelectQuery>())
|
||||||
{
|
{
|
||||||
if (!visited_nodes.count(child.get()))
|
if (!visited_nodes.count(child.get()))
|
||||||
to_visit.push_back(Edge(to_node, &*child));
|
to_visit.push_back(Edge(to_node, &*child));
|
||||||
@ -187,10 +187,9 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
inline ASTs & getFunctionOperands(ASTFunction * or_function)
|
inline ASTs & getFunctionOperands(const ASTFunction * or_function)
|
||||||
{
|
{
|
||||||
auto expression_list = static_cast<ASTExpressionList *>(&*(or_function->children[0]));
|
return or_function->children[0]->children;
|
||||||
return expression_list->children;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
@ -206,11 +205,11 @@ bool LogicalExpressionsOptimizer::mayOptimizeDisjunctiveEqualityChain(const Disj
|
|||||||
|
|
||||||
/// We check that the right-hand sides of all equalities have the same type.
|
/// We check that the right-hand sides of all equalities have the same type.
|
||||||
auto & first_operands = getFunctionOperands(equality_functions[0]);
|
auto & first_operands = getFunctionOperands(equality_functions[0]);
|
||||||
auto first_literal = static_cast<ASTLiteral *>(&*first_operands[1]);
|
const auto * first_literal = first_operands[1]->as<ASTLiteral>();
|
||||||
for (size_t i = 1; i < equality_functions.size(); ++i)
|
for (size_t i = 1; i < equality_functions.size(); ++i)
|
||||||
{
|
{
|
||||||
auto & operands = getFunctionOperands(equality_functions[i]);
|
auto & operands = getFunctionOperands(equality_functions[i]);
|
||||||
auto literal = static_cast<ASTLiteral *>(&*operands[1]);
|
const auto * literal = operands[1]->as<ASTLiteral>();
|
||||||
|
|
||||||
if (literal->value.getType() != first_literal->value.getType())
|
if (literal->value.getType() != first_literal->value.getType())
|
||||||
return false;
|
return false;
|
||||||
@ -238,8 +237,8 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
|
|||||||
/// Otherwise, they would be specified in the order of the ASTLiteral addresses, which is nondeterministic.
|
/// Otherwise, they would be specified in the order of the ASTLiteral addresses, which is nondeterministic.
|
||||||
std::sort(value_list->children.begin(), value_list->children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)
|
std::sort(value_list->children.begin(), value_list->children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)
|
||||||
{
|
{
|
||||||
const auto val_lhs = static_cast<const ASTLiteral *>(&*lhs);
|
const auto * val_lhs = lhs->as<ASTLiteral>();
|
||||||
const auto val_rhs = static_cast<const ASTLiteral *>(&*rhs);
|
const auto * val_rhs = rhs->as<ASTLiteral>();
|
||||||
return val_lhs->value < val_rhs->value;
|
return val_lhs->value < val_rhs->value;
|
||||||
});
|
});
|
||||||
|
|
||||||
@ -277,7 +276,7 @@ void LogicalExpressionsOptimizer::cleanupOrExpressions()
|
|||||||
{
|
{
|
||||||
/// Saves for each optimized OR-chain the iterator on the first element
|
/// Saves for each optimized OR-chain the iterator on the first element
|
||||||
/// list of operands to be deleted.
|
/// list of operands to be deleted.
|
||||||
std::unordered_map<ASTFunction *, ASTs::iterator> garbage_map;
|
std::unordered_map<const ASTFunction *, ASTs::iterator> garbage_map;
|
||||||
|
|
||||||
/// Initialization.
|
/// Initialization.
|
||||||
garbage_map.reserve(processed_count);
|
garbage_map.reserve(processed_count);
|
||||||
|
@ -51,11 +51,10 @@ private:
|
|||||||
*/
|
*/
|
||||||
struct OrWithExpression
|
struct OrWithExpression
|
||||||
{
|
{
|
||||||
OrWithExpression(ASTFunction * or_function_, const IAST::Hash & expression_,
|
OrWithExpression(const ASTFunction * or_function_, const IAST::Hash & expression_, const std::string & alias_);
|
||||||
const std::string & alias_);
|
|
||||||
bool operator<(const OrWithExpression & rhs) const;
|
bool operator<(const OrWithExpression & rhs) const;
|
||||||
|
|
||||||
ASTFunction * or_function;
|
const ASTFunction * or_function;
|
||||||
const IAST::Hash expression;
|
const IAST::Hash expression;
|
||||||
const std::string alias;
|
const std::string alias;
|
||||||
};
|
};
|
||||||
@ -95,8 +94,8 @@ private:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
using ParentNodes = std::vector<IAST *>;
|
using ParentNodes = std::vector<IAST *>;
|
||||||
using FunctionParentMap = std::unordered_map<IAST *, ParentNodes>;
|
using FunctionParentMap = std::unordered_map<const IAST *, ParentNodes>;
|
||||||
using ColumnToPosition = std::unordered_map<IAST *, size_t>;
|
using ColumnToPosition = std::unordered_map<const IAST *, size_t>;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
ASTSelectQuery * select_query;
|
ASTSelectQuery * select_query;
|
||||||
|
@ -16,7 +16,7 @@ namespace ErrorCodes
|
|||||||
static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value)
|
static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value)
|
||||||
{
|
{
|
||||||
/// numeric constant in condition
|
/// numeric constant in condition
|
||||||
if (const ASTLiteral * literal = typeid_cast<ASTLiteral *>(condition.get()))
|
if (const auto * literal = condition->as<ASTLiteral>())
|
||||||
{
|
{
|
||||||
if (literal->value.getType() == Field::Types::Int64 ||
|
if (literal->value.getType() == Field::Types::Int64 ||
|
||||||
literal->value.getType() == Field::Types::UInt64)
|
literal->value.getType() == Field::Types::UInt64)
|
||||||
@ -27,14 +27,14 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// cast of numeric constant in condition to UInt8
|
/// cast of numeric constant in condition to UInt8
|
||||||
if (const ASTFunction * function = typeid_cast<ASTFunction * >(condition.get()))
|
if (const auto * function = condition->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
if (function->name == "CAST")
|
if (function->name == "CAST")
|
||||||
{
|
{
|
||||||
if (ASTExpressionList * expr_list = typeid_cast<ASTExpressionList *>(function->arguments.get()))
|
if (const auto * expr_list = function->arguments->as<ASTExpressionList>())
|
||||||
{
|
{
|
||||||
const ASTPtr & type_ast = expr_list->children.at(1);
|
const ASTPtr & type_ast = expr_list->children.at(1);
|
||||||
if (const ASTLiteral * type_literal = typeid_cast<ASTLiteral *>(type_ast.get()))
|
if (const auto * type_literal = type_ast->as<ASTLiteral>())
|
||||||
{
|
{
|
||||||
if (type_literal->value.getType() == Field::Types::String &&
|
if (type_literal->value.getType() == Field::Types::String &&
|
||||||
type_literal->value.get<std::string>() == "UInt8")
|
type_literal->value.get<std::string>() == "UInt8")
|
||||||
@ -54,7 +54,7 @@ void OptimizeIfWithConstantConditionVisitor::visit(ASTPtr & current_ast)
|
|||||||
|
|
||||||
for (ASTPtr & child : current_ast->children)
|
for (ASTPtr & child : current_ast->children)
|
||||||
{
|
{
|
||||||
auto * function_node = typeid_cast<ASTFunction *>(child.get());
|
auto * function_node = child->as<ASTFunction>();
|
||||||
if (!function_node || function_node->name != "if")
|
if (!function_node || function_node->name != "if")
|
||||||
{
|
{
|
||||||
visit(child);
|
visit(child);
|
||||||
@ -62,7 +62,7 @@ void OptimizeIfWithConstantConditionVisitor::visit(ASTPtr & current_ast)
|
|||||||
}
|
}
|
||||||
|
|
||||||
visit(function_node->arguments);
|
visit(function_node->arguments);
|
||||||
auto * args = typeid_cast<ASTExpressionList *>(function_node->arguments.get());
|
const auto * args = function_node->arguments->as<ASTExpressionList>();
|
||||||
|
|
||||||
if (args->children.size() != 3)
|
if (args->children.size() != 3)
|
||||||
throw Exception("Wrong number of arguments for function 'if' (" + toString(args->children.size()) + " instead of 3)",
|
throw Exception("Wrong number of arguments for function 'if' (" + toString(args->children.size()) + " instead of 3)",
|
||||||
|
@ -151,7 +151,7 @@ std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(AST
|
|||||||
{
|
{
|
||||||
const auto expression = predicate_expressions.at(idx);
|
const auto expression = predicate_expressions.at(idx);
|
||||||
|
|
||||||
if (const auto function = typeid_cast<ASTFunction *>(expression.get()))
|
if (const auto * function = expression->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
if (function->name == and_function_name)
|
if (function->name == and_function_name)
|
||||||
{
|
{
|
||||||
@ -239,7 +239,7 @@ void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate(
|
|||||||
if (alias == qualified_name)
|
if (alias == qualified_name)
|
||||||
{
|
{
|
||||||
String name;
|
String name;
|
||||||
if (auto * id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
if (auto * id = ast->as<ASTIdentifier>())
|
||||||
{
|
{
|
||||||
name = id->tryGetAlias();
|
name = id->tryGetAlias();
|
||||||
if (name.empty())
|
if (name.empty())
|
||||||
@ -260,7 +260,7 @@ void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate(
|
|||||||
|
|
||||||
bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node)
|
bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node)
|
||||||
{
|
{
|
||||||
if (auto function = typeid_cast<ASTFunction *>(node.get()))
|
if (const auto * function = node->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
if (function->name == "arrayJoin")
|
if (function->name == "arrayJoin")
|
||||||
return true;
|
return true;
|
||||||
@ -309,7 +309,7 @@ void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr &
|
|||||||
const ASTPtr & subselect = subquery->children[0];
|
const ASTPtr & subselect = subquery->children[0];
|
||||||
|
|
||||||
ASTs select_with_union_projections;
|
ASTs select_with_union_projections;
|
||||||
auto select_with_union_query = static_cast<ASTSelectWithUnionQuery *>(subselect.get());
|
const auto * select_with_union_query = subselect->as<ASTSelectWithUnionQuery>();
|
||||||
|
|
||||||
for (auto & select : select_with_union_query->list_of_selects->children)
|
for (auto & select : select_with_union_query->list_of_selects->children)
|
||||||
{
|
{
|
||||||
@ -325,7 +325,7 @@ void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr &
|
|||||||
subquery_projections.emplace_back(std::pair(select_projection_columns[i],
|
subquery_projections.emplace_back(std::pair(select_projection_columns[i],
|
||||||
qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName()));
|
qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName()));
|
||||||
|
|
||||||
projection_columns.insert(std::pair(static_cast<ASTSelectQuery *>(select.get()), subquery_projections));
|
projection_columns.insert(std::pair(select->as<ASTSelectQuery>(), subquery_projections));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -333,7 +333,7 @@ void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr &
|
|||||||
ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast)
|
ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast)
|
||||||
{
|
{
|
||||||
ASTs projection_columns;
|
ASTs projection_columns;
|
||||||
auto select_query = static_cast<ASTSelectQuery *>(ast.get());
|
auto * select_query = ast->as<ASTSelectQuery>();
|
||||||
|
|
||||||
/// first should normalize query tree.
|
/// first should normalize query tree.
|
||||||
std::unordered_map<String, ASTPtr> aliases;
|
std::unordered_map<String, ASTPtr> aliases;
|
||||||
@ -352,7 +352,7 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
|
|||||||
|
|
||||||
for (const auto & projection_column : select_query->select_expression_list->children)
|
for (const auto & projection_column : select_query->select_expression_list->children)
|
||||||
{
|
{
|
||||||
if (typeid_cast<ASTAsterisk *>(projection_column.get()) || typeid_cast<ASTQualifiedAsterisk *>(projection_column.get()))
|
if (projection_column->as<ASTAsterisk>() || projection_column->as<ASTQualifiedAsterisk>())
|
||||||
{
|
{
|
||||||
ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column);
|
ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column);
|
||||||
|
|
||||||
@ -375,7 +375,7 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
|
|||||||
|
|
||||||
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query);
|
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query);
|
||||||
|
|
||||||
if (const auto qualified_asterisk = typeid_cast<ASTQualifiedAsterisk *>(asterisk.get()))
|
if (const auto * qualified_asterisk = asterisk->as<ASTQualifiedAsterisk>())
|
||||||
{
|
{
|
||||||
if (qualified_asterisk->children.size() != 1)
|
if (qualified_asterisk->children.size() != 1)
|
||||||
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
|
||||||
@ -399,8 +399,8 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
|
|||||||
{
|
{
|
||||||
if (table_expression->subquery)
|
if (table_expression->subquery)
|
||||||
{
|
{
|
||||||
const auto subquery = static_cast<const ASTSubquery *>(table_expression->subquery.get());
|
const auto * subquery = table_expression->subquery->as<ASTSubquery>();
|
||||||
const auto select_with_union_query = static_cast<ASTSelectWithUnionQuery *>(subquery->children[0].get());
|
const auto * select_with_union_query = subquery->children[0]->as<ASTSelectWithUnionQuery>();
|
||||||
const auto subquery_projections = getSelectQueryProjectionColumns(select_with_union_query->list_of_selects->children[0]);
|
const auto subquery_projections = getSelectQueryProjectionColumns(select_with_union_query->list_of_selects->children[0]);
|
||||||
projection_columns.insert(projection_columns.end(), subquery_projections.begin(), subquery_projections.end());
|
projection_columns.insert(projection_columns.end(), subquery_projections.begin(), subquery_projections.end());
|
||||||
}
|
}
|
||||||
@ -415,7 +415,7 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
|
|||||||
}
|
}
|
||||||
else if (table_expression->database_and_table_name)
|
else if (table_expression->database_and_table_name)
|
||||||
{
|
{
|
||||||
const auto database_and_table_ast = static_cast<ASTIdentifier*>(table_expression->database_and_table_name.get());
|
const auto * database_and_table_ast = table_expression->database_and_table_name->as<ASTIdentifier>();
|
||||||
DatabaseAndTableWithAlias database_and_table_name(*database_and_table_ast);
|
DatabaseAndTableWithAlias database_and_table_name(*database_and_table_ast);
|
||||||
storage = context.getTable(database_and_table_name.database, database_and_table_name.table);
|
storage = context.getTable(database_and_table_name.database, database_and_table_name.table);
|
||||||
}
|
}
|
||||||
|
@ -38,7 +38,7 @@ static bool isUnlimitedQuery(const IAST * ast)
|
|||||||
return false;
|
return false;
|
||||||
|
|
||||||
/// It is KILL QUERY
|
/// It is KILL QUERY
|
||||||
if (typeid_cast<const ASTKillQueryQuery *>(ast))
|
if (ast->as<ASTKillQueryQuery>())
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
/// It is SELECT FROM system.processes
|
/// It is SELECT FROM system.processes
|
||||||
@ -46,12 +46,12 @@ static bool isUnlimitedQuery(const IAST * ast)
|
|||||||
/// False negative: USE system; SELECT * FROM processes;
|
/// False negative: USE system; SELECT * FROM processes;
|
||||||
/// False positive: SELECT * FROM system.processes CROSS JOIN (SELECT ...)
|
/// False positive: SELECT * FROM system.processes CROSS JOIN (SELECT ...)
|
||||||
|
|
||||||
if (auto ast_selects = typeid_cast<const ASTSelectWithUnionQuery *>(ast))
|
if (const auto * ast_selects = ast->as<ASTSelectWithUnionQuery>())
|
||||||
{
|
{
|
||||||
if (!ast_selects->list_of_selects || ast_selects->list_of_selects->children.empty())
|
if (!ast_selects->list_of_selects || ast_selects->list_of_selects->children.empty())
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
auto ast_select = typeid_cast<const ASTSelectQuery *>(ast_selects->list_of_selects->children[0].get());
|
const auto * ast_select = ast_selects->list_of_selects->children[0]->as<ASTSelectQuery>();
|
||||||
if (!ast_select)
|
if (!ast_select)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
|
@ -32,18 +32,16 @@ static String wrongAliasMessage(const ASTPtr & ast, const ASTPtr & prev_ast, con
|
|||||||
bool QueryAliasesMatcher::needChildVisit(ASTPtr & node, const ASTPtr &)
|
bool QueryAliasesMatcher::needChildVisit(ASTPtr & node, const ASTPtr &)
|
||||||
{
|
{
|
||||||
/// Don't descent into table functions and subqueries and special case for ArrayJoin.
|
/// Don't descent into table functions and subqueries and special case for ArrayJoin.
|
||||||
if (typeid_cast<ASTTableExpression *>(node.get()) ||
|
if (node->as<ASTTableExpression>() || node->as<ASTSelectWithUnionQuery>() || node->as<ASTArrayJoin>())
|
||||||
typeid_cast<ASTSelectWithUnionQuery *>(node.get()) ||
|
|
||||||
typeid_cast<ASTArrayJoin *>(node.get()))
|
|
||||||
return false;
|
return false;
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
void QueryAliasesMatcher::visit(ASTPtr & ast, Data & data)
|
void QueryAliasesMatcher::visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * s = typeid_cast<ASTSubquery *>(ast.get()))
|
if (auto * s = ast->as<ASTSubquery>())
|
||||||
visit(*s, ast, data);
|
visit(*s, ast, data);
|
||||||
else if (auto * aj = typeid_cast<ASTArrayJoin *>(ast.get()))
|
else if (auto * aj = ast->as<ASTArrayJoin>())
|
||||||
visit(*aj, ast, data);
|
visit(*aj, ast, data);
|
||||||
else
|
else
|
||||||
visitOther(ast, data);
|
visitOther(ast, data);
|
||||||
|
@ -134,14 +134,14 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &
|
|||||||
/// mark table Identifiers as 'not a column'
|
/// mark table Identifiers as 'not a column'
|
||||||
if (node.table_expression)
|
if (node.table_expression)
|
||||||
{
|
{
|
||||||
auto & expr = static_cast<ASTTableExpression &>(*node.table_expression);
|
auto & expr = node.table_expression->as<ASTTableExpression &>();
|
||||||
setIdentifierSpecial(expr.database_and_table_name);
|
setIdentifierSpecial(expr.database_and_table_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// normalize JOIN ON section
|
/// normalize JOIN ON section
|
||||||
if (node.table_join)
|
if (node.table_join)
|
||||||
{
|
{
|
||||||
auto & join = static_cast<ASTTableJoin &>(*node.table_join);
|
auto & join = node.table_join->as<ASTTableJoin &>();
|
||||||
if (join.on_expression)
|
if (join.on_expression)
|
||||||
visit(join.on_expression, data);
|
visit(join.on_expression, data);
|
||||||
}
|
}
|
||||||
@ -149,8 +149,7 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &
|
|||||||
|
|
||||||
static bool needVisitChild(const ASTPtr & child)
|
static bool needVisitChild(const ASTPtr & child)
|
||||||
{
|
{
|
||||||
if (typeid_cast<const ASTSelectQuery *>(child.get()) ||
|
if (child->as<ASTSelectQuery>() || child->as<ASTTableExpression>())
|
||||||
typeid_cast<const ASTTableExpression *>(child.get()))
|
|
||||||
return false;
|
return false;
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -178,7 +177,7 @@ void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr & ast, Data &
|
|||||||
/// on aliases in expressions of the form 123 AS x, arrayMap(x -> 1, [2]).
|
/// on aliases in expressions of the form 123 AS x, arrayMap(x -> 1, [2]).
|
||||||
void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data)
|
void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data)
|
||||||
{
|
{
|
||||||
if (ASTFunction * func_node = typeid_cast<ASTFunction *>(node.get()))
|
if (const auto * func_node = node->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
/// We skip the first argument. We also assume that the lambda function can not have parameters.
|
/// We skip the first argument. We also assume that the lambda function can not have parameters.
|
||||||
size_t first_pos = 0;
|
size_t first_pos = 0;
|
||||||
@ -195,7 +194,7 @@ void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data)
|
|||||||
visit(child, data);
|
visit(child, data);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (!typeid_cast<ASTSelectQuery *>(node.get()))
|
else if (!node->as<ASTSelectQuery>())
|
||||||
{
|
{
|
||||||
for (auto & child : node->children)
|
for (auto & child : node->children)
|
||||||
if (needVisitChild(child))
|
if (needVisitChild(child))
|
||||||
@ -226,13 +225,13 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data)
|
|||||||
data.current_alias = my_alias;
|
data.current_alias = my_alias;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (auto * node = typeid_cast<ASTFunction *>(ast.get()))
|
if (auto * node = ast->as<ASTFunction>())
|
||||||
visit(*node, ast, data);
|
visit(*node, ast, data);
|
||||||
if (auto * node = typeid_cast<ASTIdentifier *>(ast.get()))
|
if (auto * node = ast->as<ASTIdentifier>())
|
||||||
visit(*node, ast, data);
|
visit(*node, ast, data);
|
||||||
if (auto * node = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
|
if (auto * node = ast->as<ASTTablesInSelectQueryElement>())
|
||||||
visit(*node, ast, data);
|
visit(*node, ast, data);
|
||||||
if (auto * node = typeid_cast<ASTSelectQuery *>(ast.get()))
|
if (auto * node = ast->as<ASTSelectQuery>())
|
||||||
visit(*node, ast, data);
|
visit(*node, ast, data);
|
||||||
|
|
||||||
/// If we replace the root of the subtree, we will be called again for the new root, in case the alias is replaced by an alias.
|
/// If we replace the root of the subtree, we will be called again for the new root, in case the alias is replaced by an alias.
|
||||||
|
@ -22,7 +22,7 @@ static std::vector<String> extractNamesFromLambda(const ASTFunction & node)
|
|||||||
if (node.arguments->children.size() != 2)
|
if (node.arguments->children.size() != 2)
|
||||||
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(node.arguments->children[0].get());
|
const auto * lambda_args_tuple = node.arguments->children[0]->as<ASTFunction>();
|
||||||
|
|
||||||
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
||||||
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
||||||
@ -30,7 +30,7 @@ static std::vector<String> extractNamesFromLambda(const ASTFunction & node)
|
|||||||
std::vector<String> names;
|
std::vector<String> names;
|
||||||
for (auto & child : lambda_args_tuple->arguments->children)
|
for (auto & child : lambda_args_tuple->arguments->children)
|
||||||
{
|
{
|
||||||
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(child.get());
|
const auto * identifier = child->as<ASTIdentifier>();
|
||||||
if (!identifier)
|
if (!identifier)
|
||||||
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
||||||
|
|
||||||
@ -42,16 +42,14 @@ static std::vector<String> extractNamesFromLambda(const ASTFunction & node)
|
|||||||
|
|
||||||
bool RequiredSourceColumnsMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
|
bool RequiredSourceColumnsMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
|
||||||
{
|
{
|
||||||
if (typeid_cast<ASTSelectQuery *>(child.get()))
|
if (child->as<ASTSelectQuery>())
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
/// Processed. Do not need children.
|
/// Processed. Do not need children.
|
||||||
if (typeid_cast<ASTTableExpression *>(node.get()) ||
|
if (node->as<ASTTableExpression>() || node->as<ASTArrayJoin>() || node->as<ASTSelectQuery>())
|
||||||
typeid_cast<ASTArrayJoin *>(node.get()) ||
|
|
||||||
typeid_cast<ASTSelectQuery *>(node.get()))
|
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (auto * f = typeid_cast<ASTFunction *>(node.get()))
|
if (const auto * f = node->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
/// "indexHint" is a special function for index analysis. Everything that is inside it is not calculated. @sa KeyCondition
|
/// "indexHint" is a special function for index analysis. Everything that is inside it is not calculated. @sa KeyCondition
|
||||||
/// "lambda" visit children itself.
|
/// "lambda" visit children itself.
|
||||||
@ -66,12 +64,12 @@ void RequiredSourceColumnsMatcher::visit(ASTPtr & ast, Data & data)
|
|||||||
{
|
{
|
||||||
/// results are columns
|
/// results are columns
|
||||||
|
|
||||||
if (auto * t = typeid_cast<ASTIdentifier *>(ast.get()))
|
if (auto * t = ast->as<ASTIdentifier>())
|
||||||
{
|
{
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (auto * t = typeid_cast<ASTFunction *>(ast.get()))
|
if (auto * t = ast->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
data.addColumnAliasIfAny(*ast);
|
data.addColumnAliasIfAny(*ast);
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
@ -80,24 +78,24 @@ void RequiredSourceColumnsMatcher::visit(ASTPtr & ast, Data & data)
|
|||||||
|
|
||||||
/// results are tables
|
/// results are tables
|
||||||
|
|
||||||
if (auto * t = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
|
if (auto * t = ast->as<ASTTablesInSelectQueryElement>())
|
||||||
{
|
{
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (auto * t = typeid_cast<ASTTableExpression *>(ast.get()))
|
if (auto * t = ast->as<ASTTableExpression>())
|
||||||
{
|
{
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (auto * t = typeid_cast<ASTSelectQuery *>(ast.get()))
|
if (auto * t = ast->as<ASTSelectQuery>())
|
||||||
{
|
{
|
||||||
data.addTableAliasIfAny(*ast);
|
data.addTableAliasIfAny(*ast);
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (typeid_cast<ASTSubquery *>(ast.get()))
|
if (ast->as<ASTSubquery>())
|
||||||
{
|
{
|
||||||
data.addTableAliasIfAny(*ast);
|
data.addTableAliasIfAny(*ast);
|
||||||
return;
|
return;
|
||||||
@ -105,7 +103,7 @@ void RequiredSourceColumnsMatcher::visit(ASTPtr & ast, Data & data)
|
|||||||
|
|
||||||
/// other
|
/// other
|
||||||
|
|
||||||
if (auto * t = typeid_cast<ASTArrayJoin *>(ast.get()))
|
if (auto * t = ast->as<ASTArrayJoin>())
|
||||||
{
|
{
|
||||||
data.has_array_join = true;
|
data.has_array_join = true;
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
@ -118,7 +116,7 @@ void RequiredSourceColumnsMatcher::visit(ASTSelectQuery & select, const ASTPtr &
|
|||||||
/// special case for top-level SELECT items: they are publics
|
/// special case for top-level SELECT items: they are publics
|
||||||
for (auto & node : select.select_expression_list->children)
|
for (auto & node : select.select_expression_list->children)
|
||||||
{
|
{
|
||||||
if (auto * identifier = typeid_cast<ASTIdentifier *>(node.get()))
|
if (const auto * identifier = node->as<ASTIdentifier>())
|
||||||
data.addColumnIdentifier(*identifier);
|
data.addColumnIdentifier(*identifier);
|
||||||
else
|
else
|
||||||
data.addColumnAliasIfAny(*node);
|
data.addColumnAliasIfAny(*node);
|
||||||
@ -170,9 +168,9 @@ void RequiredSourceColumnsMatcher::visit(ASTTablesInSelectQueryElement & node, c
|
|||||||
|
|
||||||
for (auto & child : node.children)
|
for (auto & child : node.children)
|
||||||
{
|
{
|
||||||
if (auto * e = typeid_cast<ASTTableExpression *>(child.get()))
|
if (auto * e = child->as<ASTTableExpression>())
|
||||||
expr = e;
|
expr = e;
|
||||||
if (auto * j = typeid_cast<ASTTableJoin *>(child.get()))
|
if (auto * j = child->as<ASTTableJoin>())
|
||||||
join = j;
|
join = j;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -207,7 +205,7 @@ void RequiredSourceColumnsMatcher::visit(const ASTArrayJoin & node, const ASTPtr
|
|||||||
{
|
{
|
||||||
data.addArrayJoinAliasIfAny(*expr);
|
data.addArrayJoinAliasIfAny(*expr);
|
||||||
|
|
||||||
if (auto * identifier = typeid_cast<ASTIdentifier *>(expr.get()))
|
if (const auto * identifier = expr->as<ASTIdentifier>())
|
||||||
{
|
{
|
||||||
data.addArrayJoinIdentifier(*identifier);
|
data.addArrayJoinIdentifier(*identifier);
|
||||||
continue;
|
continue;
|
||||||
|
@ -205,13 +205,13 @@ bool Set::insertFromBlock(const Block & block)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const Context & context)
|
static Field extractValueFromNode(const ASTPtr & node, const IDataType & type, const Context & context)
|
||||||
{
|
{
|
||||||
if (ASTLiteral * lit = typeid_cast<ASTLiteral *>(node.get()))
|
if (const auto * lit = node->as<ASTLiteral>())
|
||||||
{
|
{
|
||||||
return convertFieldToType(lit->value, type);
|
return convertFieldToType(lit->value, type);
|
||||||
}
|
}
|
||||||
else if (typeid_cast<ASTFunction *>(node.get()))
|
else if (node->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(node, context);
|
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(node, context);
|
||||||
return convertFieldToType(value_raw.first, type, value_raw.second.get());
|
return convertFieldToType(value_raw.first, type, value_raw.second.get());
|
||||||
@ -235,7 +235,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
|
|||||||
|
|
||||||
DataTypePtr tuple_type;
|
DataTypePtr tuple_type;
|
||||||
Row tuple_values;
|
Row tuple_values;
|
||||||
ASTExpressionList & list = typeid_cast<ASTExpressionList &>(*node);
|
const auto & list = node->as<ASTExpressionList &>();
|
||||||
for (auto & elem : list.children)
|
for (auto & elem : list.children)
|
||||||
{
|
{
|
||||||
if (num_columns == 1)
|
if (num_columns == 1)
|
||||||
@ -245,7 +245,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
|
|||||||
if (!value.isNull())
|
if (!value.isNull())
|
||||||
columns[0]->insert(value);
|
columns[0]->insert(value);
|
||||||
}
|
}
|
||||||
else if (ASTFunction * func = typeid_cast<ASTFunction *>(elem.get()))
|
else if (const auto * func = elem->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
Field function_result;
|
Field function_result;
|
||||||
const TupleBackend * tuple = nullptr;
|
const TupleBackend * tuple = nullptr;
|
||||||
|
@ -63,7 +63,7 @@ using LogAST = DebugASTLog<false>; /// set to true to enable logs
|
|||||||
|
|
||||||
|
|
||||||
/// Add columns from storage to source_columns list.
|
/// Add columns from storage to source_columns list.
|
||||||
void collectSourceColumns(ASTSelectQuery * select_query, StoragePtr storage, NamesAndTypesList & source_columns)
|
void collectSourceColumns(const ASTSelectQuery * select_query, StoragePtr storage, NamesAndTypesList & source_columns)
|
||||||
{
|
{
|
||||||
if (storage)
|
if (storage)
|
||||||
{
|
{
|
||||||
@ -112,12 +112,12 @@ void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query
|
|||||||
|
|
||||||
bool hasArrayJoin(const ASTPtr & ast)
|
bool hasArrayJoin(const ASTPtr & ast)
|
||||||
{
|
{
|
||||||
if (const ASTFunction * function = typeid_cast<const ASTFunction *>(&*ast))
|
if (const ASTFunction * function = ast->as<ASTFunction>())
|
||||||
if (function->name == "arrayJoin")
|
if (function->name == "arrayJoin")
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
for (const auto & child : ast->children)
|
for (const auto & child : ast->children)
|
||||||
if (!typeid_cast<ASTSelectQuery *>(child.get()) && hasArrayJoin(child))
|
if (!child->as<ASTSelectQuery>() && hasArrayJoin(child))
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
@ -213,9 +213,9 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum
|
|||||||
if (!select_query->group_expression_list)
|
if (!select_query->group_expression_list)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
const auto is_literal = [] (const ASTPtr & ast)
|
const auto is_literal = [] (const ASTPtr & ast) -> bool
|
||||||
{
|
{
|
||||||
return typeid_cast<const ASTLiteral *>(ast.get());
|
return ast->as<ASTLiteral>();
|
||||||
};
|
};
|
||||||
|
|
||||||
auto & group_exprs = select_query->group_expression_list->children;
|
auto & group_exprs = select_query->group_expression_list->children;
|
||||||
@ -232,7 +232,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum
|
|||||||
/// iterate over each GROUP BY expression, eliminate injective function calls and literals
|
/// iterate over each GROUP BY expression, eliminate injective function calls and literals
|
||||||
for (size_t i = 0; i < group_exprs.size();)
|
for (size_t i = 0; i < group_exprs.size();)
|
||||||
{
|
{
|
||||||
if (const auto function = typeid_cast<ASTFunction *>(group_exprs[i].get()))
|
if (const auto * function = group_exprs[i]->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
/// assert function is injective
|
/// assert function is injective
|
||||||
if (possibly_injective_function_names.count(function->name))
|
if (possibly_injective_function_names.count(function->name))
|
||||||
@ -244,13 +244,9 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto & dict_name = typeid_cast<const ASTLiteral &>(*function->arguments->children[0])
|
const auto & dict_name = function->arguments->children[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
.value.safeGet<String>();
|
|
||||||
|
|
||||||
const auto & dict_ptr = context.getExternalDictionaries().getDictionary(dict_name);
|
const auto & dict_ptr = context.getExternalDictionaries().getDictionary(dict_name);
|
||||||
|
const auto & attr_name = function->arguments->children[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||||
const auto & attr_name = typeid_cast<const ASTLiteral &>(*function->arguments->children[1])
|
|
||||||
.value.safeGet<String>();
|
|
||||||
|
|
||||||
if (!dict_ptr->isInjective(attr_name))
|
if (!dict_ptr->isInjective(attr_name))
|
||||||
{
|
{
|
||||||
@ -328,7 +324,7 @@ void optimizeOrderBy(const ASTSelectQuery * select_query)
|
|||||||
for (const auto & elem : elems)
|
for (const auto & elem : elems)
|
||||||
{
|
{
|
||||||
String name = elem->children.front()->getColumnName();
|
String name = elem->children.front()->getColumnName();
|
||||||
const ASTOrderByElement & order_by_elem = typeid_cast<const ASTOrderByElement &>(*elem);
|
const auto & order_by_elem = elem->as<ASTOrderByElement &>();
|
||||||
|
|
||||||
if (elems_set.emplace(name, order_by_elem.collation ? order_by_elem.collation->getColumnName() : "").second)
|
if (elems_set.emplace(name, order_by_elem.collation ? order_by_elem.collation->getColumnName() : "").second)
|
||||||
unique_elems.emplace_back(elem);
|
unique_elems.emplace_back(elem);
|
||||||
@ -363,11 +359,10 @@ void optimizeLimitBy(const ASTSelectQuery * select_query)
|
|||||||
/// Remove duplicated columns from USING(...).
|
/// Remove duplicated columns from USING(...).
|
||||||
void optimizeUsing(const ASTSelectQuery * select_query)
|
void optimizeUsing(const ASTSelectQuery * select_query)
|
||||||
{
|
{
|
||||||
auto node = const_cast<ASTTablesInSelectQueryElement *>(select_query->join());
|
if (!select_query->join())
|
||||||
if (!node)
|
|
||||||
return;
|
return;
|
||||||
|
|
||||||
auto table_join = static_cast<ASTTableJoin *>(&*node->table_join);
|
const auto * table_join = select_query->join()->table_join->as<ASTTableJoin>();
|
||||||
if (!(table_join && table_join->using_expression_list))
|
if (!(table_join && table_join->using_expression_list))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
@ -410,7 +405,7 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
|
|||||||
String result_name = expr->getAliasOrColumnName();
|
String result_name = expr->getAliasOrColumnName();
|
||||||
|
|
||||||
/// This is an array.
|
/// This is an array.
|
||||||
if (!isIdentifier(expr) || source_columns_set.count(source_name))
|
if (!expr->as<ASTIdentifier>() || source_columns_set.count(source_name))
|
||||||
{
|
{
|
||||||
result.array_join_result_to_source[result_name] = source_name;
|
result.array_join_result_to_source[result_name] = source_name;
|
||||||
}
|
}
|
||||||
@ -454,7 +449,7 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTT
|
|||||||
{
|
{
|
||||||
if (IdentifierSemantic::getColumnName(ast))
|
if (IdentifierSemantic::getColumnName(ast))
|
||||||
{
|
{
|
||||||
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
|
const auto * identifier = ast->as<ASTIdentifier>();
|
||||||
|
|
||||||
/// It's set in TranslateQualifiedNamesVisitor
|
/// It's set in TranslateQualifiedNamesVisitor
|
||||||
size_t membership = IdentifierSemantic::getMembership(*identifier);
|
size_t membership = IdentifierSemantic::getMembership(*identifier);
|
||||||
@ -498,7 +493,7 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTT
|
|||||||
/// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys.
|
/// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys.
|
||||||
auto add_columns_from_equals_expr = [&](const ASTPtr & expr)
|
auto add_columns_from_equals_expr = [&](const ASTPtr & expr)
|
||||||
{
|
{
|
||||||
auto * func_equals = typeid_cast<const ASTFunction *>(expr.get());
|
const auto * func_equals = expr->as<ASTFunction>();
|
||||||
if (!func_equals || func_equals->name != "equals")
|
if (!func_equals || func_equals->name != "equals")
|
||||||
throwSyntaxException("Expected equals expression, got " + queryToString(expr) + ".");
|
throwSyntaxException("Expected equals expression, got " + queryToString(expr) + ".");
|
||||||
|
|
||||||
@ -537,7 +532,7 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTT
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
auto * func = typeid_cast<const ASTFunction *>(table_join.on_expression.get());
|
const auto * func = table_join.on_expression->as<ASTFunction>();
|
||||||
if (func && func->name == "and")
|
if (func && func->name == "and")
|
||||||
{
|
{
|
||||||
for (const auto & expr : func->arguments->children)
|
for (const auto & expr : func->arguments->children)
|
||||||
@ -556,13 +551,13 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s
|
|||||||
if (!node)
|
if (!node)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
const auto & table_join = static_cast<const ASTTableJoin &>(*node->table_join);
|
const auto & table_join = node->table_join->as<ASTTableJoin &>();
|
||||||
const auto & table_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
|
const auto & table_expression = node->table_expression->as<ASTTableExpression &>();
|
||||||
DatabaseAndTableWithAlias joined_table_name(table_expression, current_database);
|
DatabaseAndTableWithAlias joined_table_name(table_expression, current_database);
|
||||||
|
|
||||||
if (table_join.using_expression_list)
|
if (table_join.using_expression_list)
|
||||||
{
|
{
|
||||||
auto & keys = typeid_cast<ASTExpressionList &>(*table_join.using_expression_list);
|
const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>();
|
||||||
for (const auto & key : keys.children)
|
for (const auto & key : keys.children)
|
||||||
analyzed_join.addUsingKey(key);
|
analyzed_join.addUsingKey(key);
|
||||||
|
|
||||||
@ -598,10 +593,10 @@ void replaceJoinedTable(const ASTTablesInSelectQueryElement* join)
|
|||||||
if (!join || !join->table_expression)
|
if (!join || !join->table_expression)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
auto & table_expr = static_cast<ASTTableExpression &>(*join->table_expression.get());
|
auto & table_expr = join->table_expression->as<ASTTableExpression &>();
|
||||||
if (table_expr.database_and_table_name)
|
if (table_expr.database_and_table_name)
|
||||||
{
|
{
|
||||||
auto & table_id = typeid_cast<ASTIdentifier &>(*table_expr.database_and_table_name.get());
|
const auto & table_id = table_expr.database_and_table_name->as<ASTIdentifier &>();
|
||||||
String expr = "(select * from " + table_id.name + ") as " + table_id.shortName();
|
String expr = "(select * from " + table_id.name + ") as " + table_id.shortName();
|
||||||
|
|
||||||
// FIXME: since the expression "a as b" exposes both "a" and "b" names, which is not equivalent to "(select * from a) as b",
|
// FIXME: since the expression "a as b" exposes both "a" and "b" names, which is not equivalent to "(select * from a) as b",
|
||||||
@ -610,7 +605,7 @@ void replaceJoinedTable(const ASTTablesInSelectQueryElement* join)
|
|||||||
if (table_id.alias.empty() && table_id.isShort())
|
if (table_id.alias.empty() && table_id.isShort())
|
||||||
{
|
{
|
||||||
ParserTableExpression parser;
|
ParserTableExpression parser;
|
||||||
table_expr = static_cast<ASTTableExpression &>(*parseQuery(parser, expr, 0));
|
table_expr = parseQuery(parser, expr, 0)->as<ASTTableExpression &>();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -624,7 +619,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
|||||||
const Names & required_result_columns,
|
const Names & required_result_columns,
|
||||||
StoragePtr storage) const
|
StoragePtr storage) const
|
||||||
{
|
{
|
||||||
auto * select_query = typeid_cast<ASTSelectQuery *>(query.get());
|
auto * select_query = query->as<ASTSelectQuery>();
|
||||||
if (!storage && select_query)
|
if (!storage && select_query)
|
||||||
{
|
{
|
||||||
if (auto db_and_table = getDatabaseAndTable(*select_query, 0))
|
if (auto db_and_table = getDatabaseAndTable(*select_query, 0))
|
||||||
@ -655,7 +650,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
|||||||
if (settings.enable_optimize_predicate_expression)
|
if (settings.enable_optimize_predicate_expression)
|
||||||
replaceJoinedTable(node);
|
replaceJoinedTable(node);
|
||||||
|
|
||||||
const auto & joined_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
|
const auto & joined_expression = node->table_expression->as<ASTTableExpression &>();
|
||||||
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());
|
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());
|
||||||
|
|
||||||
NamesAndTypesList joined_columns = getNamesAndTypeListFromTableExpression(joined_expression, context);
|
NamesAndTypesList joined_columns = getNamesAndTypeListFromTableExpression(joined_expression, context);
|
||||||
|
@ -31,13 +31,11 @@ namespace ErrorCodes
|
|||||||
bool TranslateQualifiedNamesMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
|
bool TranslateQualifiedNamesMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
|
||||||
{
|
{
|
||||||
/// Do not go to FROM, JOIN, subqueries.
|
/// Do not go to FROM, JOIN, subqueries.
|
||||||
if (typeid_cast<ASTTableExpression *>(child.get()) ||
|
if (child->as<ASTTableExpression>() || child->as<ASTSelectWithUnionQuery>())
|
||||||
typeid_cast<ASTSelectWithUnionQuery *>(child.get()))
|
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
/// Processed nodes. Do not go into children.
|
/// Processed nodes. Do not go into children.
|
||||||
if (typeid_cast<ASTQualifiedAsterisk *>(node.get()) ||
|
if (node->as<ASTQualifiedAsterisk>() || node->as<ASTTableJoin>())
|
||||||
typeid_cast<ASTTableJoin *>(node.get()))
|
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
/// ASTSelectQuery + others
|
/// ASTSelectQuery + others
|
||||||
@ -46,15 +44,15 @@ bool TranslateQualifiedNamesMatcher::needChildVisit(ASTPtr & node, const ASTPtr
|
|||||||
|
|
||||||
void TranslateQualifiedNamesMatcher::visit(ASTPtr & ast, Data & data)
|
void TranslateQualifiedNamesMatcher::visit(ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
if (auto * t = typeid_cast<ASTIdentifier *>(ast.get()))
|
if (auto * t = ast->as<ASTIdentifier>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
if (auto * t = typeid_cast<ASTTableJoin *>(ast.get()))
|
if (auto * t = ast->as<ASTTableJoin>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
if (auto * t = typeid_cast<ASTSelectQuery *>(ast.get()))
|
if (auto * t = ast->as<ASTSelectQuery>())
|
||||||
visit(*t, ast, data);
|
visit(*t, ast, data);
|
||||||
if (auto * node = typeid_cast<ASTExpressionList *>(ast.get()))
|
if (auto * node = ast->as<ASTExpressionList>())
|
||||||
visit(*node, ast, data);
|
visit(*node, ast, data);
|
||||||
if (auto * node = typeid_cast<ASTFunction *>(ast.get()))
|
if (auto * node = ast->as<ASTFunction>())
|
||||||
visit(*node, ast, data);
|
visit(*node, ast, data);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -91,7 +89,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTFunction & node, const ASTPtr &, D
|
|||||||
String func_name_lowercase = Poco::toLower(node.name);
|
String func_name_lowercase = Poco::toLower(node.name);
|
||||||
if (func_name_lowercase == "count" &&
|
if (func_name_lowercase == "count" &&
|
||||||
func_arguments->children.size() == 1 &&
|
func_arguments->children.size() == 1 &&
|
||||||
typeid_cast<const ASTAsterisk *>(func_arguments->children[0].get()))
|
func_arguments->children[0]->as<ASTAsterisk>())
|
||||||
func_arguments->children.clear();
|
func_arguments->children.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -173,14 +171,14 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
|||||||
bool has_asterisk = false;
|
bool has_asterisk = false;
|
||||||
for (const auto & child : node.children)
|
for (const auto & child : node.children)
|
||||||
{
|
{
|
||||||
if (typeid_cast<const ASTAsterisk *>(child.get()))
|
if (child->as<ASTAsterisk>())
|
||||||
{
|
{
|
||||||
if (tables_with_columns.empty())
|
if (tables_with_columns.empty())
|
||||||
throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR);
|
||||||
has_asterisk = true;
|
has_asterisk = true;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
else if (auto qa = typeid_cast<const ASTQualifiedAsterisk *>(child.get()))
|
else if (const auto * qa = child->as<ASTQualifiedAsterisk>())
|
||||||
{
|
{
|
||||||
visit(*qa, child, data); /// check if it's OK before rewrite
|
visit(*qa, child, data); /// check if it's OK before rewrite
|
||||||
has_asterisk = true;
|
has_asterisk = true;
|
||||||
@ -197,7 +195,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
|||||||
|
|
||||||
for (const auto & child : old_children)
|
for (const auto & child : old_children)
|
||||||
{
|
{
|
||||||
if (const auto * asterisk = typeid_cast<const ASTAsterisk *>(child.get()))
|
if (const auto * asterisk = child->as<ASTAsterisk>())
|
||||||
{
|
{
|
||||||
bool first_table = true;
|
bool first_table = true;
|
||||||
for (const auto & [table, table_columns] : tables_with_columns)
|
for (const auto & [table, table_columns] : tables_with_columns)
|
||||||
@ -214,7 +212,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
|||||||
first_table = false;
|
first_table = false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (const auto * qualified_asterisk = typeid_cast<const ASTQualifiedAsterisk *>(child.get()))
|
else if (const auto * qualified_asterisk = child->as<ASTQualifiedAsterisk>())
|
||||||
{
|
{
|
||||||
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
|
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
|
||||||
|
|
||||||
@ -239,15 +237,15 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
|||||||
/// 'select * from a join b using id' should result one 'id' column
|
/// 'select * from a join b using id' should result one 'id' column
|
||||||
void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, Data & data)
|
void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, Data & data)
|
||||||
{
|
{
|
||||||
const auto & table_join = typeid_cast<const ASTTableJoin &>(*ast);
|
const auto & table_join = ast->as<ASTTableJoin &>();
|
||||||
|
|
||||||
if (table_join.using_expression_list)
|
if (table_join.using_expression_list)
|
||||||
{
|
{
|
||||||
auto & keys = typeid_cast<ASTExpressionList &>(*table_join.using_expression_list);
|
const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>();
|
||||||
for (const auto & key : keys.children)
|
for (const auto & key : keys.children)
|
||||||
if (auto opt_column = getIdentifierName(key))
|
if (auto opt_column = getIdentifierName(key))
|
||||||
data.join_using_columns.insert(*opt_column);
|
data.join_using_columns.insert(*opt_column);
|
||||||
else if (typeid_cast<const ASTLiteral *>(key.get()))
|
else if (key->as<ASTLiteral>())
|
||||||
data.join_using_columns.insert(key->getColumnName());
|
data.join_using_columns.insert(key->getColumnName());
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
@ -60,11 +60,11 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
|
|||||||
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & context)
|
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & context)
|
||||||
{
|
{
|
||||||
/// Branch with string in query.
|
/// Branch with string in query.
|
||||||
if (typeid_cast<const ASTLiteral *>(node.get()))
|
if (node->as<ASTLiteral>())
|
||||||
return node;
|
return node;
|
||||||
|
|
||||||
/// Branch with TableFunction in query.
|
/// Branch with TableFunction in query.
|
||||||
if (auto table_func_ptr = typeid_cast<ASTFunction *>(node.get()))
|
if (const auto * table_func_ptr = node->as<ASTFunction>())
|
||||||
if (TableFunctionFactory::instance().isTableFunctionName(table_func_ptr->name))
|
if (TableFunctionFactory::instance().isTableFunctionName(table_func_ptr->name))
|
||||||
return node;
|
return node;
|
||||||
|
|
||||||
@ -73,7 +73,7 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context &
|
|||||||
|
|
||||||
ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const Context & context)
|
ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const Context & context)
|
||||||
{
|
{
|
||||||
if (auto id = typeid_cast<const ASTIdentifier *>(node.get()))
|
if (const auto * id = node->as<ASTIdentifier>())
|
||||||
return std::make_shared<ASTLiteral>(id->name);
|
return std::make_shared<ASTLiteral>(id->name);
|
||||||
|
|
||||||
return evaluateConstantExpressionAsLiteral(node, context);
|
return evaluateConstantExpressionAsLiteral(node, context);
|
||||||
@ -145,10 +145,8 @@ namespace
|
|||||||
{
|
{
|
||||||
const auto * left = fn->arguments->children.front().get();
|
const auto * left = fn->arguments->children.front().get();
|
||||||
const auto * right = fn->arguments->children.back().get();
|
const auto * right = fn->arguments->children.back().get();
|
||||||
const auto * identifier = typeid_cast<const ASTIdentifier *>(left) ? typeid_cast<const ASTIdentifier *>(left)
|
const auto * identifier = left->as<ASTIdentifier>() ? left->as<ASTIdentifier>() : right->as<ASTIdentifier>();
|
||||||
: typeid_cast<const ASTIdentifier *>(right);
|
const auto * literal = left->as<ASTLiteral>() ? left->as<ASTLiteral>() : right->as<ASTLiteral>();
|
||||||
const auto * literal = typeid_cast<const ASTLiteral *>(left) ? typeid_cast<const ASTLiteral *>(left)
|
|
||||||
: typeid_cast<const ASTLiteral *>(right);
|
|
||||||
|
|
||||||
return analyzeEquals(identifier, literal, expr);
|
return analyzeEquals(identifier, literal, expr);
|
||||||
}
|
}
|
||||||
@ -156,15 +154,15 @@ namespace
|
|||||||
{
|
{
|
||||||
const auto * left = fn->arguments->children.front().get();
|
const auto * left = fn->arguments->children.front().get();
|
||||||
const auto * right = fn->arguments->children.back().get();
|
const auto * right = fn->arguments->children.back().get();
|
||||||
const auto * identifier = typeid_cast<const ASTIdentifier *>(left);
|
const auto * identifier = left->as<ASTIdentifier>();
|
||||||
const auto * inner_fn = typeid_cast<const ASTFunction *>(right);
|
const auto * inner_fn = right->as<ASTFunction>();
|
||||||
|
|
||||||
if (!inner_fn)
|
if (!inner_fn)
|
||||||
{
|
{
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto * tuple = typeid_cast<const ASTExpressionList *>(inner_fn->children.front().get());
|
const auto * tuple = inner_fn->children.front()->as<ASTExpressionList>();
|
||||||
|
|
||||||
if (!tuple)
|
if (!tuple)
|
||||||
{
|
{
|
||||||
@ -175,7 +173,7 @@ namespace
|
|||||||
|
|
||||||
for (const auto & child : tuple->children)
|
for (const auto & child : tuple->children)
|
||||||
{
|
{
|
||||||
const auto * literal = typeid_cast<const ASTLiteral *>(child.get());
|
const auto * literal = child->as<ASTLiteral>();
|
||||||
const auto dnf = analyzeEquals(identifier, literal, expr);
|
const auto dnf = analyzeEquals(identifier, literal, expr);
|
||||||
|
|
||||||
if (dnf.empty())
|
if (dnf.empty())
|
||||||
@ -190,7 +188,7 @@ namespace
|
|||||||
}
|
}
|
||||||
else if (fn->name == "or")
|
else if (fn->name == "or")
|
||||||
{
|
{
|
||||||
const auto * args = typeid_cast<const ASTExpressionList *>(fn->children.front().get());
|
const auto * args = fn->children.front()->as<ASTExpressionList>();
|
||||||
|
|
||||||
if (!args)
|
if (!args)
|
||||||
{
|
{
|
||||||
@ -201,7 +199,7 @@ namespace
|
|||||||
|
|
||||||
for (const auto & arg : args->children)
|
for (const auto & arg : args->children)
|
||||||
{
|
{
|
||||||
const auto dnf = analyzeFunction(typeid_cast<const ASTFunction *>(arg.get()), expr);
|
const auto dnf = analyzeFunction(arg->as<ASTFunction>(), expr);
|
||||||
|
|
||||||
if (dnf.empty())
|
if (dnf.empty())
|
||||||
{
|
{
|
||||||
@ -215,7 +213,7 @@ namespace
|
|||||||
}
|
}
|
||||||
else if (fn->name == "and")
|
else if (fn->name == "and")
|
||||||
{
|
{
|
||||||
const auto * args = typeid_cast<const ASTExpressionList *>(fn->children.front().get());
|
const auto * args = fn->children.front()->as<ASTExpressionList>();
|
||||||
|
|
||||||
if (!args)
|
if (!args)
|
||||||
{
|
{
|
||||||
@ -226,7 +224,7 @@ namespace
|
|||||||
|
|
||||||
for (const auto & arg : args->children)
|
for (const auto & arg : args->children)
|
||||||
{
|
{
|
||||||
const auto dnf = analyzeFunction(typeid_cast<const ASTFunction *>(arg.get()), expr);
|
const auto dnf = analyzeFunction(arg->as<ASTFunction>(), expr);
|
||||||
|
|
||||||
if (dnf.empty())
|
if (dnf.empty())
|
||||||
{
|
{
|
||||||
@ -249,7 +247,7 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
|
|||||||
|
|
||||||
// TODO: `node` may be always-false literal.
|
// TODO: `node` may be always-false literal.
|
||||||
|
|
||||||
if (const auto fn = typeid_cast<const ASTFunction *>(node.get()))
|
if (const auto * fn = node->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
const auto dnf = analyzeFunction(fn, target_expr);
|
const auto dnf = analyzeFunction(fn, target_expr);
|
||||||
|
|
||||||
|
@ -169,7 +169,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
/// TODO Parser should fail early when max_query_size limit is reached.
|
/// TODO Parser should fail early when max_query_size limit is reached.
|
||||||
ast = parseQuery(parser, begin, end, "", max_query_size);
|
ast = parseQuery(parser, begin, end, "", max_query_size);
|
||||||
|
|
||||||
auto * insert_query = dynamic_cast<ASTInsertQuery *>(ast.get());
|
auto * insert_query = ast->as<ASTInsertQuery>();
|
||||||
if (insert_query && insert_query->data)
|
if (insert_query && insert_query->data)
|
||||||
{
|
{
|
||||||
query_end = insert_query->data;
|
query_end = insert_query->data;
|
||||||
@ -208,7 +208,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
|
|
||||||
/// Put query to process list. But don't put SHOW PROCESSLIST query itself.
|
/// Put query to process list. But don't put SHOW PROCESSLIST query itself.
|
||||||
ProcessList::EntryPtr process_list_entry;
|
ProcessList::EntryPtr process_list_entry;
|
||||||
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
|
if (!internal && !ast->as<ASTShowProcesslistQuery>())
|
||||||
{
|
{
|
||||||
process_list_entry = context.getProcessList().insert(query, ast.get(), context);
|
process_list_entry = context.getProcessList().insert(query, ast.get(), context);
|
||||||
context.setProcessListElement(&process_list_entry->get());
|
context.setProcessListElement(&process_list_entry->get());
|
||||||
@ -488,7 +488,8 @@ void executeQuery(
|
|||||||
|
|
||||||
if (streams.in)
|
if (streams.in)
|
||||||
{
|
{
|
||||||
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
|
/// FIXME: try to prettify this cast using `as<>()`
|
||||||
|
const auto * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
|
||||||
|
|
||||||
WriteBuffer * out_buf = &ostr;
|
WriteBuffer * out_buf = &ostr;
|
||||||
std::optional<WriteBufferFromFile> out_file_buf;
|
std::optional<WriteBufferFromFile> out_file_buf;
|
||||||
@ -497,7 +498,7 @@ void executeQuery(
|
|||||||
if (!allow_into_outfile)
|
if (!allow_into_outfile)
|
||||||
throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED);
|
throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED);
|
||||||
|
|
||||||
const auto & out_file = typeid_cast<const ASTLiteral &>(*ast_query_with_output->out_file).value.safeGet<std::string>();
|
const auto & out_file = ast_query_with_output->out_file->as<ASTLiteral &>().value.safeGet<std::string>();
|
||||||
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
|
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
|
||||||
out_buf = &*out_file_buf;
|
out_buf = &*out_file_buf;
|
||||||
}
|
}
|
||||||
|
@ -18,14 +18,14 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
std::string getClusterName(const IAST & node)
|
std::string getClusterName(const IAST & node)
|
||||||
{
|
{
|
||||||
if (const ASTIdentifier * ast_id = typeid_cast<const ASTIdentifier *>(&node))
|
if (const auto * ast_id = node.as<ASTIdentifier>())
|
||||||
return ast_id->name;
|
return ast_id->name;
|
||||||
|
|
||||||
if (const ASTLiteral * ast_lit = typeid_cast<const ASTLiteral *>(&node))
|
if (const auto * ast_lit = node.as<ASTLiteral>())
|
||||||
return ast_lit->value.safeGet<String>();
|
return ast_lit->value.safeGet<String>();
|
||||||
|
|
||||||
/// A hack to support hyphens in cluster names.
|
/// A hack to support hyphens in cluster names.
|
||||||
if (const ASTFunction * ast_func = typeid_cast<const ASTFunction *>(&node))
|
if (const auto * ast_func = node.as<ASTFunction>())
|
||||||
{
|
{
|
||||||
if (ast_func->name != "minus" || !ast_func->arguments || ast_func->arguments->children.size() < 2)
|
if (ast_func->name != "minus" || !ast_func->arguments || ast_func->arguments->children.size() < 2)
|
||||||
throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS);
|
throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
@ -19,9 +19,9 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
|
|||||||
const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns)
|
const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns)
|
||||||
{
|
{
|
||||||
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
|
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
|
||||||
const ASTSubquery * subquery = typeid_cast<const ASTSubquery *>(table_expression.get());
|
const auto * subquery = table_expression->as<ASTSubquery>();
|
||||||
const ASTFunction * function = typeid_cast<const ASTFunction *>(table_expression.get());
|
const auto * function = table_expression->as<ASTFunction>();
|
||||||
const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(table_expression.get());
|
const auto * table = table_expression->as<ASTIdentifier>();
|
||||||
|
|
||||||
if (!subquery && !table && !function)
|
if (!subquery && !table && !function)
|
||||||
throw Exception("Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery." , ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery." , ErrorCodes::LOGICAL_ERROR);
|
||||||
@ -65,7 +65,7 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
|
|||||||
auto query_context = const_cast<Context *>(&context.getQueryContext());
|
auto query_context = const_cast<Context *>(&context.getQueryContext());
|
||||||
const auto & storage = query_context->executeTableFunction(table_expression);
|
const auto & storage = query_context->executeTableFunction(table_expression);
|
||||||
columns = storage->getColumns().ordinary;
|
columns = storage->getColumns().ordinary;
|
||||||
select_query->addTableFunction(*const_cast<ASTPtr *>(&table_expression));
|
select_query->addTableFunction(*const_cast<ASTPtr *>(&table_expression)); // XXX: const_cast should be avoided!
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -94,9 +94,9 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
|
|||||||
std::set<std::string> all_column_names;
|
std::set<std::string> all_column_names;
|
||||||
std::set<std::string> assigned_column_names;
|
std::set<std::string> assigned_column_names;
|
||||||
|
|
||||||
if (ASTSelectWithUnionQuery * select_with_union = typeid_cast<ASTSelectWithUnionQuery *>(query.get()))
|
if (const auto * select_with_union = query->as<ASTSelectWithUnionQuery>())
|
||||||
{
|
{
|
||||||
if (ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(select_with_union->list_of_selects->children.at(0).get()))
|
if (const auto * select = select_with_union->list_of_selects->children.at(0)->as<ASTSelectQuery>())
|
||||||
{
|
{
|
||||||
for (auto & expr : select->select_expression_list->children)
|
for (auto & expr : select->select_expression_list->children)
|
||||||
all_column_names.insert(expr->getAliasOrColumnName());
|
all_column_names.insert(expr->getAliasOrColumnName());
|
||||||
|
@ -39,7 +39,7 @@ static void executeCreateQuery(
|
|||||||
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);
|
||||||
|
|
||||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||||
ast_create_query.attach = true;
|
ast_create_query.attach = true;
|
||||||
ast_create_query.database = database;
|
ast_create_query.database = database;
|
||||||
|
|
||||||
|
@ -123,7 +123,7 @@ public:
|
|||||||
|
|
||||||
void add(const ASTPtr & command)
|
void add(const ASTPtr & command)
|
||||||
{
|
{
|
||||||
commands.push_back(static_cast<ASTAlterCommand *>(command.get()));
|
commands.push_back(command->as<ASTAlterCommand>());
|
||||||
children.push_back(command);
|
children.push_back(command);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -62,7 +62,7 @@ ASTPtr ASTFunction::clone() const
|
|||||||
*/
|
*/
|
||||||
static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const IAST::FormatSettings & settings, const char * metacharacters)
|
static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const IAST::FormatSettings & settings, const char * metacharacters)
|
||||||
{
|
{
|
||||||
if (auto literal = dynamic_cast<const ASTLiteral *>(node.get()))
|
if (const auto * literal = node->as<ASTLiteral>())
|
||||||
{
|
{
|
||||||
if (literal->value.getType() == Field::Types::String)
|
if (literal->value.getType() == Field::Types::String)
|
||||||
{
|
{
|
||||||
@ -132,7 +132,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
|||||||
* Instead, add a space.
|
* Instead, add a space.
|
||||||
* PS. You can not just ask to add parentheses - see formatImpl for ASTLiteral.
|
* PS. You can not just ask to add parentheses - see formatImpl for ASTLiteral.
|
||||||
*/
|
*/
|
||||||
if (name == "negate" && typeid_cast<const ASTLiteral *>(&*arguments->children[0]))
|
if (name == "negate" && arguments->children[0]->as<ASTLiteral>())
|
||||||
settings.ostr << ' ';
|
settings.ostr << ' ';
|
||||||
|
|
||||||
arguments->formatImpl(settings, state, nested_need_parens);
|
arguments->formatImpl(settings, state, nested_need_parens);
|
||||||
@ -203,7 +203,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
|||||||
if (!written && 0 == strcmp(name.c_str(), "tupleElement"))
|
if (!written && 0 == strcmp(name.c_str(), "tupleElement"))
|
||||||
{
|
{
|
||||||
/// It can be printed in a form of 'x.1' only if right hand side is unsigned integer literal.
|
/// It can be printed in a form of 'x.1' only if right hand side is unsigned integer literal.
|
||||||
if (const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(arguments->children[1].get()))
|
if (const auto * lit = arguments->children[1]->as<ASTLiteral>())
|
||||||
{
|
{
|
||||||
if (lit->value.getType() == Field::Types::UInt64)
|
if (lit->value.getType() == Field::Types::UInt64)
|
||||||
{
|
{
|
||||||
@ -222,7 +222,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
|||||||
if (frame.need_parens)
|
if (frame.need_parens)
|
||||||
settings.ostr << '(';
|
settings.ostr << '(';
|
||||||
|
|
||||||
const ASTFunction * first_arg_func = typeid_cast<const ASTFunction *>(arguments->children[0].get());
|
const auto * first_arg_func = arguments->children[0]->as<ASTFunction>();
|
||||||
if (first_arg_func
|
if (first_arg_func
|
||||||
&& first_arg_func->name == "tuple"
|
&& first_arg_func->name == "tuple"
|
||||||
&& first_arg_func->arguments
|
&& first_arg_func->arguments
|
||||||
|
@ -83,17 +83,10 @@ ASTPtr createTableIdentifier(const String & database_name, const String & table_
|
|||||||
return database_and_table;
|
return database_and_table;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isIdentifier(const IAST * const ast)
|
|
||||||
{
|
|
||||||
if (ast)
|
|
||||||
return typeid_cast<const ASTIdentifier *>(ast);
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
std::optional<String> getIdentifierName(const IAST * const ast)
|
std::optional<String> getIdentifierName(const IAST * const ast)
|
||||||
{
|
{
|
||||||
if (ast)
|
if (ast)
|
||||||
if (auto node = typeid_cast<const ASTIdentifier *>(ast))
|
if (const auto * node = ast->as<ASTIdentifier>())
|
||||||
return node->name;
|
return node->name;
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
@ -101,7 +94,7 @@ std::optional<String> getIdentifierName(const IAST * const ast)
|
|||||||
bool getIdentifierName(const ASTPtr & ast, String & name)
|
bool getIdentifierName(const ASTPtr & ast, String & name)
|
||||||
{
|
{
|
||||||
if (ast)
|
if (ast)
|
||||||
if (auto node = typeid_cast<const ASTIdentifier *>(ast.get()))
|
if (const auto * node = ast->as<ASTIdentifier>())
|
||||||
{
|
{
|
||||||
name = node->name;
|
name = node->name;
|
||||||
return true;
|
return true;
|
||||||
@ -112,7 +105,7 @@ bool getIdentifierName(const ASTPtr & ast, String & name)
|
|||||||
void setIdentifierSpecial(ASTPtr & ast)
|
void setIdentifierSpecial(ASTPtr & ast)
|
||||||
{
|
{
|
||||||
if (ast)
|
if (ast)
|
||||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(ast.get()))
|
if (auto * id = ast->as<ASTIdentifier>())
|
||||||
id->semantic->special = true;
|
id->semantic->special = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -69,9 +69,6 @@ private:
|
|||||||
ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
|
ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
|
||||||
void setIdentifierSpecial(ASTPtr & ast);
|
void setIdentifierSpecial(ASTPtr & ast);
|
||||||
|
|
||||||
bool isIdentifier(const IAST * const ast);
|
|
||||||
inline bool isIdentifier(const ASTPtr & ast) { return isIdentifier(ast.get()); }
|
|
||||||
|
|
||||||
std::optional<String> getIdentifierName(const IAST * const ast);
|
std::optional<String> getIdentifierName(const IAST * const ast);
|
||||||
inline std::optional<String> getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
inline std::optional<String> getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
||||||
bool getIdentifierName(const ASTPtr & ast, String & name);
|
bool getIdentifierName(const ASTPtr & ast, String & name);
|
||||||
|
@ -49,7 +49,8 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat
|
|||||||
|
|
||||||
bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast)
|
bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast)
|
||||||
{
|
{
|
||||||
if (auto ast_with_output = dynamic_cast<ASTQueryWithOutput *>(&ast))
|
/// FIXME: try to prettify this cast using `as<>()`
|
||||||
|
if (auto * ast_with_output = dynamic_cast<ASTQueryWithOutput *>(&ast))
|
||||||
{
|
{
|
||||||
ast_with_output->format.reset();
|
ast_with_output->format.reset();
|
||||||
ast_with_output->out_file.reset();
|
ast_with_output->out_file.reset();
|
||||||
|
@ -41,7 +41,7 @@ public:
|
|||||||
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override
|
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override
|
||||||
{
|
{
|
||||||
auto query_ptr = clone();
|
auto query_ptr = clone();
|
||||||
auto & query = static_cast<ASTRenameQuery &>(*query_ptr);
|
auto & query = query_ptr->as<ASTRenameQuery &>();
|
||||||
|
|
||||||
query.cluster.clear();
|
query.cluster.clear();
|
||||||
for (Element & elem : query.elements)
|
for (Element & elem : query.elements)
|
||||||
|
@ -65,7 +65,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
|
|||||||
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH " << (s.hilite ? hilite_none : "");
|
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH " << (s.hilite ? hilite_none : "");
|
||||||
s.one_line
|
s.one_line
|
||||||
? with_expression_list->formatImpl(s, state, frame)
|
? with_expression_list->formatImpl(s, state, frame)
|
||||||
: typeid_cast<const ASTExpressionList &>(*with_expression_list).formatImplMultiline(s, state, frame);
|
: with_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
|
||||||
s.ostr << s.nl_or_ws;
|
s.ostr << s.nl_or_ws;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -73,7 +73,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
|
|||||||
|
|
||||||
s.one_line
|
s.one_line
|
||||||
? select_expression_list->formatImpl(s, state, frame)
|
? select_expression_list->formatImpl(s, state, frame)
|
||||||
: typeid_cast<const ASTExpressionList &>(*select_expression_list).formatImplMultiline(s, state, frame);
|
: select_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
|
||||||
|
|
||||||
if (tables)
|
if (tables)
|
||||||
{
|
{
|
||||||
@ -98,7 +98,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
|
|||||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : "");
|
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : "");
|
||||||
s.one_line
|
s.one_line
|
||||||
? group_expression_list->formatImpl(s, state, frame)
|
? group_expression_list->formatImpl(s, state, frame)
|
||||||
: typeid_cast<const ASTExpressionList &>(*group_expression_list).formatImplMultiline(s, state, frame);
|
: group_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (group_by_with_rollup)
|
if (group_by_with_rollup)
|
||||||
@ -121,7 +121,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
|
|||||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : "");
|
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : "");
|
||||||
s.one_line
|
s.one_line
|
||||||
? order_expression_list->formatImpl(s, state, frame)
|
? order_expression_list->formatImpl(s, state, frame)
|
||||||
: typeid_cast<const ASTExpressionList &>(*order_expression_list).formatImplMultiline(s, state, frame);
|
: order_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (limit_by_value)
|
if (limit_by_value)
|
||||||
@ -131,7 +131,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
|
|||||||
s.ostr << (s.hilite ? hilite_keyword : "") << " BY " << (s.hilite ? hilite_none : "");
|
s.ostr << (s.hilite ? hilite_keyword : "") << " BY " << (s.hilite ? hilite_none : "");
|
||||||
s.one_line
|
s.one_line
|
||||||
? limit_by_expression_list->formatImpl(s, state, frame)
|
? limit_by_expression_list->formatImpl(s, state, frame)
|
||||||
: typeid_cast<const ASTExpressionList &>(*limit_by_expression_list).formatImplMultiline(s, state, frame);
|
: limit_by_expression_list->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (limit_length)
|
if (limit_length)
|
||||||
@ -161,15 +161,15 @@ static const ASTTableExpression * getFirstTableExpression(const ASTSelectQuery &
|
|||||||
if (!select.tables)
|
if (!select.tables)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const ASTTablesInSelectQuery & tables_in_select_query = static_cast<const ASTTablesInSelectQuery &>(*select.tables);
|
const auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
|
||||||
if (tables_in_select_query.children.empty())
|
if (tables_in_select_query.children.empty())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const ASTTablesInSelectQueryElement & tables_element = static_cast<const ASTTablesInSelectQueryElement &>(*tables_in_select_query.children[0]);
|
const auto & tables_element = tables_in_select_query.children[0]->as<ASTTablesInSelectQueryElement &>();
|
||||||
if (!tables_element.table_expression)
|
if (!tables_element.table_expression)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
return static_cast<const ASTTableExpression *>(tables_element.table_expression.get());
|
return tables_element.table_expression->as<ASTTableExpression>();
|
||||||
}
|
}
|
||||||
|
|
||||||
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select)
|
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select)
|
||||||
@ -177,15 +177,15 @@ static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select)
|
|||||||
if (!select.tables)
|
if (!select.tables)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
ASTTablesInSelectQuery & tables_in_select_query = static_cast<ASTTablesInSelectQuery &>(*select.tables);
|
auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
|
||||||
if (tables_in_select_query.children.empty())
|
if (tables_in_select_query.children.empty())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
ASTTablesInSelectQueryElement & tables_element = static_cast<ASTTablesInSelectQueryElement &>(*tables_in_select_query.children[0]);
|
auto & tables_element = tables_in_select_query.children[0]->as<ASTTablesInSelectQueryElement &>();
|
||||||
if (!tables_element.table_expression)
|
if (!tables_element.table_expression)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
return static_cast<ASTTableExpression *>(tables_element.table_expression.get());
|
return tables_element.table_expression->as<ASTTableExpression>();
|
||||||
}
|
}
|
||||||
|
|
||||||
static const ASTArrayJoin * getFirstArrayJoin(const ASTSelectQuery & select)
|
static const ASTArrayJoin * getFirstArrayJoin(const ASTSelectQuery & select)
|
||||||
@ -193,18 +193,18 @@ static const ASTArrayJoin * getFirstArrayJoin(const ASTSelectQuery & select)
|
|||||||
if (!select.tables)
|
if (!select.tables)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const ASTTablesInSelectQuery & tables_in_select_query = static_cast<const ASTTablesInSelectQuery &>(*select.tables);
|
const auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
|
||||||
if (tables_in_select_query.children.empty())
|
if (tables_in_select_query.children.empty())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const ASTArrayJoin * array_join = nullptr;
|
const ASTArrayJoin * array_join = nullptr;
|
||||||
for (const auto & child : tables_in_select_query.children)
|
for (const auto & child : tables_in_select_query.children)
|
||||||
{
|
{
|
||||||
const ASTTablesInSelectQueryElement & tables_element = static_cast<const ASTTablesInSelectQueryElement &>(*child);
|
const auto & tables_element = child->as<ASTTablesInSelectQueryElement &>();
|
||||||
if (tables_element.array_join)
|
if (tables_element.array_join)
|
||||||
{
|
{
|
||||||
if (!array_join)
|
if (!array_join)
|
||||||
array_join = static_cast<const ASTArrayJoin *>(tables_element.array_join.get());
|
array_join = tables_element.array_join->as<ASTArrayJoin>();
|
||||||
else
|
else
|
||||||
throw Exception("Support for more than one ARRAY JOIN in query is not implemented", ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Support for more than one ARRAY JOIN in query is not implemented", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
@ -218,14 +218,14 @@ static const ASTTablesInSelectQueryElement * getFirstTableJoin(const ASTSelectQu
|
|||||||
if (!select.tables)
|
if (!select.tables)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const ASTTablesInSelectQuery & tables_in_select_query = static_cast<const ASTTablesInSelectQuery &>(*select.tables);
|
const auto & tables_in_select_query = select.tables->as<ASTTablesInSelectQuery &>();
|
||||||
if (tables_in_select_query.children.empty())
|
if (tables_in_select_query.children.empty())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const ASTTablesInSelectQueryElement * joined_table = nullptr;
|
const ASTTablesInSelectQueryElement * joined_table = nullptr;
|
||||||
for (const auto & child : tables_in_select_query.children)
|
for (const auto & child : tables_in_select_query.children)
|
||||||
{
|
{
|
||||||
const ASTTablesInSelectQueryElement & tables_element = static_cast<const ASTTablesInSelectQueryElement &>(*child);
|
const auto & tables_element = child->as<ASTTablesInSelectQueryElement &>();
|
||||||
if (tables_element.table_join)
|
if (tables_element.table_join)
|
||||||
{
|
{
|
||||||
if (!joined_table)
|
if (!joined_table)
|
||||||
@ -357,4 +357,3 @@ void ASTSelectQuery::addTableFunction(ASTPtr & table_function_ptr)
|
|||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -208,7 +208,7 @@ void ASTArrayJoin::formatImpl(const FormatSettings & settings, FormatState & sta
|
|||||||
|
|
||||||
settings.one_line
|
settings.one_line
|
||||||
? expression_list->formatImpl(settings, state, frame)
|
? expression_list->formatImpl(settings, state, frame)
|
||||||
: typeid_cast<const ASTExpressionList &>(*expression_list).formatImplMultiline(settings, state, frame);
|
: expression_list->as<ASTExpressionList &>().formatImplMultiline(settings, state, frame);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -218,7 +218,7 @@ void ASTTablesInSelectQueryElement::formatImpl(const FormatSettings & settings,
|
|||||||
{
|
{
|
||||||
if (table_join)
|
if (table_join)
|
||||||
{
|
{
|
||||||
static_cast<const ASTTableJoin &>(*table_join).formatImplBeforeTable(settings, state, frame);
|
table_join->as<ASTTableJoin &>().formatImplBeforeTable(settings, state, frame);
|
||||||
settings.ostr << " ";
|
settings.ostr << " ";
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -226,7 +226,7 @@ void ASTTablesInSelectQueryElement::formatImpl(const FormatSettings & settings,
|
|||||||
settings.ostr << " ";
|
settings.ostr << " ";
|
||||||
|
|
||||||
if (table_join)
|
if (table_join)
|
||||||
static_cast<const ASTTableJoin &>(*table_join).formatImplAfterTable(settings, state, frame);
|
table_join->as<ASTTableJoin &>().formatImplAfterTable(settings, state, frame);
|
||||||
}
|
}
|
||||||
else if (array_join)
|
else if (array_join)
|
||||||
{
|
{
|
||||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user