mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
ISSUES-3134 fix merge and distributed engine query stage
This commit is contained in:
parent
882125dd6d
commit
80d83bb50f
@ -21,6 +21,7 @@
|
|||||||
#include <Columns/ColumnString.h>
|
#include <Columns/ColumnString.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Databases/IDatabase.h>
|
#include <Databases/IDatabase.h>
|
||||||
|
#include <Interpreters/SettingsCommon.h>
|
||||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||||
|
|
||||||
|
|
||||||
@ -203,6 +204,12 @@ BlockInputStreams StorageMerge::read(
|
|||||||
*/
|
*/
|
||||||
StorageListWithLocks selected_tables = getSelectedTables(query_info.query, has_table_virtual_column, true);
|
StorageListWithLocks selected_tables = getSelectedTables(query_info.query, has_table_virtual_column, true);
|
||||||
|
|
||||||
|
if (selected_tables.empty())
|
||||||
|
{
|
||||||
|
BlockInputStreams streams{std::make_shared<NullBlockInputStream>(header)};
|
||||||
|
return streams;
|
||||||
|
}
|
||||||
|
|
||||||
size_t remaining_streams = num_streams;
|
size_t remaining_streams = num_streams;
|
||||||
size_t tables_count = selected_tables.size();
|
size_t tables_count = selected_tables.size();
|
||||||
|
|
||||||
@ -220,16 +227,16 @@ BlockInputStreams StorageMerge::read(
|
|||||||
if (current_streams)
|
if (current_streams)
|
||||||
{
|
{
|
||||||
source_streams = createSourceStreams(
|
source_streams = createSourceStreams(
|
||||||
query_info, processed_stage, max_block_size, real_column_names, modified_context, header, storage,
|
query_info, processed_stage, max_block_size, modified_context, header, storage,
|
||||||
struct_lock, current_streams, has_table_virtual_column);
|
struct_lock, real_column_names, current_streams, has_table_virtual_column);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
source_streams.emplace_back(std::make_shared<LazyBlockInputStream>(header, [=]() -> BlockInputStreamPtr
|
source_streams.emplace_back(std::make_shared<LazyBlockInputStream>(header, [=, &real_column_names]() -> BlockInputStreamPtr
|
||||||
{
|
{
|
||||||
BlockInputStreams streams = createSourceStreams(query_info, processed_stage, max_block_size, real_column_names,
|
BlockInputStreams streams = createSourceStreams(query_info, processed_stage, max_block_size,
|
||||||
modified_context, header, storage,
|
modified_context, header, storage,
|
||||||
struct_lock, current_streams, has_table_virtual_column);
|
struct_lock, real_column_names, current_streams, has_table_virtual_column);
|
||||||
|
|
||||||
if (streams.size() != 1)
|
if (streams.size() != 1)
|
||||||
throw Exception("LogicalError: the lazy stream size must to be one.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("LogicalError: the lazy stream size must to be one.", ErrorCodes::LOGICAL_ERROR);
|
||||||
@ -249,10 +256,10 @@ BlockInputStreams StorageMerge::read(
|
|||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage,
|
BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage,
|
||||||
const size_t max_block_size, const Names & real_column_names,
|
const size_t max_block_size, const Context & modified_context,
|
||||||
const Context & modified_context, const Block & header, const StoragePtr & storage,
|
const Block & header, const StoragePtr & storage,
|
||||||
const TableStructureReadLockPtr & struct_lock, size_t current_streams,
|
const TableStructureReadLockPtr & struct_lock, Names & real_column_names,
|
||||||
bool has_table_virtual_column) const
|
size_t current_streams, bool has_table_virtual_column)
|
||||||
{
|
{
|
||||||
SelectQueryInfo modified_query_info;
|
SelectQueryInfo modified_query_info;
|
||||||
modified_query_info.query = query_info.query->clone();
|
modified_query_info.query = query_info.query->clone();
|
||||||
@ -264,6 +271,10 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
|
|||||||
|
|
||||||
if (processed_stage <= storage->getQueryProcessingStage(modified_context))
|
if (processed_stage <= storage->getQueryProcessingStage(modified_context))
|
||||||
{
|
{
|
||||||
|
/// If there are only virtual columns in query, you must request at least one other column.
|
||||||
|
if (real_column_names.size() ==0)
|
||||||
|
real_column_names.push_back(ExpressionActions::getSmallestColumn(storage->getColumns().getAllPhysical()));
|
||||||
|
|
||||||
source_streams = storage->read(real_column_names, modified_query_info, modified_context, processed_stage, max_block_size,
|
source_streams = storage->read(real_column_names, modified_query_info, modified_context, processed_stage, max_block_size,
|
||||||
current_streams);
|
current_streams);
|
||||||
}
|
}
|
||||||
@ -282,6 +293,12 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
|
|||||||
source_streams.emplace_back(std::make_shared<MaterializingBlockInputStream>(interpreter_stream));
|
source_streams.emplace_back(std::make_shared<MaterializingBlockInputStream>(interpreter_stream));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (source_streams.empty())
|
||||||
|
{
|
||||||
|
source_streams.emplace_back(std::make_shared<OneBlockInputStream>(header));
|
||||||
|
return source_streams;
|
||||||
|
}
|
||||||
|
|
||||||
if (!current_streams)
|
if (!current_streams)
|
||||||
{
|
{
|
||||||
BlockInputStreamPtr stream =
|
BlockInputStreamPtr stream =
|
||||||
|
@ -75,9 +75,9 @@ protected:
|
|||||||
const Context & context, QueryProcessingStage::Enum processed_stage);
|
const Context & context, QueryProcessingStage::Enum processed_stage);
|
||||||
|
|
||||||
BlockInputStreams createSourceStreams(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage,
|
BlockInputStreams createSourceStreams(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage,
|
||||||
const size_t max_block_size, const Names &real_column_names, const Context & modified_context,
|
const size_t max_block_size, const Context & modified_context,
|
||||||
const Block & header, const StoragePtr & storage, const TableStructureReadLockPtr & struct_lock,
|
const Block & header, const StoragePtr & storage, const TableStructureReadLockPtr & struct_lock,
|
||||||
size_t current_streams, bool has_table_virtual_column) const;
|
Names & real_column_names, size_t current_streams, bool has_table_virtual_column);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -14,7 +14,8 @@
|
|||||||
1:
|
1:
|
||||||
1
|
1
|
||||||
1
|
1
|
||||||
-1:
|
-1:
|
||||||
|
18446744073709551615
|
||||||
Int32 | UInt64
|
Int32 | UInt64
|
||||||
1
|
1
|
||||||
1
|
1
|
||||||
|
@ -3,6 +3,3 @@
|
|||||||
1000
|
1000
|
||||||
1000
|
1000
|
||||||
1000
|
1000
|
||||||
0
|
|
||||||
0
|
|
||||||
0
|
|
||||||
|
Loading…
Reference in New Issue
Block a user