mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
Fix explain for ISourceStep.
This commit is contained in:
parent
dea90009e3
commit
576ffadb17
@ -90,6 +90,8 @@ std::unique_ptr<QueryPlan> createLocalPlan(
|
||||
auto converting = std::make_unique<ConvertingStep>(query_plan->getCurrentDataStream(), header, true);
|
||||
converting->setStepDescription("Convert block structure for query from local replica");
|
||||
query_plan->addStep(std::move(converting));
|
||||
|
||||
return query_plan;
|
||||
}
|
||||
|
||||
String formattedAST(const ASTPtr & ast)
|
||||
|
@ -39,7 +39,7 @@ public:
|
||||
const String & query, const ASTPtr & query_ast,
|
||||
const Context & context, const ThrottlerPtr & throttler,
|
||||
const SelectQueryInfo & query_info,
|
||||
std::vector<QueryPlanPtr> & res,
|
||||
std::vector<QueryPlanPtr> & plans,
|
||||
Pipes & remote_pipes,
|
||||
Pipes & delayed_pipes) override;
|
||||
|
||||
|
@ -1107,9 +1107,9 @@ static StreamLocalLimits getLimitsForStorage(const Settings & settings, const Se
|
||||
return limits;
|
||||
}
|
||||
|
||||
static void addEmptySource(QueryPlan & query_plan, const Block & header, SelectQueryInfo & query_info)
|
||||
void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info)
|
||||
{
|
||||
Pipe pipe(std::make_shared<NullSource>(header));
|
||||
Pipe pipe(std::make_shared<NullSource>(source_header));
|
||||
|
||||
if (query_info.prewhere_info)
|
||||
{
|
||||
@ -1511,7 +1511,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
{
|
||||
auto header = metadata_snapshot->getSampleBlockForColumns(
|
||||
required_columns, storage->getVirtuals(), storage->getStorageID());
|
||||
addEmptySource(query_plan, header, query_info);
|
||||
addEmptySourceToQueryPlan(query_plan, header, query_info);
|
||||
}
|
||||
|
||||
/// Extend lifetime of context, table lock, storage. Set limits and quota.
|
||||
|
@ -94,6 +94,8 @@ public:
|
||||
|
||||
const SelectQueryInfo & getQueryInfo() const { return query_info; }
|
||||
|
||||
static void addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info);
|
||||
|
||||
private:
|
||||
InterpreterSelectQuery(
|
||||
const ASTPtr & query_ptr_,
|
||||
|
@ -14,7 +14,8 @@ QueryPipelinePtr ISourceStep::updatePipeline(QueryPipelines)
|
||||
auto pipeline = std::make_unique<QueryPipeline>();
|
||||
QueryPipelineProcessorsCollector collector(*pipeline, this);
|
||||
initializePipeline(*pipeline);
|
||||
processors = collector.detachProcessors();
|
||||
auto added_processors = collector.detachProcessors();
|
||||
processors.insert(processors.end(), added_processors.begin(), added_processors.end());
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
|
@ -16,7 +16,7 @@ public:
|
||||
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
protected:
|
||||
/// We collect processors got after pipeline transformation.
|
||||
Processors processors;
|
||||
};
|
||||
|
@ -5,7 +5,7 @@ namespace DB
|
||||
{
|
||||
|
||||
ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr<Context> context_)
|
||||
: ISourceStep(DataStream{.header = pipe_.getHeader(), .has_single_port = true})
|
||||
: ISourceStep(DataStream{.header = pipe_.getHeader()})
|
||||
, pipe(std::move(pipe_))
|
||||
, context(std::move(context_))
|
||||
{
|
||||
@ -13,6 +13,9 @@ ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr<Conte
|
||||
|
||||
void ReadFromPreparedSource::initializePipeline(QueryPipeline & pipeline)
|
||||
{
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
|
||||
if (context)
|
||||
|
@ -23,8 +23,8 @@ private:
|
||||
class ReadFromStorageStep : public ReadFromPreparedSource
|
||||
{
|
||||
public:
|
||||
ReadFromStorageStep(Pipe pipe, String storage_name)
|
||||
: ReadFromPreparedSource(std::move(pipe))
|
||||
ReadFromStorageStep(Pipe pipe_, String storage_name)
|
||||
: ReadFromPreparedSource(std::move(pipe_))
|
||||
{
|
||||
setStepDescription(storage_name);
|
||||
}
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -103,8 +104,16 @@ void IStorage::read(
|
||||
unsigned num_streams)
|
||||
{
|
||||
auto pipe = read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
|
||||
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), getName());
|
||||
query_plan.addStep(std::move(read_step));
|
||||
if (pipe.empty())
|
||||
{
|
||||
auto header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID());
|
||||
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), getName());
|
||||
query_plan.addStep(std::move(read_step));
|
||||
}
|
||||
}
|
||||
|
||||
Pipe IStorage::alterPartition(
|
||||
|
@ -495,6 +495,20 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con
|
||||
}
|
||||
|
||||
Pipe StorageDistributed::read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
const size_t max_block_size,
|
||||
const unsigned num_streams)
|
||||
{
|
||||
QueryPlan plan;
|
||||
read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
|
||||
return QueryPipeline::getPipe(std::move(*plan.buildQueryPipeline()));
|
||||
}
|
||||
|
||||
void StorageDistributed::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
|
@ -77,7 +77,7 @@ public:
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
Pipe StorageDistributed::read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
|
Loading…
Reference in New Issue
Block a user