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
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258
3259
3260
3261
3262
3263
3264
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274
3275
3276
3277
3278
3279
3280
3281
3282
3283
3284
3285
3286
3287
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311
3312
3313
3314
3315
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363
3364
3365
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375
3376
3377
3378
3379
3380
3381
3382
3383
3384
3385
3386
3387
3388
3389
3390
3391
3392
3393
3394
3395
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405
3406
3407
3408
3409
3410
3411
3412
3413
3414
3415
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
|
#include <algorithm>
#include <future>
#include <numeric>
#include <Poco/Util/Application.h>
#ifdef OS_LINUX
# include <unistd.h>
#endif
#include <base/sort.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnSparse.h>
#include <Formats/NativeWriter.h>
#include <IO/WriteBufferFromFile.h>
#include <Compression/CompressedWriteBuffer.h>
#include <Interpreters/Aggregator.h>
#include <AggregateFunctions/AggregateFunctionArray.h>
#include <AggregateFunctions/AggregateFunctionState.h>
#include <IO/Operators.h>
#include <Interpreters/JIT/compileFunction.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Core/ProtocolDefines.h>
#include <Disks/TemporaryFileOnDisk.h>
#include <Interpreters/TemporaryDataOnDisk.h>
#include <Common/Stopwatch.h>
#include <Common/setThreadName.h>
#include <Common/formatReadable.h>
#include <Common/logger_useful.h>
#include <Common/CacheBase.h>
#include <Common/MemoryTracker.h>
#include <Common/CurrentThread.h>
#include <Common/CurrentMetrics.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Common/JSONBuilder.h>
#include <Common/filesystemHelpers.h>
#include <Common/scope_guard_safe.h>
#include <Parsers/ASTSelectQuery.h>
#include <Interpreters/AggregationUtils.h>
namespace ProfileEvents
{
extern const Event ExternalAggregationWritePart;
extern const Event ExternalAggregationCompressedBytes;
extern const Event ExternalAggregationUncompressedBytes;
extern const Event ExternalProcessingCompressedBytesTotal;
extern const Event ExternalProcessingUncompressedBytesTotal;
extern const Event AggregationPreallocatedElementsInHashTables;
extern const Event AggregationHashTablesInitializedAsTwoLevel;
extern const Event OverflowThrow;
extern const Event OverflowBreak;
extern const Event OverflowAny;
}
namespace CurrentMetrics
{
extern const Metric TemporaryFilesForAggregation;
extern const Metric AggregatorThreads;
extern const Metric AggregatorThreadsActive;
}
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_AGGREGATED_DATA_VARIANT;
extern const int TOO_MANY_ROWS;
extern const int EMPTY_DATA_PASSED;
extern const int CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS;
extern const int LOGICAL_ERROR;
}
}
namespace
{
/** Collects observed HashMap-s sizes to avoid redundant intermediate resizes.
*/
class HashTablesStatistics
{
public:
struct Entry
{
size_t sum_of_sizes; // used to determine if it's better to convert aggregation to two-level from the beginning
size_t median_size; // roughly the size we're going to preallocate on each thread
};
using Cache = DB::CacheBase<UInt64, Entry>;
using CachePtr = std::shared_ptr<Cache>;
using Params = DB::Aggregator::Params::StatsCollectingParams;
/// Collection and use of the statistics should be enabled.
std::optional<Entry> getSizeHint(const Params & params)
{
if (!params.isCollectionAndUseEnabled())
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled.");
std::lock_guard lock(mutex);
const auto cache = getHashTableStatsCache(params, lock);
if (const auto hint = cache->get(params.key))
{
LOG_TRACE(
&Poco::Logger::get("Aggregator"),
"An entry for key={} found in cache: sum_of_sizes={}, median_size={}",
params.key,
hint->sum_of_sizes,
hint->median_size);
return *hint;
}
return std::nullopt;
}
/// Collection and use of the statistics should be enabled.
void update(size_t sum_of_sizes, size_t median_size, const Params & params)
{
if (!params.isCollectionAndUseEnabled())
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled.");
std::lock_guard lock(mutex);
const auto cache = getHashTableStatsCache(params, lock);
const auto hint = cache->get(params.key);
// We'll maintain the maximum among all the observed values until the next prediction turns out to be too wrong.
if (!hint || sum_of_sizes < hint->sum_of_sizes / 2 || hint->sum_of_sizes < sum_of_sizes || median_size < hint->median_size / 2
|| hint->median_size < median_size)
{
LOG_TRACE(
&Poco::Logger::get("Aggregator"),
"Statistics updated for key={}: new sum_of_sizes={}, median_size={}",
params.key,
sum_of_sizes,
median_size);
cache->set(params.key, std::make_shared<Entry>(Entry{.sum_of_sizes = sum_of_sizes, .median_size = median_size}));
}
}
std::optional<DB::HashTablesCacheStatistics> getCacheStats() const
{
std::lock_guard lock(mutex);
if (hash_table_stats)
{
size_t hits = 0, misses = 0;
hash_table_stats->getStats(hits, misses);
return DB::HashTablesCacheStatistics{.entries = hash_table_stats->count(), .hits = hits, .misses = misses};
}
return std::nullopt;
}
static size_t calculateCacheKey(const DB::ASTPtr & select_query)
{
if (!select_query)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Query ptr cannot be null");
const auto & select = select_query->as<DB::ASTSelectQuery &>();
// It may happen in some corner cases like `select 1 as num group by num`.
if (!select.tables())
return 0;
SipHash hash;
hash.update(select.tables()->getTreeHash());
if (const auto where = select.where())
hash.update(where->getTreeHash());
if (const auto group_by = select.groupBy())
hash.update(group_by->getTreeHash());
return hash.get64();
}
private:
CachePtr getHashTableStatsCache(const Params & params, const std::lock_guard<std::mutex> &)
{
if (!hash_table_stats || hash_table_stats->maxSizeInBytes() != params.max_entries_for_hash_table_stats)
hash_table_stats = std::make_shared<Cache>(params.max_entries_for_hash_table_stats);
return hash_table_stats;
}
mutable std::mutex mutex;
CachePtr hash_table_stats;
};
HashTablesStatistics & getHashTablesStatistics()
{
static HashTablesStatistics hash_tables_stats;
return hash_tables_stats;
}
bool worthConvertToTwoLevel(
size_t group_by_two_level_threshold, size_t result_size, size_t group_by_two_level_threshold_bytes, auto result_size_bytes)
{
// params.group_by_two_level_threshold will be equal to 0 if we have only one thread to execute aggregation (refer to AggregatingStep::transformPipeline).
return (group_by_two_level_threshold && result_size >= group_by_two_level_threshold)
|| (group_by_two_level_threshold_bytes && result_size_bytes >= static_cast<Int64>(group_by_two_level_threshold_bytes));
}
DB::AggregatedDataVariants::Type convertToTwoLevelTypeIfPossible(DB::AggregatedDataVariants::Type type)
{
using Type = DB::AggregatedDataVariants::Type;
switch (type)
{
#define M(NAME) \
case Type::NAME: \
return Type::NAME##_two_level;
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
#undef M
default:
return type;
}
UNREACHABLE();
}
void initDataVariantsWithSizeHint(
DB::AggregatedDataVariants & result, DB::AggregatedDataVariants::Type method_chosen, const DB::Aggregator::Params & params)
{
const auto & stats_collecting_params = params.stats_collecting_params;
if (stats_collecting_params.isCollectionAndUseEnabled())
{
if (auto hint = getHashTablesStatistics().getSizeHint(stats_collecting_params))
{
const auto max_threads = params.group_by_two_level_threshold != 0 ? std::max(params.max_threads, 1ul) : 1;
const auto lower_limit = hint->sum_of_sizes / max_threads;
const auto upper_limit = stats_collecting_params.max_size_to_preallocate_for_aggregation / max_threads;
if (hint->median_size > upper_limit)
{
/// Since we cannot afford to preallocate as much as we want, we will likely need to do resize anyway.
/// But we will also work with the big (i.e. not so cache friendly) HT from the beginning which may result in a slight slowdown.
/// So let's just do nothing.
LOG_TRACE(
&Poco::Logger::get("Aggregator"),
"No space were preallocated in hash tables because 'max_size_to_preallocate_for_aggregation' has too small value: {}, "
"should be at least {}",
stats_collecting_params.max_size_to_preallocate_for_aggregation,
hint->median_size * max_threads);
}
/// https://github.com/ClickHouse/ClickHouse/issues/44402#issuecomment-1359920703
else if ((max_threads > 1 && hint->sum_of_sizes > 100'000) || hint->sum_of_sizes > 500'000)
{
const auto adjusted = std::max(lower_limit, hint->median_size);
if (worthConvertToTwoLevel(
params.group_by_two_level_threshold,
hint->sum_of_sizes,
/*group_by_two_level_threshold_bytes*/ 0,
/*result_size_bytes*/ 0))
method_chosen = convertToTwoLevelTypeIfPossible(method_chosen);
result.init(method_chosen, adjusted);
ProfileEvents::increment(ProfileEvents::AggregationHashTablesInitializedAsTwoLevel, result.isTwoLevel());
return;
}
}
}
result.init(method_chosen);
}
/// Collection and use of the statistics should be enabled.
void updateStatistics(const DB::ManyAggregatedDataVariants & data_variants, const DB::Aggregator::Params::StatsCollectingParams & params)
{
if (!params.isCollectionAndUseEnabled())
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled.");
std::vector<size_t> sizes(data_variants.size());
for (size_t i = 0; i < data_variants.size(); ++i)
sizes[i] = data_variants[i]->size();
const auto median_size = sizes.begin() + sizes.size() / 2; // not precisely though...
std::nth_element(sizes.begin(), median_size, sizes.end());
const auto sum_of_sizes = std::accumulate(sizes.begin(), sizes.end(), 0ull);
getHashTablesStatistics().update(sum_of_sizes, *median_size, params);
}
// The std::is_constructible trait isn't suitable here because some classes have template constructors with semantics different from providing size hints.
// Also string hash table variants are not supported due to the fact that both local perf tests and tests in CI showed slowdowns for them.
template <typename...>
struct HasConstructorOfNumberOfElements : std::false_type
{
};
template <typename... Ts>
struct HasConstructorOfNumberOfElements<HashMapTable<Ts...>> : std::true_type
{
};
template <typename Key, typename Cell, typename Hash, typename Grower, typename Allocator, template <typename...> typename ImplTable>
struct HasConstructorOfNumberOfElements<TwoLevelHashMapTable<Key, Cell, Hash, Grower, Allocator, ImplTable>> : std::true_type
{
};
template <typename... Ts>
struct HasConstructorOfNumberOfElements<HashTable<Ts...>> : std::true_type
{
};
template <typename... Ts>
struct HasConstructorOfNumberOfElements<TwoLevelHashTable<Ts...>> : std::true_type
{
};
template <template <typename> typename Method, typename Base>
struct HasConstructorOfNumberOfElements<Method<Base>> : HasConstructorOfNumberOfElements<Base>
{
};
template <typename Method>
auto constructWithReserveIfPossible(size_t size_hint)
{
if constexpr (HasConstructorOfNumberOfElements<typename Method::Data>::value)
{
ProfileEvents::increment(ProfileEvents::AggregationPreallocatedElementsInHashTables, size_hint);
return std::make_unique<Method>(size_hint);
}
else
return std::make_unique<Method>();
}
DB::ColumnNumbers calculateKeysPositions(const DB::Block & header, const DB::Aggregator::Params & params)
{
DB::ColumnNumbers keys_positions(params.keys_size);
for (size_t i = 0; i < params.keys_size; ++i)
keys_positions[i] = header.getPositionByName(params.keys[i]);
return keys_positions;
}
template <typename HashTable, typename KeyHolder>
concept HasPrefetchMemberFunc = requires
{
{std::declval<HashTable>().prefetch(std::declval<KeyHolder>())};
};
size_t getMinBytesForPrefetch()
{
size_t l2_size = 0;
#if defined(OS_LINUX) && defined(_SC_LEVEL2_CACHE_SIZE)
if (auto ret = sysconf(_SC_LEVEL2_CACHE_SIZE); ret != -1)
l2_size = ret;
#endif
/// 256KB looks like a reasonable default L2 size. 4 is empirical constant.
return 4 * std::max<size_t>(l2_size, 256 * 1024);
}
}
namespace DB
{
AggregatedDataVariants::~AggregatedDataVariants()
{
if (aggregator && !aggregator->all_aggregates_has_trivial_destructor)
{
try
{
aggregator->destroyAllAggregateStates(*this);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
std::optional<HashTablesCacheStatistics> getHashTablesCacheStatistics()
{
return getHashTablesStatistics().getCacheStats();
}
void AggregatedDataVariants::convertToTwoLevel()
{
if (aggregator)
LOG_TRACE(aggregator->log, "Converting aggregation data to two-level.");
switch (type)
{
#define M(NAME) \
case Type::NAME: \
NAME ## _two_level = std::make_unique<decltype(NAME ## _two_level)::element_type>(*(NAME)); \
(NAME).reset(); \
type = Type::NAME ## _two_level; \
break;
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
#undef M
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong data variant passed.");
}
}
void AggregatedDataVariants::init(Type type_, std::optional<size_t> size_hint)
{
switch (type_)
{
case Type::EMPTY:
case Type::without_key:
break;
#define M(NAME, IS_TWO_LEVEL) \
case Type::NAME: \
if (size_hint) \
(NAME) = constructWithReserveIfPossible<decltype(NAME)::element_type>(*size_hint); \
else \
(NAME) = std::make_unique<decltype(NAME)::element_type>(); \
break;
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}
type = type_;
}
Aggregator::Params::StatsCollectingParams::StatsCollectingParams() = default;
Aggregator::Params::StatsCollectingParams::StatsCollectingParams(
const ASTPtr & select_query_,
bool collect_hash_table_stats_during_aggregation_,
size_t max_entries_for_hash_table_stats_,
size_t max_size_to_preallocate_for_aggregation_)
: key(collect_hash_table_stats_during_aggregation_ ? HashTablesStatistics::calculateCacheKey(select_query_) : 0)
, max_entries_for_hash_table_stats(max_entries_for_hash_table_stats_)
, max_size_to_preallocate_for_aggregation(max_size_to_preallocate_for_aggregation_)
{
}
Block Aggregator::getHeader(bool final) const
{
return params.getHeader(header, final);
}
Block Aggregator::Params::getHeader(
const Block & header, bool only_merge, const Names & keys, const AggregateDescriptions & aggregates, bool final)
{
Block res;
if (only_merge)
{
NameSet needed_columns(keys.begin(), keys.end());
for (const auto & aggregate : aggregates)
needed_columns.emplace(aggregate.column_name);
for (const auto & column : header)
{
if (needed_columns.contains(column.name))
res.insert(column.cloneEmpty());
}
if (final)
{
for (const auto & aggregate : aggregates)
{
auto & elem = res.getByName(aggregate.column_name);
elem.type = aggregate.function->getResultType();
elem.column = elem.type->createColumn();
}
}
}
else
{
for (const auto & key : keys)
res.insert(header.getByName(key).cloneEmpty());
for (const auto & aggregate : aggregates)
{
size_t arguments_size = aggregate.argument_names.size();
DataTypes argument_types(arguments_size);
for (size_t j = 0; j < arguments_size; ++j)
argument_types[j] = header.getByName(aggregate.argument_names[j]).type;
DataTypePtr type;
if (final)
type = aggregate.function->getResultType();
else
type = std::make_shared<DataTypeAggregateFunction>(aggregate.function, argument_types, aggregate.parameters);
res.insert({ type, aggregate.column_name });
}
}
return materializeBlock(res);
}
ColumnRawPtrs Aggregator::Params::makeRawKeyColumns(const Block & block) const
{
ColumnRawPtrs key_columns(keys_size);
for (size_t i = 0; i < keys_size; ++i)
key_columns[i] = block.safeGetByPosition(i).column.get();
return key_columns;
}
Aggregator::AggregateColumnsConstData Aggregator::Params::makeAggregateColumnsData(const Block & block) const
{
AggregateColumnsConstData aggregate_columns(aggregates_size);
for (size_t i = 0; i < aggregates_size; ++i)
{
const auto & aggregate_column_name = aggregates[i].column_name;
aggregate_columns[i] = &typeid_cast<const ColumnAggregateFunction &>(*block.getByName(aggregate_column_name).column).getData();
}
return aggregate_columns;
}
void Aggregator::Params::explain(WriteBuffer & out, size_t indent) const
{
String prefix(indent, ' ');
{
/// Dump keys.
out << prefix << "Keys: ";
bool first = true;
for (const auto & key : keys)
{
if (!first)
out << ", ";
first = false;
out << key;
}
out << '\n';
}
if (!aggregates.empty())
{
out << prefix << "Aggregates:\n";
for (const auto & aggregate : aggregates)
aggregate.explain(out, indent + 4);
}
}
void Aggregator::Params::explain(JSONBuilder::JSONMap & map) const
{
auto keys_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & key : keys)
keys_array->add(key);
map.add("Keys", std::move(keys_array));
if (!aggregates.empty())
{
auto aggregates_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & aggregate : aggregates)
{
auto aggregate_map = std::make_unique<JSONBuilder::JSONMap>();
aggregate.explain(*aggregate_map);
aggregates_array->add(std::move(aggregate_map));
}
map.add("Aggregates", std::move(aggregates_array));
}
}
#if USE_EMBEDDED_COMPILER
static CHJIT & getJITInstance()
{
static CHJIT jit;
return jit;
}
class CompiledAggregateFunctionsHolder final : public CompiledExpressionCacheEntry
{
public:
explicit CompiledAggregateFunctionsHolder(CompiledAggregateFunctions compiled_function_)
: CompiledExpressionCacheEntry(compiled_function_.compiled_module.size)
, compiled_aggregate_functions(compiled_function_)
{}
~CompiledAggregateFunctionsHolder() override
{
getJITInstance().deleteCompiledModule(compiled_aggregate_functions.compiled_module);
}
CompiledAggregateFunctions compiled_aggregate_functions;
};
#endif
Aggregator::Aggregator(const Block & header_, const Params & params_)
: header(header_)
, keys_positions(calculateKeysPositions(header, params_))
, params(params_)
, tmp_data(params.tmp_data_scope ? std::make_unique<TemporaryDataOnDisk>(params.tmp_data_scope, CurrentMetrics::TemporaryFilesForAggregation) : nullptr)
, min_bytes_for_prefetch(getMinBytesForPrefetch())
{
/// Use query-level memory tracker
if (auto * memory_tracker_child = CurrentThread::getMemoryTracker())
if (auto * memory_tracker = memory_tracker_child->getParent())
memory_usage_before_aggregation = memory_tracker->get();
aggregate_functions.resize(params.aggregates_size);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i] = params.aggregates[i].function.get();
/// Initialize sizes of aggregation states and its offsets.
offsets_of_aggregate_states.resize(params.aggregates_size);
total_size_of_aggregate_states = 0;
all_aggregates_has_trivial_destructor = true;
// aggregate_states will be aligned as below:
// |<-- state_1 -->|<-- pad_1 -->|<-- state_2 -->|<-- pad_2 -->| .....
//
// pad_N will be used to match alignment requirement for each next state.
// The address of state_1 is aligned based on maximum alignment requirements in states
for (size_t i = 0; i < params.aggregates_size; ++i)
{
offsets_of_aggregate_states[i] = total_size_of_aggregate_states;
total_size_of_aggregate_states += params.aggregates[i].function->sizeOfData();
// aggregate states are aligned based on maximum requirement
align_aggregate_states = std::max(align_aggregate_states, params.aggregates[i].function->alignOfData());
// If not the last aggregate_state, we need pad it so that next aggregate_state will be aligned.
if (i + 1 < params.aggregates_size)
{
size_t alignment_of_next_state = params.aggregates[i + 1].function->alignOfData();
if ((alignment_of_next_state & (alignment_of_next_state - 1)) != 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: alignOfData is not 2^N");
/// Extend total_size to next alignment requirement
/// Add padding by rounding up 'total_size_of_aggregate_states' to be a multiplier of alignment_of_next_state.
total_size_of_aggregate_states = (total_size_of_aggregate_states + alignment_of_next_state - 1) / alignment_of_next_state * alignment_of_next_state;
}
if (!params.aggregates[i].function->hasTrivialDestructor())
all_aggregates_has_trivial_destructor = false;
}
method_chosen = chooseAggregationMethod();
HashMethodContext::Settings cache_settings;
cache_settings.max_threads = params.max_threads;
aggregation_state_cache = AggregatedDataVariants::createCache(method_chosen, cache_settings);
#if USE_EMBEDDED_COMPILER
compileAggregateFunctionsIfNeeded();
#endif
}
#if USE_EMBEDDED_COMPILER
void Aggregator::compileAggregateFunctionsIfNeeded()
{
static std::unordered_map<UInt128, UInt64, UInt128Hash> aggregate_functions_description_to_count;
static std::mutex mutex;
if (!params.compile_aggregate_expressions)
return;
std::vector<AggregateFunctionWithOffset> functions_to_compile;
String functions_description;
is_aggregate_function_compiled.resize(aggregate_functions.size());
/// Add values to the aggregate functions.
for (size_t i = 0; i < aggregate_functions.size(); ++i)
{
const auto * function = aggregate_functions[i];
size_t offset_of_aggregate_function = offsets_of_aggregate_states[i];
if (function->isCompilable())
{
AggregateFunctionWithOffset function_to_compile
{
.function = function,
.aggregate_data_offset = offset_of_aggregate_function
};
functions_to_compile.emplace_back(std::move(function_to_compile));
functions_description += function->getDescription();
functions_description += ' ';
functions_description += std::to_string(offset_of_aggregate_function);
functions_description += ' ';
}
is_aggregate_function_compiled[i] = function->isCompilable();
}
if (functions_to_compile.empty())
return;
SipHash aggregate_functions_description_hash;
aggregate_functions_description_hash.update(functions_description);
const auto aggregate_functions_description_hash_key = aggregate_functions_description_hash.get128();
{
std::lock_guard<std::mutex> lock(mutex);
if (aggregate_functions_description_to_count[aggregate_functions_description_hash_key]++ < params.min_count_to_compile_aggregate_expression)
return;
}
if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
{
auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(aggregate_functions_description_hash_key, [&] ()
{
LOG_TRACE(log, "Compile expression {}", functions_description);
auto compiled_aggregate_functions = compileAggregateFunctions(getJITInstance(), functions_to_compile, functions_description);
return std::make_shared<CompiledAggregateFunctionsHolder>(std::move(compiled_aggregate_functions));
});
compiled_aggregate_functions_holder = std::static_pointer_cast<CompiledAggregateFunctionsHolder>(compiled_function_cache_entry);
}
else
{
LOG_TRACE(log, "Compile expression {}", functions_description);
auto compiled_aggregate_functions = compileAggregateFunctions(getJITInstance(), functions_to_compile, functions_description);
compiled_aggregate_functions_holder = std::make_shared<CompiledAggregateFunctionsHolder>(std::move(compiled_aggregate_functions));
}
}
#endif
AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
{
/// If no keys. All aggregating to single row.
if (params.keys_size == 0)
return AggregatedDataVariants::Type::without_key;
/// Check if at least one of the specified keys is nullable.
DataTypes types_removed_nullable;
types_removed_nullable.reserve(params.keys.size());
bool has_nullable_key = false;
bool has_low_cardinality = false;
for (const auto & key : params.keys)
{
DataTypePtr type = header.getByName(key).type;
if (type->lowCardinality())
{
has_low_cardinality = true;
type = removeLowCardinality(type);
}
if (type->isNullable())
{
has_nullable_key = true;
type = removeNullable(type);
}
types_removed_nullable.push_back(type);
}
/** Returns ordinary (not two-level) methods, because we start from them.
* Later, during aggregation process, data may be converted (partitioned) to two-level structure, if cardinality is high.
*/
size_t keys_bytes = 0;
size_t num_fixed_contiguous_keys = 0;
key_sizes.resize(params.keys_size);
for (size_t j = 0; j < params.keys_size; ++j)
{
if (types_removed_nullable[j]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
{
if (types_removed_nullable[j]->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion())
{
++num_fixed_contiguous_keys;
key_sizes[j] = types_removed_nullable[j]->getSizeOfValueInMemory();
keys_bytes += key_sizes[j];
}
}
}
if (has_nullable_key)
{
/// Optimization for one key
if (params.keys_size == 1 && !has_low_cardinality)
{
if (types_removed_nullable[0]->isValueRepresentedByNumber())
{
size_t size_of_field = types_removed_nullable[0]->getSizeOfValueInMemory();
if (size_of_field == 1)
return AggregatedDataVariants::Type::nullable_key8;
if (size_of_field == 2)
return AggregatedDataVariants::Type::nullable_key16;
if (size_of_field == 4)
return AggregatedDataVariants::Type::nullable_key32;
if (size_of_field == 8)
return AggregatedDataVariants::Type::nullable_key64;
}
if (isFixedString(types_removed_nullable[0]))
{
return AggregatedDataVariants::Type::nullable_key_fixed_string;
}
if (isString(types_removed_nullable[0]))
{
return AggregatedDataVariants::Type::nullable_key_string;
}
}
if (params.keys_size == num_fixed_contiguous_keys && !has_low_cardinality)
{
/// Pack if possible all the keys along with information about which key values are nulls
/// into a fixed 16- or 32-byte blob.
if (std::tuple_size<KeysNullMap<UInt128>>::value + keys_bytes <= 16)
return AggregatedDataVariants::Type::nullable_keys128;
if (std::tuple_size<KeysNullMap<UInt256>>::value + keys_bytes <= 32)
return AggregatedDataVariants::Type::nullable_keys256;
}
if (has_low_cardinality && params.keys_size == 1)
{
if (types_removed_nullable[0]->isValueRepresentedByNumber())
{
size_t size_of_field = types_removed_nullable[0]->getSizeOfValueInMemory();
if (size_of_field == 1)
return AggregatedDataVariants::Type::low_cardinality_key8;
if (size_of_field == 2)
return AggregatedDataVariants::Type::low_cardinality_key16;
if (size_of_field == 4)
return AggregatedDataVariants::Type::low_cardinality_key32;
if (size_of_field == 8)
return AggregatedDataVariants::Type::low_cardinality_key64;
}
else if (isString(types_removed_nullable[0]))
return AggregatedDataVariants::Type::low_cardinality_key_string;
else if (isFixedString(types_removed_nullable[0]))
return AggregatedDataVariants::Type::low_cardinality_key_fixed_string;
}
/// Fallback case.
return AggregatedDataVariants::Type::serialized;
}
/// No key has been found to be nullable.
/// Single numeric key.
if (params.keys_size == 1 && types_removed_nullable[0]->isValueRepresentedByNumber())
{
size_t size_of_field = types_removed_nullable[0]->getSizeOfValueInMemory();
if (has_low_cardinality)
{
if (size_of_field == 1)
return AggregatedDataVariants::Type::low_cardinality_key8;
if (size_of_field == 2)
return AggregatedDataVariants::Type::low_cardinality_key16;
if (size_of_field == 4)
return AggregatedDataVariants::Type::low_cardinality_key32;
if (size_of_field == 8)
return AggregatedDataVariants::Type::low_cardinality_key64;
if (size_of_field == 16)
return AggregatedDataVariants::Type::low_cardinality_keys128;
if (size_of_field == 32)
return AggregatedDataVariants::Type::low_cardinality_keys256;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: low cardinality numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
}
if (size_of_field == 1)
return AggregatedDataVariants::Type::key8;
if (size_of_field == 2)
return AggregatedDataVariants::Type::key16;
if (size_of_field == 4)
return AggregatedDataVariants::Type::key32;
if (size_of_field == 8)
return AggregatedDataVariants::Type::key64;
if (size_of_field == 16)
return AggregatedDataVariants::Type::keys128;
if (size_of_field == 32)
return AggregatedDataVariants::Type::keys256;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
}
if (params.keys_size == 1 && isFixedString(types_removed_nullable[0]))
{
if (has_low_cardinality)
return AggregatedDataVariants::Type::low_cardinality_key_fixed_string;
else
return AggregatedDataVariants::Type::key_fixed_string;
}
/// If all keys fits in N bits, will use hash table with all keys packed (placed contiguously) to single N-bit key.
if (params.keys_size == num_fixed_contiguous_keys)
{
if (has_low_cardinality)
{
if (keys_bytes <= 16)
return AggregatedDataVariants::Type::low_cardinality_keys128;
if (keys_bytes <= 32)
return AggregatedDataVariants::Type::low_cardinality_keys256;
}
if (keys_bytes <= 2)
return AggregatedDataVariants::Type::keys16;
if (keys_bytes <= 4)
return AggregatedDataVariants::Type::keys32;
if (keys_bytes <= 8)
return AggregatedDataVariants::Type::keys64;
if (keys_bytes <= 16)
return AggregatedDataVariants::Type::keys128;
if (keys_bytes <= 32)
return AggregatedDataVariants::Type::keys256;
}
/// If single string key - will use hash table with references to it. Strings itself are stored separately in Arena.
if (params.keys_size == 1 && isString(types_removed_nullable[0]))
{
if (has_low_cardinality)
return AggregatedDataVariants::Type::low_cardinality_key_string;
else
return AggregatedDataVariants::Type::key_string;
}
return AggregatedDataVariants::Type::serialized;
}
template <bool skip_compiled_aggregate_functions>
void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const
{
for (size_t j = 0; j < params.aggregates_size; ++j)
{
if constexpr (skip_compiled_aggregate_functions)
if (is_aggregate_function_compiled[j])
continue;
try
{
/** An exception may occur if there is a shortage of memory.
* In order that then everything is properly destroyed, we "roll back" some of the created states.
* The code is not very convenient.
*/
aggregate_functions[j]->create(aggregate_data + offsets_of_aggregate_states[j]);
}
catch (...)
{
for (size_t rollback_j = 0; rollback_j < j; ++rollback_j)
{
if constexpr (skip_compiled_aggregate_functions)
if (is_aggregate_function_compiled[j])
continue;
aggregate_functions[rollback_j]->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]);
}
throw;
}
}
}
bool Aggregator::hasSparseArguments(AggregateFunctionInstruction * aggregate_instructions)
{
for (auto * inst = aggregate_instructions; inst->that; ++inst)
if (inst->has_sparse_arguments)
return true;
return false;
}
void Aggregator::executeOnBlockSmall(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result.aggregator = this;
/// How to perform the aggregation?
if (result.empty())
{
if (method_chosen != AggregatedDataVariants::Type::without_key)
initDataVariantsWithSizeHint(result, method_chosen, params);
else
result.init(method_chosen);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
}
executeImpl(result, row_begin, row_end, key_columns, aggregate_instructions);
CurrentMemoryTracker::check();
}
void Aggregator::mergeOnBlockSmall(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data,
const ColumnRawPtrs & key_columns) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result.aggregator = this;
/// How to perform the aggregation?
if (result.empty())
{
initDataVariantsWithSizeHint(result, method_chosen, params);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
}
if ((params.overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key)
{
AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(place);
result.without_key = place;
}
if (false) {} // NOLINT
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(result.aggregates_pool, *result.NAME, result.NAME->data, \
result.without_key, /* no_more_keys= */ false, \
row_begin, row_end, \
aggregate_columns_data, key_columns, result.aggregates_pool);
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
CurrentMemoryTracker::check();
}
void Aggregator::executeImpl(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
bool no_more_keys,
AggregateDataPtr overflow_row) const
{
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
executeImpl(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_instructions, no_more_keys, overflow_row);
if (false) {} // NOLINT
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}
/** It's interesting - if you remove `noinline`, then gcc for some reason will inline this function, and the performance decreases (~ 10%).
* (Probably because after the inline of this function, more internal functions no longer be inlined.)
* Inline does not make sense, since the inner loop is entirely inside this function.
*/
template <typename Method>
void NO_INLINE Aggregator::executeImpl(
Method & method,
Arena * aggregates_pool,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
bool no_more_keys,
AggregateDataPtr overflow_row) const
{
typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
if (!no_more_keys)
{
/// Prefetching doesn't make sense for small hash tables, because they fit in caches entirely.
const bool prefetch = Method::State::has_cheap_key_calculation && params.enable_prefetch
&& (method.data.getBufferSizeInBytes() > min_bytes_for_prefetch);
#if USE_EMBEDDED_COMPILER
if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions))
{
if (prefetch)
executeImplBatch<false, true, true>(
method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
else
executeImplBatch<false, true, false>(
method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
}
else
#endif
{
if (prefetch)
executeImplBatch<false, false, true>(
method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
else
executeImplBatch<false, false, false>(
method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
}
}
else
{
executeImplBatch<true, false, false>(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
}
}
template <bool no_more_keys, bool use_compiled_functions, bool prefetch, typename Method>
void NO_INLINE Aggregator::executeImplBatch(
Method & method,
typename Method::State & state,
Arena * aggregates_pool,
size_t row_begin,
size_t row_end,
AggregateFunctionInstruction * aggregate_instructions,
AggregateDataPtr overflow_row) const
{
using KeyHolder = decltype(state.getKeyHolder(0, std::declval<Arena &>()));
/// During processing of row #i we will prefetch HashTable cell for row #(i + prefetch_look_ahead).
PrefetchingHelper prefetching;
size_t prefetch_look_ahead = prefetching.getInitialLookAheadValue();
/// Optimization for special case when there are no aggregate functions.
if (params.aggregates_size == 0)
{
if constexpr (no_more_keys)
return;
/// For all rows.
AggregateDataPtr place = aggregates_pool->alloc(0);
for (size_t i = row_begin; i < row_end; ++i)
{
if constexpr (prefetch && HasPrefetchMemberFunc<decltype(method.data), KeyHolder>)
{
if (i == row_begin + prefetching.iterationsToMeasure())
prefetch_look_ahead = prefetching.calcPrefetchLookAhead();
if (i + prefetch_look_ahead < row_end)
{
auto && key_holder = state.getKeyHolder(i + prefetch_look_ahead, *aggregates_pool);
method.data.prefetch(std::move(key_holder));
}
}
state.emplaceKey(method.data, i, *aggregates_pool).setMapped(place);
}
return;
}
/// Optimization for special case when aggregating by 8bit key.
if constexpr (!no_more_keys && std::is_same_v<Method, typename decltype(AggregatedDataVariants::key8)::element_type>)
{
/// We use another method if there are aggregate functions with -Array combinator.
bool has_arrays = false;
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
{
if (inst->offsets)
{
has_arrays = true;
break;
}
}
if (!has_arrays && !hasSparseArguments(aggregate_instructions))
{
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
{
inst->batch_that->addBatchLookupTable8(
row_begin,
row_end,
reinterpret_cast<AggregateDataPtr *>(method.data.data()),
inst->state_offset,
[&](AggregateDataPtr & aggregate_data)
{
aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(aggregate_data);
},
state.getKeyData(),
inst->batch_arguments,
aggregates_pool);
}
return;
}
}
/// NOTE: only row_end-row_start is required, but:
/// - this affects only optimize_aggregation_in_order,
/// - this is just a pointer, so it should not be significant,
/// - and plus this will require other changes in the interface.
std::unique_ptr<AggregateDataPtr[]> places(new AggregateDataPtr[row_end]);
/// For all rows.
for (size_t i = row_begin; i < row_end; ++i)
{
AggregateDataPtr aggregate_data = nullptr;
if constexpr (!no_more_keys)
{
if constexpr (prefetch && HasPrefetchMemberFunc<decltype(method.data), KeyHolder>)
{
if (i == row_begin + prefetching.iterationsToMeasure())
prefetch_look_ahead = prefetching.calcPrefetchLookAhead();
if (i + prefetch_look_ahead < row_end)
{
auto && key_holder = state.getKeyHolder(i + prefetch_look_ahead, *aggregates_pool);
method.data.prefetch(std::move(key_holder));
}
}
auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool);
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
if (emplace_result.isInserted())
{
/// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
emplace_result.setMapped(nullptr);
aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
{
const auto & compiled_aggregate_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions;
compiled_aggregate_functions.create_aggregate_states_function(aggregate_data);
if (compiled_aggregate_functions.functions_count != aggregate_functions.size())
{
static constexpr bool skip_compiled_aggregate_functions = true;
createAggregateStates<skip_compiled_aggregate_functions>(aggregate_data);
}
#if defined(MEMORY_SANITIZER)
/// We compile only functions that do not allocate some data in Arena. Only store necessary state in AggregateData place.
for (size_t aggregate_function_index = 0; aggregate_function_index < aggregate_functions.size(); ++aggregate_function_index)
{
if (!is_aggregate_function_compiled[aggregate_function_index])
continue;
auto aggregate_data_with_offset = aggregate_data + offsets_of_aggregate_states[aggregate_function_index];
auto data_size = params.aggregates[aggregate_function_index].function->sizeOfData();
__msan_unpoison(aggregate_data_with_offset, data_size);
}
#endif
}
else
#endif
{
createAggregateStates(aggregate_data);
}
emplace_result.setMapped(aggregate_data);
}
else
aggregate_data = emplace_result.getMapped();
assert(aggregate_data != nullptr);
}
else
{
/// Add only if the key already exists.
auto find_result = state.findKey(method.data, i, *aggregates_pool);
if (find_result.isFound())
aggregate_data = find_result.getMapped();
else
aggregate_data = overflow_row;
}
places[i] = aggregate_data;
}
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
{
std::vector<ColumnData> columns_data;
for (size_t i = 0; i < aggregate_functions.size(); ++i)
{
if (!is_aggregate_function_compiled[i])
continue;
AggregateFunctionInstruction * inst = aggregate_instructions + i;
size_t arguments_size = inst->that->getArgumentTypes().size(); // NOLINT
for (size_t argument_index = 0; argument_index < arguments_size; ++argument_index)
columns_data.emplace_back(getColumnData(inst->batch_arguments[argument_index]));
}
auto add_into_aggregate_states_function = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function;
add_into_aggregate_states_function(row_begin, row_end, columns_data.data(), places.get());
}
#endif
/// Add values to the aggregate functions.
for (size_t i = 0; i < aggregate_functions.size(); ++i)
{
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
if (is_aggregate_function_compiled[i])
continue;
#endif
AggregateFunctionInstruction * inst = aggregate_instructions + i;
if (inst->offsets)
inst->batch_that->addBatchArray(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, inst->offsets, aggregates_pool);
else if (inst->has_sparse_arguments)
inst->batch_that->addBatchSparse(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool);
else
inst->batch_that->addBatch(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool);
}
}
template <bool use_compiled_functions>
void NO_INLINE Aggregator::executeWithoutKeyImpl(
AggregatedDataWithoutKey & res,
size_t row_begin, size_t row_end,
AggregateFunctionInstruction * aggregate_instructions,
Arena * arena) const
{
if (row_begin == row_end)
return;
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
{
std::vector<ColumnData> columns_data;
for (size_t i = 0; i < aggregate_functions.size(); ++i)
{
if (!is_aggregate_function_compiled[i])
continue;
AggregateFunctionInstruction * inst = aggregate_instructions + i;
size_t arguments_size = inst->that->getArgumentTypes().size();
for (size_t argument_index = 0; argument_index < arguments_size; ++argument_index)
{
columns_data.emplace_back(getColumnData(inst->batch_arguments[argument_index]));
}
}
auto add_into_aggregate_states_function_single_place = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function_single_place;
add_into_aggregate_states_function_single_place(row_begin, row_end, columns_data.data(), res);
#if defined(MEMORY_SANITIZER)
/// We compile only functions that do not allocate some data in Arena. Only store necessary state in AggregateData place.
for (size_t aggregate_function_index = 0; aggregate_function_index < aggregate_functions.size(); ++aggregate_function_index)
{
if (!is_aggregate_function_compiled[aggregate_function_index])
continue;
auto aggregate_data_with_offset = res + offsets_of_aggregate_states[aggregate_function_index];
auto data_size = params.aggregates[aggregate_function_index].function->sizeOfData();
__msan_unpoison(aggregate_data_with_offset, data_size);
}
#endif
}
#endif
/// Adding values
for (size_t i = 0; i < aggregate_functions.size(); ++i)
{
AggregateFunctionInstruction * inst = aggregate_instructions + i;
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
if (is_aggregate_function_compiled[i])
continue;
#endif
if (inst->offsets)
inst->batch_that->addBatchSinglePlace(
inst->offsets[static_cast<ssize_t>(row_begin) - 1],
inst->offsets[row_end - 1],
res + inst->state_offset,
inst->batch_arguments,
arena);
else if (inst->has_sparse_arguments)
inst->batch_that->addBatchSparseSinglePlace(
row_begin, row_end,
res + inst->state_offset,
inst->batch_arguments,
arena);
else
inst->batch_that->addBatchSinglePlace(
row_begin, row_end,
res + inst->state_offset,
inst->batch_arguments,
arena);
}
}
void NO_INLINE Aggregator::executeOnIntervalWithoutKey(
AggregatedDataVariants & data_variants, size_t row_begin, size_t row_end, AggregateFunctionInstruction * aggregate_instructions) const
{
/// `data_variants` will destroy the states of aggregate functions in the destructor
data_variants.aggregator = this;
data_variants.init(AggregatedDataVariants::Type::without_key);
AggregatedDataWithoutKey & res = data_variants.without_key;
/// Adding values
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
{
if (inst->offsets)
inst->batch_that->addBatchSinglePlaceFromInterval(
inst->offsets[static_cast<ssize_t>(row_begin) - 1],
inst->offsets[row_end - 1],
res + inst->state_offset,
inst->batch_arguments, data_variants.aggregates_pool);
else
inst->batch_that->addBatchSinglePlaceFromInterval(
row_begin,
row_end,
res + inst->state_offset,
inst->batch_arguments,
data_variants.aggregates_pool);
}
}
void NO_INLINE Aggregator::mergeOnIntervalWithoutKey(
AggregatedDataVariants & data_variants,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data) const
{
/// `data_variants` will destroy the states of aggregate functions in the destructor
data_variants.aggregator = this;
data_variants.init(AggregatedDataVariants::Type::without_key);
mergeWithoutKeyStreamsImpl(data_variants, row_begin, row_end, aggregate_columns_data);
}
void Aggregator::prepareAggregateInstructions(
Columns columns,
AggregateColumns & aggregate_columns,
Columns & materialized_columns,
AggregateFunctionInstructions & aggregate_functions_instructions,
NestedColumnsHolder & nested_columns_holder) const
{
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i].resize(params.aggregates[i].argument_names.size());
aggregate_functions_instructions.resize(params.aggregates_size + 1);
aggregate_functions_instructions[params.aggregates_size].that = nullptr;
for (size_t i = 0; i < params.aggregates_size; ++i)
{
bool allow_sparse_arguments = aggregate_columns[i].size() == 1;
bool has_sparse_arguments = false;
for (size_t j = 0; j < aggregate_columns[i].size(); ++j)
{
const auto pos = header.getPositionByName(params.aggregates[i].argument_names[j]);
materialized_columns.push_back(columns.at(pos)->convertToFullColumnIfConst());
aggregate_columns[i][j] = materialized_columns.back().get();
/// Sparse columns without defaults may be handled incorrectly.
if (aggregate_columns[i][j]->isSparse()
&& aggregate_columns[i][j]->getNumberOfDefaultRows() == 0)
allow_sparse_arguments = false;
auto full_column = allow_sparse_arguments
? aggregate_columns[i][j]->getPtr()
: recursiveRemoveSparse(aggregate_columns[i][j]->getPtr());
full_column = recursiveRemoveLowCardinality(full_column);
if (full_column.get() != aggregate_columns[i][j])
{
materialized_columns.emplace_back(std::move(full_column));
aggregate_columns[i][j] = materialized_columns.back().get();
}
if (aggregate_columns[i][j]->isSparse())
has_sparse_arguments = true;
}
aggregate_functions_instructions[i].has_sparse_arguments = has_sparse_arguments;
aggregate_functions_instructions[i].arguments = aggregate_columns[i].data();
aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i];
const auto * that = aggregate_functions[i];
/// Unnest consecutive trailing -State combinators
while (const auto * func = typeid_cast<const AggregateFunctionState *>(that))
that = func->getNestedFunction().get();
aggregate_functions_instructions[i].that = that;
if (const auto * func = typeid_cast<const AggregateFunctionArray *>(that))
{
/// Unnest consecutive -State combinators before -Array
that = func->getNestedFunction().get();
while (const auto * nested_func = typeid_cast<const AggregateFunctionState *>(that))
that = nested_func->getNestedFunction().get();
auto [nested_columns, offsets] = checkAndGetNestedArrayOffset(aggregate_columns[i].data(), that->getArgumentTypes().size());
nested_columns_holder.push_back(std::move(nested_columns));
aggregate_functions_instructions[i].batch_arguments = nested_columns_holder.back().data();
aggregate_functions_instructions[i].offsets = offsets;
}
else
aggregate_functions_instructions[i].batch_arguments = aggregate_columns[i].data();
aggregate_functions_instructions[i].batch_that = that;
}
}
bool Aggregator::executeOnBlock(const Block & block,
AggregatedDataVariants & result,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns,
bool & no_more_keys) const
{
return executeOnBlock(block.getColumns(),
/* row_begin= */ 0, block.rows(),
result,
key_columns,
aggregate_columns,
no_more_keys);
}
bool Aggregator::executeOnBlock(Columns columns,
size_t row_begin, size_t row_end,
AggregatedDataVariants & result,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns,
bool & no_more_keys) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result.aggregator = this;
/// How to perform the aggregation?
if (result.empty())
{
initDataVariantsWithSizeHint(result, method_chosen, params);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
LOG_TRACE(log, "Aggregation method: {}", result.getMethodName());
}
/** Constant columns are not supported directly during aggregation.
* To make them work anyway, we materialize them.
*/
Columns materialized_columns;
/// Remember the columns we will work with
for (size_t i = 0; i < params.keys_size; ++i)
{
materialized_columns.push_back(recursiveRemoveSparse(columns.at(keys_positions[i]))->convertToFullColumnIfConst());
key_columns[i] = materialized_columns.back().get();
if (!result.isLowCardinality())
{
auto column_no_lc = recursiveRemoveLowCardinality(key_columns[i]->getPtr());
if (column_no_lc.get() != key_columns[i])
{
materialized_columns.emplace_back(std::move(column_no_lc));
key_columns[i] = materialized_columns.back().get();
}
}
}
NestedColumnsHolder nested_columns_holder;
AggregateFunctionInstructions aggregate_functions_instructions;
prepareAggregateInstructions(columns, aggregate_columns, materialized_columns, aggregate_functions_instructions, nested_columns_holder);
if ((params.overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key)
{
AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(place);
result.without_key = place;
}
/// We select one of the aggregation methods and call it.
/// For the case when there are no keys (all aggregate into one row).
if (result.type == AggregatedDataVariants::Type::without_key)
{
/// TODO: Enable compilation after investigation
// #if USE_EMBEDDED_COMPILER
// if (compiled_aggregate_functions_holder)
// {
// executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
// }
// else
// #endif
{
executeWithoutKeyImpl<false>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
}
}
else
{
/// This is where data is written that does not fit in `max_rows_to_group_by` with `group_by_overflow_mode = any`.
AggregateDataPtr overflow_row_ptr = params.overflow_row ? result.without_key : nullptr;
executeImpl(result, row_begin, row_end, key_columns, aggregate_functions_instructions.data(), no_more_keys, overflow_row_ptr);
}
size_t result_size = result.sizeWithoutOverflowRow();
Int64 current_memory_usage = 0;
if (auto * memory_tracker_child = CurrentThread::getMemoryTracker())
if (auto * memory_tracker = memory_tracker_child->getParent())
current_memory_usage = memory_tracker->get();
/// Here all the results in the sum are taken into account, from different threads.
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation;
bool worth_convert_to_two_level = worthConvertToTwoLevel(
params.group_by_two_level_threshold, result_size, params.group_by_two_level_threshold_bytes, result_size_bytes);
/** Converting to a two-level data structure.
* It allows you to make, in the subsequent, an effective merge - either economical from memory or parallel.
*/
if (result.isConvertibleToTwoLevel() && worth_convert_to_two_level)
result.convertToTwoLevel();
/// Checking the constraints.
if (!checkLimits(result_size, no_more_keys))
return false;
/** Flush data to disk if too much RAM is consumed.
* Data can only be flushed to disk if a two-level aggregation structure is used.
*/
if (params.max_bytes_before_external_group_by
&& result.isTwoLevel()
&& current_memory_usage > static_cast<Int64>(params.max_bytes_before_external_group_by)
&& worth_convert_to_two_level)
{
size_t size = current_memory_usage + params.min_free_disk_space;
writeToTemporaryFile(result, size);
}
return true;
}
void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, size_t max_temp_file_size) const
{
if (!tmp_data)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write to temporary file because temporary file is not initialized");
Stopwatch watch;
size_t rows = data_variants.size();
auto & out_stream = tmp_data->createStream(getHeader(false), max_temp_file_size);
ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart);
LOG_DEBUG(log, "Writing part of aggregation data into temporary file {}", out_stream.getPath());
/// Flush only two-level data and possibly overflow data.
#define M(NAME) \
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
writeToTemporaryFileImpl(data_variants, *data_variants.NAME, out_stream);
if (false) {} // NOLINT
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant");
/// NOTE Instead of freeing up memory and creating new hash tables and arenas, you can re-use the old ones.
data_variants.init(data_variants.type);
data_variants.aggregates_pools = Arenas(1, std::make_shared<Arena>());
data_variants.aggregates_pool = data_variants.aggregates_pools.back().get();
if (params.overflow_row || data_variants.type == AggregatedDataVariants::Type::without_key)
{
AggregateDataPtr place = data_variants.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(place);
data_variants.without_key = place;
}
auto stat = out_stream.finishWriting();
ProfileEvents::increment(ProfileEvents::ExternalAggregationCompressedBytes, stat.compressed_size);
ProfileEvents::increment(ProfileEvents::ExternalAggregationUncompressedBytes, stat.uncompressed_size);
ProfileEvents::increment(ProfileEvents::ExternalProcessingCompressedBytesTotal, stat.compressed_size);
ProfileEvents::increment(ProfileEvents::ExternalProcessingUncompressedBytesTotal, stat.uncompressed_size);
double elapsed_seconds = watch.elapsedSeconds();
double compressed_size = stat.compressed_size;
double uncompressed_size = stat.uncompressed_size;
LOG_DEBUG(log,
"Written part in {:.3f} sec., {} rows, {} uncompressed, {} compressed,"
" {:.3f} uncompressed bytes per row, {:.3f} compressed bytes per row, compression rate: {:.3f}"
" ({:.3f} rows/sec., {}/sec. uncompressed, {}/sec. compressed)",
elapsed_seconds,
rows,
ReadableSize(uncompressed_size),
ReadableSize(compressed_size),
static_cast<double>(uncompressed_size) / rows,
static_cast<double>(compressed_size) / rows,
static_cast<double>(uncompressed_size) / compressed_size,
static_cast<double>(rows) / elapsed_seconds,
ReadableSize(static_cast<double>(uncompressed_size) / elapsed_seconds),
ReadableSize(static_cast<double>(compressed_size) / elapsed_seconds));
}
template <typename Method>
Block Aggregator::convertOneBucketToBlock(
AggregatedDataVariants & data_variants,
Method & method,
Arena * arena,
bool final,
Int32 bucket) const
{
// Used in ConvertingAggregatedToChunksSource -> ConvertingAggregatedToChunksTransform (expects single chunk for each bucket_id).
constexpr bool return_single_block = true;
Block block = convertToBlockImpl<return_single_block>(
method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size());
block.info.bucket_num = static_cast<int>(bucket);
return block;
}
Block Aggregator::mergeAndConvertOneBucketToBlock(
ManyAggregatedDataVariants & variants,
Arena * arena,
bool final,
Int32 bucket,
std::atomic<bool> * is_cancelled) const
{
auto & merged_data = *variants[0];
auto method = merged_data.type;
Block block;
if (false) {} // NOLINT
#define M(NAME) \
else if (method == AggregatedDataVariants::Type::NAME) \
{ \
mergeBucketImpl<decltype(merged_data.NAME)::element_type>(variants, bucket, arena); \
if (is_cancelled && is_cancelled->load(std::memory_order_seq_cst)) \
return {}; \
block = convertOneBucketToBlock(merged_data, *merged_data.NAME, arena, final, bucket); \
}
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
return block;
}
Block Aggregator::convertOneBucketToBlock(AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket) const
{
const auto method = variants.type;
Block block;
if (false) {} // NOLINT
#define M(NAME) \
else if (method == AggregatedDataVariants::Type::NAME) \
block = convertOneBucketToBlock(variants, *variants.NAME, arena, final, bucket); \
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
return block;
}
template <typename Method>
void Aggregator::writeToTemporaryFileImpl(
AggregatedDataVariants & data_variants,
Method & method,
TemporaryFileStream & out) const
{
size_t max_temporary_block_size_rows = 0;
size_t max_temporary_block_size_bytes = 0;
auto update_max_sizes = [&](const Block & block)
{
size_t block_size_rows = block.rows();
size_t block_size_bytes = block.bytes();
if (block_size_rows > max_temporary_block_size_rows)
max_temporary_block_size_rows = block_size_rows;
if (block_size_bytes > max_temporary_block_size_bytes)
max_temporary_block_size_bytes = block_size_bytes;
};
for (UInt32 bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
{
Block block = convertOneBucketToBlock(data_variants, method, data_variants.aggregates_pool, false, bucket);
out.write(block);
update_max_sizes(block);
}
if (params.overflow_row)
{
Block block = prepareBlockAndFillWithoutKey(data_variants, false, true);
out.write(block);
update_max_sizes(block);
}
/// Pass ownership of the aggregate functions states:
/// `data_variants` will not destroy them in the destructor, they are now owned by ColumnAggregateFunction objects.
data_variants.aggregator = nullptr;
LOG_DEBUG(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, ReadableSize(max_temporary_block_size_bytes));
}
bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const
{
if (!no_more_keys && params.max_rows_to_group_by && result_size > params.max_rows_to_group_by)
{
switch (params.group_by_overflow_mode)
{
case OverflowMode::THROW:
ProfileEvents::increment(ProfileEvents::OverflowThrow);
throw Exception(ErrorCodes::TOO_MANY_ROWS, "Limit for rows to GROUP BY exceeded: has {} rows, maximum: {}",
result_size, params.max_rows_to_group_by);
case OverflowMode::BREAK:
ProfileEvents::increment(ProfileEvents::OverflowBreak);
return false;
case OverflowMode::ANY:
ProfileEvents::increment(ProfileEvents::OverflowAny);
no_more_keys = true;
break;
}
}
/// Some aggregate functions cannot throw exceptions on allocations (e.g. from C malloc)
/// but still tracks memory. Check it here.
CurrentMemoryTracker::check();
return true;
}
template <bool return_single_block, typename Method, typename Table>
Aggregator::ConvertToBlockRes<return_single_block>
Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const
{
if (data.empty())
{
auto && out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, rows);
return {finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows)};
}
ConvertToBlockRes<return_single_block> res;
if (final)
{
#if USE_EMBEDDED_COMPILER
if (compiled_aggregate_functions_holder)
{
static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization;
res = convertToBlockImplFinal<Method, use_compiled_functions, return_single_block>(method, data, arena, aggregates_pools, rows);
}
else
#endif
{
res = convertToBlockImplFinal<Method, false, return_single_block>(method, data, arena, aggregates_pools, rows);
}
}
else
{
res = convertToBlockImplNotFinal<return_single_block>(method, data, aggregates_pools, rows);
}
/// In order to release memory early.
data.clearAndShrink();
return res;
}
template <typename Mapped>
inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColumns & final_aggregate_columns, Arena * arena) const
{
/** Final values of aggregate functions are inserted to columns.
* Then states of aggregate functions, that are not longer needed, are destroyed.
*
* We mark already destroyed states with "nullptr" in data,
* so they will not be destroyed in destructor of Aggregator
* (other values will be destroyed in destructor in case of exception).
*
* But it becomes tricky, because we have multiple aggregate states pointed by a single pointer in data.
* So, if exception is thrown in the middle of moving states for different aggregate functions,
* we have to catch exceptions and destroy all the states that are no longer needed,
* to keep the data in consistent state.
*
* It is also tricky, because there are aggregate functions with "-State" modifier.
* When we call "insertResultInto" for them, they insert a pointer to the state to ColumnAggregateFunction
* and ColumnAggregateFunction will take ownership of this state.
* So, for aggregate functions with "-State" modifier, only states of all combinators that are used
* after -State will be destroyed after result has been transferred to ColumnAggregateFunction.
* For example, if we have function `uniqStateForEachMap` after aggregation we should destroy all states that
* were created by combinators `-ForEach` and `-Map`, because resulting ColumnAggregateFunction will be
* responsible only for destruction of the states created by `uniq` function.
* But we should mark that the data no longer owns these states.
*/
size_t insert_i = 0;
std::exception_ptr exception;
try
{
/// Insert final values of aggregate functions into columns.
for (; insert_i < params.aggregates_size; ++insert_i)
aggregate_functions[insert_i]->insertResultInto(
mapped + offsets_of_aggregate_states[insert_i],
*final_aggregate_columns[insert_i],
arena);
}
catch (...)
{
exception = std::current_exception();
}
/** Destroy states that are no longer needed. This loop does not throw.
*
* For functions with -State combinator we destroy only states of all combinators that are used
* after -State, because the ownership of the rest states is transferred to ColumnAggregateFunction
* and ColumnAggregateFunction will take care.
*
* But it's only for states that has been transferred to ColumnAggregateFunction
* before exception has been thrown;
*/
for (size_t destroy_i = 0; destroy_i < params.aggregates_size; ++destroy_i)
{
if (destroy_i < insert_i)
aggregate_functions[destroy_i]->destroyUpToState(mapped + offsets_of_aggregate_states[destroy_i]);
else
aggregate_functions[destroy_i]->destroy(mapped + offsets_of_aggregate_states[destroy_i]);
}
/// Mark the cell as destroyed so it will not be destroyed in destructor.
mapped = nullptr;
if (exception)
std::rethrow_exception(exception);
}
template <bool use_compiled_functions>
Block Aggregator::insertResultsIntoColumns(PaddedPODArray<AggregateDataPtr> & places, OutputBlockColumns && out_cols, Arena * arena, bool has_null_key_data [[maybe_unused]]) const
{
std::exception_ptr exception;
size_t aggregate_functions_destroy_index = 0;
try
{
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
{
/** For JIT compiled functions we need to resize columns before pass them into compiled code.
* insert_aggregates_into_columns_function function does not throw exception.
*/
std::vector<ColumnData> columns_data;
auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions;
for (size_t i = 0; i < params.aggregates_size; ++i)
{
if (!is_aggregate_function_compiled[i])
continue;
auto & final_aggregate_column = out_cols.final_aggregate_columns[i];
/**
* In convertToBlockImplFinal, additional data with a key of null may be written,
* and additional memory for null data needs to be allocated when using the compiled function
*/
final_aggregate_column = final_aggregate_column->cloneResized(places.size() + (has_null_key_data ? 1 : 0));
columns_data.emplace_back(getColumnData(final_aggregate_column.get(), (has_null_key_data ? 1 : 0)));
}
auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function;
insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data());
}
#endif
for (; aggregate_functions_destroy_index < params.aggregates_size;)
{
if constexpr (use_compiled_functions)
{
if (is_aggregate_function_compiled[aggregate_functions_destroy_index])
{
++aggregate_functions_destroy_index;
continue;
}
}
auto & final_aggregate_column = out_cols.final_aggregate_columns[aggregate_functions_destroy_index];
size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index];
/** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch
* throws exception, it also must destroy all necessary states.
* Then code need to continue to destroy other aggregate function states with next function index.
*/
size_t destroy_index = aggregate_functions_destroy_index;
++aggregate_functions_destroy_index;
aggregate_functions[destroy_index]->insertResultIntoBatch(0, places.size(), places.data(), offset, *final_aggregate_column, arena);
}
}
catch (...)
{
exception = std::current_exception();
}
for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index)
{
if constexpr (use_compiled_functions)
{
if (is_aggregate_function_compiled[aggregate_functions_destroy_index])
{
++aggregate_functions_destroy_index;
continue;
}
}
size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index];
aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset);
}
if (exception)
std::rethrow_exception(exception);
return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size());
}
template <typename Method, bool use_compiled_functions, bool return_single_block, typename Table>
Aggregator::ConvertToBlockRes<return_single_block> NO_INLINE
Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const
{
/// +1 for nullKeyData, if `data` doesn't have it - not a problem, just some memory for one excessive row will be preallocated
const size_t max_block_size = (return_single_block ? data.size() : std::min(params.max_block_size, data.size())) + 1;
const bool final = true;
ConvertToBlockRes<return_single_block> res;
std::optional<OutputBlockColumns> out_cols;
std::optional<Sizes> shuffled_key_sizes;
PaddedPODArray<AggregateDataPtr> places;
bool has_null_key_data = false;
auto init_out_cols = [&]()
{
out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size);
if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization)
{
/**
* When one_key_nullable_optimization is enabled, null data will be written to the key column and result column in advance.
* And in insertResultsIntoColumns need to allocate memory for null data.
*/
if (data.hasNullKeyData())
{
has_null_key_data = true;
out_cols->key_columns[0]->insertDefault();
insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena);
data.hasNullKeyData() = false;
}
}
shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes);
places.reserve(max_block_size);
};
// should be invoked at least once, because null data might be the only content of the `data`
init_out_cols();
data.forEachValue(
[&](const auto & key, auto & mapped)
{
if (!out_cols.has_value())
init_out_cols();
const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes;
method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref);
places.emplace_back(mapped);
/// Mark the cell as destroyed so it will not be destroyed in destructor.
mapped = nullptr;
if constexpr (!return_single_block)
{
if (places.size() >= max_block_size)
{
res.emplace_back(insertResultsIntoColumns<use_compiled_functions>(places, std::move(out_cols.value()), arena, has_null_key_data));
places.clear();
out_cols.reset();
has_null_key_data = false;
}
}
});
if constexpr (return_single_block)
{
return insertResultsIntoColumns<use_compiled_functions>(places, std::move(out_cols.value()), arena, has_null_key_data);
}
else
{
if (out_cols.has_value())
res.emplace_back(insertResultsIntoColumns<use_compiled_functions>(places, std::move(out_cols.value()), arena, has_null_key_data));
return res;
}
}
template <bool return_single_block, typename Method, typename Table>
Aggregator::ConvertToBlockRes<return_single_block> NO_INLINE
Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const
{
/// +1 for nullKeyData, if `data` doesn't have it - not a problem, just some memory for one excessive row will be preallocated
const size_t max_block_size = (return_single_block ? data.size() : std::min(params.max_block_size, data.size())) + 1;
const bool final = false;
ConvertToBlockRes<return_single_block> res;
std::optional<OutputBlockColumns> out_cols;
std::optional<Sizes> shuffled_key_sizes;
size_t rows_in_current_block = 0;
auto init_out_cols = [&]()
{
out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size);
if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization)
{
if (data.hasNullKeyData())
{
out_cols->raw_key_columns[0]->insertDefault();
for (size_t i = 0; i < params.aggregates_size; ++i)
out_cols->aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]);
++rows_in_current_block;
data.getNullKeyData() = nullptr;
data.hasNullKeyData() = false;
}
}
shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes);
};
// should be invoked at least once, because null data might be the only content of the `data`
init_out_cols();
data.forEachValue(
[&](const auto & key, auto & mapped)
{
if (!out_cols.has_value())
init_out_cols();
const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes;
method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref);
/// reserved, so push_back does not throw exceptions
for (size_t i = 0; i < params.aggregates_size; ++i)
out_cols->aggregate_columns_data[i]->push_back(mapped + offsets_of_aggregate_states[i]);
mapped = nullptr;
++rows_in_current_block;
if constexpr (!return_single_block)
{
if (rows_in_current_block >= max_block_size)
{
res.emplace_back(finalizeBlock(params, getHeader(final), std::move(out_cols.value()), final, rows_in_current_block));
out_cols.reset();
rows_in_current_block = 0;
}
}
});
if constexpr (return_single_block)
{
return finalizeBlock(params, getHeader(final), std::move(out_cols).value(), final, rows_in_current_block);
}
else
{
if (rows_in_current_block)
res.emplace_back(finalizeBlock(params, getHeader(final), std::move(out_cols).value(), final, rows_in_current_block));
return res;
}
return res;
}
void Aggregator::addSingleKeyToAggregateColumns(
AggregatedDataVariants & data_variants,
MutableColumns & aggregate_columns) const
{
auto & data = data_variants.without_key;
size_t i = 0;
try
{
for (i = 0; i < params.aggregates_size; ++i)
{
auto & column_aggregate_func = assert_cast<ColumnAggregateFunction &>(*aggregate_columns[i]);
column_aggregate_func.getData().push_back(data + offsets_of_aggregate_states[i]);
}
}
catch (...)
{
/// Rollback
for (size_t rollback_i = 0; rollback_i < i; ++rollback_i)
{
auto & column_aggregate_func = assert_cast<ColumnAggregateFunction &>(*aggregate_columns[rollback_i]);
column_aggregate_func.getData().pop_back();
}
throw;
}
data = nullptr;
}
void Aggregator::addArenasToAggregateColumns(
const AggregatedDataVariants & data_variants,
MutableColumns & aggregate_columns) const
{
for (size_t i = 0; i < params.aggregates_size; ++i)
{
auto & column_aggregate_func = assert_cast<ColumnAggregateFunction &>(*aggregate_columns[i]);
for (const auto & pool : data_variants.aggregates_pools)
column_aggregate_func.addArena(pool);
}
}
void Aggregator::createStatesAndFillKeyColumnsWithSingleKey(
AggregatedDataVariants & data_variants,
Columns & key_columns,
size_t key_row,
MutableColumns & final_key_columns) const
{
AggregateDataPtr place = data_variants.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(place);
data_variants.without_key = place;
for (size_t i = 0; i < params.keys_size; ++i)
{
final_key_columns[i]->insertFrom(*key_columns[i].get(), key_row);
}
}
Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const
{
size_t rows = 1;
auto && out_cols
= prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), data_variants.aggregates_pools, final, rows);
auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols;
if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
{
AggregatedDataWithoutKey & data = data_variants.without_key;
if (!data)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong data variant passed.");
if (!final)
{
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns_data[i]->push_back(data + offsets_of_aggregate_states[i]);
data = nullptr;
}
else
{
/// Always single-thread. It's safe to pass current arena from 'aggregates_pool'.
insertAggregatesIntoColumns(data, final_aggregate_columns, data_variants.aggregates_pool);
}
if (params.overflow_row)
for (size_t i = 0; i < params.keys_size; ++i)
key_columns[i]->insertDefault();
}
Block block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows);
if (is_overflows)
block.info.is_overflows = true;
if (final)
destroyWithoutKey(data_variants);
return block;
}
template <bool return_single_block>
Aggregator::ConvertToBlockRes<return_single_block>
Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const
{
const size_t rows = data_variants.sizeWithoutOverflowRow();
#define M(NAME) \
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
{ \
return convertToBlockImpl<return_single_block>( \
*data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \
}
if (false) {} // NOLINT
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
#undef M
else throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
}
BlocksList Aggregator::prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const
{
#define M(NAME) \
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
return prepareBlocksAndFillTwoLevelImpl(data_variants, *data_variants.NAME, final, thread_pool);
if (false) {} // NOLINT
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
}
template <typename Method>
BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
AggregatedDataVariants & data_variants,
Method & method,
bool final,
ThreadPool * thread_pool) const
{
size_t max_threads = thread_pool ? thread_pool->getMaxThreads() : 1;
if (max_threads > data_variants.aggregates_pools.size())
for (size_t i = data_variants.aggregates_pools.size(); i < max_threads; ++i)
data_variants.aggregates_pools.push_back(std::make_shared<Arena>());
std::atomic<UInt32> next_bucket_to_merge = 0;
auto converter = [&](size_t thread_id, ThreadGroupPtr thread_group)
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToGroupIfDetached(thread_group);
BlocksList blocks;
while (true)
{
UInt32 bucket = next_bucket_to_merge.fetch_add(1);
if (bucket >= Method::Data::NUM_BUCKETS)
break;
if (method.data.impls[bucket].empty())
continue;
/// Select Arena to avoid race conditions
Arena * arena = data_variants.aggregates_pools.at(thread_id).get();
blocks.emplace_back(convertOneBucketToBlock(data_variants, method, arena, final, bucket));
}
return blocks;
};
/// packaged_task is used to ensure that exceptions are automatically thrown into the main stream.
std::vector<std::packaged_task<BlocksList()>> tasks(max_threads);
try
{
for (size_t thread_id = 0; thread_id < max_threads; ++thread_id)
{
tasks[thread_id] = std::packaged_task<BlocksList()>(
[group = CurrentThread::getGroup(), thread_id, &converter] { return converter(thread_id, group); });
if (thread_pool)
thread_pool->scheduleOrThrowOnError([thread_id, &tasks] { tasks[thread_id](); });
else
tasks[thread_id]();
}
}
catch (...)
{
/// If this is not done, then in case of an exception, tasks will be destroyed before the threads are completed, and it will be bad.
if (thread_pool)
thread_pool->wait();
throw;
}
if (thread_pool)
thread_pool->wait();
BlocksList blocks;
for (auto & task : tasks)
{
if (!task.valid())
continue;
blocks.splice(blocks.end(), task.get_future().get());
}
return blocks;
}
BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, bool final, size_t max_threads) const
{
LOG_TRACE(log, "Converting aggregated data to blocks");
Stopwatch watch;
BlocksList blocks;
/// In what data structure is the data aggregated?
if (data_variants.empty())
return blocks;
std::unique_ptr<ThreadPool> thread_pool;
if (max_threads > 1 && data_variants.sizeWithoutOverflowRow() > 100000 /// TODO Make a custom threshold.
&& data_variants.isTwoLevel()) /// TODO Use the shared thread pool with the `merge` function.
thread_pool = std::make_unique<ThreadPool>(CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, max_threads);
if (data_variants.without_key)
blocks.emplace_back(prepareBlockAndFillWithoutKey(
data_variants, final, data_variants.type != AggregatedDataVariants::Type::without_key));
if (data_variants.type != AggregatedDataVariants::Type::without_key)
{
if (!data_variants.isTwoLevel())
blocks.splice(blocks.end(), prepareBlockAndFillSingleLevel</* return_single_block */ false>(data_variants, final));
else
blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get()));
}
if (!final)
{
/// data_variants will not destroy the states of aggregate functions in the destructor.
/// Now ColumnAggregateFunction owns the states.
data_variants.aggregator = nullptr;
}
size_t rows = 0;
size_t bytes = 0;
for (const auto & block : blocks)
{
rows += block.rows();
bytes += block.bytes();
}
double elapsed_seconds = watch.elapsedSeconds();
LOG_DEBUG(log,
"Converted aggregated data to blocks. {} rows, {} in {} sec. ({:.3f} rows/sec., {}/sec.)",
rows, ReadableSize(bytes),
elapsed_seconds, rows / elapsed_seconds,
ReadableSize(bytes / elapsed_seconds));
return blocks;
}
template <typename Method, typename Table>
void NO_INLINE Aggregator::mergeDataNullKey(
Table & table_dst,
Table & table_src,
Arena * arena) const
{
if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization)
{
if (table_src.hasNullKeyData())
{
if (!table_dst.hasNullKeyData())
{
table_dst.hasNullKeyData() = true;
table_dst.getNullKeyData() = table_src.getNullKeyData();
}
else
{
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(
table_dst.getNullKeyData() + offsets_of_aggregate_states[i],
table_src.getNullKeyData() + offsets_of_aggregate_states[i],
arena);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(
table_src.getNullKeyData() + offsets_of_aggregate_states[i]);
}
table_src.hasNullKeyData() = false;
table_src.getNullKeyData() = nullptr;
}
}
}
template <typename Method, bool use_compiled_functions, bool prefetch, typename Table>
void NO_INLINE Aggregator::mergeDataImpl(Table & table_dst, Table & table_src, Arena * arena) const
{
if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization)
mergeDataNullKey<Method, Table>(table_dst, table_src, arena);
PaddedPODArray<AggregateDataPtr> dst_places;
PaddedPODArray<AggregateDataPtr> src_places;
auto merge = [&](AggregateDataPtr & __restrict dst, AggregateDataPtr & __restrict src, bool inserted)
{
if (!inserted)
{
dst_places.push_back(dst);
src_places.push_back(src);
}
else
{
dst = src;
}
src = nullptr;
};
table_src.template mergeToViaEmplace<decltype(merge), prefetch>(table_dst, std::move(merge));
table_src.clearAndShrink();
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
{
const auto & compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions;
compiled_functions.merge_aggregate_states_function(dst_places.data(), src_places.data(), dst_places.size());
for (size_t i = 0; i < params.aggregates_size; ++i)
{
if (!is_aggregate_function_compiled[i])
aggregate_functions[i]->mergeAndDestroyBatch(
dst_places.data(), src_places.data(), dst_places.size(), offsets_of_aggregate_states[i], arena);
}
return;
}
#endif
for (size_t i = 0; i < params.aggregates_size; ++i)
{
aggregate_functions[i]->mergeAndDestroyBatch(
dst_places.data(), src_places.data(), dst_places.size(), offsets_of_aggregate_states[i], arena);
}
}
template <typename Method, typename Table>
void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl(
Table & table_dst,
AggregatedDataWithoutKey & overflows,
Table & table_src,
Arena * arena) const
{
/// Note : will create data for NULL key if not exist
if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization)
mergeDataNullKey<Method, Table>(table_dst, table_src, arena);
table_src.mergeToViaFind(table_dst, [&](AggregateDataPtr dst, AggregateDataPtr & src, bool found)
{
AggregateDataPtr res_data = found ? dst : overflows;
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(
res_data + offsets_of_aggregate_states[i],
src + offsets_of_aggregate_states[i],
arena);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(src + offsets_of_aggregate_states[i]);
src = nullptr;
});
table_src.clearAndShrink();
}
template <typename Method, typename Table>
void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl(
Table & table_dst,
Table & table_src,
Arena * arena) const
{
/// Note : will create data for NULL key if not exist
if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization)
mergeDataNullKey<Method, Table>(table_dst, table_src, arena);
table_src.mergeToViaFind(table_dst,
[&](AggregateDataPtr dst, AggregateDataPtr & src, bool found)
{
if (!found)
return;
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(
dst + offsets_of_aggregate_states[i],
src + offsets_of_aggregate_states[i],
arena);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(src + offsets_of_aggregate_states[i]);
src = nullptr;
});
table_src.clearAndShrink();
}
void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(
ManyAggregatedDataVariants & non_empty_data) const
{
ThreadPool thread_pool{CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, params.max_threads};
AggregatedDataVariantsPtr & res = non_empty_data[0];
for (size_t i = 0; i < params.aggregates_size; ++i)
{
if (aggregate_functions[i]->isParallelizeMergePrepareNeeded())
{
size_t size = non_empty_data.size();
std::vector<AggregateDataPtr> data_vec;
for (size_t result_num = 0; result_num < size; ++result_num)
data_vec.emplace_back(non_empty_data[result_num]->without_key + offsets_of_aggregate_states[i]);
aggregate_functions[i]->parallelizeMergePrepare(data_vec, thread_pool);
}
}
/// We merge all aggregation results to the first.
for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num)
{
AggregatedDataWithoutKey & res_data = res->without_key;
AggregatedDataWithoutKey & current_data = non_empty_data[result_num]->without_key;
for (size_t i = 0; i < params.aggregates_size; ++i)
if (aggregate_functions[i]->isAbleToParallelizeMerge())
aggregate_functions[i]->merge(
res_data + offsets_of_aggregate_states[i],
current_data + offsets_of_aggregate_states[i],
thread_pool,
res->aggregates_pool);
else
aggregate_functions[i]->merge(
res_data + offsets_of_aggregate_states[i], current_data + offsets_of_aggregate_states[i], res->aggregates_pool);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(current_data + offsets_of_aggregate_states[i]);
current_data = nullptr;
}
}
template <typename Method>
void NO_INLINE Aggregator::mergeSingleLevelDataImpl(
ManyAggregatedDataVariants & non_empty_data) const
{
AggregatedDataVariantsPtr & res = non_empty_data[0];
bool no_more_keys = false;
const bool prefetch = Method::State::has_cheap_key_calculation && params.enable_prefetch
&& (getDataVariant<Method>(*res).data.getBufferSizeInBytes() > min_bytes_for_prefetch);
/// We merge all aggregation results to the first.
for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num)
{
if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
break;
AggregatedDataVariants & current = *non_empty_data[result_num];
if (!no_more_keys)
{
#if USE_EMBEDDED_COMPILER
if (compiled_aggregate_functions_holder)
{
if (prefetch)
mergeDataImpl<Method, true, true>(
getDataVariant<Method>(*res).data, getDataVariant<Method>(current).data, res->aggregates_pool);
else
mergeDataImpl<Method, true, false>(
getDataVariant<Method>(*res).data, getDataVariant<Method>(current).data, res->aggregates_pool);
}
else
#endif
{
if (prefetch)
mergeDataImpl<Method, false, true>(
getDataVariant<Method>(*res).data, getDataVariant<Method>(current).data, res->aggregates_pool);
else
mergeDataImpl<Method, false, false>(
getDataVariant<Method>(*res).data, getDataVariant<Method>(current).data, res->aggregates_pool);
}
}
else if (res->without_key)
{
mergeDataNoMoreKeysImpl<Method>(
getDataVariant<Method>(*res).data,
res->without_key,
getDataVariant<Method>(current).data,
res->aggregates_pool);
}
else
{
mergeDataOnlyExistingKeysImpl<Method>(
getDataVariant<Method>(*res).data,
getDataVariant<Method>(current).data,
res->aggregates_pool);
}
/// `current` will not destroy the states of aggregate functions in the destructor
current.aggregator = nullptr;
}
}
#define M(NAME) \
template void NO_INLINE Aggregator::mergeSingleLevelDataImpl<decltype(AggregatedDataVariants::NAME)::element_type>( \
ManyAggregatedDataVariants & non_empty_data) const;
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
#undef M
template <typename Method>
void NO_INLINE Aggregator::mergeBucketImpl(
ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena, std::atomic<bool> * is_cancelled) const
{
/// We merge all aggregation results to the first.
AggregatedDataVariantsPtr & res = data[0];
const bool prefetch = Method::State::has_cheap_key_calculation && params.enable_prefetch
&& (Method::Data::NUM_BUCKETS * getDataVariant<Method>(*res).data.impls[bucket].getBufferSizeInBytes() > min_bytes_for_prefetch);
for (size_t result_num = 1, size = data.size(); result_num < size; ++result_num)
{
if (is_cancelled && is_cancelled->load(std::memory_order_seq_cst))
return;
AggregatedDataVariants & current = *data[result_num];
#if USE_EMBEDDED_COMPILER
if (compiled_aggregate_functions_holder)
{
if (prefetch)
mergeDataImpl<Method, true, true>(
getDataVariant<Method>(*res).data.impls[bucket], getDataVariant<Method>(current).data.impls[bucket], arena);
else
mergeDataImpl<Method, true, false>(
getDataVariant<Method>(*res).data.impls[bucket], getDataVariant<Method>(current).data.impls[bucket], arena);
}
else
#endif
{
if (prefetch)
mergeDataImpl<Method, false, true>(
getDataVariant<Method>(*res).data.impls[bucket], getDataVariant<Method>(current).data.impls[bucket], arena);
else
mergeDataImpl<Method, false, false>(
getDataVariant<Method>(*res).data.impls[bucket], getDataVariant<Method>(current).data.impls[bucket], arena);
}
}
}
ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedDataVariants & data_variants) const
{
if (data_variants.empty())
throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Empty data passed to Aggregator::prepareVariantsToMerge.");
LOG_TRACE(log, "Merging aggregated data");
if (params.stats_collecting_params.isCollectionAndUseEnabled())
updateStatistics(data_variants, params.stats_collecting_params);
ManyAggregatedDataVariants non_empty_data;
non_empty_data.reserve(data_variants.size());
for (auto & data : data_variants)
if (!data->empty())
non_empty_data.push_back(data);
if (non_empty_data.empty())
return {};
if (non_empty_data.size() > 1)
{
/// Sort the states in descending order so that the merge is more efficient (since all states are merged into the first).
::sort(non_empty_data.begin(), non_empty_data.end(),
[](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs)
{
return lhs->sizeWithoutOverflowRow() > rhs->sizeWithoutOverflowRow();
});
}
/// If at least one of the options is two-level, then convert all the options into two-level ones, if there are not such.
/// Note - perhaps it would be more optimal not to convert single-level versions before the merge, but merge them separately, at the end.
bool has_at_least_one_two_level = false;
for (const auto & variant : non_empty_data)
{
if (variant->isTwoLevel())
{
has_at_least_one_two_level = true;
break;
}
}
if (has_at_least_one_two_level)
for (auto & variant : non_empty_data)
if (!variant->isTwoLevel())
variant->convertToTwoLevel();
AggregatedDataVariantsPtr & first = non_empty_data[0];
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
{
if (first->type != non_empty_data[i]->type)
throw Exception(ErrorCodes::CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS, "Cannot merge different aggregated data variants.");
/** Elements from the remaining sets can be moved to the first data set.
* Therefore, it must own all the arenas of all other sets.
*/
first->aggregates_pools.insert(first->aggregates_pools.end(),
non_empty_data[i]->aggregates_pools.begin(), non_empty_data[i]->aggregates_pools.end());
}
return non_empty_data;
}
template <bool no_more_keys, typename Method, typename Table>
void NO_INLINE Aggregator::mergeStreamsImplCase(
Arena * aggregates_pool,
Method & method [[maybe_unused]],
Table & data,
AggregateDataPtr overflow_row,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data,
const ColumnRawPtrs & key_columns,
Arena * arena_for_keys) const
{
typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
std::unique_ptr<AggregateDataPtr[]> places(new AggregateDataPtr[row_end]);
if (!arena_for_keys)
arena_for_keys = aggregates_pool;
for (size_t i = row_begin; i < row_end; ++i)
{
AggregateDataPtr aggregate_data = nullptr;
if (!no_more_keys)
{
auto emplace_result = state.emplaceKey(data, i, *arena_for_keys); // NOLINT
if (emplace_result.isInserted())
{
emplace_result.setMapped(nullptr);
aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(aggregate_data);
emplace_result.setMapped(aggregate_data);
}
else
aggregate_data = emplace_result.getMapped();
}
else
{
auto find_result = state.findKey(data, i, *arena_for_keys);
if (find_result.isFound())
aggregate_data = find_result.getMapped();
}
/// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys.
AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row;
places[i] = value;
}
for (size_t j = 0; j < params.aggregates_size; ++j)
{
/// Merge state of aggregate functions.
aggregate_functions[j]->mergeBatch(
row_begin, row_end,
places.get(), offsets_of_aggregate_states[j],
aggregate_columns_data[j]->data(),
aggregates_pool);
}
}
template <typename Method, typename Table>
void NO_INLINE Aggregator::mergeStreamsImpl(
Block block,
Arena * aggregates_pool,
Method & method,
Table & data,
AggregateDataPtr overflow_row,
bool no_more_keys,
Arena * arena_for_keys) const
{
const AggregateColumnsConstData & aggregate_columns_data = params.makeAggregateColumnsData(block);
const ColumnRawPtrs & key_columns = params.makeRawKeyColumns(block);
mergeStreamsImpl<Method, Table>(
aggregates_pool, method, data, overflow_row, no_more_keys, 0, block.rows(), aggregate_columns_data, key_columns, arena_for_keys);
}
template <typename Method, typename Table>
void NO_INLINE Aggregator::mergeStreamsImpl(
Arena * aggregates_pool,
Method & method,
Table & data,
AggregateDataPtr overflow_row,
bool no_more_keys,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data,
const ColumnRawPtrs & key_columns,
Arena * arena_for_keys) const
{
if (!no_more_keys)
mergeStreamsImplCase<false>(
aggregates_pool, method, data, overflow_row, row_begin, row_end, aggregate_columns_data, key_columns, arena_for_keys);
else
mergeStreamsImplCase<true>(
aggregates_pool, method, data, overflow_row, row_begin, row_end, aggregate_columns_data, key_columns, arena_for_keys);
}
void NO_INLINE Aggregator::mergeBlockWithoutKeyStreamsImpl(
Block block,
AggregatedDataVariants & result) const
{
AggregateColumnsConstData aggregate_columns = params.makeAggregateColumnsData(block);
mergeWithoutKeyStreamsImpl(result, 0, block.rows(), aggregate_columns);
}
void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
const AggregateColumnsConstData & aggregate_columns_data) const
{
AggregatedDataWithoutKey & res = result.without_key;
if (!res)
{
AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(place);
res = place;
}
for (size_t row = row_begin; row < row_end; ++row)
{
/// Adding Values
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(res + offsets_of_aggregate_states[i], (*aggregate_columns_data[i])[row], result.aggregates_pool);
}
}
bool Aggregator::mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result.aggregator = this;
/// How to perform the aggregation?
if (result.empty())
{
result.init(method_chosen);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
LOG_TRACE(log, "Aggregation method: {}", result.getMethodName());
}
if ((params.overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key)
{
AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(place);
result.without_key = place;
}
if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows)
mergeBlockWithoutKeyStreamsImpl(std::move(block), result);
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
else if (result.type != AggregatedDataVariants::Type::without_key)
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
size_t result_size = result.sizeWithoutOverflowRow();
Int64 current_memory_usage = 0;
if (auto * memory_tracker_child = CurrentThread::getMemoryTracker())
if (auto * memory_tracker = memory_tracker_child->getParent())
current_memory_usage = memory_tracker->get();
/// Here all the results in the sum are taken into account, from different threads.
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation;
bool worth_convert_to_two_level = worthConvertToTwoLevel(
params.group_by_two_level_threshold, result_size, params.group_by_two_level_threshold_bytes, result_size_bytes);
/** Converting to a two-level data structure.
* It allows you to make, in the subsequent, an effective merge - either economical from memory or parallel.
*/
if (result.isConvertibleToTwoLevel() && worth_convert_to_two_level)
result.convertToTwoLevel();
/// Checking the constraints.
if (!checkLimits(result_size, no_more_keys))
return false;
/** Flush data to disk if too much RAM is consumed.
* Data can only be flushed to disk if a two-level aggregation structure is used.
*/
if (params.max_bytes_before_external_group_by
&& result.isTwoLevel()
&& current_memory_usage > static_cast<Int64>(params.max_bytes_before_external_group_by)
&& worth_convert_to_two_level)
{
size_t size = current_memory_usage + params.min_free_disk_space;
writeToTemporaryFile(result, size);
}
return true;
}
void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVariants & result, size_t max_threads)
{
if (bucket_to_blocks.empty())
return;
UInt64 total_input_rows = 0;
for (auto & bucket : bucket_to_blocks)
for (auto & block : bucket.second)
total_input_rows += block.rows();
/** `minus one` means the absence of information about the bucket
* - in the case of single-level aggregation, as well as for blocks with "overflowing" values.
* If there is at least one block with a bucket number greater or equal than zero, then there was a two-level aggregation.
*/
auto max_bucket = bucket_to_blocks.rbegin()->first;
bool has_two_level = max_bucket >= 0;
if (has_two_level)
{
#define M(NAME) \
if (method_chosen == AggregatedDataVariants::Type::NAME) \
method_chosen = AggregatedDataVariants::Type::NAME ## _two_level;
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
#undef M
}
/// result will destroy the states of aggregate functions in the destructor
result.aggregator = this;
result.init(method_chosen);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
bool has_blocks_with_unknown_bucket = bucket_to_blocks.contains(-1);
/// First, parallel the merge for the individual buckets. Then we continue merge the data not allocated to the buckets.
if (has_two_level)
{
/** In this case, no_more_keys is not supported due to the fact that
* from different threads it is difficult to update the general state for "other" keys (overflows).
* That is, the keys in the end can be significantly larger than max_rows_to_group_by.
*/
LOG_TRACE(log, "Merging partially aggregated two-level data.");
auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupPtr thread_group)
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToGroupIfDetached(thread_group);
for (Block & block : bucket_to_blocks[bucket])
{
#define M(NAME) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(std::move(block), aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, false);
if (false) {} // NOLINT
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
}
};
std::unique_ptr<ThreadPool> thread_pool;
if (max_threads > 1 && total_input_rows > 100000) /// TODO Make a custom threshold.
thread_pool = std::make_unique<ThreadPool>(CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, max_threads);
for (const auto & bucket_blocks : bucket_to_blocks)
{
const auto bucket = bucket_blocks.first;
if (bucket == -1)
continue;
result.aggregates_pools.push_back(std::make_shared<Arena>());
Arena * aggregates_pool = result.aggregates_pools.back().get();
auto task = [group = CurrentThread::getGroup(), bucket, &merge_bucket, aggregates_pool]{ return merge_bucket(bucket, aggregates_pool, group); };
if (thread_pool)
thread_pool->scheduleOrThrowOnError(task);
else
task();
}
if (thread_pool)
thread_pool->wait();
LOG_TRACE(log, "Merged partially aggregated two-level data.");
}
if (has_blocks_with_unknown_bucket)
{
LOG_TRACE(log, "Merging partially aggregated single-level data.");
bool no_more_keys = false;
BlocksList & blocks = bucket_to_blocks[-1];
for (Block & block : blocks)
{
if (!checkLimits(result.sizeWithoutOverflowRow(), no_more_keys))
break;
if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows)
mergeBlockWithoutKeyStreamsImpl(std::move(block), result);
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
else if (result.type != AggregatedDataVariants::Type::without_key)
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
}
LOG_TRACE(log, "Merged partially aggregated single-level data.");
}
CurrentMemoryTracker::check();
}
Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
{
if (blocks.empty())
return {};
auto bucket_num = blocks.front().info.bucket_num;
bool is_overflows = blocks.front().info.is_overflows;
LOG_TRACE(log, "Merging partially aggregated blocks (bucket = {}).", bucket_num);
Stopwatch watch;
/** If possible, change 'method' to some_hash64. Otherwise, leave as is.
* Better hash function is needed because during external aggregation,
* we may merge partitions of data with total number of keys far greater than 4 billion.
*/
auto merge_method = method_chosen;
#define APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION(M) \
M(key64) \
M(key_string) \
M(key_fixed_string) \
M(keys128) \
M(keys256) \
M(serialized) \
#define M(NAME) \
if (merge_method == AggregatedDataVariants::Type::NAME) \
merge_method = AggregatedDataVariants::Type::NAME ## _hash64; \
APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION(M)
#undef M
#undef APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION
/// Temporary data for aggregation.
AggregatedDataVariants result;
/// result will destroy the states of aggregate functions in the destructor
result.aggregator = this;
result.init(merge_method);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
size_t source_rows = 0;
/// In some aggregation methods (e.g. serialized) aggregates pools are used also to store serialized aggregation keys.
/// Memory occupied by them will have the same lifetime as aggregate function states, while it is not actually necessary and leads to excessive memory consumption.
/// To avoid this we use a separate arena to allocate memory for aggregation keys. Its memory will be freed at this function return.
auto arena_for_keys = std::make_shared<Arena>();
for (Block & block : blocks)
{
source_rows += block.rows();
if (bucket_num >= 0 && block.info.bucket_num != bucket_num)
bucket_num = -1;
if (result.type == AggregatedDataVariants::Type::without_key || is_overflows)
mergeBlockWithoutKeyStreamsImpl(std::move(block), result);
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false, arena_for_keys.get());
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
else if (result.type != AggregatedDataVariants::Type::without_key)
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
}
Block block;
if (result.type == AggregatedDataVariants::Type::without_key || is_overflows)
{
block = prepareBlockAndFillWithoutKey(result, final, is_overflows);
}
else
{
// Used during memory efficient merging (SortingAggregatedTransform expects single chunk for each bucket_id).
constexpr bool return_single_block = true;
block = prepareBlockAndFillSingleLevel<return_single_block>(result, final);
}
/// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods.
if (!final)
{
/// Pass ownership of aggregate function states from result to ColumnAggregateFunction objects in the resulting block.
result.aggregator = nullptr;
}
size_t rows = block.rows();
size_t bytes = block.bytes();
double elapsed_seconds = watch.elapsedSeconds();
LOG_DEBUG(
log,
"Merged partially aggregated blocks for bucket #{}. Got {} rows, {} from {} source rows in {} sec. ({:.3f} rows/sec., {}/sec.)",
bucket_num,
rows,
ReadableSize(bytes),
source_rows,
elapsed_seconds,
rows / elapsed_seconds,
ReadableSize(bytes / elapsed_seconds));
block.info.bucket_num = bucket_num;
return block;
}
template <typename Method>
void NO_INLINE Aggregator::convertBlockToTwoLevelImpl(
Method & method,
Arena * pool,
ColumnRawPtrs & key_columns,
const Block & source,
std::vector<Block> & destinations) const
{
typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
size_t rows = source.rows();
size_t columns = source.columns();
/// Create a 'selector' that will contain bucket index for every row. It will be used to scatter rows to buckets.
IColumn::Selector selector(rows);
/// For every row.
for (size_t i = 0; i < rows; ++i)
{
if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization)
{
if (state.isNullAt(i))
{
selector[i] = 0;
continue;
}
}
/// Calculate bucket number from row hash.
auto hash = state.getHash(method.data, i, *pool);
auto bucket = method.data.getBucketFromHash(hash);
selector[i] = bucket;
}
UInt32 num_buckets = static_cast<UInt32>(destinations.size());
for (size_t column_idx = 0; column_idx < columns; ++column_idx)
{
const ColumnWithTypeAndName & src_col = source.getByPosition(column_idx);
MutableColumns scattered_columns = src_col.column->scatter(num_buckets, selector);
for (UInt32 bucket = 0, size = num_buckets; bucket < size; ++bucket)
{
if (!scattered_columns[bucket]->empty())
{
Block & dst = destinations[bucket];
dst.info.bucket_num = static_cast<int>(bucket);
dst.insert({std::move(scattered_columns[bucket]), src_col.type, src_col.name});
}
/** Inserted columns of type ColumnAggregateFunction will own states of aggregate functions
* by holding shared_ptr to source column. See ColumnAggregateFunction.h
*/
}
}
}
std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block) const
{
if (!block)
return {};
AggregatedDataVariants data;
ColumnRawPtrs key_columns(params.keys_size);
/// Remember the columns we will work with
for (size_t i = 0; i < params.keys_size; ++i)
key_columns[i] = block.safeGetByPosition(i).column.get();
AggregatedDataVariants::Type type = method_chosen;
data.keys_size = params.keys_size;
data.key_sizes = key_sizes;
#define M(NAME) \
else if (type == AggregatedDataVariants::Type::NAME) \
type = AggregatedDataVariants::Type::NAME ## _two_level;
if (false) {} // NOLINT
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
data.init(type);
size_t num_buckets = 0;
#define M(NAME) \
else if (data.type == AggregatedDataVariants::Type::NAME) \
num_buckets = data.NAME->data.NUM_BUCKETS;
if (false) {} // NOLINT
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
std::vector<Block> splitted_blocks(num_buckets);
#define M(NAME) \
else if (data.type == AggregatedDataVariants::Type::NAME) \
convertBlockToTwoLevelImpl(*data.NAME, data.aggregates_pool, \
key_columns, block, splitted_blocks);
if (false) {} // NOLINT
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
else
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
return splitted_blocks;
}
template <typename Method, typename Table>
void NO_INLINE Aggregator::destroyImpl(Table & table) const
{
table.forEachMapped([&](AggregateDataPtr & data)
{
/** If an exception (usually a lack of memory, the MemoryTracker throws) arose
* after inserting the key into a hash table, but before creating all states of aggregate functions,
* then data will be equal nullptr.
*/
if (nullptr == data)
return;
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(data + offsets_of_aggregate_states[i]);
data = nullptr;
});
}
void Aggregator::destroyWithoutKey(AggregatedDataVariants & result) const
{
AggregatedDataWithoutKey & res_data = result.without_key;
if (nullptr != res_data)
{
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(res_data + offsets_of_aggregate_states[i]);
res_data = nullptr;
}
}
void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) const
{
if (result.empty())
return;
/// In what data structure is the data aggregated?
if (result.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
destroyWithoutKey(result);
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
destroyImpl<decltype(result.NAME)::element_type>(result.NAME->data);
if (false) {} // NOLINT
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
else if (result.type != AggregatedDataVariants::Type::without_key)
throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant.");
}
template Aggregator::ConvertToBlockRes<false>
Aggregator::prepareBlockAndFillSingleLevel<false>(AggregatedDataVariants & data_variants, bool final) const;
}
|