Skip to content

Commit b25b04e

Browse files
committed
support fill missing tumble windows for aggr by setting
1 parent 8ac7bbb commit b25b04e

13 files changed

+187
-17
lines changed

src/Core/Settings.h

+1
Original file line numberDiff line numberDiff line change
@@ -808,6 +808,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
808808
M(Float, replay_speed, 0., "Control the replay speed..0 < replay_speed < 1, means replay slower.replay_speed == 1, means replay by actual ingest interval.1 < replay_speed < <max_limit>, means replay faster", 0) \
809809
M(UInt64, max_events, 0, "Total events to generate for random stream", 0) \
810810
M(Int64, eps, -1, "control the random stream eps in query time, defalut value is -1, if it is 0 means no limit.", 0) \
811+
M(Bool, fill_missing_window_for_aggr, false, "fill missing window if not exist for aggr query", 0) \
811812
// End of GLOBAL_SETTINGS
812813

813814
#define CONFIGURABLE_GLOBAL_SETTINGS(M) \

src/Interpreters/InterpreterSelectQuery.cpp

+14-2
Original file line numberDiff line numberDiff line change
@@ -3268,10 +3268,22 @@ void InterpreterSelectQuery::executeStreamingAggregation(
32683268
/// 2) `shuffle by`: calculating light substream without substream ID (The data have been shuffled by `LightShufflingTransform`)
32693269
if (query_info.hasPartitionByKeys() || light_shuffled)
32703270
query_plan.addStep(std::make_unique<Streaming::AggregatingStepWithSubstream>(
3271-
query_plan.getCurrentDataStream(), std::move(params), final, emit_version, data_stream_semantic_pair.isChangelogOutput()));
3271+
query_plan.getCurrentDataStream(),
3272+
std::move(params),
3273+
final,
3274+
emit_version,
3275+
data_stream_semantic_pair.isChangelogOutput(),
3276+
settings.fill_missing_window_for_aggr));
32723277
else
32733278
query_plan.addStep(std::make_unique<Streaming::AggregatingStep>(
3274-
query_plan.getCurrentDataStream(), std::move(params), final, merge_threads, temporary_data_merge_threads, emit_version, data_stream_semantic_pair.isChangelogOutput()));
3279+
query_plan.getCurrentDataStream(),
3280+
std::move(params),
3281+
final,
3282+
merge_threads,
3283+
temporary_data_merge_threads,
3284+
emit_version,
3285+
data_stream_semantic_pair.isChangelogOutput(),
3286+
settings.fill_missing_window_for_aggr));
32753287
}
32763288

32773289
/// Resolve input / output data stream semantic.

src/Interpreters/Streaming/WindowCommon.cpp

+34
Original file line numberDiff line numberDiff line change
@@ -873,5 +873,39 @@ void reassignWindow(Chunk & chunk, const Window & window, bool time_col_is_datet
873873
chunk.setColumns(std::move(columns), rows);
874874
}
875875

876+
void addMissingWindow(
877+
Chunk & chunk, const Window & window, bool time_col_is_datetime64, std::optional<size_t> start_pos, std::optional<size_t> end_pos)
878+
{
879+
auto add_window_time = [&](ColumnPtr & column, Int64 ts) {
880+
auto col = IColumn::mutate(std::move(column));
881+
if (time_col_is_datetime64)
882+
col->insert(static_cast<DateTime64>(ts));
883+
else
884+
col->insert(static_cast<UInt32>(ts));
885+
column = std::move(col);
886+
};
887+
888+
auto add_default = [&](ColumnPtr & column) {
889+
auto col = IColumn::mutate(std::move(column));
890+
col->insertDefault();
891+
column = std::move(col);
892+
};
893+
894+
auto rows = chunk.rows();
895+
auto columns = chunk.detachColumns();
896+
for (size_t pos = 0; auto & column : columns)
897+
{
898+
if (start_pos.has_value() && pos == *start_pos)
899+
add_window_time(column, window.start);
900+
else if (end_pos.has_value() && pos == *end_pos)
901+
add_window_time(column, window.end);
902+
else
903+
add_default(column);
904+
905+
++pos;
906+
}
907+
908+
chunk.setColumns(std::move(columns), rows + 1);
909+
}
876910
}
877911
}

src/Interpreters/Streaming/WindowCommon.h

