mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
Merge branch 'master' into trace_id_order
This commit is contained in:
commit
08c24cd8a3
@ -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
|
||||
|
@ -1 +0,0 @@
|
||||
../../../../en/sql-reference/statements/alter/update.md
|
29
docs/zh/sql-reference/statements/alter/update.md
Normal file
29
docs/zh/sql-reference/statements/alter/update.md
Normal 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
|
||||
|
@ -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_,
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
};
|
||||
}
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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
|
39
tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.sh
Executable file
39
tests/queries/0_stateless/01071_window_view_event_tumble_asc_join.sh
Executable 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"
|
Loading…
Reference in New Issue
Block a user