mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Review fixes and better test
This commit is contained in:
parent
30052ea46c
commit
69fdf2a6a3
@ -236,15 +236,6 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n
|
|||||||
|
|
||||||
if (view.query)
|
if (view.query)
|
||||||
{
|
{
|
||||||
/// We prepare columns set for our temporary storage from a single
|
|
||||||
/// block. It's union of columns from the block, and alias columns
|
|
||||||
/// of source storage, because block doesn't contain aliases.
|
|
||||||
ColumnsDescription columns(block.getNamesAndTypesList());
|
|
||||||
const auto & columns_from_storage = storage->getColumns();
|
|
||||||
for (const auto & column : columns_from_storage.getAliases())
|
|
||||||
if (!columns.has(column.name))
|
|
||||||
columns.add(columns_from_storage.get(column.name));
|
|
||||||
|
|
||||||
/// We create a table with the same name as original table and the same alias columns,
|
/// We create a table with the same name as original table and the same alias columns,
|
||||||
/// but it will contain single block (that is INSERT-ed into main table).
|
/// but it will contain single block (that is INSERT-ed into main table).
|
||||||
/// InterpreterSelectQuery will do processing of alias columns.
|
/// InterpreterSelectQuery will do processing of alias columns.
|
||||||
@ -252,7 +243,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n
|
|||||||
Context local_context = *views_context;
|
Context local_context = *views_context;
|
||||||
local_context.addViewSource(
|
local_context.addViewSource(
|
||||||
StorageValues::create(
|
StorageValues::create(
|
||||||
storage->getStorageID(), columns, block));
|
storage->getStorageID(), storage->getColumns(), block, storage->getVirtuals()));
|
||||||
select.emplace(view.query, local_context, SelectQueryOptions());
|
select.emplace(view.query, local_context, SelectQueryOptions());
|
||||||
in = std::make_shared<MaterializingBlockInputStream>(select->execute().in);
|
in = std::make_shared<MaterializingBlockInputStream>(select->execute().in);
|
||||||
|
|
||||||
|
@ -566,13 +566,14 @@ std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQu
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Add columns from storage to source_columns list. Deduplicate resulted list.
|
/// Add columns from storage to source_columns list. Deduplicate resulted list.
|
||||||
void SyntaxAnalyzerResult::collectSourceColumns(bool add_virtuals)
|
/// Special columns are non physical columns, for example ALIAS
|
||||||
|
void SyntaxAnalyzerResult::collectSourceColumns(bool add_special)
|
||||||
{
|
{
|
||||||
if (storage)
|
if (storage)
|
||||||
{
|
{
|
||||||
const ColumnsDescription & columns = storage->getColumns();
|
const ColumnsDescription & columns = storage->getColumns();
|
||||||
|
|
||||||
auto columns_from_storage = add_virtuals ? columns.getAll() : columns.getAllPhysical();
|
auto columns_from_storage = add_special ? columns.getAll() : columns.getAllPhysical();
|
||||||
if (source_columns.empty())
|
if (source_columns.empty())
|
||||||
source_columns.swap(columns_from_storage);
|
source_columns.swap(columns_from_storage);
|
||||||
else
|
else
|
||||||
|
@ -51,11 +51,11 @@ struct SyntaxAnalyzerResult
|
|||||||
|
|
||||||
bool maybe_optimize_trivial_count = false;
|
bool maybe_optimize_trivial_count = false;
|
||||||
|
|
||||||
SyntaxAnalyzerResult(const NamesAndTypesList & source_columns_, ConstStoragePtr storage_ = {}, bool add_virtuals = true)
|
SyntaxAnalyzerResult(const NamesAndTypesList & source_columns_, ConstStoragePtr storage_ = {}, bool add_special = true)
|
||||||
: storage(storage_)
|
: storage(storage_)
|
||||||
, source_columns(source_columns_)
|
, source_columns(source_columns_)
|
||||||
{
|
{
|
||||||
collectSourceColumns(add_virtuals);
|
collectSourceColumns(add_special);
|
||||||
}
|
}
|
||||||
|
|
||||||
void collectSourceColumns(bool add_virtuals);
|
void collectSourceColumns(bool add_virtuals);
|
||||||
|
@ -9,8 +9,12 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
StorageValues::StorageValues(const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_)
|
StorageValues::StorageValues(
|
||||||
: IStorage(table_id_), res_block(res_block_)
|
const StorageID & table_id_,
|
||||||
|
const ColumnsDescription & columns_,
|
||||||
|
const Block & res_block_,
|
||||||
|
const NamesAndTypesList & virtuals_)
|
||||||
|
: IStorage(table_id_), res_block(res_block_), virtuals(virtuals_)
|
||||||
{
|
{
|
||||||
setColumns(columns_);
|
setColumns(columns_);
|
||||||
}
|
}
|
||||||
|
@ -23,11 +23,20 @@ public:
|
|||||||
size_t max_block_size,
|
size_t max_block_size,
|
||||||
unsigned num_streams) override;
|
unsigned num_streams) override;
|
||||||
|
|
||||||
|
/// Why we may have virtual columns in the storage from a single block?
|
||||||
|
/// Because it used as tmp storage for pushing blocks into views, and some
|
||||||
|
/// views may contain virtual columns from original storage.
|
||||||
|
NamesAndTypesList getVirtuals() const override
|
||||||
|
{
|
||||||
|
return virtuals;
|
||||||
|
}
|
||||||
private:
|
private:
|
||||||
Block res_block;
|
Block res_block;
|
||||||
|
NamesAndTypesList virtuals;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
StorageValues(const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_);
|
StorageValues(
|
||||||
|
const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_, const NamesAndTypesList & virtuals_ = {});
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1 +1 @@
|
|||||||
2018-12-10 1 1
|
2018-12-10 1 1 3 5
|
||||||
|
@ -1,23 +1,28 @@
|
|||||||
DROP TABLE IF EXISTS test1;
|
DROP TABLE IF EXISTS table_view;
|
||||||
|
DROP TABLE IF EXISTS source_table;
|
||||||
|
|
||||||
CREATE TABLE test1 (
|
CREATE TABLE source_table (
|
||||||
date Date,
|
date Date,
|
||||||
datetime DateTime,
|
datetime DateTime,
|
||||||
zoneId UInt64,
|
zoneId UInt64,
|
||||||
test ALIAS zoneId == 1
|
test1 ALIAS zoneId == 1,
|
||||||
|
test2 DEFAULT zoneId * 3,
|
||||||
|
test3 MATERIALIZED zoneId * 5
|
||||||
) ENGINE = MergeTree(date, (date, zoneId), 8192);
|
) ENGINE = MergeTree(date, (date, zoneId), 8192);
|
||||||
|
|
||||||
CREATE MATERIALIZED VIEW test1_view
|
CREATE MATERIALIZED VIEW table_view
|
||||||
ENGINE = MergeTree(date, (date, zoneId), 8192)
|
ENGINE = MergeTree(date, (date, zoneId), 8192)
|
||||||
AS SELECT
|
AS SELECT
|
||||||
date,
|
date,
|
||||||
zoneId,
|
zoneId,
|
||||||
test
|
test1,
|
||||||
FROM test1;
|
test2,
|
||||||
|
test3
|
||||||
|
FROM source_table;
|
||||||
|
|
||||||
INSERT INTO test1 VALUES ('2018-12-10', '2018-12-10 23:59:59', 1);
|
INSERT INTO source_table (date, datetime, zoneId) VALUES ('2018-12-10', '2018-12-10 23:59:59', 1);
|
||||||
|
|
||||||
SELECT * from test1_view;
|
SELECT * from table_view;
|
||||||
|
|
||||||
DROP TABLE test1_view;
|
DROP TABLE IF EXISTS table_view;
|
||||||
DROP TABLE test1;
|
DROP TABLE IF EXISTS source_table;
|
||||||
|
Loading…
Reference in New Issue
Block a user