+2
Original file line numberDiff line numberDiff line change
@@ -298,5 +298,7 @@ void assignWindow(
298298
Columns & columns, const WindowInterval & interval, size_t time_col_pos, bool time_col_is_datetime64, const DateLUTImpl & time_zone);
299299
void reassignWindow(
300300
Chunk & chunk, const Window & window, bool time_col_is_datetime64, std::optional<size_t> start_pos, std::optional<size_t> end_pos);
301+
void addMissingWindow(
302+
Chunk & chunk, const Window & window, bool time_col_is_datetime64, std::optional<size_t> start_pos, std::optional<size_t> end_pos);
301303
}
302304
}

src/Processors/QueryPlan/Streaming/AggregatingStep.cpp

+7-3
Original file line numberDiff line numberDiff line change
@@ -37,14 +37,17 @@ AggregatingStep::AggregatingStep(
3737
size_t merge_threads_,
3838
size_t temporary_data_merge_threads_,
3939
bool emit_version_,
40-
bool emit_changelog_)
41-
: ITransformingStep(input_stream_, AggregatingTransformParams::getHeader(params_, final_, emit_version_, emit_changelog_), getTraits(), false)
40+
bool emit_changelog_,
41+
bool fill_missing_window_)
42+
: ITransformingStep(
43+
input_stream_, AggregatingTransformParams::getHeader(params_, final_, emit_version_, emit_changelog_), getTraits(), false)
4244
, params(std::move(params_))
4345
, final(std::move(final_))
4446
, merge_threads(merge_threads_)
4547
, temporary_data_merge_threads(temporary_data_merge_threads_)
4648
, emit_version(emit_version_)
4749
, emit_changelog(emit_changelog_)
50+
, fill_missing_window(fill_missing_window_)
4851
{
4952
}
5053

@@ -69,7 +72,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
6972
* 1. Parallel aggregation is done, and the results should be merged in parallel.
7073
* 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way.
7174
*/
72-
auto transform_params = std::make_shared<AggregatingTransformParams>(std::move(params), final, emit_version, emit_changelog);
75+
auto transform_params
76+
= std::make_shared<AggregatingTransformParams>(std::move(params), final, emit_version, emit_changelog, fill_missing_window);
7377

7478
/// If there are several sources, then we perform parallel aggregation
7579
if (pipeline.getNumStreams() > 1)

src/Processors/QueryPlan/Streaming/AggregatingStep.h

+3-1
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,8 @@ class AggregatingStep : public ITransformingStep
2222
size_t merge_threads_,
2323
size_t temporary_data_merge_threads_,
2424
bool emit_version_,
25-
bool emit_changelog_);
25+
bool emit_changelog_,
26+
bool fill_missing_window_);
2627

2728
String getName() const override { return "StreamingAggregating"; }
2829

@@ -43,6 +44,7 @@ class AggregatingStep : public ITransformingStep
4344

4445
bool emit_version;
4546
bool emit_changelog;
47+
bool fill_missing_window;
4648

4749
Processors aggregating;
4850
};

src/Processors/QueryPlan/Streaming/AggregatingStepWithSubstream.cpp

+8-2
Original file line numberDiff line numberDiff line change
@@ -37,13 +37,19 @@ ITransformingStep::Traits getTraits()
3737
}
3838

3939
AggregatingStepWithSubstream::AggregatingStepWithSubstream(
40-
const DataStream & input_stream_, Aggregator::Params params_, bool final_, bool emit_version_, bool emit_changelog_)
40+
const DataStream & input_stream_,
41+
Aggregator::Params params_,
42+
bool final_,
43+
bool emit_version_,
44+
bool emit_changelog_,
45+
bool fill_missing_window_)
4146
: ITransformingStep(
4247
input_stream_, AggregatingTransformParams::getHeader(params_, final_, emit_version_, emit_changelog_), getTraits(), false)
4348
, params(std::move(params_))
4449
, final(std::move(final_))
4550
, emit_version(emit_version_)
4651
, emit_changelog(emit_changelog_)
52+
, fill_missing_window(fill_missing_window_)
4753
{
4854
}
4955

@@ -64,7 +70,7 @@ void AggregatingStepWithSubstream::transformPipeline(QueryPipelineBuilder & pipe
6470
params.group_by_two_level_threshold_bytes = 0;
6571
}
6672

67-
auto transform_params = std::make_shared<AggregatingTransformParams>(std::move(params), final, emit_version, emit_changelog);
73+
auto transform_params = std::make_shared<AggregatingTransformParams>(std::move(params), final, emit_version, emit_changelog, fill_missing_window);
6874

