getSampleBlockWithVirtuals in StorageInMemoryMetadata

This commit is contained in:
alesapin 2020-06-16 15:58:05 +03:00
parent 53cb5210de
commit 08b9aa6b2e
7 changed files with 48 additions and 24 deletions

View File

@ -19,8 +19,14 @@ namespace DB
PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
const StoragePtr & storage_,
const Context & context_, const ASTPtr & query_ptr_, bool no_destination)
: storage(storage_), context(context_), query_ptr(query_ptr_)
const StorageMetadataPtr & metadata_snapshot_,
const Context & context_,
const ASTPtr & query_ptr_,
bool no_destination)
: storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, context(context_)
, query_ptr(query_ptr_)
{
/** TODO This is a very important line. At any insertion into the table one of streams should own lock.
* Although now any insertion into the table is done via PushingToViewsBlockOutputStream,
@ -60,6 +66,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
for (const auto & database_table : dependencies)
{
auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context);
auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr();
ASTPtr query;
BlockOutputStreamPtr out;
@ -97,9 +104,11 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
out = io.out;
}
else if (dynamic_cast<const StorageLiveView *>(dependent_table.get()))
out = std::make_shared<PushingToViewsBlockOutputStream>(dependent_table, *insert_context, ASTPtr(), true);
out = std::make_shared<PushingToViewsBlockOutputStream>(
dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr(), true);
else
out = std::make_shared<PushingToViewsBlockOutputStream>(dependent_table, *insert_context, ASTPtr());
out = std::make_shared<PushingToViewsBlockOutputStream>(
dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr());
views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr});
}
@ -118,9 +127,9 @@ Block PushingToViewsBlockOutputStream::getHeader() const
/// If we don't write directly to the destination
/// then expect that we're inserting with precalculated virtual columns
if (output)
return storage->getSampleBlock();
return metadata_snapshot->getSampleBlock();
else
return storage->getSampleBlockWithVirtuals();
return metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals());
}

View File

@ -17,8 +17,12 @@ class ReplicatedMergeTreeBlockOutputStream;
class PushingToViewsBlockOutputStream : public IBlockOutputStream
{
public:
PushingToViewsBlockOutputStream(const StoragePtr & storage_,
const Context & context_, const ASTPtr & query_ptr_, bool no_destination = false);
PushingToViewsBlockOutputStream(
const StoragePtr & storage_,
const StorageMetadataPtr & metadata_snapshot_,
const Context & context_,
const ASTPtr & query_ptr_,
bool no_destination = false);
Block getHeader() const override;
void write(const Block & block) override;
@ -29,6 +33,7 @@ public:
private:
StoragePtr storage;
StorageMetadataPtr metadata_snapshot;
BlockOutputStreamPtr output;
ReplicatedMergeTreeBlockOutputStream * replicated_output = nullptr;

View File

@ -83,7 +83,7 @@ Block InterpreterInsertQuery::getSampleBlock(
if (!query.columns)
{
if (no_destination)
return table->getSampleBlockWithVirtuals();
return metadata_snapshot->getSampleBlockWithVirtuals(table->getVirtuals());
else
return table_sample_non_materialized;
}
@ -232,7 +232,7 @@ BlockIO InterpreterInsertQuery::execute()
if (table->noPushingToViews() && !no_destination)
out = table->write(query_ptr, metadata_snapshot, context);
else
out = std::make_shared<PushingToViewsBlockOutputStream>(table, context, query_ptr, no_destination);
out = std::make_shared<PushingToViewsBlockOutputStream>(table, metadata_snapshot, context, query_ptr, no_destination);
/// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order.

View File

@ -62,19 +62,6 @@ Block IStorage::getSampleBlock() const
return res;
}
Block IStorage::getSampleBlockWithVirtuals() const
{
auto res = getSampleBlock();
/// Virtual columns must be appended after ordinary, because user can
/// override them.
for (const auto & column : getVirtuals())
res.insert({column.type->createColumn(), column.type, column.name});
return res;
}
Block IStorage::getSampleBlockForColumns(const Names & column_names) const
{
Block res;

View File

@ -159,7 +159,6 @@ public: /// thread-unsafe part. lockStructure must be acquired
void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_) { metadata = std::make_shared<StorageInMemoryMetadata>(metadata_); }
Block getSampleBlock() const; /// ordinary + materialized.
Block getSampleBlockWithVirtuals() const; /// ordinary + materialized + virtuals.
Block getSampleBlockForColumns(const Names & column_names) const; /// ordinary + materialized + aliases + virtuals.
/// Verify that all the requested names are in the table and are set correctly:

View File

@ -218,4 +218,26 @@ Block StorageInMemoryMetadata::getSampleBlockNonMaterialized() const
return res;
}
Block StorageInMemoryMetadata::getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const
{
auto res = getSampleBlock();
/// Virtual columns must be appended after ordinary, because user can
/// override them.
for (const auto & column : virtuals)
res.insert({column.type->createColumn(), column.type, column.name});
return res;
}
Block StorageInMemoryMetadata::getSampleBlock() const
{
Block res;
for (const auto & column : getColumns().getAllPhysical())
res.insert({column.type->createColumn(), column.type, column.name});
return res;
}
}

View File

@ -107,7 +107,9 @@ struct StorageInMemoryMetadata
/// indices, TTL expressions) if we update @updated_columns set of columns.
ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const;
Block getSampleBlock() const; /// ordinary + materialized.
Block getSampleBlockNonMaterialized() const; /// ordinary.
Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; /// ordinary + materialized + virtuals.
};
using StorageMetadataPtr = std::shared_ptr<StorageInMemoryMetadata>;