mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 02:21:59 +00:00
Allow UNION in matviews
This commit is contained in:
parent
1aceb608f3
commit
9c0683e157
@ -11,7 +11,7 @@
|
||||
#include <Processors/Transforms/PlanSquashingTransform.h>
|
||||
#include <Processors/Transforms/SquashingTransform.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
#include <Storages/WindowView/StorageWindowView.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeSink.h>
|
||||
@ -148,7 +148,7 @@ private:
|
||||
struct State
|
||||
{
|
||||
QueryPipeline pipeline;
|
||||
PullingPipelineExecutor executor;
|
||||
PullingAsyncPipelineExecutor executor;
|
||||
|
||||
explicit State(QueryPipeline pipeline_)
|
||||
: pipeline(std::move(pipeline_))
|
||||
|
@ -64,8 +64,6 @@ StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, ContextPt
|
||||
ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW,
|
||||
"StorageMaterializedView cannot be created from table functions ({})",
|
||||
serializeAST(*subquery));
|
||||
if (ast_select->list_of_selects->children.size() != 1)
|
||||
throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "UNION is not supported for MATERIALIZED VIEW");
|
||||
|
||||
auto & inner_query = ast_select->list_of_selects->children.at(0);
|
||||
|
||||
@ -75,69 +73,47 @@ StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, ContextPt
|
||||
}
|
||||
|
||||
|
||||
void checkAllowedQueries(const ASTSelectQuery & query)
|
||||
void checkAllowedQueries(const ASTSelectWithUnionQuery & select)
|
||||
{
|
||||
if (query.prewhere() || query.final() || query.sampleSize())
|
||||
throw Exception(DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL.");
|
||||
|
||||
ASTPtr subquery = extractTableExpression(query, 0);
|
||||
if (!subquery)
|
||||
return;
|
||||
|
||||
if (const auto * ast_select = subquery->as<ASTSelectWithUnionQuery>())
|
||||
for (const auto & children : select.list_of_selects->children)
|
||||
{
|
||||
if (ast_select->list_of_selects->children.size() != 1)
|
||||
throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "UNION is not supported for MATERIALIZED VIEW");
|
||||
auto * query = children->as<ASTSelectQuery>();
|
||||
|
||||
const auto & inner_query = ast_select->list_of_selects->children.at(0);
|
||||
if (query->prewhere() || query->final() || query->sampleSize())
|
||||
throw Exception(DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL.");
|
||||
|
||||
checkAllowedQueries(inner_query->as<ASTSelectQuery &>());
|
||||
ASTPtr subquery = extractTableExpression(*query, 0);
|
||||
if (!subquery)
|
||||
return;
|
||||
|
||||
if (const auto * ast_select_with_union = subquery->as<ASTSelectWithUnionQuery>())
|
||||
{
|
||||
checkAllowedQueries(*ast_select_with_union);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/// check if only one single select query in SelectWithUnionQuery
|
||||
static bool isSingleSelect(const ASTPtr & select, ASTPtr & res)
|
||||
{
|
||||
auto * new_select = select->as<ASTSelectWithUnionQuery>();
|
||||
if (new_select == nullptr)
|
||||
return false;
|
||||
|
||||
if (new_select->list_of_selects->children.size() != 1)
|
||||
return false;
|
||||
|
||||
auto & new_inner_query = new_select->list_of_selects->children.at(0);
|
||||
if (new_inner_query->as<ASTSelectQuery>())
|
||||
{
|
||||
res = new_inner_query;
|
||||
return true;
|
||||
}
|
||||
|
||||
return isSingleSelect(new_inner_query, res);
|
||||
}
|
||||
|
||||
SelectQueryDescription SelectQueryDescription::getSelectQueryFromASTForMatView(const ASTPtr & select, bool refreshable, ContextPtr context)
|
||||
{
|
||||
SelectQueryDescription result;
|
||||
result.select_query = select->as<ASTSelectWithUnionQuery &>().clone();
|
||||
|
||||
ASTSelectWithUnionQuery & query = result.select_query->as<ASTSelectWithUnionQuery &>();
|
||||
|
||||
/// Skip all the checks, none of them apply to refreshable views.
|
||||
/// Don't assign select_table_id. This way no materialized view dependency gets registered,
|
||||
/// so data doesn't get pushed to the refreshable view on source table inserts.
|
||||
if (refreshable)
|
||||
return result;
|
||||
|
||||
ASTPtr new_inner_query;
|
||||
|
||||
if (!isSingleSelect(select, new_inner_query))
|
||||
throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "UNION is not supported for MATERIALIZED VIEW");
|
||||
|
||||
auto & select_query = new_inner_query->as<ASTSelectQuery &>();
|
||||
checkAllowedQueries(select_query);
|
||||
|
||||
result.select_table_id = extractDependentTableFromSelectQuery(select_query, context);
|
||||
result.inner_query = new_inner_query->clone();
|
||||
checkAllowedQueries(query);
|
||||
/// We trigger only for the first found table
|
||||
ASTSelectQuery & new_inner_query = query.list_of_selects->children.at(0)->as<ASTSelectQuery &>();
|
||||
/// Extracting first found table ID
|
||||
result.select_table_id = extractDependentTableFromSelectQuery(new_inner_query, context);
|
||||
result.inner_query = new_inner_query.clone();
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -0,0 +1,6 @@
|
||||
4000
|
||||
10 0
|
||||
10 1
|
||||
10 2
|
||||
10 3
|
||||
10 4
|
82
tests/queries/0_stateless/03275_matview_with_union.sql
Normal file
82
tests/queries/0_stateless/03275_matview_with_union.sql
Normal file
@ -0,0 +1,82 @@
|
||||
DROP TABLE IF EXISTS src;
|
||||
DROP TABLE IF EXISTS dst;
|
||||
DROP TABLE IF EXISTS matview;
|
||||
|
||||
CREATE TABLE src (
|
||||
event_time DateTime,
|
||||
key UInt64,
|
||||
value Int64
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY (event_time, key);
|
||||
|
||||
CREATE TABLE dst (
|
||||
step UInt16,
|
||||
rounded_event_time DateTime,
|
||||
key UInt64,
|
||||
value AggregateFunction(max, Int64)
|
||||
)
|
||||
ENGINE = AggregatingMergeTree()
|
||||
ORDER BY (step, rounded_event_time, key);
|
||||
|
||||
CREATE MATERIALIZED VIEW matview TO dst
|
||||
(
|
||||
step UInt16,
|
||||
rounded_event_time DateTime,
|
||||
key UInt64,
|
||||
value AggregateFunction(max, Int64)
|
||||
) AS
|
||||
SELECT * FROM (
|
||||
SELECT
|
||||
1 AS step,
|
||||
key,
|
||||
intDiv(toUnixTimestamp(event_time), step) * step AS rounded_event_time,
|
||||
initializeAggregation('maxState', value) AS value
|
||||
FROM src
|
||||
ORDER BY
|
||||
rounded_event_time,
|
||||
key
|
||||
UNION ALL
|
||||
SELECT
|
||||
5 AS step,
|
||||
key,
|
||||
intDiv(toUnixTimestamp(event_time), step) * step AS rounded_event_time,
|
||||
initializeAggregation('maxState', value) AS value
|
||||
FROM src
|
||||
ORDER BY
|
||||
rounded_event_time,
|
||||
key
|
||||
UNION ALL
|
||||
SELECT
|
||||
15 AS step,
|
||||
key,
|
||||
intDiv(toUnixTimestamp(event_time), step) * step AS rounded_event_time,
|
||||
initializeAggregation('maxState', value) AS value
|
||||
FROM src
|
||||
ORDER BY
|
||||
rounded_event_time,
|
||||
key
|
||||
UNION ALL
|
||||
SELECT
|
||||
30 AS step,
|
||||
key,
|
||||
intDiv(toUnixTimestamp(event_time), step) * step AS rounded_event_time,
|
||||
initializeAggregation('maxState', value) AS value
|
||||
FROM src
|
||||
ORDER BY
|
||||
rounded_event_time,
|
||||
key
|
||||
)
|
||||
ORDER BY step, rounded_event_time, key SETTINGS query_plan_remove_redundant_sorting = 0;
|
||||
|
||||
set optimize_on_insert = 1;
|
||||
|
||||
INSERT INTO src SELECT toDateTime('2020-10-01 00:00:00') + number, number % 100, number from numbers(1000);
|
||||
|
||||
SELECT count() FROM dst;
|
||||
|
||||
SELECT count(), key FROM dst WHERE step = 30 group by key ORDER BY key LIMIT 5;
|
||||
|
||||
DROP TABLE IF EXISTS src;
|
||||
DROP TABLE IF EXISTS dst;
|
||||
DROP TABLE IF EXISTS matview;
|
Loading…
Reference in New Issue
Block a user