6975
/// If there are several sources, we perform aggregation separately (Assume it's shuffled data by substream keys)
7076
pipeline.addSimpleTransform([&](const Block & header) -> std::shared_ptr<IProcessor> {

src/Processors/QueryPlan/Streaming/AggregatingStepWithSubstream.h

+3-1
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,8 @@ class AggregatingStepWithSubstream final : public ITransformingStep
1919
Aggregator::Params params_,
2020
bool final_,
2121
bool emit_version_,
22-
bool emit_changelog_);
22+
bool emit_changelog_,
23+
bool fill_missing_window_);
2324

2425
String getName() const override { return "StreamingAggregatingWithSubstream"; }
2526

@@ -37,6 +38,7 @@ class AggregatingStepWithSubstream final : public ITransformingStep
3738
bool final;
3839
bool emit_version;
3940
bool emit_changelog;
41+
bool fill_missing_window;
4042

4143
Processors aggregating;
4244
};

src/Processors/Transforms/Streaming/AggregatingTransform.h

+3-1
Original file line numberDiff line numberDiff line change
@@ -21,14 +21,16 @@ struct AggregatingTransformParams
2121
bool only_merge = false;
2222
bool emit_version = false;
2323
bool emit_changelog = false;
24+
bool fill_missing_window = false;
2425
DataTypePtr version_type;
2526

26-
AggregatingTransformParams(const Aggregator::Params & params_, bool final_, bool emit_version_, bool emit_changelog_)
27+
AggregatingTransformParams(const Aggregator::Params & params_, bool final_, bool emit_version_, bool emit_changelog_, bool fill_missing_window_)
2728
: aggregator(params_)
2829
, params(aggregator.getParams())
2930
, final(final_)
3031
, emit_version(emit_version_)
3132
, emit_changelog(emit_changelog_)
33+
, fill_missing_window(fill_missing_window_)
3234
{
3335
if (emit_version)
3436
version_type = DataTypeFactory::instance().get("int64");

src/Processors/Transforms/Streaming/TumbleAggregatingTransform.cpp

+37
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,43 @@ WindowsWithBuckets TumbleAggregatingTransform::getLocalFinalizedWindowsWithBucke
7979
{time_bucket}});
8080
}
8181

82+
if (params->fill_missing_window && many_data->finalized_window_end.load(std::memory_order_relaxed) != INVALID_WATERMARK)
83+
{
84+
auto next_window_end = addTime(
85+
many_data->finalized_window_end.load(std::memory_order_relaxed),
86+
window_params.interval_kind,
87+
window_params.window_interval,
88+
*window_params.time_zone,
89+
window_params.time_scale);
90+
91+
auto it = windows_with_buckets.begin();
92+
while (next_window_end <= watermark_)
93+
{
94+
/// Add missing window if not exist
95+
if (it == windows_with_buckets.end() || next_window_end != it->window.end) [[unlikely]]
96+
{
97+
Window missing_window
98+
= {addTime(
99+
next_window_end,
100+
window_params.interval_kind,
101+
-window_params.window_interval,
102+
*window_params.time_zone,
103+
window_params.time_scale),
104+
next_window_end};
105+
it = windows_with_buckets.insert(it, WindowWithBuckets{.window = std::move(missing_window), .buckets = {}});
106+
}
107+
108+
next_window_end = addTime(
109+
next_window_end,
110+
window_params.interval_kind,
111+
window_params.window_interval,
112+
*window_params.time_zone,
113+
window_params.time_scale);
114+
115+
++it;
116+
}
117+
}
118+
82119
return windows_with_buckets;
83120
}
84121

src/Processors/Transforms/Streaming/TumbleAggregatingTransformWithSubstream.cpp

+44
Original file line numberDiff line numberDiff line change
@@ -65,6 +65,50 @@ TumbleAggregatingTransformWithSubstream::getFinalizedWindowsWithBuckets(Int64 wa
6565
{time_bucket}});
6666
}
6767

