2018-02-25 00:50:53 +00:00
|
|
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
|
|
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
2018-02-27 20:16:58 +00:00
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
2018-02-25 00:50:53 +00:00
|
|
|
#include <DataStreams/UnionBlockInputStream.h>
|
|
|
|
#include <DataStreams/NullBlockInputStream.h>
|
2018-02-28 02:33:04 +00:00
|
|
|
#include <DataStreams/ConcatBlockInputStream.h>
|
2018-02-26 06:12:59 +00:00
|
|
|
#include <DataStreams/ConvertingBlockInputStream.h>
|
|
|
|
#include <DataTypes/getLeastSupertype.h>
|
|
|
|
#include <Columns/ColumnConst.h>
|
2018-02-25 07:39:45 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2018-05-30 19:23:15 +00:00
|
|
|
#include <Parsers/queryToString.h>
|
2018-02-25 00:50:53 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
2018-02-26 06:12:59 +00:00
|
|
|
extern const int UNION_ALL_RESULT_STRUCTURES_MISMATCH;
|
2018-02-25 00:50:53 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
|
|
|
|
const ASTPtr & query_ptr_,
|
|
|
|
const Context & context_,
|
2018-02-27 20:16:58 +00:00
|
|
|
const Names & required_result_column_names,
|
2018-02-25 00:50:53 +00:00
|
|
|
QueryProcessingStage::Enum to_stage_,
|
2018-07-19 13:36:21 +00:00
|
|
|
size_t subquery_depth_,
|
|
|
|
bool only_analyze)
|
2018-02-25 00:50:53 +00:00
|
|
|
: query_ptr(query_ptr_),
|
|
|
|
context(context_),
|
|
|
|
to_stage(to_stage_),
|
|
|
|
subquery_depth(subquery_depth_)
|
|
|
|
{
|
2018-03-02 05:44:17 +00:00
|
|
|
if (!context.hasQueryContext())
|
|
|
|
context.setQueryContext(context);
|
|
|
|
|
2018-02-25 07:39:45 +00:00
|
|
|
const ASTSelectWithUnionQuery & ast = typeid_cast<const ASTSelectWithUnionQuery &>(*query_ptr);
|
|
|
|
|
|
|
|
size_t num_selects = ast.list_of_selects->children.size();
|
2018-02-25 00:50:53 +00:00
|
|
|
|
|
|
|
if (!num_selects)
|
|
|
|
throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2018-02-27 20:16:58 +00:00
|
|
|
/// Initialize interpreters for each SELECT query.
|
|
|
|
/// Note that we pass 'required_result_column_names' to first SELECT.
|
2018-02-27 20:43:42 +00:00
|
|
|
/// And for the rest, we pass names at the corresponding positions of 'required_result_column_names' in the result of first SELECT,
|
2018-02-27 20:16:58 +00:00
|
|
|
/// because names could be different.
|
2018-02-25 00:50:53 +00:00
|
|
|
|
2018-02-27 20:16:58 +00:00
|
|
|
nested_interpreters.reserve(num_selects);
|
2018-02-25 00:50:53 +00:00
|
|
|
|
2018-02-27 20:43:42 +00:00
|
|
|
std::vector<Names> required_result_column_names_for_other_selects(num_selects);
|
2018-03-02 04:05:20 +00:00
|
|
|
if (!required_result_column_names.empty() && num_selects > 1)
|
2018-02-27 20:16:58 +00:00
|
|
|
{
|
2018-02-27 20:43:42 +00:00
|
|
|
/// Result header if there are no filtering by 'required_result_column_names'.
|
|
|
|
/// We use it to determine positions of 'required_result_column_names' in SELECT clause.
|
|
|
|
|
|
|
|
Block full_result_header = InterpreterSelectQuery(
|
2018-07-19 13:36:21 +00:00
|
|
|
ast.list_of_selects->children.at(0), context, Names(), to_stage, subquery_depth, nullptr, true).getSampleBlock();
|
2018-02-27 20:16:58 +00:00
|
|
|
|
2018-02-27 20:43:42 +00:00
|
|
|
std::vector<size_t> positions_of_required_result_columns(required_result_column_names.size());
|
2018-02-27 20:16:58 +00:00
|
|
|
for (size_t required_result_num = 0, size = required_result_column_names.size(); required_result_num < size; ++required_result_num)
|
2018-02-27 20:43:42 +00:00
|
|
|
positions_of_required_result_columns[required_result_num] = full_result_header.getPositionByName(required_result_column_names[required_result_num]);
|
|
|
|
|
|
|
|
for (size_t query_num = 1; query_num < num_selects; ++query_num)
|
2018-02-27 20:16:58 +00:00
|
|
|
{
|
2018-02-27 20:43:42 +00:00
|
|
|
Block full_result_header_for_current_select = InterpreterSelectQuery(
|
2018-07-19 13:36:21 +00:00
|
|
|
ast.list_of_selects->children.at(query_num), context, Names(), to_stage, subquery_depth, nullptr, true).getSampleBlock();
|
2018-02-27 20:43:42 +00:00
|
|
|
|
|
|
|
if (full_result_header_for_current_select.columns() != full_result_header.columns())
|
|
|
|
throw Exception("Different number of columns in UNION ALL elements", ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
|
|
|
|
|
|
|
|
required_result_column_names_for_other_selects[query_num].reserve(required_result_column_names.size());
|
|
|
|
for (const auto & pos : positions_of_required_result_columns)
|
|
|
|
required_result_column_names_for_other_selects[query_num].push_back(full_result_header_for_current_select.getByPosition(pos).name);
|
2018-02-27 20:16:58 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for (size_t query_num = 0; query_num < num_selects; ++query_num)
|
|
|
|
{
|
2018-02-27 20:43:42 +00:00
|
|
|
const Names & current_required_result_column_names = query_num == 0
|
|
|
|
? required_result_column_names
|
|
|
|
: required_result_column_names_for_other_selects[query_num];
|
2018-02-27 20:16:58 +00:00
|
|
|
|
2018-02-27 20:43:42 +00:00
|
|
|
nested_interpreters.emplace_back(std::make_unique<InterpreterSelectQuery>(
|
2018-07-19 13:36:21 +00:00
|
|
|
ast.list_of_selects->children.at(query_num), context, current_required_result_column_names, to_stage, subquery_depth, nullptr, only_analyze));
|
2018-02-27 20:16:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Determine structure of result.
|
2018-02-26 06:12:59 +00:00
|
|
|
|
|
|
|
if (num_selects == 1)
|
|
|
|
{
|
|
|
|
result_header = nested_interpreters.front()->getSampleBlock();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
Blocks headers(num_selects);
|
|
|
|
for (size_t query_num = 0; query_num < num_selects; ++query_num)
|
|
|
|
headers[query_num] = nested_interpreters[query_num]->getSampleBlock();
|
|
|
|
|
|
|
|
result_header = headers.front();
|
|
|
|
size_t num_columns = result_header.columns();
|
|
|
|
|
|
|
|
for (size_t query_num = 1; query_num < num_selects; ++query_num)
|
|
|
|
if (headers[query_num].columns() != num_columns)
|
|
|
|
throw Exception("Different number of columns in UNION ALL elements", ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
|
|
|
|
|
|
|
|
for (size_t column_num = 0; column_num < num_columns; ++column_num)
|
|
|
|
{
|
|
|
|
ColumnWithTypeAndName & result_elem = result_header.getByPosition(column_num);
|
|
|
|
|
|
|
|
/// Determine common type.
|
|
|
|
|
|
|
|
DataTypes types(num_selects);
|
|
|
|
for (size_t query_num = 0; query_num < num_selects; ++query_num)
|
|
|
|
types[query_num] = headers[query_num].getByPosition(column_num).type;
|
|
|
|
|
|
|
|
result_elem.type = getLeastSupertype(types);
|
|
|
|
|
|
|
|
/// If there are different constness or different values of constants, the result must be non-constant.
|
|
|
|
|
|
|
|
if (result_elem.column->isColumnConst())
|
|
|
|
{
|
|
|
|
bool need_materialize = false;
|
|
|
|
for (size_t query_num = 1; query_num < num_selects; ++query_num)
|
|
|
|
{
|
|
|
|
const ColumnWithTypeAndName & source_elem = headers[query_num].getByPosition(column_num);
|
|
|
|
|
|
|
|
if (!source_elem.column->isColumnConst()
|
|
|
|
|| (static_cast<const ColumnConst &>(*result_elem.column).getField()
|
|
|
|
!= static_cast<const ColumnConst &>(*source_elem.column).getField()))
|
|
|
|
{
|
|
|
|
need_materialize = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (need_materialize)
|
|
|
|
result_elem.column = result_elem.type->createColumn();
|
|
|
|
}
|
|
|
|
|
|
|
|
/// BTW, result column names are from first SELECT.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-02-27 20:16:58 +00:00
|
|
|
InterpreterSelectWithUnionQuery::~InterpreterSelectWithUnionQuery() = default;
|
|
|
|
|
|
|
|
|
2018-02-25 00:50:53 +00:00
|
|
|
Block InterpreterSelectWithUnionQuery::getSampleBlock()
|
|
|
|
{
|
2018-02-26 06:12:59 +00:00
|
|
|
return result_header;
|
2018-02-25 00:50:53 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Block InterpreterSelectWithUnionQuery::getSampleBlock(
|
|
|
|
const ASTPtr & query_ptr,
|
|
|
|
const Context & context)
|
|
|
|
{
|
2018-05-30 19:23:15 +00:00
|
|
|
auto & cache = context.getSampleBlockCache();
|
|
|
|
/// Using query string because query_ptr changes for every internal SELECT
|
|
|
|
auto key = queryToString(query_ptr);
|
|
|
|
if (cache.find(key) != cache.end())
|
|
|
|
{
|
|
|
|
return cache[key];
|
|
|
|
}
|
|
|
|
|
2018-07-19 13:36:21 +00:00
|
|
|
return cache[key] = InterpreterSelectWithUnionQuery(query_ptr, context, {}, QueryProcessingStage::Complete, 0, true).getSampleBlock();
|
2018-02-25 00:50:53 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams()
|
|
|
|
{
|
|
|
|
BlockInputStreams nested_streams;
|
|
|
|
|
|
|
|
for (auto & interpreter : nested_interpreters)
|
|
|
|
{
|
|
|
|
BlockInputStreams streams = interpreter->executeWithMultipleStreams();
|
|
|
|
nested_streams.insert(nested_streams.end(), streams.begin(), streams.end());
|
|
|
|
}
|
|
|
|
|
2018-02-26 09:10:11 +00:00
|
|
|
/// Unify data structure.
|
|
|
|
if (nested_interpreters.size() > 1)
|
|
|
|
for (auto & stream : nested_streams)
|
|
|
|
stream = std::make_shared<ConvertingBlockInputStream>(context, stream, result_header, ConvertingBlockInputStream::MatchColumnsMode::Position);
|
|
|
|
|
2018-02-25 00:50:53 +00:00
|
|
|
return nested_streams;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
BlockIO InterpreterSelectWithUnionQuery::execute()
|
|
|
|
{
|
2018-02-28 02:33:04 +00:00
|
|
|
const Settings & settings = context.getSettingsRef();
|
|
|
|
|
2018-02-25 00:50:53 +00:00
|
|
|
BlockInputStreams nested_streams = executeWithMultipleStreams();
|
|
|
|
BlockInputStreamPtr result_stream;
|
|
|
|
|
|
|
|
if (nested_streams.empty())
|
|
|
|
{
|
2018-02-25 06:34:20 +00:00
|
|
|
result_stream = std::make_shared<NullBlockInputStream>(getSampleBlock());
|
2018-02-25 00:50:53 +00:00
|
|
|
}
|
|
|
|
else if (nested_streams.size() == 1)
|
|
|
|
{
|
|
|
|
result_stream = nested_streams.front();
|
2018-02-25 06:34:20 +00:00
|
|
|
nested_streams.clear();
|
2018-02-25 00:50:53 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-02-28 18:35:49 +00:00
|
|
|
result_stream = std::make_shared<UnionBlockInputStream<>>(nested_streams, nullptr, settings.max_threads);
|
2018-02-25 00:50:53 +00:00
|
|
|
nested_streams.clear();
|
|
|
|
}
|
|
|
|
|
|
|
|
BlockIO res;
|
|
|
|
res.in = result_stream;
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2018-02-25 06:34:20 +00:00
|
|
|
|
|
|
|
void InterpreterSelectWithUnionQuery::ignoreWithTotals()
|
|
|
|
{
|
|
|
|
for (auto & interpreter : nested_interpreters)
|
|
|
|
interpreter->ignoreWithTotals();
|
|
|
|
}
|
|
|
|
|
2018-02-25 00:50:53 +00:00
|
|
|
}
|