提交 224bc4df 编写于 作者: A Alexey Milovidov

Minor modifications + a comment #3796

上级 b045e854
......@@ -10,7 +10,7 @@
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/ThreadPool.h>
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
#include <Storages/StorageBlock.h>
#include <Storages/StorageBlockInserted.h>
namespace DB
{
......@@ -183,7 +183,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n
/// but it will contain single block (that is INSERT-ed into main table).
/// InterpreterSelectQuery will do processing of alias columns.
Context local_context = *views_context;
local_context.addViewSource(StorageBlock::create(block, storage));
local_context.addViewSource(StorageBlockInserted::create(block, storage));
InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions());
BlockInputStreamPtr in = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
......
......@@ -133,7 +133,7 @@ private:
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
TableAndCreateASTs external_tables; /// Temporary tables.
StoragePtr view_source; /// Temporary StorageBlock used to generate alias columns for materialized views
StoragePtr view_source; /// Temporary StorageBlockInserted used to generate alias columns for materialized views
Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id.
Context * query_context = nullptr;
Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this.
......
......@@ -48,7 +48,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
#include <Storages/IStorage.h>
#include <Storages/StorageBlock.h>
#include <Storages/StorageBlockInserted.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
......@@ -274,7 +274,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (auto view_source = context.getViewSource())
{
auto & storage_block = static_cast<const StorageBlock &>(*view_source);
auto & storage_block = static_cast<const StorageBlockInserted &>(*view_source);
if (storage_block.getDatabaseName() == database_name && storage_block.getTableName() == table_name)
{
/// Read from view source.
......
#pragma once
#include <ext/shared_ptr_helper.h>
#include <Core/NamesAndTypes.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Storages/IStorage.h>
namespace DB
{
/// Allows to look at the Block that was inserted or is going to be inserted
/// to a table as if we read this block from that table.
/// The table has all the properties of another storage,
/// but will read single prepared block instead.
/// Used in PushingToViewsBlockOutputStream for generating alias columns
/// NOTE: Some of the properties seems redundant.
class StorageBlockInserted : public ext::shared_ptr_helper<StorageBlockInserted>, public IStorage
{
public:
std::string getName() const override { return storage->getName(); }
std::string getTableName() const override { return storage->getTableName(); }
std::string getDatabaseName() const override { return storage->getDatabaseName(); }
bool isRemote() const override { return storage->isRemote(); }
bool supportsSampling() const override { return storage->supportsSampling(); }
bool supportsFinal() const override { return storage->supportsFinal(); }
bool supportsPrewhere() const override { return storage->supportsPrewhere(); }
bool supportsReplication() const override { return storage->supportsReplication(); }
bool supportsDeduplication() const override { return storage->supportsDeduplication(); }
bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); }
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override
{
return storage->mayBenefitFromIndexForIn(left_in_operand, query_context);
}
ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); }
ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); }
ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); }
ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); }
Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); }
Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); }
Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); }
Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); }
Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); }
BlockInputStreams read(const Names &, const SelectQueryInfo &, const Context &, QueryProcessingStage::Enum, size_t, unsigned) override
{
return {std::make_shared<OneBlockInputStream>(std::move(block))};
}
private:
Block block;
StoragePtr storage;
protected:
StorageBlockInserted(Block block_, StoragePtr storage_)
: IStorage{storage_->getColumns()}, block(std::move(block_)), storage(storage_)
{
}
};
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册