68+
if (params->fill_missing_window && substream_ctx->finalized_watermark != INVALID_WATERMARK)
69+
{
70+
auto finalized_window_start = toStartTime(
71+
substream_ctx->finalized_watermark,
72+
window_params.interval_kind,
73+
window_params.window_interval,
74+
*window_params.time_zone,
75+
window_params.time_scale);
76+
77+
auto next_window_end = addTime(
78+
finalized_window_start,
79+
window_params.interval_kind,
80+
2 * window_params.window_interval,
81+
*window_params.time_zone,
82+
window_params.time_scale);
83+
84+
auto it = windows_with_buckets.begin();
85+
while (next_window_end <= watermark)
86+
{
87+
/// Add missing window if not exist
88+
if (it == windows_with_buckets.end() || next_window_end != it->window.end) [[unlikely]]
89+
{
90+
Window missing_window
91+
= {addTime(
92+
next_window_end,
93+
window_params.interval_kind,
94+
-window_params.window_interval,
95+
*window_params.time_zone,
96+
window_params.time_scale),
97+
next_window_end};
98+
it = windows_with_buckets.insert(it, WindowWithBuckets{.window = std::move(missing_window), .buckets = {}});
99+
}
100+
101+
next_window_end = addTime(
102+
next_window_end,
103+
window_params.interval_kind,
104+
window_params.window_interval,
105+
*window_params.time_zone,
106+
window_params.time_scale);
107+
108+
++it;
109+
}
110+
}
111+
68112
return windows_with_buckets;
69113
}
70114

src/Processors/Transforms/Streaming/WindowAggregatingTransform.cpp

+15-3
Original file line numberDiff line numberDiff line change
@@ -131,10 +131,22 @@ void WindowAggregatingTransform::finalize(const ChunkContextPtr & chunk_ctx)
131131
assert(!prepared_windows_with_buckets.empty());
132132
for (const auto & window_with_buckets : prepared_windows_with_buckets)
133133
{
134-
chunk = AggregatingHelper::mergeAndSpliceAndConvertBucketsToChunk(many_data->variants, *params, window_with_buckets.buckets);
134+
/// No buckets means it's empty window. We should fill it with default values, it's only possible for enable fill missing window
135+
if (window_with_buckets.buckets.empty())
136+
{
137+
assert(params->fill_missing_window);
138+
if (!chunk.hasColumns())
139+
chunk.setColumns(params->aggregator.getHeader(params->final).cloneEmptyColumns(), 0);
140+
141+
addMissingWindow(chunk, window_with_buckets.window, params->params.window_params->time_col_is_datetime64, window_start_col_pos, window_end_col_pos);
142+
}
143+
else
144+
{
145+
chunk = AggregatingHelper::mergeAndSpliceAndConvertBucketsToChunk(many_data->variants, *params, window_with_buckets.buckets);
135146

136-
if (needReassignWindow())
137-
reassignWindow(chunk, window_with_buckets.window, params->params.window_params->time_col_is_datetime64, window_start_col_pos, window_end_col_pos);
147+
if (needReassignWindow())
148+
reassignWindow(chunk, window_with_buckets.window, params->params.window_params->time_col_is_datetime64, window_start_col_pos, window_end_col_pos);
149+
}
138150

139151
if (params->emit_version && params->final)
140152
emitVersion(chunk);

src/Processors/Transforms/Streaming/WindowAggregatingTransformWithSubstream.cpp

+16-4
Original file line numberDiff line numberDiff line change
@@ -73,10 +73,22 @@ void WindowAggregatingTransformWithSubstream::doFinalize(
7373
&& window_with_buckets.window.end <= last_finalized_windows_with_buckets.back().window.end)
7474
continue;
7575

76-
chunk = AggregatingHelper::spliceAndConvertBucketsToChunk(data_variant, *params, window_with_buckets.buckets);
77-
78-
if (needReassignWindow())
79-
reassignWindow(chunk, window_with_buckets.window, params->params.window_params->time_col_is_datetime64, window_start_col_pos, window_end_col_pos);
76+
/// No buckets means it's empty window. We should fill it with default values, it's only possible for enable fill missing window
77+
if (window_with_buckets.buckets.empty())
78+
{
79+
assert(params->fill_missing_window);
80+
if (!chunk.hasColumns())
81+
chunk.setColumns(params->aggregator.getHeader(params->final).cloneEmptyColumns(), 0);
82+
83+
addMissingWindow(chunk, window_with_buckets.window, params->params.window_params->time_col_is_datetime64, window_start_col_pos, window_end_col_pos);
84+
}
85+
else
86+
{
87+
chunk = AggregatingHelper::spliceAndConvertBucketsToChunk(data_variant, *params, window_with_buckets.buckets);
88+
89+
if (needReassignWindow())
90+
reassignWindow(chunk, window_with_buckets.window, params->params.window_params->time_col_is_datetime64, window_start_col_pos, window_end_col_pos);
91+
}
8092

8193
if (params->emit_version && params->final)
8294
emitVersion(chunk, substream_ctx);

0 commit comments

Comments
 (0)