1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
|
#include <Processors/Transforms/AggregatingTransform.h>
#include <Formats/NativeReader.h>
#include <Processors/ISource.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
#include <Core/ProtocolDefines.h>
#include <Common/logger_useful.h>
#include <Processors/Transforms/SquashingChunksTransform.h>
namespace ProfileEvents
{
extern const Event ExternalAggregationMerge;
}
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_AGGREGATED_DATA_VARIANT;
extern const int LOGICAL_ERROR;
}
/// Convert block to chunk.
/// Adds additional info about aggregation.
Chunk convertToChunk(const Block & block)
{
auto info = std::make_shared<AggregatedChunkInfo>();
info->bucket_num = block.info.bucket_num;
info->is_overflows = block.info.is_overflows;
UInt64 num_rows = block.rows();
Chunk chunk(block.getColumns(), num_rows);
chunk.setChunkInfo(std::move(info));
return chunk;
}
namespace
{
const AggregatedChunkInfo * getInfoFromChunk(const Chunk & chunk)
{
const auto & info = chunk.getChunkInfo();
if (!info)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk.");
const auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(info.get());
if (!agg_info)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo.");
return agg_info;
}
/// Reads chunks from file in native format. Provide chunks with aggregation info.
class SourceFromNativeStream : public ISource
{
public:
explicit SourceFromNativeStream(TemporaryFileStream * tmp_stream_)
: ISource(tmp_stream_->getHeader())
, tmp_stream(tmp_stream_)
{}
String getName() const override { return "SourceFromNativeStream"; }
Chunk generate() override
{
if (!tmp_stream)
return {};
auto block = tmp_stream->read();
if (!block)
{
tmp_stream = nullptr;
return {};
}
return convertToChunk(block);
}
private:
TemporaryFileStream * tmp_stream;
};
}
/// Worker which merges buckets for two-level aggregation.
/// Atomically increments bucket counter and returns merged result.
class ConvertingAggregatedToChunksWithMergingSource : public ISource
{
public:
static constexpr UInt32 NUM_BUCKETS = 256;
struct SharedData
{
std::atomic<UInt32> next_bucket_to_merge = 0;
std::array<std::atomic<bool>, NUM_BUCKETS> is_bucket_processed{};
std::atomic<bool> is_cancelled = false;
SharedData()
{
for (auto & flag : is_bucket_processed)
flag = false;
}
};
using SharedDataPtr = std::shared_ptr<SharedData>;
ConvertingAggregatedToChunksWithMergingSource(
AggregatingTransformParamsPtr params_, ManyAggregatedDataVariantsPtr data_, SharedDataPtr shared_data_, Arena * arena_)
: ISource(params_->getHeader(), false)
, params(std::move(params_))
, data(std::move(data_))
, shared_data(std::move(shared_data_))
, arena(arena_)
{
}
String getName() const override { return "ConvertingAggregatedToChunksWithMergingSource"; }
protected:
Chunk generate() override
{
UInt32 bucket_num = shared_data->next_bucket_to_merge.fetch_add(1);
if (bucket_num >= NUM_BUCKETS)
return {};
Block block = params->aggregator.mergeAndConvertOneBucketToBlock(*data, arena, params->final, bucket_num, &shared_data->is_cancelled);
Chunk chunk = convertToChunk(block);
shared_data->is_bucket_processed[bucket_num] = true;
return chunk;
}
private:
AggregatingTransformParamsPtr params;
ManyAggregatedDataVariantsPtr data;
SharedDataPtr shared_data;
Arena * arena;
};
/// Asks Aggregator to convert accumulated aggregation state into blocks (without merging) and pushes them to later steps.
class ConvertingAggregatedToChunksSource : public ISource
{
public:
ConvertingAggregatedToChunksSource(AggregatingTransformParamsPtr params_, AggregatedDataVariantsPtr variant_)
: ISource(params_->getHeader(), false), params(params_), variant(variant_)
{
}
String getName() const override { return "ConvertingAggregatedToChunksSource"; }
protected:
Chunk generate() override
{
if (variant->isTwoLevel())
{
if (current_bucket_num < NUM_BUCKETS)
{
Arena * arena = variant->aggregates_pool;
Block block = params->aggregator.convertOneBucketToBlock(*variant, arena, params->final, current_bucket_num++);
return convertToChunk(block);
}
}
else if (!single_level_converted)
{
Block block = params->aggregator.prepareBlockAndFillSingleLevel<true /* return_single_block */>(*variant, params->final);
single_level_converted = true;
return convertToChunk(block);
}
return {};
}
private:
static constexpr UInt32 NUM_BUCKETS = 256;
AggregatingTransformParamsPtr params;
AggregatedDataVariantsPtr variant;
UInt32 current_bucket_num = 0;
bool single_level_converted = false;
};
/// Reads chunks from GroupingAggregatedTransform (stored in ChunksToMerge structure) and outputs them.
class FlattenChunksToMergeTransform : public IProcessor
{
public:
explicit FlattenChunksToMergeTransform(const Block & input_header, const Block & output_header)
: IProcessor({input_header}, {output_header})
{
}
String getName() const override { return "FlattenChunksToMergeTransform"; }
private:
void work() override
{
}
void process(Chunk && chunk)
{
if (!chunk.hasChunkInfo())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected chunk with chunk info in {}", getName());
const auto & info = chunk.getChunkInfo();
const auto * chunks_to_merge = typeid_cast<const ChunksToMerge *>(info.get());
if (!chunks_to_merge)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected chunk with ChunksToMerge info in {}", getName());
if (chunks_to_merge->chunks)
for (auto & cur_chunk : *chunks_to_merge->chunks)
chunks.emplace_back(std::move(cur_chunk));
}
Status prepare() override
{
auto & input = inputs.front();
auto & output = outputs.front();
if (output.isFinished())
{
input.close();
return Status::Finished;
}
if (!output.canPush())
{
input.setNotNeeded();
return Status::PortFull;
}
if (!chunks.empty())
{
output.push(std::move(chunks.front()));
chunks.pop_front();
if (!chunks.empty())
return Status::Ready;
}
if (input.isFinished() && chunks.empty())
{
output.finish();
return Status::Finished;
}
if (input.isFinished())
return Status::Ready;
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
Chunk chunk = input.pull(true /* set_not_needed */);
process(std::move(chunk));
return Status::Ready;
}
std::list<Chunk> chunks;
};
/// Generates chunks with aggregated data.
/// In single level case, aggregates data itself.
/// In two-level case, creates `ConvertingAggregatedToChunksWithMergingSource` workers:
///
/// ConvertingAggregatedToChunksWithMergingSource ->
/// ConvertingAggregatedToChunksWithMergingSource -> ConvertingAggregatedToChunksTransform -> AggregatingTransform
/// ConvertingAggregatedToChunksWithMergingSource ->
///
/// Result chunks guaranteed to be sorted by bucket number.
class ConvertingAggregatedToChunksTransform : public IProcessor
{
public:
ConvertingAggregatedToChunksTransform(AggregatingTransformParamsPtr params_, ManyAggregatedDataVariantsPtr data_, size_t num_threads_)
: IProcessor({}, {params_->getHeader()}), params(std::move(params_)), data(std::move(data_)), num_threads(num_threads_)
{
}
String getName() const override { return "ConvertingAggregatedToChunksTransform"; }
void work() override
{
if (data->empty())
{
finished = true;
return;
}
if (!is_initialized)
{
initialize();
return;
}
if (data->at(0)->isTwoLevel())
{
/// In two-level case will only create sources.
if (inputs.empty())
createSources();
}
else
{
mergeSingleLevel();
}
}
Processors expandPipeline() override
{
for (auto & source : processors)
{
auto & out = source->getOutputs().front();
inputs.emplace_back(out.getHeader(), this);
connect(out, inputs.back());
inputs.back().setNeeded();
}
return std::move(processors);
}
IProcessor::Status prepare() override
{
auto & output = outputs.front();
if (finished && single_level_chunks.empty())
{
output.finish();
return Status::Finished;
}
/// Check can output.
if (output.isFinished())
{
for (auto & input : inputs)
input.close();
if (shared_data)
shared_data->is_cancelled.store(true);
return Status::Finished;
}
if (!output.canPush())
return Status::PortFull;
if (!is_initialized)
return Status::Ready;
if (!processors.empty())
return Status::ExpandPipeline;
if (!single_level_chunks.empty())
return preparePushToOutput();
/// Single level case.
if (inputs.empty())
return Status::Ready;
/// Two-level case.
return prepareTwoLevel();
}
private:
IProcessor::Status preparePushToOutput()
{
if (single_level_chunks.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Some ready chunks expected");
auto & output = outputs.front();
auto chunk = std::move(single_level_chunks.back());
single_level_chunks.pop_back();
const auto has_rows = chunk.hasRows();
if (has_rows)
output.push(std::move(chunk));
if (finished && single_level_chunks.empty())
{
output.finish();
return Status::Finished;
}
return has_rows ? Status::PortFull : Status::Ready;
}
/// Read all sources and try to push current bucket.
IProcessor::Status prepareTwoLevel()
{
auto & output = outputs.front();
for (auto & input : inputs)
{
if (!input.isFinished() && input.hasData())
{
auto chunk = input.pull();
auto bucket = getInfoFromChunk(chunk)->bucket_num;
two_level_chunks[bucket] = std::move(chunk);
}
}
if (!shared_data->is_bucket_processed[current_bucket_num])
return Status::NeedData;
if (!two_level_chunks[current_bucket_num])
return Status::NeedData;
auto chunk = std::move(two_level_chunks[current_bucket_num]);
const auto has_rows = chunk.hasRows();
if (has_rows)
output.push(std::move(chunk));
++current_bucket_num;
if (current_bucket_num == NUM_BUCKETS)
{
output.finish();
/// Do not close inputs, they must be finished.
return Status::Finished;
}
return has_rows ? Status::PortFull : Status::Ready;
}
AggregatingTransformParamsPtr params;
ManyAggregatedDataVariantsPtr data;
ConvertingAggregatedToChunksWithMergingSource::SharedDataPtr shared_data;
size_t num_threads;
bool is_initialized = false;
bool finished = false;
Chunks single_level_chunks;
UInt32 current_bucket_num = 0;
static constexpr Int32 NUM_BUCKETS = 256;
std::array<Chunk, NUM_BUCKETS> two_level_chunks;
Processors processors;
void initialize()
{
is_initialized = true;
AggregatedDataVariantsPtr & first = data->at(0);
/// At least we need one arena in first data item per thread
if (num_threads > first->aggregates_pools.size())
{
Arenas & first_pool = first->aggregates_pools;
for (size_t j = first_pool.size(); j < num_threads; ++j)
first_pool.emplace_back(std::make_shared<Arena>());
}
if (first->type == AggregatedDataVariants::Type::without_key || params->params.overflow_row)
{
params->aggregator.mergeWithoutKeyDataImpl(*data);
auto block = params->aggregator.prepareBlockAndFillWithoutKey(
*first, params->final, first->type != AggregatedDataVariants::Type::without_key);
single_level_chunks.emplace_back(convertToChunk(block));
}
}
void mergeSingleLevel()
{
AggregatedDataVariantsPtr & first = data->at(0);
if (current_bucket_num > 0 || first->type == AggregatedDataVariants::Type::without_key)
{
finished = true;
return;
}
++current_bucket_num;
#define M(NAME) \
else if (first->type == AggregatedDataVariants::Type::NAME) \
params->aggregator.mergeSingleLevelDataImpl<decltype(first->NAME)::element_type>(*data);
if (false) {} // NOLINT
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
auto blocks = params->aggregator.prepareBlockAndFillSingleLevel</* return_single_block */ false>(*first, params->final);
for (auto & block : blocks)
single_level_chunks.emplace_back(convertToChunk(block));
finished = true;
}
void createSources()
{
AggregatedDataVariantsPtr & first = data->at(0);
shared_data = std::make_shared<ConvertingAggregatedToChunksWithMergingSource::SharedData>();
for (size_t thread = 0; thread < num_threads; ++thread)
{
/// Select Arena to avoid race conditions
Arena * arena = first->aggregates_pools.at(thread).get();
auto source = std::make_shared<ConvertingAggregatedToChunksWithMergingSource>(params, data, shared_data, arena);
processors.emplace_back(std::move(source));
}
}
};
AggregatingTransform::AggregatingTransform(Block header, AggregatingTransformParamsPtr params_)
: AggregatingTransform(
std::move(header),
std::move(params_),
std::make_unique<ManyAggregatedData>(1),
0,
1,
1,
true /* should_produce_results_in_order_of_bucket_number */,
false /* skip_merging */)
{
}
AggregatingTransform::AggregatingTransform(
Block header,
AggregatingTransformParamsPtr params_,
ManyAggregatedDataPtr many_data_,
size_t current_variant,
size_t max_threads_,
size_t temporary_data_merge_threads_,
bool should_produce_results_in_order_of_bucket_number_,
bool skip_merging_)
: IProcessor({std::move(header)}, {params_->getHeader()})
, params(std::move(params_))
, key_columns(params->params.keys_size)
, aggregate_columns(params->params.aggregates_size)
, many_data(std::move(many_data_))
, variants(*many_data->variants[current_variant])
, max_threads(std::min(many_data->variants.size(), max_threads_))
, temporary_data_merge_threads(temporary_data_merge_threads_)
, should_produce_results_in_order_of_bucket_number(should_produce_results_in_order_of_bucket_number_)
, skip_merging(skip_merging_)
{
}
AggregatingTransform::~AggregatingTransform() = default;
IProcessor::Status AggregatingTransform::prepare()
{
/// There are one or two input ports.
/// The first one is used at aggregation step, the second one - while reading merged data from ConvertingAggregated
auto & output = outputs.front();
/// Last output is current. All other outputs should already be closed.
auto & input = inputs.back();
/// Check can output.
if (output.isFinished())
{
input.close();
return Status::Finished;
}
if (!output.canPush())
{
input.setNotNeeded();
return Status::PortFull;
}
/// Finish data processing, prepare to generating.
if (is_consume_finished && !is_generate_initialized)
{
/// Close input port in case max_rows_to_group_by was reached but not all data was read.
inputs.front().close();
return Status::Ready;
}
if (is_generate_initialized && !is_pipeline_created && !processors.empty())
return Status::ExpandPipeline;
/// Only possible while consuming.
if (read_current_chunk)
return Status::Ready;
/// Get chunk from input.
if (input.isFinished())
{
if (is_consume_finished)
{
output.finish();
return Status::Finished;
}
else
{
/// Finish data processing and create another pipe.
is_consume_finished = true;
return Status::Ready;
}
}
if (!input.hasData())
{
input.setNeeded();
return Status::NeedData;
}
if (is_consume_finished)
input.setNeeded();
current_chunk = input.pull(/*set_not_needed = */ !is_consume_finished);
read_current_chunk = true;
if (is_consume_finished)
{
output.push(std::move(current_chunk));
read_current_chunk = false;
return Status::PortFull;
}
return Status::Ready;
}
void AggregatingTransform::work()
{
if (is_consume_finished)
initGenerate();
else
{
consume(std::move(current_chunk));
read_current_chunk = false;
}
}
Processors AggregatingTransform::expandPipeline()
{
if (processors.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not expandPipeline in AggregatingTransform. This is a bug.");
auto & out = processors.back()->getOutputs().front();
inputs.emplace_back(out.getHeader(), this);
connect(out, inputs.back());
is_pipeline_created = true;
return std::move(processors);
}
void AggregatingTransform::consume(Chunk chunk)
{
const UInt64 num_rows = chunk.getNumRows();
if (num_rows == 0 && params->params.empty_result_for_aggregation_by_empty_set)
return;
if (!is_consume_started)
{
LOG_TRACE(log, "Aggregating");
is_consume_started = true;
}
src_rows += num_rows;
src_bytes += chunk.bytes();
if (params->params.only_merge)
{
auto block = getInputs().front().getHeader().cloneWithColumns(chunk.detachColumns());
block = materializeBlock(block);
if (!params->aggregator.mergeOnBlock(block, variants, no_more_keys))
is_consume_finished = true;
}
else
{
if (!params->aggregator.executeOnBlock(chunk.detachColumns(), 0, num_rows, variants, key_columns, aggregate_columns, no_more_keys))
is_consume_finished = true;
}
}
void AggregatingTransform::initGenerate()
{
if (is_generate_initialized)
return;
is_generate_initialized = true;
/// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation.
/// To do this, we pass a block with zero rows to aggregate.
if (variants.empty() && params->params.keys_size == 0 && !params->params.empty_result_for_aggregation_by_empty_set)
{
if (params->params.only_merge)
params->aggregator.mergeOnBlock(getInputs().front().getHeader(), variants, no_more_keys);
else
params->aggregator.executeOnBlock(getInputs().front().getHeader(), variants, key_columns, aggregate_columns, no_more_keys);
}
double elapsed_seconds = watch.elapsedSeconds();
size_t rows = variants.sizeWithoutOverflowRow();
LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({:.3f} rows/sec., {}/sec.)",
src_rows, rows, ReadableSize(src_bytes),
elapsed_seconds, src_rows / elapsed_seconds,
ReadableSize(src_bytes / elapsed_seconds));
if (params->aggregator.hasTemporaryData())
{
if (variants.isConvertibleToTwoLevel())
variants.convertToTwoLevel();
/// Flush data in the RAM to disk also. It's easier than merging on-disk and RAM data.
if (!variants.empty())
params->aggregator.writeToTemporaryFile(variants);
}
if (many_data->num_finished.fetch_add(1) + 1 < many_data->variants.size())
return;
if (!params->aggregator.hasTemporaryData())
{
if (!skip_merging)
{
auto prepared_data = params->aggregator.prepareVariantsToMerge(many_data->variants);
auto prepared_data_ptr = std::make_shared<ManyAggregatedDataVariants>(std::move(prepared_data));
processors.emplace_back(
std::make_shared<ConvertingAggregatedToChunksTransform>(params, std::move(prepared_data_ptr), max_threads));
}
else
{
auto prepared_data = params->aggregator.prepareVariantsToMerge(many_data->variants);
Pipes pipes;
for (auto & variant : prepared_data)
/// Converts hash tables to blocks with data (finalized or not).
pipes.emplace_back(std::make_shared<ConvertingAggregatedToChunksSource>(params, variant));
Pipe pipe = Pipe::unitePipes(std::move(pipes));
if (!pipe.empty())
{
if (should_produce_results_in_order_of_bucket_number)
{
/// Groups chunks with the same bucket_id and outputs them (as a vector of chunks) in order of bucket_id.
pipe.addTransform(std::make_shared<GroupingAggregatedTransform>(pipe.getHeader(), pipe.numOutputPorts(), params));
/// Outputs one chunk from group at a time in order of bucket_id.
pipe.addTransform(std::make_shared<FlattenChunksToMergeTransform>(pipe.getHeader(), params->getHeader()));
}
else
{
/// If this is a final stage, we no longer have to keep chunks from different buckets into different chunks.
/// So now we can insert transform that will keep chunks size under control. It makes few times difference in exec time in some cases.
if (params->final)
{
pipe.addSimpleTransform(
[this](const Block & header)
{
/// Just a reasonable constant, matches default value for the setting `preferred_block_size_bytes`
static constexpr size_t oneMB = 1024 * 1024;
return std::make_shared<SimpleSquashingChunksTransform>(header, params->params.max_block_size, oneMB);
});
}
/// AggregatingTransform::expandPipeline expects single output port.
/// It's not a big problem because we do resize() to max_threads after AggregatingTransform.
pipe.resize(1);
}
}
processors = Pipe::detachProcessors(std::move(pipe));
}
}
else
{
/// If there are temporary files with partially-aggregated data on the disk,
/// then read and merge them, spending the minimum amount of memory.
ProfileEvents::increment(ProfileEvents::ExternalAggregationMerge);
if (many_data->variants.size() > 1)
{
/// It may happen that some data has not yet been flushed,
/// because at the time thread has finished, no data has been flushed to disk, and then some were.
for (auto & cur_variants : many_data->variants)
{
if (cur_variants->isConvertibleToTwoLevel())
cur_variants->convertToTwoLevel();
if (!cur_variants->empty())
params->aggregator.writeToTemporaryFile(*cur_variants);
}
}
const auto & tmp_data = params->aggregator.getTemporaryData();
Pipe pipe;
{
Pipes pipes;
for (auto * tmp_stream : tmp_data.getStreams())
pipes.emplace_back(Pipe(std::make_unique<SourceFromNativeStream>(tmp_stream)));
pipe = Pipe::unitePipes(std::move(pipes));
}
size_t num_streams = tmp_data.getStreams().size();
size_t compressed_size = tmp_data.getStat().compressed_size;
size_t uncompressed_size = tmp_data.getStat().uncompressed_size;
LOG_DEBUG(
log,
"Will merge {} temporary files of size {} compressed, {} uncompressed.",
num_streams,
ReadableSize(compressed_size),
ReadableSize(uncompressed_size));
addMergingAggregatedMemoryEfficientTransform(pipe, params, temporary_data_merge_threads);
processors = Pipe::detachProcessors(std::move(pipe));
}
}
}
|