From f526ff2a5163550fe024fcfc321b4137acb21fe1 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 18 Jan 2022 09:03:16 +0000 Subject: [PATCH 1/5] remove unused variable --- src/Storages/WindowView/StorageWindowView.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index a81a5a9649a..d14ce7e1c52 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -200,7 +200,6 @@ namespace { String window_id_name; String window_id_alias; - Aliases * aliases; }; static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } From ba0a7028075a3501d2d1d7c729ac871cdf022271 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 18 Jan 2022 09:20:15 +0000 Subject: [PATCH 2/5] enable stream to table join in windowview --- src/Storages/LiveView/StorageBlocks.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 170 +++++++++--------- .../WindowView/WindowViewProxyStorage.h | 48 ----- ...7_window_view_parser_inner_table.reference | 10 +- .../01047_window_view_parser_inner_table.sql | 31 ++-- .../01048_window_view_parser.reference | 36 ++-- .../0_stateless/01048_window_view_parser.sql | 26 ++- ...indow_view_event_tumble_asc_join.reference | 3 + ...01071_window_view_event_tumble_asc_join.sh | 39 ++++ 9 files changed, 201 insertions(+), 164 deletions(-) delete mode 100644 src/Storages/WindowView/WindowViewProxyStorage.h create mode 100644 tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.reference create mode 100755 tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.sh diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index f010ef50ecd..c085e600d39 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -10,7 +10,7 @@ namespace DB class StorageBlocks : public IStorage { /* Storage based on the prepared streams that already contain data blocks. - * Used by Live Views to complete stored query based on the mergeable blocks. + * Used by Live and Window Views to complete stored query based on the mergeable blocks. */ public: StorageBlocks(const StorageID & table_id_, diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index d14ce7e1c52..be3cda1c980 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -40,9 +40,10 @@ #include #include +#include + #include #include -#include #include @@ -238,6 +239,23 @@ namespace } }; + struct DropTableIdentifierMatcher + { + using Visitor = InDepthNodeVisitor; + + struct Data{}; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data &) + { + if (auto * t = ast->as()) + { + ast = std::make_shared(t->shortName()); + } + } + }; + IntervalKind strToIntervalKind(const String& interval_str) { if (interval_str == "Second") @@ -462,16 +480,26 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) current_header, getWindowIDColumnPosition(current_header), window_column_name_and_type, window_value); }); + Pipes pipes; auto pipe = QueryPipelineBuilder::getPipe(std::move(builder)); - auto parent_table_metadata = getParentStorage()->getInMemoryMetadataPtr(); - auto required_columns = parent_table_metadata->getColumns(); - required_columns.add(ColumnDescription("____timestamp", std::make_shared())); - auto proxy_storage = std::make_shared( - StorageID(getStorageID().database_name, "WindowViewProxyStorage"), required_columns, - std::move(pipe), QueryProcessingStage::WithMergeableState); + pipes.emplace_back(std::move(pipe)); + + auto creator = [&](const StorageID & blocks_id_global) { + auto parent_table_metadata = getParentStorage()->getInMemoryMetadataPtr(); + auto required_columns = parent_table_metadata->getColumns(); + required_columns.add(ColumnDescription("____timestamp", std::make_shared())); + return StorageBlocks::createStorage(blocks_id_global, required_columns, std::move(pipes), QueryProcessingStage::WithMergeableState); + }; + + TemporaryTableHolder blocks_storage(window_view_context, creator); InterpreterSelectQuery select( - getFinalQuery(), window_view_context, proxy_storage, nullptr, SelectQueryOptions(QueryProcessingStage::Complete)); + getFinalQuery(), + window_view_context, + blocks_storage.getTable(), + blocks_storage.getTable()->getInMemoryMetadataPtr(), + SelectQueryOptions(QueryProcessingStage::Complete)); + builder = select.buildQueryPipeline(); builder.addSimpleTransform([&](const Block & current_header) @@ -576,16 +604,16 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( auto columns_list = std::make_shared(); + String window_id_column_name; if (is_time_column_func_now) { auto column_window = std::make_shared(); column_window->name = window_id_name; column_window->type = std::make_shared("UInt32"); columns_list->children.push_back(column_window); + window_id_column_name = window_id_name; } - bool has_window_id = false; - for (const auto & column : t_sample_block.getColumnsWithTypeAndName()) { ParserIdentifierWithOptionalParameters parser; @@ -595,28 +623,17 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( column_dec->name = column.name; column_dec->type = ast; columns_list->children.push_back(column_dec); - if (!is_time_column_func_now && !has_window_id) + if (!is_time_column_func_now && window_id_column_name.empty() && startsWith(column.name, "windowID")) { - if (startsWith(column.name, "windowID")) - has_window_id = true; + window_id_column_name = column.name; } } - if (!is_time_column_func_now && !has_window_id) + if (window_id_column_name.empty()) throw Exception( "The first argument of time window function should not be a constant value.", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); - ToIdentifierMatcher::Data query_data; - query_data.window_id_name = window_id_name; - query_data.window_id_alias = window_id_alias; - ToIdentifierMatcher::Visitor to_identifier_visitor(query_data); - - ReplaceFunctionNowData time_now_data; - ReplaceFunctionNowVisitor time_now_visitor(time_now_data); - ReplaceFunctionWindowMatcher::Data func_hop_data; - ReplaceFunctionWindowMatcher::Visitor func_window_visitor(func_hop_data); - auto new_storage = std::make_shared(); /// storage != nullptr in case create window view with ENGINE syntax if (storage) @@ -632,6 +649,19 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( "The ENGINE of WindowView must be MergeTree family of table engines " "including the engines with replication support"); + ToIdentifierMatcher::Data query_data; + query_data.window_id_name = window_id_name; + query_data.window_id_alias = window_id_alias; + ToIdentifierMatcher::Visitor to_identifier_visitor(query_data); + + ReplaceFunctionNowData time_now_data; + ReplaceFunctionNowVisitor time_now_visitor(time_now_data); + ReplaceFunctionWindowMatcher::Data func_hop_data; + ReplaceFunctionWindowMatcher::Visitor func_window_visitor(func_hop_data); + + DropTableIdentifierMatcher::Data drop_table_identifier_data; + DropTableIdentifierMatcher::Visitor drop_table_identifier_visitor(drop_table_identifier_data); + new_storage->set(new_storage->engine, storage->engine->clone()); auto visit = [&](const IAST * ast, IAST *& field) @@ -639,24 +669,22 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( if (ast) { auto node = ast->clone(); + QueryNormalizer(normalizer_data).visit(node); /// now() -> ____timestamp if (is_time_column_func_now) { time_now_visitor.visit(node); function_now_timezone = time_now_data.now_timezone; } + drop_table_identifier_visitor.visit(node); /// tumble/hop -> windowID func_window_visitor.visit(node); to_identifier_visitor.visit(node); - QueryNormalizer(normalizer_data).visit(node); node->setAlias(""); new_storage->set(field, node); } }; - for (auto & [alias_name, ast] : aliases) - ast = std::make_shared(ast->getColumnName()); - visit(storage->partition_by, new_storage->partition_by); visit(storage->primary_key, new_storage->primary_key); visit(storage->order_by, new_storage->order_by); @@ -669,35 +697,8 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( { new_storage->set(new_storage->engine, makeASTFunction("AggregatingMergeTree")); - for (auto & child : inner_select_query->groupBy()->children) - if (auto * ast_with_alias = dynamic_cast(child.get())) - ast_with_alias->setAlias(""); - - auto order_by = std::make_shared(); - order_by->name = "tuple"; - order_by->arguments = inner_select_query->groupBy(); - order_by->children.push_back(order_by->arguments); - - ASTPtr order_by_ptr = order_by; - if (is_time_column_func_now) - { - time_now_visitor.visit(order_by_ptr); - function_now_timezone = time_now_data.now_timezone; - } - to_identifier_visitor.visit(order_by_ptr); - - for (auto & child : order_by->arguments->children) - { - if (child->getColumnName() == window_id_name) - { - ASTPtr tmp = child; - child = order_by->arguments->children[0]; - order_by->arguments->children[0] = tmp; - break; - } - } - new_storage->set(new_storage->order_by, order_by_ptr); - new_storage->set(new_storage->primary_key, std::make_shared(window_id_name)); + new_storage->set(new_storage->order_by, std::make_shared(window_id_column_name)); + new_storage->set(new_storage->primary_key, std::make_shared(window_id_column_name)); } auto new_columns = std::make_shared(); @@ -1235,35 +1236,36 @@ void StorageWindowView::writeIntoWindowView( return std::make_shared(stream_header, adding_column_actions); }); } - - InterpreterSelectQuery select_block( - window_view.getMergeableQuery(), local_context, {std::move(pipe)}, - QueryProcessingStage::WithMergeableState); - - builder = select_block.buildQueryPipeline(); - builder.addSimpleTransform([&](const Block & current_header) - { - return std::make_shared( - current_header, - local_context->getSettingsRef().min_insert_block_size_rows, - local_context->getSettingsRef().min_insert_block_size_bytes); - }); } - else + + Pipes pipes; + pipes.emplace_back(std::move(pipe)); + + auto creator = [&](const StorageID & blocks_id_global) { + auto parent_metadata = window_view.getParentStorage()->getInMemoryMetadataPtr(); + auto required_columns = parent_metadata->getColumns(); + required_columns.add(ColumnDescription("____timestamp", std::make_shared())); + return StorageBlocks::createStorage(blocks_id_global, required_columns, std::move(pipes), QueryProcessingStage::FetchColumns); + }; + TemporaryTableHolder blocks_storage(local_context, creator); + + InterpreterSelectQuery select_block( + window_view.getMergeableQuery(), + local_context, + blocks_storage.getTable(), + blocks_storage.getTable()->getInMemoryMetadataPtr(), + QueryProcessingStage::WithMergeableState); + + builder = select_block.buildQueryPipeline(); + builder.addSimpleTransform([&](const Block & current_header) { + return std::make_shared( + current_header, + local_context->getSettingsRef().min_insert_block_size_rows, + local_context->getSettingsRef().min_insert_block_size_bytes); + }); + + if (!window_view.is_proctime) { - InterpreterSelectQuery select_block( - window_view.getMergeableQuery(), local_context, {std::move(pipe)}, - QueryProcessingStage::WithMergeableState); - - builder = select_block.buildQueryPipeline(); - builder.addSimpleTransform([&](const Block & current_header) - { - return std::make_shared( - current_header, - local_context->getSettingsRef().min_insert_block_size_rows, - local_context->getSettingsRef().min_insert_block_size_bytes); - }); - UInt32 block_max_timestamp = 0; if (window_view.is_watermark_bounded || window_view.allowed_lateness) { diff --git a/src/Storages/WindowView/WindowViewProxyStorage.h b/src/Storages/WindowView/WindowViewProxyStorage.h deleted file mode 100644 index 55426bddd5d..00000000000 --- a/src/Storages/WindowView/WindowViewProxyStorage.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class WindowViewProxyStorage : public IStorage -{ -public: - WindowViewProxyStorage(const StorageID & table_id_, ColumnsDescription columns_, Pipe pipe_, QueryProcessingStage::Enum to_stage_) - : IStorage(table_id_) - , pipe(std::move(pipe_)) - , to_stage(to_stage_) - { - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - setInMemoryMetadata(storage_metadata); - } - -public: - std::string getName() const override { return "WindowViewProxy"; } - - QueryProcessingStage::Enum - getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageMetadataPtr &, SelectQueryInfo &) const override - { - return to_stage; - } - - Pipe read( - const Names &, - const StorageMetadataPtr & /*metadata_snapshot*/, - SelectQueryInfo & /*query_info*/, - ContextPtr /*context*/, - QueryProcessingStage::Enum /*processed_stage*/, - size_t /*max_block_size*/, - unsigned /*num_streams*/) override - { - return std::move(pipe); - } - -private: - Pipe pipe; - QueryProcessingStage::Enum to_stage; -}; -} diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference index 96f7cbb1d69..319cbef0914 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference @@ -1,6 +1,4 @@ ---TUMBLE--- -||---DEFAULT ENGINE WITH DATA COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN NAME--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- @@ -13,9 +11,10 @@ CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(____timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +||---JOIN--- +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- -||---DEFAULT ENGINE WITH DATA COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN NAME--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- @@ -28,3 +27,6 @@ CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +||---JOIN--- +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index 595d93e0771..3e4f95d098e 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -5,16 +5,12 @@ DROP DATABASE IF EXISTS test_01047; CREATE DATABASE test_01047 ENGINE=Ordinary; DROP TABLE IF EXISTS test_01047.mt; +DROP TABLE IF EXISTS test_01047.mt_2; CREATE TABLE test_01047.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test_01047.mt_2(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---TUMBLE---'; -SELECT '||---DEFAULT ENGINE WITH DATA COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND); -SHOW CREATE TABLE test_01047.`.inner.wv`; - SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; @@ -51,14 +47,17 @@ DROP TABLE IF EXISTS test_01047.`.inner.wv`; CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, tumble(now(), INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; SHOW CREATE TABLE test_01047.`.inner.wv`; - -SELECT '---HOP---'; -SELECT '||---DEFAULT ENGINE WITH DATA COLUMN ALIAS---'; +SELECT '||---JOIN---'; DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND); +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; + + +SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; @@ -95,5 +94,15 @@ DROP TABLE IF EXISTS test_01047.`.inner.wv`; CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; SHOW CREATE TABLE test_01047.`.inner.wv`; +SELECT '||---JOIN---'; +DROP TABLE IF EXISTS test_01047.wv; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; + +DROP TABLE IF EXISTS test_01047.wv; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; + DROP TABLE test_01047.wv; DROP TABLE test_01047.mt; +DROP TABLE test_01047.mt_2; diff --git a/tests/queries/0_stateless/01048_window_view_parser.reference b/tests/queries/0_stateless/01048_window_view_parser.reference index 6625313f572..c055971bef3 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.reference +++ b/tests/queries/0_stateless/01048_window_view_parser.reference @@ -1,26 +1,34 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1))`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(1))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1))`\nORDER BY `windowID(timestamp, toIntervalSecond(1))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---TimeZone--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nSETTINGS index_granularity = 8192 +||---DATA COLUMN ALIAS--- +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +||---JOIN--- +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---TimeZone--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nORDER BY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nSETTINGS index_granularity = 8192 +||---DATA COLUMN ALIAS--- +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +||---JOIN--- +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_parser.sql b/tests/queries/0_stateless/01048_window_view_parser.sql index 3f57f6fbd91..e17352205e3 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_parser.sql @@ -5,8 +5,10 @@ DROP DATABASE IF EXISTS test_01048; CREATE DATABASE test_01048 ENGINE=Ordinary; DROP TABLE IF EXISTS test_01048.mt; +DROP TABLE IF EXISTS test_01048.mt_2; CREATE TABLE test_01048.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test_01048.mt_2(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; @@ -42,6 +44,16 @@ DROP TABLE IF EXISTS test_01048.wv; CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND, 'Asia/Shanghai') AS wid FROM test_01048.mt GROUP BY wid; SHOW CREATE TABLE test_01048.`.inner.wv`; +SELECT '||---DATA COLUMN ALIAS---'; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, b as id FROM test_01048.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND); +SHOW CREATE TABLE test_01048.`.inner.wv`; + +SELECT '||---JOIN---'; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(test_01048.mt.a), count(test_01048.mt_2.b), wid FROM test_01048.mt JOIN test_01048.mt_2 ON test_01048.mt.timestamp = test_01048.mt_2.timestamp GROUP BY tumble(test_01048.mt.timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; + SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; @@ -68,15 +80,25 @@ DROP TABLE IF EXISTS test_01048.wv; CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE test_01048.`.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); +SHOW CREATE TABLE test_01048.`.inner.wv`; + SELECT '||---TimeZone---'; DROP TABLE IF EXISTS test_01048.wv; CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, hopEnd(wid) as wend FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'Asia/Shanghai') as wid; SHOW CREATE TABLE test_01048.`.inner.wv`; - +SELECT '||---DATA COLUMN ALIAS---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, b as id FROM test_01048.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND); +SHOW CREATE TABLE test_01048.`.inner.wv`; + +SELECT '||---JOIN---'; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(test_01048.mt.a), count(test_01048.mt_2.b), wid FROM test_01048.mt JOIN test_01048.mt_2 ON test_01048.mt.timestamp = test_01048.mt_2.timestamp GROUP BY hop(test_01048.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE test_01048.`.inner.wv`; DROP TABLE test_01048.wv; DROP TABLE test_01048.mt; +DROP TABLE test_01048.mt_2; diff --git a/tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.reference b/tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.reference new file mode 100644 index 00000000000..e433cdedd69 --- /dev/null +++ b/tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.reference @@ -0,0 +1,3 @@ +3 6 1990-01-01 12:00:05 +2 4 1990-01-01 12:00:10 +2 4 1990-01-01 12:00:15 diff --git a/tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.sh b/tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.sh new file mode 100755 index 00000000000..263dd5597c4 --- /dev/null +++ b/tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --multiquery < Date: Wed, 19 Jan 2022 02:24:27 +0000 Subject: [PATCH 3/5] update code style --- src/Storages/WindowView/StorageWindowView.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 9d0d2aeec87..37c913f58a9 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -488,7 +488,8 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) auto pipe = QueryPipelineBuilder::getPipe(std::move(builder)); pipes.emplace_back(std::move(pipe)); - auto creator = [&](const StorageID & blocks_id_global) { + auto creator = [&](const StorageID & blocks_id_global) + { auto parent_table_metadata = getParentStorage()->getInMemoryMetadataPtr(); auto required_columns = parent_table_metadata->getColumns(); required_columns.add(ColumnDescription("____timestamp", std::make_shared())); @@ -1245,7 +1246,8 @@ void StorageWindowView::writeIntoWindowView( Pipes pipes; pipes.emplace_back(std::move(pipe)); - auto creator = [&](const StorageID & blocks_id_global) { + auto creator = [&](const StorageID & blocks_id_global) + { auto parent_metadata = window_view.getParentStorage()->getInMemoryMetadataPtr(); auto required_columns = parent_metadata->getColumns(); required_columns.add(ColumnDescription("____timestamp", std::make_shared())); @@ -1261,7 +1263,8 @@ void StorageWindowView::writeIntoWindowView( QueryProcessingStage::WithMergeableState); builder = select_block.buildQueryPipeline(); - builder.addSimpleTransform([&](const Block & current_header) { + builder.addSimpleTransform([&](const Block & current_header) + { return std::make_shared( current_header, local_context->getSettingsRef().min_insert_block_size_rows, From c49a49b41ff9c2eeda615677ad8fd0d64d4ae261 Mon Sep 17 00:00:00 2001 From: cnmade Date: Thu, 20 Jan 2022 09:50:25 +0800 Subject: [PATCH 4/5] Translate zh/sql-reference/statements/alter/update.md --- .../sql-reference/statements/alter/update.md | 30 ++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) mode change 120000 => 100644 docs/zh/sql-reference/statements/alter/update.md diff --git a/docs/zh/sql-reference/statements/alter/update.md b/docs/zh/sql-reference/statements/alter/update.md deleted file mode 120000 index fa9be21c070..00000000000 --- a/docs/zh/sql-reference/statements/alter/update.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/alter/update.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/alter/update.md b/docs/zh/sql-reference/statements/alter/update.md new file mode 100644 index 00000000000..08eccdf1aa2 --- /dev/null +++ b/docs/zh/sql-reference/statements/alter/update.md @@ -0,0 +1,29 @@ +--- +toc_priority: 40 +toc_title: UPDATE +--- + +# ALTER TABLE … UPDATE 语句 {#alter-table-update-statements} + +``` sql +ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr +``` + +操作与指定过滤表达式相匹配的数据。作为一个[变更 mutation](../../../sql-reference/statements/alter/index.md#mutations)来实现. + +!!! note "Note" + `ALTER TABLE` 的前缀使这个语法与其他大多数支持SQL的系统不同。它的目的是表明,与OLTP数据库中的类似查询不同,这是一个繁重的操作,不是为频繁使用而设计。 + +`filter_expr`必须是`UInt8`类型。这个查询将指定列的值更新为行中相应表达式的值,对于这些行,`filter_expr`取值为非零。使用`CAST`操作符将数值映射到列的类型上。不支持更新用于计算主键或分区键的列。 + +一个查询可以包含几个由逗号分隔的命令。 + +查询处理的同步性由 [mutations_sync](../../../operations/settings/settings.md#mutations_sync) 设置定义。 默认情况下,它是异步操作。 + + +**更多详情请参阅** + +- [变更 Mutations](../../../sql-reference/statements/alter/index.md#mutations) +- [ALTER查询的同步性问题](../../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) +- [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting + From eabef7b1ee127b0beb8d22a52fe8c6a0dd8ac387 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 20 Jan 2022 10:50:57 +0300 Subject: [PATCH 5/5] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 87860deea9d..166492e293a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,7 +23,6 @@ * Added function `arrayLast`. Closes [#33390](https://github.com/ClickHouse/ClickHouse/issues/33390). [#33415](https://github.com/ClickHouse/ClickHouse/pull/33415) Added function `arrayLastIndex`. [#33465](https://github.com/ClickHouse/ClickHouse/pull/33465) ([Maksim Kita](https://github.com/kitaisreal)). * Add function `decodeURLFormComponent` slightly different to `decodeURLComponent`. Close [#10298](https://github.com/ClickHouse/ClickHouse/issues/10298). [#33451](https://github.com/ClickHouse/ClickHouse/pull/33451) ([SuperDJY](https://github.com/cmsxbc)). * Allow to split `GraphiteMergeTree` rollup rules for plain/tagged metrics (optional rule_type field). [#33494](https://github.com/ClickHouse/ClickHouse/pull/33494) ([Michail Safronov](https://github.com/msaf1980)). -* Potential issue, cannot be exploited: integer overflow may happen in array resize. [#33024](https://github.com/ClickHouse/ClickHouse/pull/33024) ([varadarajkumar](https://github.com/varadarajkumar)). #### Performance Improvement @@ -100,6 +99,7 @@ * Use `--database` option for clickhouse-local. [#32797](https://github.com/ClickHouse/ClickHouse/pull/32797) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix surprisingly bad code in SQL ordinary function `file`. Now it supports symlinks. [#32640](https://github.com/ClickHouse/ClickHouse/pull/32640) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Updating `modification_time` for data part in `system.parts` after part movement [#32964](https://github.com/ClickHouse/ClickHouse/issues/32964). [#32965](https://github.com/ClickHouse/ClickHouse/pull/32965) ([save-my-heart](https://github.com/save-my-heart)). +* Potential issue, cannot be exploited: integer overflow may happen in array resize. [#33024](https://github.com/ClickHouse/ClickHouse/pull/33024) ([varadarajkumar](https://github.com/varadarajkumar)). #### Build/Testing/Packaging Improvement