mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
change windowview engine behaviour
This commit is contained in:
parent
fe2aa1861f
commit
f459dd5aba
@ -394,6 +394,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
|
||||
frame.expression_list_always_start_on_new_line = false; //-V519
|
||||
|
||||
if (inner_storage)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " INNER" << (settings.hilite ? hilite_none : "");
|
||||
inner_storage->formatImpl(settings, state, frame);
|
||||
}
|
||||
|
||||
if (storage)
|
||||
storage->formatImpl(settings, state, frame);
|
||||
|
||||
|
@ -76,6 +76,7 @@ public:
|
||||
|
||||
StorageID to_table_id = StorageID::createEmpty(); /// For CREATE MATERIALIZED VIEW mv TO table.
|
||||
UUID to_inner_uuid = UUIDHelpers::Nil; /// For materialized view with inner table
|
||||
ASTStorage * inner_storage = nullptr; /// For window view with inner table
|
||||
ASTStorage * storage = nullptr;
|
||||
ASTPtr watermark_function;
|
||||
ASTPtr lateness_function;
|
||||
|
@ -835,6 +835,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
ParserToken s_rparen(TokenType::ClosingRoundBracket);
|
||||
ParserStorage storage_p;
|
||||
ParserStorage storage_inner;
|
||||
ParserTablePropertiesDeclarationList table_properties_p;
|
||||
ParserIntervalOperatorExpression watermark_p;
|
||||
ParserIntervalOperatorExpression lateness_p;
|
||||
@ -844,6 +845,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
ASTPtr to_table;
|
||||
ASTPtr columns_list;
|
||||
ASTPtr storage;
|
||||
ASTPtr inner_storage;
|
||||
ASTPtr watermark;
|
||||
ASTPtr lateness;
|
||||
ASTPtr as_database;
|
||||
@ -901,8 +903,17 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Inner table ENGINE for WINDOW VIEW
|
||||
storage_p.parse(pos, storage, expected);
|
||||
if (ParserKeyword{"INNER"}.ignore(pos, expected))
|
||||
{
|
||||
/// Inner table ENGINE for WINDOW VIEW
|
||||
storage_inner.parse(pos, inner_storage, expected);
|
||||
}
|
||||
|
||||
if (!to_table)
|
||||
{
|
||||
/// Target table ENGINE for WINDOW VIEW
|
||||
storage_p.parse(pos, storage, expected);
|
||||
}
|
||||
|
||||
// WATERMARK
|
||||
if (ParserKeyword{"WATERMARK"}.ignore(pos, expected))
|
||||
@ -955,6 +966,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
|
||||
query->set(query->columns_list, columns_list);
|
||||
query->set(query->storage, storage);
|
||||
query->set(query->inner_storage, inner_storage);
|
||||
query->is_watermark_strictly_ascending = is_watermark_strictly_ascending;
|
||||
query->is_watermark_ascending = is_watermark_ascending;
|
||||
query->is_watermark_bounded = is_watermark_bounded;
|
||||
|
@ -391,7 +391,7 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/// CREATE|ATTACH WINDOW VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE [db.]name] [WATERMARK function] AS SELECT ...
|
||||
/// CREATE|ATTACH WINDOW VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [INNER ENGINE [db.]name] [ENGINE [db.]name] [WATERMARK function] [ALLOWED_LATENESS = interval_function] AS SELECT ...
|
||||
class ParserCreateWindowViewQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
|
@ -349,6 +349,37 @@ namespace
|
||||
|
||||
String getName() const override { return "AddingAggregatedChunkInfoTransform"; }
|
||||
};
|
||||
|
||||
static inline String generateInnerTableName(const StorageID & storage_id)
|
||||
{
|
||||
if (storage_id.hasUUID())
|
||||
return ".inner." + toString(storage_id.uuid);
|
||||
return ".inner." + storage_id.getTableName();
|
||||
}
|
||||
|
||||
static inline String generateTargetTableName(const StorageID & storage_id)
|
||||
{
|
||||
if (storage_id.hasUUID())
|
||||
return ".inner.target." + toString(storage_id.uuid);
|
||||
return ".inner.target." + storage_id.table_name;
|
||||
}
|
||||
|
||||
static ASTPtr generateInnerFetchQuery(StorageID inner_table_id)
|
||||
{
|
||||
auto fetch_query = std::make_shared<ASTSelectQuery>();
|
||||
auto select = std::make_shared<ASTExpressionList>();
|
||||
select->children.push_back(std::make_shared<ASTAsterisk>());
|
||||
fetch_query->setExpression(ASTSelectQuery::Expression::SELECT, select);
|
||||
fetch_query->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared<ASTTablesInSelectQuery>());
|
||||
auto tables_elem = std::make_shared<ASTTablesInSelectQueryElement>();
|
||||
auto table_expr = std::make_shared<ASTTableExpression>();
|
||||
fetch_query->tables()->children.push_back(tables_elem);
|
||||
tables_elem->table_expression = table_expr;
|
||||
tables_elem->children.push_back(table_expr);
|
||||
table_expr->database_and_table_name = std::make_shared<ASTTableIdentifier>(inner_table_id);
|
||||
table_expr->children.push_back(table_expr->database_and_table_name);
|
||||
return fetch_query;
|
||||
}
|
||||
}
|
||||
|
||||
static void extractDependentTable(ContextPtr context, ASTPtr & query, String & select_database_name, String & select_table_name)
|
||||
@ -461,15 +492,52 @@ std::pair<BlocksPtr, Block> StorageWindowView::getNewBlocks(UInt32 watermark)
|
||||
{
|
||||
UInt32 w_start = addTime(watermark, window_kind, -window_num_units, *time_zone);
|
||||
|
||||
auto inner_storage = getInnerStorage();
|
||||
InterpreterSelectQuery fetch(
|
||||
getFetchColumnQuery(w_start, watermark),
|
||||
inner_fetch_query,
|
||||
getContext(),
|
||||
getInnerStorage(),
|
||||
nullptr,
|
||||
inner_storage,
|
||||
inner_storage->getInMemoryMetadataPtr(),
|
||||
SelectQueryOptions(QueryProcessingStage::FetchColumns));
|
||||
|
||||
auto builder = fetch.buildQueryPipeline();
|
||||
|
||||
ASTPtr filter_function;
|
||||
if (is_tumble)
|
||||
{
|
||||
/// SELECT * FROM inner_table WHERE window_id_name == w_end
|
||||
/// (because we fire at the end of windows)
|
||||
filter_function = makeASTFunction("equals", std::make_shared<ASTIdentifier>(window_id_name), std::make_shared<ASTLiteral>(watermark));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto func_array = makeASTFunction("array");
|
||||
auto w_end = watermark;
|
||||
while (w_start < w_end)
|
||||
{
|
||||
/// slice_num_units = std::gcd(hop_num_units, window_num_units);
|
||||
/// We use std::gcd(hop_num_units, window_num_units) as the new window size
|
||||
/// to split the overlapped windows into non-overlapped.
|
||||
/// For a hopping window with window_size=3 slice=1, the windows might be
|
||||
/// [1,3],[2,4],[3,5], which will cause recomputation.
|
||||
/// In this case, the slice_num_units will be `gcd(1,3)=1' and the non-overlapped
|
||||
/// windows will split into [1], [2], [3]... We compute each split window into
|
||||
/// mergeable state and merge them when the window is triggering.
|
||||
func_array ->arguments->children.push_back(std::make_shared<ASTLiteral>(w_end));
|
||||
w_end = addTime(w_end, window_kind, -slice_num_units, *time_zone);
|
||||
}
|
||||
filter_function = makeASTFunction("has", func_array, std::make_shared<ASTIdentifier>(window_id_name));
|
||||
}
|
||||
|
||||
auto syntax_result = TreeRewriter(getContext()).analyze(filter_function, builder.getHeader().getNamesAndTypesList());
|
||||
auto filter_expression = ExpressionAnalyzer(filter_function, syntax_result, getContext()).getActionsDAG(false);
|
||||
|
||||
builder.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<FilterTransform>(
|
||||
header, std::make_shared<ExpressionActions>(filter_expression), filter_function->getColumnName(), true);
|
||||
});
|
||||
|
||||
/// Adding window column
|
||||
DataTypes window_column_type{std::make_shared<DataTypeDateTime>(), std::make_shared<DataTypeDateTime>()};
|
||||
ColumnWithTypeAndName column;
|
||||
@ -565,10 +633,15 @@ inline void StorageWindowView::fire(UInt32 watermark)
|
||||
|
||||
BlocksPtr blocks;
|
||||
Block header;
|
||||
try
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
std::tie(blocks, header) = getNewBlocks(watermark);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
for (const auto & block : *blocks)
|
||||
{
|
||||
@ -689,33 +762,30 @@ std::shared_ptr<ASTCreateQuery> StorageWindowView::getInnerTableCreateQuery(
|
||||
};
|
||||
|
||||
auto new_storage = std::make_shared<ASTStorage>();
|
||||
/// storage != nullptr in case create window view with ENGINE syntax
|
||||
/// storage != nullptr in case create window view with INNER ENGINE syntax
|
||||
if (storage)
|
||||
{
|
||||
new_storage->set(new_storage->engine, storage->engine->clone());
|
||||
|
||||
if (storage->ttl_table)
|
||||
throw Exception(
|
||||
ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW,
|
||||
"TTL is not supported for inner table in Window View");
|
||||
|
||||
if (!endsWith(storage->engine->name, "MergeTree"))
|
||||
throw Exception(
|
||||
ErrorCodes::INCORRECT_QUERY,
|
||||
"The ENGINE of WindowView must be MergeTree family of table engines "
|
||||
"including the engines with replication support");
|
||||
new_storage->set(new_storage->engine, storage->engine->clone());
|
||||
|
||||
if (storage->partition_by)
|
||||
new_storage->set(new_storage->partition_by, visit(storage->partition_by));
|
||||
if (storage->primary_key)
|
||||
new_storage->set(new_storage->primary_key, visit(storage->primary_key));
|
||||
if (storage->order_by)
|
||||
new_storage->set(new_storage->order_by, visit(storage->order_by));
|
||||
if (storage->sample_by)
|
||||
new_storage->set(new_storage->sample_by, visit(storage->sample_by));
|
||||
if (endsWith(storage->engine->name, "MergeTree"))
|
||||
{
|
||||
if (storage->partition_by)
|
||||
new_storage->set(new_storage->partition_by, visit(storage->partition_by));
|
||||
if (storage->primary_key)
|
||||
new_storage->set(new_storage->primary_key, visit(storage->primary_key));
|
||||
if (storage->order_by)
|
||||
new_storage->set(new_storage->order_by, visit(storage->order_by));
|
||||
if (storage->sample_by)
|
||||
new_storage->set(new_storage->sample_by, visit(storage->sample_by));
|
||||
|
||||
if (storage->settings)
|
||||
new_storage->set(new_storage->settings, storage->settings->clone());
|
||||
if (storage->settings)
|
||||
new_storage->set(new_storage->settings, storage->settings->clone());
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1052,6 +1122,13 @@ StorageWindowView::StorageWindowView(
|
||||
if (!query.select)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "SELECT query is not specified for {}", getName());
|
||||
|
||||
/// If the target table is not set, use inner target table
|
||||
inner_target_table = query.to_table_id.empty();
|
||||
if (inner_target_table && !query.storage)
|
||||
throw Exception(
|
||||
"You must specify where to save results of a WindowView query: either ENGINE or an existing table in a TO clause",
|
||||
ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (query.select->list_of_selects->children.size() != 1)
|
||||
throw Exception(
|
||||
ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW,
|
||||
@ -1095,7 +1172,6 @@ StorageWindowView::StorageWindowView(
|
||||
is_watermark_strictly_ascending = query.is_watermark_strictly_ascending;
|
||||
is_watermark_ascending = query.is_watermark_ascending;
|
||||
is_watermark_bounded = query.is_watermark_bounded;
|
||||
target_table_id = query.to_table_id;
|
||||
|
||||
/// Extract information about watermark, lateness.
|
||||
eventTimeParser(query);
|
||||
@ -1105,29 +1181,52 @@ StorageWindowView::StorageWindowView(
|
||||
else
|
||||
window_column_name = std::regex_replace(window_id_name, std::regex("windowID"), "hop");
|
||||
|
||||
auto generate_inner_table_name = [](const StorageID & storage_id)
|
||||
{
|
||||
if (storage_id.hasUUID())
|
||||
return ".inner." + toString(storage_id.uuid);
|
||||
return ".inner." + storage_id.table_name;
|
||||
};
|
||||
|
||||
if (attach_)
|
||||
{
|
||||
inner_table_id = StorageID(table_id_.database_name, generate_inner_table_name(table_id_));
|
||||
inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_));
|
||||
if (inner_target_table)
|
||||
target_table_id = StorageID(table_id_.database_name, generateTargetTableName(table_id_));
|
||||
else
|
||||
target_table_id = query.to_table_id;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// create inner table
|
||||
auto inner_create_query
|
||||
= getInnerTableCreateQuery(inner_query, query.storage, table_id_.database_name, generate_inner_table_name(table_id_));
|
||||
= getInnerTableCreateQuery(inner_query, query.inner_storage, table_id_.database_name, generateInnerTableName(table_id_));
|
||||
|
||||
auto create_context = Context::createCopy(context_);
|
||||
InterpreterCreateQuery create_interpreter(inner_create_query, create_context);
|
||||
create_interpreter.setInternal(true);
|
||||
create_interpreter.execute();
|
||||
inner_table_id = StorageID(inner_create_query->getDatabase(), inner_create_query->getTable());
|
||||
|
||||
if (inner_target_table)
|
||||
{
|
||||
/// create inner target table
|
||||
auto create_context = Context::createCopy(context_);
|
||||
auto target_create_query = std::make_shared<ASTCreateQuery>();
|
||||
target_create_query->setDatabase(table_id_.database_name);
|
||||
target_create_query->setTable(generateTargetTableName(table_id_));
|
||||
|
||||
auto new_columns_list = std::make_shared<ASTColumns>();
|
||||
new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr());
|
||||
|
||||
target_create_query->set(target_create_query->columns_list, new_columns_list);
|
||||
target_create_query->set(target_create_query->storage, query.storage->ptr());
|
||||
|
||||
InterpreterCreateQuery create_interpreter(target_create_query, create_context);
|
||||
create_interpreter.setInternal(true);
|
||||
create_interpreter.execute();
|
||||
|
||||
target_table_id = StorageID(target_create_query->getDatabase(), target_create_query->getTable());
|
||||
}
|
||||
else
|
||||
target_table_id = query.to_table_id;
|
||||
}
|
||||
|
||||
inner_fetch_query = generateInnerFetchQuery(inner_table_id);
|
||||
|
||||
clean_interval_ms = getContext()->getSettingsRef().window_view_clean_interval.totalMilliseconds();
|
||||
next_fire_signal = getWindowUpperBound(std::time(nullptr));
|
||||
|
||||
@ -1463,6 +1562,9 @@ void StorageWindowView::dropInnerTableIfAny(bool no_delay, ContextPtr local_cont
|
||||
{
|
||||
InterpreterDropQuery::executeDropQuery(
|
||||
ASTDropQuery::Kind::Drop, getContext(), local_context, inner_table_id, no_delay);
|
||||
|
||||
if (inner_target_table)
|
||||
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, no_delay);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -1497,51 +1599,6 @@ StoragePtr StorageWindowView::getInnerStorage() const
|
||||
return DatabaseCatalog::instance().getTable(inner_table_id, getContext());
|
||||
}
|
||||
|
||||
ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 w_start, UInt32 w_end) const
|
||||
{
|
||||
auto res_query = std::make_shared<ASTSelectQuery>();
|
||||
auto select = std::make_shared<ASTExpressionList>();
|
||||
select->children.push_back(std::make_shared<ASTAsterisk>());
|
||||
res_query->setExpression(ASTSelectQuery::Expression::SELECT, select);
|
||||
res_query->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared<ASTTablesInSelectQuery>());
|
||||
auto tables_elem = std::make_shared<ASTTablesInSelectQueryElement>();
|
||||
auto table_expr = std::make_shared<ASTTableExpression>();
|
||||
res_query->tables()->children.push_back(tables_elem);
|
||||
tables_elem->table_expression = table_expr;
|
||||
tables_elem->children.push_back(table_expr);
|
||||
table_expr->database_and_table_name = std::make_shared<ASTTableIdentifier>(inner_table_id);
|
||||
table_expr->children.push_back(table_expr->database_and_table_name);
|
||||
|
||||
if (is_tumble)
|
||||
{
|
||||
/// SELECT * FROM inner_table PREWHERE window_id_name == w_end
|
||||
/// (because we fire at the end of windows)
|
||||
auto func_equals = makeASTFunction("equals", std::make_shared<ASTIdentifier>(window_id_name), std::make_shared<ASTLiteral>(w_end));
|
||||
res_query->setExpression(ASTSelectQuery::Expression::PREWHERE, func_equals);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto func_array = makeASTFunction("array");
|
||||
while (w_start < w_end)
|
||||
{
|
||||
/// slice_num_units = std::gcd(hop_num_units, window_num_units);
|
||||
/// We use std::gcd(hop_num_units, window_num_units) as the new window size
|
||||
/// to split the overlapped windows into non-overlapped.
|
||||
/// For a hopping window with window_size=3 slice=1, the windows might be
|
||||
/// [1,3],[2,4],[3,5], which will cause recomputation.
|
||||
/// In this case, the slice_num_units will be `gcd(1,3)=1' and the non-overlapped
|
||||
/// windows will split into [1], [2], [3]... We compute each split window into
|
||||
/// mergeable state and merge them when the window is triggering.
|
||||
func_array ->arguments->children.push_back(std::make_shared<ASTLiteral>(w_end));
|
||||
w_end = addTime(w_end, window_kind, -slice_num_units, *time_zone);
|
||||
}
|
||||
auto func_has = makeASTFunction("has", func_array, std::make_shared<ASTIdentifier>(window_id_name));
|
||||
res_query->setExpression(ASTSelectQuery::Expression::PREWHERE, func_has);
|
||||
}
|
||||
|
||||
return res_query;
|
||||
}
|
||||
|
||||
StoragePtr StorageWindowView::getTargetStorage() const
|
||||
{
|
||||
return DatabaseCatalog::instance().getTable(target_table_id, getContext());
|
||||
|
@ -180,11 +180,15 @@ private:
|
||||
/// Used to fetch the mergeable state and generate the final result. e.g. SELECT * FROM * GROUP BY tumble(____timestamp, *)
|
||||
ASTPtr final_query;
|
||||
|
||||
/// Used to fetch the data from inner storage.
|
||||
ASTPtr inner_fetch_query;
|
||||
|
||||
bool is_proctime{true};
|
||||
bool is_time_column_func_now;
|
||||
bool is_tumble; // false if is hop
|
||||
std::atomic<bool> shutdown_called{false};
|
||||
bool has_inner_table{true};
|
||||
bool inner_target_table{false};
|
||||
mutable Block sample_block;
|
||||
UInt64 clean_interval_ms;
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
@ -253,7 +257,6 @@ private:
|
||||
void updateMaxTimestamp(UInt32 timestamp);
|
||||
|
||||
ASTPtr getFinalQuery() const { return final_query->clone(); }
|
||||
ASTPtr getFetchColumnQuery(UInt32 w_start, UInt32 w_end) const;
|
||||
|
||||
StoragePtr getParentStorage() const;
|
||||
|
||||
|
@ -14,46 +14,46 @@ SELECT '---TUMBLE---';
|
||||
SELECT '||---WINDOW COLUMN NAME---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY tumble(timestamp, INTERVAL '1' SECOND) AS SELECT count(a), tumbleEnd(wid) AS count FROM test_01047.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) as wid;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a), tumbleEnd(wid) AS count FROM test_01047.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) as wid;
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---WINDOW COLUMN ALIAS---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid;
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---DATA COLUMN ALIAS---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY id AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND);
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory 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 '||---IDENTIFIER---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.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 test_01047.mt GROUP BY b, tumble(timestamp, INTERVAL '1' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, tumble(timestamp, INTERVAL '1' SECOND) AS wid;
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---FUNCTION---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.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 test_01047.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' SECOND) AS wid;
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---PARTITION---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
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;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory 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 '||---JOIN---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
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;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) ENGINE Memory 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;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory 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`;
|
||||
|
||||
|
||||
@ -61,46 +61,46 @@ SELECT '---HOP---';
|
||||
SELECT '||---WINDOW COLUMN NAME---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid;
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---WINDOW COLUMN ALIAS---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01047.mt GROUP BY wid;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01047.mt GROUP BY wid;
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---DATA COLUMN ALIAS---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY id 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 INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND);
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---IDENTIFIER---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.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 test_01047.mt GROUP BY b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---FUNCTION---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
DROP TABLE IF EXISTS test_01047.`.inner.wv`;
|
||||
CREATE WINDOW VIEW test_01047.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 test_01047.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER 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) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
|
||||
SHOW CREATE TABLE test_01047.`.inner.wv`;
|
||||
|
||||
SELECT '||---PARTITION---';
|
||||
DROP TABLE IF EXISTS test_01047.wv;
|
||||
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;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory 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;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory 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;
|
||||
CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory 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;
|
||||
|
@ -13,90 +13,90 @@ CREATE TABLE test_01048.mt_2(a Int32, b Int32, timestamp DateTime) ENGINE=MergeT
|
||||
SELECT '---TUMBLE---';
|
||||
SELECT '||---WINDOW COLUMN NAME---';
|
||||
DROP TABLE IF EXISTS test_01048.wv;
|
||||
CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, tumbleEnd(wid) as wend FROM test_01048.mt GROUP BY tumble(timestamp, INTERVAL 1 SECOND) as wid;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count, tumbleEnd(wid) as wend FROM test_01048.mt GROUP BY tumble(timestamp, INTERVAL 1 SECOND) as wid;
|
||||
SHOW CREATE TABLE test_01048.`.inner.wv`;
|
||||
|
||||
SELECT '||---WINDOW COLUMN ALIAS---';
|
||||
DROP TABLE IF EXISTS test_01048.wv;
|
||||
CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01048.mt GROUP BY wid;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01048.mt GROUP BY wid;
|
||||
SHOW CREATE TABLE test_01048.`.inner.wv`;
|
||||
|
||||
SELECT '||---IDENTIFIER---';
|
||||
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 b, tumble(timestamp, INTERVAL '1' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count FROM test_01048.mt GROUP BY b, tumble(timestamp, INTERVAL '1' 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 tumble(timestamp, INTERVAL '1' SECOND) AS wid, b;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count FROM test_01048.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) AS wid, b;
|
||||
SHOW CREATE TABLE test_01048.`.inner.wv`;
|
||||
|
||||
SELECT '||---FUNCTION---';
|
||||
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, tumble(timestamp, INTERVAL '1' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count FROM test_01048.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' 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 tumble(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b);
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count FROM test_01048.mt GROUP BY tumble(timestamp, INTERVAL '1' 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, tumble(timestamp, INTERVAL '1' SECOND, 'Asia/Shanghai') AS wid FROM test_01048.mt GROUP BY wid;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory 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);
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory 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;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory 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---';
|
||||
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) as wid;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) as wend FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND) as wid;
|
||||
SHOW CREATE TABLE test_01048.`.inner.wv`;
|
||||
|
||||
SELECT '||---WINDOW COLUMN ALIAS---';
|
||||
DROP TABLE IF EXISTS test_01048.wv;
|
||||
CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01048.mt GROUP BY wid;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01048.mt GROUP BY wid;
|
||||
SHOW CREATE TABLE test_01048.`.inner.wv`;
|
||||
|
||||
SELECT '||---IDENTIFIER---';
|
||||
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 b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count FROM test_01048.mt GROUP BY b, 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, b;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory AS SELECT count(a) AS count FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, b;
|
||||
SHOW CREATE TABLE test_01048.`.inner.wv`;
|
||||
|
||||
SELECT '||---FUNCTION---';
|
||||
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;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory 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);
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory 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;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory 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, b as id FROM test_01048.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND);
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory 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;
|
||||
CREATE WINDOW VIEW test_01048.wv ENGINE Memory 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;
|
||||
|
@ -6,28 +6,28 @@ CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
|
||||
|
||||
SELECT '---WATERMARK---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
|
||||
SELECT '---With w_end---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(tumble(timestamp, INTERVAL '3' SECOND)) AS w_start, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), tumbleStart(tumble(timestamp, INTERVAL '3' SECOND)) AS w_start, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
|
||||
SELECT '---WithOut w_end---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
|
||||
SELECT '---WITH---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(wid) AS w_end, date_time FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(wid) AS w_end, date_time FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
|
||||
SELECT '---WHERE---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid;
|
||||
|
||||
SELECT '---ORDER_BY---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start;
|
||||
|
||||
SELECT '---With now---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(tumble(now(), INTERVAL '3' SECOND)) AS w_end FROM mt GROUP BY tumble(now(), INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(tumble(now(), INTERVAL '3' SECOND)) AS w_end FROM mt GROUP BY tumble(now(), INTERVAL '3' SECOND) AS wid;
|
||||
|
@ -6,28 +6,28 @@ CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
|
||||
|
||||
SELECT '---WATERMARK---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
|
||||
SELECT '---With w_end---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
|
||||
SELECT '---WithOut w_end---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), hopStart(wid) AS w_start FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
|
||||
SELECT '---WITH---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end, date_time FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end, date_time FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
|
||||
SELECT '---WHERE---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), hopStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
|
||||
|
||||
SELECT '---ORDER_BY---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), hopStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start;
|
||||
|
||||
SELECT '---With now---';
|
||||
DROP TABLE IF EXISTS wv NO DELAY;
|
||||
CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND)) as w_end FROM mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
|
||||
CREATE WINDOW VIEW wv ENGINE Memory AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND)) as w_end FROM mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid;
|
||||
|
@ -39,7 +39,7 @@ with client(name="client1>", log=log) as client1, client(
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv AS SELECT count(a) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
|
||||
"CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
|
@ -35,7 +35,7 @@ with client(name="client1>", log=log) as client1, client(
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW db_01059_event_hop_watch_strict_asc.wv WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, hopEnd(wid) as w_end FROM db_01059_event_hop_watch_strict_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid;"
|
||||
"CREATE WINDOW VIEW db_01059_event_hop_watch_strict_asc.wv ENGINE Memory WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, hopEnd(wid) as w_end FROM db_01059_event_hop_watch_strict_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid;"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
|
@ -39,7 +39,7 @@ with client(name="client1>", log=log) as client1, client(
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01062_window_view_event_hop_watch_asc.wv WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01062_window_view_event_hop_watch_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
"CREATE WINDOW VIEW 01062_window_view_event_hop_watch_asc.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01062_window_view_event_hop_watch_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
|
@ -39,7 +39,7 @@ with client(name="client1>", log=log) as client1, client(
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01069_window_view_proc_tumble_watch.wv AS SELECT count(a) AS count FROM 01069_window_view_proc_tumble_watch.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
|
||||
"CREATE WINDOW VIEW 01069_window_view_proc_tumble_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM 01069_window_view_proc_tumble_watch.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
|
@ -39,7 +39,7 @@ with client(name="client1>", log=log) as client1, client(
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01070_window_view_watch_events.wv WATERMARK=ASCENDING AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM 01070_window_view_watch_events.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid"
|
||||
"CREATE WINDOW VIEW 01070_window_view_watch_events.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM 01070_window_view_watch_events.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user