use AggregatingMergeTree as default inner table.

This commit is contained in:
Vxider 2020-03-28 00:44:09 +08:00
parent 220c29600d
commit 5a23306ebb
60 changed files with 310 additions and 414 deletions

View File

@ -16,6 +16,7 @@
#include <Interpreters/InDepthNodeVisitor.h> #include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/InterpreterAlterQuery.h> #include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/InterpreterDropQuery.h> #include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/QueryAliasesVisitor.h>
#include <Interpreters/getTableExpressions.h> #include <Interpreters/getTableExpressions.h>
#include <Parsers/ASTAlterQuery.h> #include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTAsterisk.h> #include <Parsers/ASTAsterisk.h>
@ -146,6 +147,7 @@ namespace
{ {
String window_column_name; String window_column_name;
String window_column_alias; String window_column_alias;
Aliases * aliases;
}; };
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
@ -159,19 +161,22 @@ namespace
} }
private: private:
static void visit(const ASTFunction & node, ASTPtr & node_ptr, Data & data) static void visit(const ASTFunction & node, ASTPtr & node_ptr, Data &)
{ {
if (node.name == "TUMBLE" || node.name == "HOP") if (node.name == "tuple")
{ return;
if (queryToString(node) == data.window_column_name) if (node.name == "HOP")
node_ptr = std::make_shared<ASTIdentifier>(data.window_column_name); node_ptr = std::make_shared<ASTIdentifier>("arrayJoin(" + node.getColumnName() + ")");
} else
node_ptr = std::make_shared<ASTIdentifier>(node.getColumnName());
} }
static void visit(const ASTIdentifier & node, ASTPtr & node_ptr, Data & data) static void visit(const ASTIdentifier & node, ASTPtr & node_ptr, Data & data)
{ {
if (node.name == data.window_column_alias) if (node.getColumnName() == data.window_column_alias)
node_ptr = std::make_shared<ASTIdentifier>(data.window_column_name); dynamic_cast<ASTIdentifier *>(node_ptr.get())->name = data.window_column_name;
else if (auto it = data.aliases->find(node.getColumnName()); it != data.aliases->end())
dynamic_cast<ASTIdentifier *>(node_ptr.get())->name = it->second->getColumnName();
} }
}; };
@ -327,9 +332,7 @@ void StorageWindowView::drop(TableStructureWriteLockHolder &)
{ {
auto table_id = getStorageID(); auto table_id = getStorageID();
DatabaseCatalog::instance().removeDependency(select_table_id, table_id); DatabaseCatalog::instance().removeDependency(select_table_id, table_id);
executeDropQuery(ASTDropQuery::Kind::Drop, global_context, inner_table_id);
if (!inner_table_id.empty())
executeDropQuery(ASTDropQuery::Kind::Drop, global_context, inner_table_id);
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
is_dropped = true; is_dropped = true;
@ -338,20 +341,11 @@ void StorageWindowView::drop(TableStructureWriteLockHolder &)
void StorageWindowView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) void StorageWindowView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
{ {
if (!inner_table_id.empty()) executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, inner_table_id);
executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, inner_table_id);
else
{
std::lock_guard lock(mutex);
mergeable_blocks.clear();
}
} }
bool StorageWindowView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) bool StorageWindowView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
{ {
if (inner_table_id.empty())
throw Exception(
"OPTIMIZE only supported when creating WINDOW VIEW within INNER table.", ErrorCodes::INCORRECT_QUERY);
return getInnerStorage()->optimize(query, partition, final, deduplicate, context); return getInnerStorage()->optimize(query, partition, final, deduplicate, context);
} }
@ -391,28 +385,9 @@ inline void StorageWindowView::cleanCache()
w_bound = is_tumble ? addTime(w_bound, window_kind, -1 * window_num_units, time_zone) w_bound = is_tumble ? addTime(w_bound, window_kind, -1 * window_num_units, time_zone)
: addTime(w_bound, hop_kind, -1 * hop_num_units, time_zone); : addTime(w_bound, hop_kind, -1 * hop_num_units, time_zone);
if (!inner_table_id.empty()) auto sql = generateCleanCacheQuery(w_bound);
{ InterpreterAlterQuery alt_query(sql, global_context);
auto sql = generateCleanCacheQuery(w_bound); alt_query.execute();
InterpreterAlterQuery alt_query(sql, global_context);
alt_query.execute();
}
else
{
std::lock_guard lock(mutex);
mergeable_blocks.remove_if([&](Block & block)
{
auto & column = block.getByName(window_column_name).column;
const ColumnTuple & column_tuple = typeid_cast<const ColumnTuple &>(*column);
const ColumnUInt32::Container & data = static_cast<const ColumnUInt32 &>(*column_tuple.getColumnPtr(1)).getData();
for (size_t i = 0; i < column->size(); ++i)
{
if (data[i] >= w_bound)
return false;
}
return true;
});
}
std::lock_guard lock(fire_signal_mutex); std::lock_guard lock(fire_signal_mutex);
watch_streams.remove_if([](std::weak_ptr<WindowViewBlockInputStream> & ptr) { return ptr.expired(); }); watch_streams.remove_if([](std::weak_ptr<WindowViewBlockInputStream> & ptr) { return ptr.expired(); });
@ -471,17 +446,21 @@ inline void StorageWindowView::fire(UInt32 watermark)
fire_condition.notify_all(); fire_condition.notify_all();
} }
std::shared_ptr<ASTCreateQuery> StorageWindowView::generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name) std::shared_ptr<ASTCreateQuery> StorageWindowView::generateInnerTableCreateQuery(ASTStorage * storage, const String & database_name, const String & table_name)
{ {
/// We will create a query to create an internal table. /// We will create a query to create an internal table.
auto manual_create_query = std::make_shared<ASTCreateQuery>(); auto inner_create_query = std::make_shared<ASTCreateQuery>();
manual_create_query->database = database_name; inner_create_query->database = database_name;
manual_create_query->table = table_name; inner_create_query->table = table_name;
auto new_columns_list = std::make_shared<ASTColumns>(); auto inner_select_query = dynamic_pointer_cast<ASTSelectQuery>(getInnerQuery());
Aliases aliases;
QueryAliasesVisitor::Data query_aliases_data{aliases};
QueryAliasesVisitor(query_aliases_data).visit(inner_select_query);
auto t_sample_block auto t_sample_block
= InterpreterSelectQuery(getInnerQuery(), global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::WithMergeableState)) = InterpreterSelectQuery(inner_select_query, global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::WithMergeableState))
.getSampleBlock(); .getSampleBlock();
auto columns_list = std::make_shared<ASTExpressionList>(); auto columns_list = std::make_shared<ASTExpressionList>();
@ -506,59 +485,100 @@ std::shared_ptr<ASTCreateQuery> StorageWindowView::generateInnerTableCreateQuery
columns_list->children.push_back(column_dec); columns_list->children.push_back(column_dec);
} }
if (inner_create_query.storage->ttl_table)
throw Exception("TTL is not supported for inner table in Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW);
ReplaceFunctionWindowMatcher::Data query_data; ReplaceFunctionWindowMatcher::Data query_data;
query_data.window_column_name = window_column_name; query_data.window_column_name = window_column_name;
query_data.window_column_alias = window_column_alias; query_data.window_column_alias = window_column_alias;
query_data.aliases = &aliases;
ReplaceFunctionWindowMatcher::Visitor visitor(query_data); ReplaceFunctionWindowMatcher::Visitor visitor(query_data);
ReplaceFuncNowVisitorData parser_proc_time_data; ReplaceFuncNowVisitorData parser_proc_time_data;
InDepthNodeVisitor<OneTypeMatcher<ReplaceFuncNowVisitorData>, true> time_now_visitor(parser_proc_time_data); InDepthNodeVisitor<OneTypeMatcher<ReplaceFuncNowVisitorData>, true> time_now_visitor(parser_proc_time_data);
auto storage = std::make_shared<ASTStorage>(); auto new_storage = std::make_shared<ASTStorage>();
storage->set(storage->engine, inner_create_query.storage->engine->clone()); if (storage == nullptr)
if (inner_create_query.storage->partition_by)
{ {
auto partition_by = inner_create_query.storage->partition_by->clone(); new_storage->set(new_storage->engine, makeASTFunction("AggregatingMergeTree"));
if (is_time_column_func_now)
time_now_visitor.visit(partition_by);
visitor.visit(partition_by);
storage->set(storage->partition_by, partition_by);
}
if (inner_create_query.storage->primary_key)
{
auto primary_key = inner_create_query.storage->primary_key->clone();
if (is_time_column_func_now)
time_now_visitor.visit(primary_key);
visitor.visit(primary_key);
storage->set(storage->primary_key, primary_key);
}
if (inner_create_query.storage->order_by)
{
auto order_by = inner_create_query.storage->order_by->clone();
if (is_time_column_func_now)
time_now_visitor.visit(order_by);
visitor.visit(order_by);
storage->set(storage->order_by, order_by);
}
if (inner_create_query.storage->sample_by)
{
auto sample_by = inner_create_query.storage->sample_by->clone();
if (is_time_column_func_now)
time_now_visitor.visit(sample_by);
visitor.visit(sample_by);
storage->set(storage->sample_by, sample_by);
}
if (inner_create_query.storage->settings)
storage->set(storage->settings, inner_create_query.storage->settings->clone());
new_columns_list->set(new_columns_list->columns, columns_list); // auto inner_select_query = std::dynamic_pointer_cast<ASTSelectQuery>(getInnerQuery());
manual_create_query->set(manual_create_query->columns_list, new_columns_list); for (auto & child : inner_select_query->groupBy()->children)
manual_create_query->set(manual_create_query->storage, storage); if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(child.get()))
ast_with_alias->setAlias("");
return manual_create_query; 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);
visitor.visit(order_by_ptr);
for (auto & child : order_by->arguments->children)
{
if (child->getColumnName() == window_column_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_column_name));
}
else
{
if (storage->ttl_table)
throw Exception("TTL is not supported for inner table in Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW);
if (!endsWith(storage->engine->name, "MergeTree"))
throw Exception(
"The ENGINE of WindowView must be MergeTree family of table engines including the engines with replication support",
ErrorCodes::INCORRECT_QUERY);
new_storage->set(new_storage->engine, storage->engine->clone());
if (storage->partition_by)
{
auto partition_by = storage->partition_by->clone();
if (is_time_column_func_now)
time_now_visitor.visit(partition_by);
visitor.visit(partition_by);
new_storage->set(new_storage->partition_by, partition_by);
}
if (storage->primary_key)
{
auto primary_key = storage->primary_key->clone();
if (is_time_column_func_now)
time_now_visitor.visit(primary_key);
visitor.visit(primary_key);
new_storage->set(new_storage->primary_key, primary_key);
}
if (storage->order_by)
{
auto order_by = storage->order_by->clone();
if (is_time_column_func_now)
time_now_visitor.visit(order_by);
visitor.visit(order_by);
new_storage->set(new_storage->order_by, order_by);
}
if (storage->sample_by)
{
auto sample_by = storage->sample_by->clone();
if (is_time_column_func_now)
time_now_visitor.visit(sample_by);
visitor.visit(sample_by);
new_storage->set(new_storage->sample_by, sample_by);
}
if (storage->settings)
new_storage->set(new_storage->settings, storage->settings->clone());
}
auto new_columns = std::make_shared<ASTColumns>();
new_columns->set(new_columns->columns, columns_list);
inner_create_query->set(inner_create_query->columns_list, new_columns);
inner_create_query->set(inner_create_query->storage, new_storage);
return inner_create_query;
} }
inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec)
@ -862,29 +882,22 @@ StorageWindowView::StorageWindowView(
throw Exception("Value for ALLOWED_LATENESS function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); throw Exception("Value for ALLOWED_LATENESS function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
} }
if (query.storage) if (attach_)
{ {
if (attach_) inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name));
{
inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name));
}
else
{
if (!endsWith(query.storage->engine->name, "MergeTree"))
throw Exception(
"The ENGINE of WindowView must be MergeTree family of table engines including the engines with replication support",
ErrorCodes::INCORRECT_QUERY);
auto manual_create_query
= generateInnerTableCreateQuery(query, table_id_.database_name, generateInnerTableName(table_id_.table_name));
InterpreterCreateQuery create_interpreter(manual_create_query, local_context);
create_interpreter.setInternal(true);
create_interpreter.execute();
inner_storage = DatabaseCatalog::instance().getTable(StorageID(manual_create_query->database, manual_create_query->table));
inner_table_id = inner_storage->getStorageID();
}
fetch_column_query = generateFetchColumnsQuery(inner_table_id);
} }
else
{
auto inner_create_query
= generateInnerTableCreateQuery(query.storage, table_id_.database_name, generateInnerTableName(table_id_.table_name));
InterpreterCreateQuery create_interpreter(inner_create_query, local_context);
create_interpreter.setInternal(true);
create_interpreter.execute();
inner_storage = DatabaseCatalog::instance().getTable(StorageID(inner_create_query->database, inner_create_query->table));
inner_table_id = inner_storage->getStorageID();
}
fetch_column_query = generateFetchColumnsQuery(inner_table_id);
clean_cache_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); clean_cache_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); });
if (is_proctime) if (is_proctime)
@ -1032,23 +1045,10 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con
std::static_pointer_cast<WatermarkBlockInputStream>(source_stream)->setAllowedLateness(t_max_fired_watermark); std::static_pointer_cast<WatermarkBlockInputStream>(source_stream)->setAllowedLateness(t_max_fired_watermark);
} }
if (!window_view.inner_table_id.empty()) auto & inner_storage = window_view.getInnerStorage();
{ auto lock = inner_storage->lockStructureForShare(true, context.getCurrentQueryId());
auto & inner_storage = window_view.getInnerStorage(); auto stream = inner_storage->write(window_view.getInnerQuery(), context);
auto lock = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); copyData(*source_stream, *stream);
auto stream = inner_storage->write(window_view.getInnerQuery(), context);
copyData(*source_stream, *stream);
}
else
{
source_stream->readPrefix();
{
std::lock_guard lock(window_view.mutex);
while (Block t_block = source_stream->read())
window_view.mergeable_blocks.push_back(std::move(t_block));
}
source_stream->readSuffix();
}
} }
void StorageWindowView::startup() void StorageWindowView::startup()
@ -1093,16 +1093,9 @@ StoragePtr StorageWindowView::getParentStorage() const
return parent_storage; return parent_storage;
} }
Block & StorageWindowView::getMergeableHeader() const
{
if (!mergeable_sample_block)
mergeable_sample_block = mergeable_blocks.front().cloneEmpty();
return mergeable_sample_block;
}
StoragePtr & StorageWindowView::getInnerStorage() const StoragePtr & StorageWindowView::getInnerStorage() const
{ {
if (inner_storage == nullptr && !inner_table_id.empty()) if (inner_storage == nullptr)
inner_storage = DatabaseCatalog::instance().getTable(inner_table_id); inner_storage = DatabaseCatalog::instance().getTable(inner_table_id);
return inner_storage; return inner_storage;
} }
@ -1118,21 +1111,12 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma
{ {
Pipes pipes; Pipes pipes;
if (!inner_table_id.empty()) auto & storage = getInnerStorage();
{ InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns));
auto & storage = getInnerStorage(); QueryPipeline pipeline;
InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); BlockInputStreams streams = fetch.executeWithMultipleStreams(pipeline);
QueryPipeline pipeline; for (auto & stream : streams)
BlockInputStreams streams = fetch.executeWithMultipleStreams(pipeline); pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::move(stream)));
for (auto & stream : streams)
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::move(stream)));
}
else
{
if (mergeable_blocks.empty())
return std::make_shared<NullBlockInputStream>(getHeader());
pipes = blocksToPipes(mergeable_blocks, getMergeableHeader());
}
ColumnsWithTypeAndName t_columns; ColumnsWithTypeAndName t_columns;
t_columns.emplace_back( t_columns.emplace_back(

View File

@ -63,7 +63,6 @@ private:
bool is_tumble; // false if is hop bool is_tumble; // false if is hop
std::atomic<bool> shutdown_called{false}; std::atomic<bool> shutdown_called{false};
mutable Block sample_block; mutable Block sample_block;
mutable Block mergeable_sample_block;
UInt64 clean_interval; UInt64 clean_interval;
const DateLUTImpl & time_zone; const DateLUTImpl & time_zone;
UInt32 max_timestamp = 0; UInt32 max_timestamp = 0;
@ -78,7 +77,6 @@ private:
std::list<std::weak_ptr<WindowViewBlockInputStream>> watch_streams; std::list<std::weak_ptr<WindowViewBlockInputStream>> watch_streams;
std::condition_variable_any fire_signal_condition; std::condition_variable_any fire_signal_condition;
std::condition_variable fire_condition; std::condition_variable fire_condition;
BlocksList mergeable_blocks;
/// Mutex for the blocks and ready condition /// Mutex for the blocks and ready condition
std::mutex mutex; std::mutex mutex;
@ -110,7 +108,7 @@ private:
ASTPtr innerQueryParser(ASTSelectQuery & inner_query); ASTPtr innerQueryParser(ASTSelectQuery & inner_query);
std::shared_ptr<ASTCreateQuery> generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name); std::shared_ptr<ASTCreateQuery> generateInnerTableCreateQuery(ASTStorage * storage, const String & database_name, const String & table_name);
ASTPtr generateCleanCacheQuery(UInt32 timestamp); ASTPtr generateCleanCacheQuery(UInt32 timestamp);
UInt32 getWindowLowerBound(UInt32 time_sec); UInt32 getWindowLowerBound(UInt32 time_sec);
@ -138,8 +136,6 @@ private:
Block & getHeader() const; Block & getHeader() const;
Block & getMergeableHeader() const;
StorageWindowView( StorageWindowView(
const StorageID & table_id_, const StorageID & table_id_,
Context & local_context, Context & local_context,

View File

@ -0,0 +1,22 @@
---TUMBLE---
||---WINDOW COLUMN NAME---
CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` SETTINGS index_granularity = 8192
||---WINDOW COLUMN ALIAS---
CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` SETTINGS index_granularity = 8192
||---IDENTIFIER---
CREATE TABLE default.`.inner.wv` (`b` Int32, `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b) SETTINGS index_granularity = 8192
||---FUNCTION---
CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192
||---PARTITION---
CREATE TABLE default.`.inner.wv` (`TUMBLE(____timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PARTITION BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))` ORDER BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))` SETTINGS index_granularity = 8192
---HOP---
||---WINDOW COLUMN NAME---
CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192
||---WINDOW COLUMN ALIAS---
CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192
||---IDENTIFIER---
CREATE TABLE default.`.inner.wv` (`b` Int32, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192
||---FUNCTION---
CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192
||---PARTITION---
CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PARTITION BY `arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY `arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192

View File

@ -0,0 +1,61 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
CREATE TABLE mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
SELECT '---TUMBLE---';
SELECT '||---WINDOW COLUMN NAME---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a), TUMBLE_END(wid) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---WINDOW COLUMN ALIAS---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---IDENTIFIER---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---FUNCTION---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---PARTITION---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, TUMBLE(now(), INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '---HOP---';
SELECT '||---WINDOW COLUMN NAME---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count, HOP_END(wid) FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---WINDOW COLUMN ALIAS---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM mt GROUP BY wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---IDENTIFIER---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---FUNCTION---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---PARTITION---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, HOP_END(wid) FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid;
SHOW CREATE TABLE `.inner.wv`;
DROP TABLE wv;
DROP TABLE mt;

View File

@ -0,0 +1,22 @@
---TUMBLE---
||---WINDOW COLUMN NAME---
CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY tuple(`TUMBLE(timestamp, toIntervalSecond(\'1\'))`) SETTINGS index_granularity = 8192
||---WINDOW COLUMN ALIAS---
CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY tuple(`TUMBLE(timestamp, toIntervalSecond(\'1\'))`) SETTINGS index_granularity = 8192
||---IDENTIFIER---
CREATE TABLE default.`.inner.wv` (`b` Int32, `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b) SETTINGS index_granularity = 8192
CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `b` Int32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b) SETTINGS index_granularity = 8192
||---FUNCTION---
CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192
CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `plus(a, b)` Int64, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192
---HOP---
||---WINDOW COLUMN NAME---
CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY tuple(`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`) SETTINGS index_granularity = 8192
||---WINDOW COLUMN ALIAS---
CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY tuple(`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`) SETTINGS index_granularity = 8192
||---IDENTIFIER---
CREATE TABLE default.`.inner.wv` (`b` Int32, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192
CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `b` Int32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192
||---FUNCTION---
CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192
CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `plus(a, b)` Int64, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192

View File

@ -0,0 +1,68 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS `.inner.wv`;
CREATE TABLE mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
SELECT '---TUMBLE---';
SELECT '||---WINDOW COLUMN NAME---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE_END(wid) as wend FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---WINDOW COLUMN ALIAS---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---IDENTIFIER---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;
SHOW CREATE TABLE `.inner.wv`;
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, b;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---FUNCTION---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;
SHOW CREATE TABLE `.inner.wv`;
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b);
SHOW CREATE TABLE `.inner.wv`;
SELECT '---HOP---';
SELECT '||---WINDOW COLUMN NAME---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---WINDOW COLUMN ALIAS---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM mt GROUP BY wid;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---IDENTIFIER---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
SHOW CREATE TABLE `.inner.wv`;
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, b;
SHOW CREATE TABLE `.inner.wv`;
SELECT '||---FUNCTION---';
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
SHOW CREATE TABLE `.inner.wv`;
DROP TABLE IF EXISTS wv;
CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b);
SHOW CREATE TABLE `.inner.wv`;
DROP TABLE wv;
DROP TABLE mt;

View File

@ -1,17 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree PARTITION BY wid ORDER BY tuple(TUMBLE(now(), INTERVAL '1' SECOND)) AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;
INSERT INTO mt VALUES (1);
SELECT sleep(2);
SELECT count from dst;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,17 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv to dst ENGINE=AggregatingMergeTree ORDER BY tuple(wid) AS SELECT count(a) AS count FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;
INSERT INTO mt VALUES (1);
SELECT sleep(2);
SELECT count from dst;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,17 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;
INSERT INTO mt VALUES (1, now());
SELECT sleep(2);
SELECT count from dst;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,17 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;
INSERT INTO mt VALUES (1, now());
SELECT sleep(2);
SELECT count from dst;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,4 +0,0 @@
0
3 1990-01-01 12:00:05
1 1990-01-01 12:00:10
2 1990-01-01 12:00:15

View File

@ -1,24 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid;
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:10');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:11');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:30');
SELECT sleep(1);
SELECT * from dst order by w_end;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,7 +0,0 @@
0
2 1990-01-01 12:00:02
2 1990-01-01 12:00:04
1 1990-01-01 12:00:06
1 1990-01-01 12:00:08
2 1990-01-01 12:00:12
1 1990-01-01 12:00:14

View File

@ -1,24 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree() order by wid WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid;
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:10');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:11');
INSERT INTO mt VALUES (1, '1990/01/01 12:00:30');
SELECT sleep(1);
SELECT * from dst order by w_end;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,25 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid;
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');
SELECT sleep(1);
SELECT * from dst order by w_end;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,25 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid;
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');
SELECT sleep(1);
SELECT * from dst order by w_end;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,4 +0,0 @@
0
3 1990-01-01 12:00:05
2 1990-01-01 12:00:10
2 1990-01-01 12:00:15

View File

@ -1,4 +0,0 @@
0
3 1990-01-01 12:00:05
2 1990-01-01 12:00:10
2 1990-01-01 12:00:15

View File

@ -1,25 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid;
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');
SELECT sleep(1);
SELECT * from dst order by w_end;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;

View File

@ -1,7 +0,0 @@
0
2 1990-01-01 12:00:02
2 1990-01-01 12:00:04
1 1990-01-01 12:00:06
2 1990-01-01 12:00:08
2 1990-01-01 12:00:12
1 1990-01-01 12:00:14

View File

@ -1,7 +0,0 @@
0
2 1990-01-01 12:00:02
2 1990-01-01 12:00:04
1 1990-01-01 12:00:06
2 1990-01-01 12:00:08
2 1990-01-01 12:00:12
1 1990-01-01 12:00:14

View File

@ -1,25 +0,0 @@
SET allow_experimental_window_view = 1;
DROP TABLE IF EXISTS mt;
DROP TABLE IF EXISTS dst;
DROP TABLE IF EXISTS wv;
CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple();
CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid;
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');
SELECT sleep(1);
SELECT * from dst order by w_end;
DROP TABLE wv;
DROP TABLE mt;
DROP TABLE dst;