Merge branch 'master' into trace_id_order

This commit is contained in:
mergify[bot] 2022-01-20 08:20:32 +00:00 committed by GitHub
commit 08c24cd8a3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 232 additions and 165 deletions

View File

@ -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

View File

@ -1 +0,0 @@
../../../../en/sql-reference/statements/alter/update.md

View File

@ -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

View File

@ -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_,

View File

@ -44,9 +44,10 @@
#include <base/sleep.h>
#include <base/logger_useful.h>
#include <Storages/LiveView/StorageBlocks.h>
#include <Storages/WindowView/StorageWindowView.h>
#include <Storages/WindowView/WindowViewSource.h>
#include <Storages/WindowView/WindowViewProxyStorage.h>
#include <QueryPipeline/printPipeline.h>
@ -204,7 +205,6 @@ namespace
{
String window_id_name;
String window_id_alias;
Aliases * aliases;
};
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
@ -243,6 +243,23 @@ namespace
}
};
struct DropTableIdentifierMatcher
{
using Visitor = InDepthNodeVisitor<DropTableIdentifierMatcher, true>;
struct Data{};
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, Data &)
{
if (auto * t = ast->as<ASTIdentifier>())
{
ast = std::make_shared<ASTIdentifier>(t->shortName());
}
}
};
IntervalKind strToIntervalKind(const String& interval_str)
{
if (interval_str == "Second")
@ -467,16 +484,27 @@ std::pair<BlocksPtr, Block> 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<DataTypeDateTime>()));
auto proxy_storage = std::make_shared<WindowViewProxyStorage>(
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<DataTypeDateTime>()));
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)
@ -581,16 +609,16 @@ std::shared_ptr<ASTCreateQuery> StorageWindowView::getInnerTableCreateQuery(
auto columns_list = std::make_shared<ASTExpressionList>();
String window_id_column_name;
if (is_time_column_func_now)
{
auto column_window = std::make_shared<ASTColumnDeclaration>();
column_window->name = window_id_name;
column_window->type = std::make_shared<ASTIdentifier>("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;
@ -600,28 +628,17 @@ std::shared_ptr<ASTCreateQuery> 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<ASTStorage>();
/// storage != nullptr in case create window view with ENGINE syntax
if (storage)
@ -637,6 +654,19 @@ std::shared_ptr<ASTCreateQuery> 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)
@ -644,24 +674,22 @@ std::shared_ptr<ASTCreateQuery> 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<ASTIdentifier>(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);
@ -674,35 +702,8 @@ std::shared_ptr<ASTCreateQuery> 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<ASTWithAlias *>(child.get()))
ast_with_alias->setAlias("");
auto order_by = std::make_shared<ASTFunction>();
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<ASTIdentifier>(window_id_name));
new_storage->set(new_storage->order_by, std::make_shared<ASTIdentifier>(window_id_column_name));
new_storage->set(new_storage->primary_key, std::make_shared<ASTIdentifier>(window_id_column_name));
}
auto new_columns = std::make_shared<ASTColumns>();
@ -1240,35 +1241,38 @@ void StorageWindowView::writeIntoWindowView(
return std::make_shared<ExpressionTransform>(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<SquashingChunksTransform>(
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)
{
InterpreterSelectQuery select_block(
window_view.getMergeableQuery(), local_context, {std::move(pipe)},
QueryProcessingStage::WithMergeableState);
auto parent_metadata = window_view.getParentStorage()->getInMemoryMetadataPtr();
auto required_columns = parent_metadata->getColumns();
required_columns.add(ColumnDescription("____timestamp", std::make_shared<DataTypeDateTime>()));
return StorageBlocks::createStorage(blocks_id_global, required_columns, std::move(pipes), QueryProcessingStage::FetchColumns);
};
TemporaryTableHolder blocks_storage(local_context, creator);
builder = select_block.buildQueryPipeline();
builder.addSimpleTransform([&](const Block & current_header)
{
return std::make_shared<SquashingChunksTransform>(
current_header,
local_context->getSettingsRef().min_insert_block_size_rows,
local_context->getSettingsRef().min_insert_block_size_bytes);
});
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<SquashingChunksTransform>(
current_header,
local_context->getSettingsRef().min_insert_block_size_rows,
local_context->getSettingsRef().min_insert_block_size_bytes);
});
if (!window_view.is_proctime)
{
UInt32 block_max_timestamp = 0;
if (window_view.is_watermark_bounded || window_view.allowed_lateness)
{

View File

@ -1,48 +0,0 @@
#pragma once
#include <DataTypes/DataTypeDateTime.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/IStorage.h>
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;
};
}

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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 <<EOF
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS info;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64, sum UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE TABLE info(key Int32, value Int32) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, sum(info.value) as sum, tumbleEnd(wid) AS w_end FROM mt JOIN info ON mt.a = info.key GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid;
INSERT INTO info VALUES (1, 2);
INSERT INTO mt VALUES (1, '1990/01/01 12:00:00');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:01');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:02');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:05');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:06');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:10');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:11');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:30');
EOF
while true; do
$CLICKHOUSE_CLIENT --query="SELECT count(*) FROM dst" | grep -q "3" && break || sleep .5 ||:
done
$CLICKHOUSE_CLIENT --query="SELECT * FROM dst ORDER BY w_end;"
$CLICKHOUSE_CLIENT --query="DROP TABLE wv"
$CLICKHOUSE_CLIENT --query="DROP TABLE mt"
$CLICKHOUSE_CLIENT --query="DROP TABLE info"
$CLICKHOUSE_CLIENT --query="DROP TABLE dst"