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
3426
3427
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442
3443
3444
3445
3446
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458
3459
3460
3461
3462
3463
3464
3465
3466
3467
3468
3469
3470
3471
3472
3473
3474
3475
3476
3477
3478
3479
3480
3481
3482
3483
3484
3485
3486
3487
3488
3489
3490
3491
3492
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516
3517
3518
3519
3520
3521
3522
3523
3524
3525
3526
3527
3528
3529
3530
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548
3549
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599
3600
3601
3602
3603
3604
3605
3606
3607
3608
3609
3610
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636
3637
3638
3639
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651
3652
3653
3654
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691
3692
3693
3694
3695
3696
3697
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711
3712
3713
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745
3746
3747
3748
3749
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
3767
3768
3769
3770
3771
3772
3773
3774
3775
3776
3777
3778
3779
3780
3781
3782
3783
3784
3785
3786
3787
3788
3789
3790
3791
3792
3793
3794
3795
3796
3797
3798
3799
3800
3801
3802
3803
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818
3819
3820
3821
3822
3823
3824
3825
3826
3827
3828
3829
3830
3831
3832
3833
3834
3835
3836
3837
3838
3839
3840
3841
3842
3843
3844
3845
3846
3847
3848
3849
3850
3851
3852
3853
3854
3855
3856
3857
3858
3859
3860
3861
3862
3863
3864
3865
3866
3867
3868
3869
use self::latency_awareness::LatencyAwareness;
pub use self::latency_awareness::LatencyAwarenessBuilder;

use super::{FallbackPlan, LoadBalancingPolicy, NodeRef, RoutingInfo};
use crate::{
    routing::{Shard, Token},
    transport::errors::QueryError,
    transport::{cluster::ClusterData, locator::ReplicaSet, node::Node, topology::Strategy},
};
use itertools::{Either, Itertools};
use rand::{prelude::SliceRandom, thread_rng, Rng};
use rand_pcg::Pcg32;
use scylla_cql::frame::response::result::TableSpec;
use scylla_cql::frame::types::SerialConsistency;
use scylla_cql::Consistency;
use std::hash::{Hash, Hasher};
use std::{fmt, sync::Arc, time::Duration};
use tracing::{debug, warn};
use uuid::Uuid;

#[derive(Clone, Copy)]
enum NodeLocationCriteria<'a> {
    Any,
    Datacenter(&'a str),
    DatacenterAndRack(&'a str, &'a str),
}

impl<'a> NodeLocationCriteria<'a> {
    fn datacenter(&self) -> Option<&'a str> {
        match self {
            Self::Any => None,
            Self::Datacenter(dc) | Self::DatacenterAndRack(dc, _) => Some(dc),
        }
    }
}

#[derive(Debug, Clone)]
enum NodeLocationPreference {
    Any,
    Datacenter(String),
    DatacenterAndRack(String, String),
}

impl NodeLocationPreference {
    fn datacenter(&self) -> Option<&str> {
        match self {
            Self::Any => None,
            Self::Datacenter(dc) | Self::DatacenterAndRack(dc, _) => Some(dc),
        }
    }

    #[allow(unused)]
    fn rack(&self) -> Option<&str> {
        match self {
            Self::Any | Self::Datacenter(_) => None,
            Self::DatacenterAndRack(_, rack) => Some(rack),
        }
    }
}

/// An ordering requirement for replicas.
#[derive(Clone, Copy)]
enum ReplicaOrder {
    /// No requirement. Replicas can be returned in arbitrary order.
    Arbitrary,

    /// A requirement for the order to be deterministic, not only across statement executions
    /// but also across drivers. This is used for LWT optimisation, to avoid Paxos conflicts.
    Deterministic,
}

/// Statement kind, used to enable specific load balancing patterns for certain cases.
///
/// Currently, there is a distinguished case of LWT statements, which should always be routed
/// to replicas in a deterministic order to avoid Paxos conflicts. Other statements
/// are routed to random replicas to balance the load.
#[derive(Clone, Copy)]
enum StatementType {
    /// The statement is a confirmed LWT. It's to be routed specifically.
    Lwt,

    /// The statement is not a confirmed LWT. It's to be routed in a default way.
    NonLwt,
}

/// A result of `pick_replica`.
enum PickedReplica<'a> {
    /// A replica that could be computed cheaply.
    Computed((NodeRef<'a>, Shard)),

    /// A replica that could not be computed cheaply. `pick` should therefore return None
    /// and `fallback` will then return that replica as the first in the iterator.
    ToBeComputedInFallback,
}

/// The default load balancing policy.
///
/// It can be configured to be datacenter-aware, rack-aware and token-aware.
/// Datacenter failover (sending query to a node from a remote datacenter)
/// for queries with non local consistency mode is also supported.
///
/// Latency awareness is available, although **not recommended**:
/// the penalisation mechanism it involves may interact badly with other
/// mechanisms, such as LWT optimisation. Also, the very tactics of penalising
/// nodes for recently measures latencies is believed to not be very stable
/// and beneficial. The number of in-flight requests, for instance, seems
/// to be a better metric showing how (over)loaded a target node/shard is.
/// For now, however, we don't have an implementation of the
/// in-flight-requests-aware policy.
#[allow(clippy::type_complexity)]
pub struct DefaultPolicy {
    /// Preferences regarding node location. One of: rack and DC, DC, or no preference.
    preferences: NodeLocationPreference,

    /// Configures whether the policy takes token into consideration when creating plans.
    /// If this is set to `true` AND token, keyspace and table are available,
    /// then policy prefers replicas and puts them earlier in the query plan.
    is_token_aware: bool,

    /// Whether to permit remote nodes (those not located in the preferred DC) in plans.
    /// If no preferred DC is set, this has no effect.
    permit_dc_failover: bool,

    /// A predicate that a target (node + shard) must satisfy in order to be picked.
    /// This was introduced to make latency awareness cleaner.
    /// - if latency awareness is disabled, then `pick_predicate` is just `Self::is_alive()`;
    /// - if latency awareness is enabled, then it is `Self::is_alive() && latency_predicate()`,
    ///   which checks that the target is not penalised due to high latencies.
    pick_predicate: Box<dyn Fn(NodeRef<'_>, Option<Shard>) -> bool + Send + Sync>,

    /// Additional layer that penalises targets that are too slow compared to others
    /// in terms of latency. It works in the following way:
    /// - for `pick`, it uses `latency_predicate` to filter out penalised nodes,
    ///   so that a penalised node will never be `pick`ed;
    /// - for `fallback`, it wraps the returned iterator, moving all penalised nodes
    ///   to the end, in a stable way.
    ///
    /// Penalisation is done based on collected and updated latencies.
    latency_awareness: Option<LatencyAwareness>,

    /// The policy chooses (in `pick`) and shuffles (in `fallback`) replicas and nodes
    /// based on random number generator. For sake of deterministic testing,
    /// a fixed seed can be used.
    fixed_seed: Option<u64>,
}

impl fmt::Debug for DefaultPolicy {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        f.debug_struct("DefaultPolicy")
            .field("preferences", &self.preferences)
            .field("is_token_aware", &self.is_token_aware)
            .field("permit_dc_failover", &self.permit_dc_failover)
            .field("latency_awareness", &self.latency_awareness)
            .field("fixed_seed", &self.fixed_seed)
            .finish_non_exhaustive()
    }
}

impl LoadBalancingPolicy for DefaultPolicy {
    fn pick<'a>(
        &'a self,
        query: &'a RoutingInfo,
        cluster: &'a ClusterData,
    ) -> Option<(NodeRef<'a>, Option<Shard>)> {
        /* For prepared statements, token-aware logic is available, we know what are the replicas
         * for the statement, so that we can pick one of them. */
        let routing_info = self.routing_info(query, cluster);

        if let Some(ref token_with_strategy) = routing_info.token_with_strategy {
            if self.preferences.datacenter().is_some()
                && !self.permit_dc_failover
                && matches!(
                    token_with_strategy.strategy,
                    Strategy::SimpleStrategy { .. }
                )
            {
                warn!("\
Combining SimpleStrategy with preferred_datacenter set to Some and disabled datacenter failover may lead to empty query plans for some tokens.\
It is better to give up using one of them: either operate in a keyspace with NetworkTopologyStrategy, which explicitly states\
how many replicas there are in each datacenter (you probably want at least 1 to avoid empty plans while preferring that datacenter), \
or refrain from preferring datacenters (which may ban all other datacenters, if datacenter failover happens to be not possible)."
                );
            }
        }

        /* LWT statements need to be routed differently: always to the same replica, to avoid Paxos contention. */
        let statement_type = if query.is_confirmed_lwt {
            StatementType::Lwt
        } else {
            StatementType::NonLwt
        };

        /* Token-aware logic - if routing info is available, we know what are the replicas
         * for the statement. Try to pick one of them. */
        if let (Some(ts), Some(table_spec)) = (&routing_info.token_with_strategy, query.table) {
            if let NodeLocationPreference::DatacenterAndRack(dc, rack) = &self.preferences {
                // Try to pick some alive local rack random replica.
                let local_rack_picked = self.pick_replica(
                    ts,
                    NodeLocationCriteria::DatacenterAndRack(dc, rack),
                    |node, shard| (self.pick_predicate)(node, Some(shard)),
                    cluster,
                    statement_type,
                    table_spec,
                );

                if let Some(picked) = local_rack_picked {
                    return match picked {
                        PickedReplica::Computed((alive_local_rack_replica, shard)) => {
                            Some((alive_local_rack_replica, Some(shard)))
                        }
                        // Let call to fallback() compute the replica, because it requires allocation.
                        PickedReplica::ToBeComputedInFallback => None,
                    };
                }
            }

            if let NodeLocationPreference::DatacenterAndRack(dc, _)
            | NodeLocationPreference::Datacenter(dc) = &self.preferences
            {
                // Try to pick some alive local random replica.
                let picked = self.pick_replica(
                    ts,
                    NodeLocationCriteria::Datacenter(dc),
                    |node, shard| (self.pick_predicate)(node, Some(shard)),
                    cluster,
                    statement_type,
                    table_spec,
                );

                if let Some(picked) = picked {
                    return match picked {
                        PickedReplica::Computed((alive_local_replica, shard)) => {
                            Some((alive_local_replica, Some(shard)))
                        }
                        // Let call to fallback() compute the replica, because it requires allocation.
                        PickedReplica::ToBeComputedInFallback => None,
                    };
                }
            }

            // If preferred datacenter is not specified, or if datacenter failover is possible, loosen restriction about locality.
            if self.preferences.datacenter().is_none()
                || self.is_datacenter_failover_possible(&routing_info)
            {
                // Try to pick some alive random replica.
                let picked = self.pick_replica(
                    ts,
                    NodeLocationCriteria::Any,
                    |node, shard| (self.pick_predicate)(node, Some(shard)),
                    cluster,
                    statement_type,
                    table_spec,
                );
                if let Some(picked) = picked {
                    return match picked {
                        PickedReplica::Computed((alive_remote_replica, shard)) => {
                            Some((alive_remote_replica, Some(shard)))
                        }
                        // Let call to fallback() compute the replica, because it requires allocation.
                        PickedReplica::ToBeComputedInFallback => None,
                    };
                }
            }
        };

        /* Token-unaware logic - if routing info is not available (e.g. for unprepared statements),
         * or no replica was suitable for targeting it (e.g. disabled or down), try to choose
         * a random node, not necessarily a replica. */

        /* We start having not alive nodes filtered out. This is done by `pick_predicate`,
         * which always contains `Self::is_alive()`. */

        // Let's start with local nodes, i.e. those in the preferred datacenter.
        // If there was no preferred datacenter specified, all nodes are treated as local.
        let local_nodes = self.preferred_node_set(cluster);

        if let NodeLocationPreference::DatacenterAndRack(dc, rack) = &self.preferences {
            // Try to pick some alive random local rack node.
            let rack_predicate = Self::make_rack_predicate(
                |node| (self.pick_predicate)(node, None),
                NodeLocationCriteria::DatacenterAndRack(dc, rack),
            );
            let local_rack_node_picked = self.pick_node(local_nodes, rack_predicate);

            if let Some(alive_local_rack_node) = local_rack_node_picked {
                return Some((alive_local_rack_node, None));
            }
        }

        // Try to pick some alive random local node.
        let local_node_picked =
            self.pick_node(local_nodes, |node| (self.pick_predicate)(node, None));
        if let Some(alive_local_node) = local_node_picked {
            return Some((alive_local_node, None));
        }

        let all_nodes = cluster.replica_locator().unique_nodes_in_global_ring();
        // If a datacenter failover is possible, loosen restriction about locality.
        if self.is_datacenter_failover_possible(&routing_info) {
            let maybe_remote_node_picked =
                self.pick_node(all_nodes, |node| (self.pick_predicate)(node, None));
            if let Some(alive_maybe_remote_node) = maybe_remote_node_picked {
                return Some((alive_maybe_remote_node, None));
            }
        }

        /* As we are here, we failed to pick any alive node. Now let's consider even down nodes. */

        // Previous checks imply that every node we could have selected is down.
        // Let's try to return a down node that wasn't disabled.
        let maybe_down_local_node_picked = self.pick_node(local_nodes, |node| node.is_enabled());
        if let Some(down_but_enabled_local_node) = maybe_down_local_node_picked {
            return Some((down_but_enabled_local_node, None));
        }

        // If a datacenter failover is possible, loosen restriction about locality.
        if self.is_datacenter_failover_possible(&routing_info) {
            let maybe_down_maybe_remote_node_picked =
                self.pick_node(all_nodes, |node| node.is_enabled());
            if let Some(down_but_enabled_maybe_remote_node) = maybe_down_maybe_remote_node_picked {
                return Some((down_but_enabled_maybe_remote_node, None));
            }
        }

        // Every node is disabled. This could be due to a bad host filter - configuration error.
        // It makes no sense to return disabled nodes (there are no open connections to them anyway),
        // so let's return None. `fallback()` will return empty iterator, and so the whole plan
        // will be empty.
        None
    }

    fn fallback<'a>(
        &'a self,
        query: &'a RoutingInfo,
        cluster: &'a ClusterData,
    ) -> FallbackPlan<'a> {
        /* For prepared statements, token-aware logic is available, we know what are the replicas
         * for the statement, so that we can pick one of them. */
        let routing_info = self.routing_info(query, cluster);

        /* LWT statements need to be routed differently: always to the same replica, to avoid Paxos contention. */
        let statement_type = if query.is_confirmed_lwt {
            StatementType::Lwt
        } else {
            StatementType::NonLwt
        };

        /* Token-aware logic - if routing info is available, we know what are the replicas for the statement.
         * Get a list of alive replicas:
         * - shuffled list in case of non-LWTs,
         * - deterministically ordered in case of LWTs. */
        let maybe_replicas = if let (Some(ts), Some(table_spec)) =
            (&routing_info.token_with_strategy, query.table)
        {
            // Iterator over alive local rack replicas (shuffled or deterministically ordered,
            // depending on the statement being LWT or not).
            let maybe_local_rack_replicas =
                if let NodeLocationPreference::DatacenterAndRack(dc, rack) = &self.preferences {
                    let local_rack_replicas = self.maybe_shuffled_replicas(
                        ts,
                        NodeLocationCriteria::DatacenterAndRack(dc, rack),
                        |node, shard| Self::is_alive(node, Some(shard)),
                        cluster,
                        statement_type,
                        table_spec,
                    );
                    Either::Left(local_rack_replicas)
                } else {
                    Either::Right(std::iter::empty())
                };

            // Iterator over alive local datacenter replicas (shuffled or deterministically ordered,
            // depending on the statement being LWT or not).
            let maybe_local_replicas = if let NodeLocationPreference::DatacenterAndRack(dc, _)
            | NodeLocationPreference::Datacenter(dc) =
                &self.preferences
            {
                let local_replicas = self.maybe_shuffled_replicas(
                    ts,
                    NodeLocationCriteria::Datacenter(dc),
                    |node, shard| Self::is_alive(node, Some(shard)),
                    cluster,
                    statement_type,
                    table_spec,
                );
                Either::Left(local_replicas)
            } else {
                Either::Right(std::iter::empty())
            };

            // If no datacenter is preferred, or datacenter failover is possible, loosen restriction about locality.
            let maybe_remote_replicas = if self.preferences.datacenter().is_none()
                || self.is_datacenter_failover_possible(&routing_info)
            {
                // Iterator over alive replicas (shuffled or deterministically ordered,
                // depending on the statement being LWT or not).
                let remote_replicas = self.maybe_shuffled_replicas(
                    ts,
                    NodeLocationCriteria::Any,
                    |node, shard| Self::is_alive(node, Some(shard)),
                    cluster,
                    statement_type,
                    table_spec,
                );
                Either::Left(remote_replicas)
            } else {
                Either::Right(std::iter::empty())
            };

            // Produce an iterator, prioritizing local replicas.
            // If preferred datacenter is not specified, every replica is treated as a remote one.
            Either::Left(
                maybe_local_rack_replicas
                    .chain(maybe_local_replicas)
                    .chain(maybe_remote_replicas)
                    .map(|(node, shard)| (node, Some(shard))),
            )
        } else {
            Either::Right(std::iter::empty::<(NodeRef<'a>, Option<Shard>)>())
        };

        /* Token-unaware logic - if routing info is not available (e.g. for unprepared statements),
         * or no replica is suitable for targeting it (e.g. disabled or down), try targetting nodes
         * that are not necessarily replicas. */

        /* We start having not alive nodes filtered out. */

        // All nodes in the local datacenter (if one is given).
        let local_nodes = self.preferred_node_set(cluster);

        let robinned_local_rack_nodes =
            if let NodeLocationPreference::DatacenterAndRack(dc, rack) = &self.preferences {
                let rack_predicate = Self::make_rack_predicate(
                    |node| Self::is_alive(node, None),
                    NodeLocationCriteria::DatacenterAndRack(dc, rack),
                );
                Either::Left(
                    self.round_robin_nodes(local_nodes, rack_predicate)
                        .map(|node| (node, None)),
                )
            } else {
                Either::Right(std::iter::empty::<(NodeRef<'a>, Option<Shard>)>())
            };

        let robinned_local_nodes = self
            .round_robin_nodes(local_nodes, |node| Self::is_alive(node, None))
            .map(|node| (node, None));

        // All nodes in the cluster.
        let all_nodes = cluster.replica_locator().unique_nodes_in_global_ring();

        // If a datacenter failover is possible, loosen restriction about locality.
        let maybe_remote_nodes = if self.is_datacenter_failover_possible(&routing_info) {
            let robinned_all_nodes =
                self.round_robin_nodes(all_nodes, |node| Self::is_alive(node, None));

            Either::Left(robinned_all_nodes.map(|node| (node, None)))
        } else {
            Either::Right(std::iter::empty::<(NodeRef<'a>, Option<Shard>)>())
        };

        // Even if we consider some enabled nodes to be down, we should try contacting them in the last resort.
        let maybe_down_local_nodes = local_nodes
            .iter()
            .filter(|node| node.is_enabled())
            .map(|node| (node, None));

        // If a datacenter failover is possible, loosen restriction about locality.
        let maybe_down_nodes = if self.is_datacenter_failover_possible(&routing_info) {
            Either::Left(
                all_nodes
                    .iter()
                    .filter(|node| node.is_enabled())
                    .map(|node| (node, None)),
            )
        } else {
            Either::Right(std::iter::empty())
        };

        /// *Plan* should return unique elements. It is not however obvious what it means,
        /// because some nodes in the plan may have shards and some may not.
        ///
        /// This helper structure defines equality of plan elements.
        /// How the comparison works:
        /// - If at least one of elements is shard-less, then compare just nodes.
        /// - If both elements have shards, then compare both nodes and shards.
        ///
        /// Why is it implemented this way?
        /// Driver should not attempt to send a request to the same destination twice.
        /// If a plan element doesn't have shard specified, then a random shard will be
        /// chosen by the driver. If the plan also contains the same node but with
        /// a shard present, and we randomly choose the same shard for the shard-less element,
        /// then we have duplication.
        ///
        /// Example: plan is `[(Node1, Some(1)), (Node1, None)]` - if the driver uses
        /// the second element and randomly chooses shard 1, then we have duplication.
        ///
        /// On the other hand, if a plan has a duplicate node, but with different shards,
        /// then we want to use both elements - so we can't just make the list unique by node,
        /// and so this struct was created.
        struct DefaultPolicyTargetComparator {
            host_id: Uuid,
            shard: Option<Shard>,
        }

        impl PartialEq for DefaultPolicyTargetComparator {
            fn eq(&self, other: &Self) -> bool {
                match (self.shard, other.shard) {
                    (_, None) | (None, _) => self.host_id.eq(&other.host_id),
                    (Some(shard_left), Some(shard_right)) => {
                        self.host_id.eq(&other.host_id) && shard_left.eq(&shard_right)
                    }
                }
            }
        }

        impl Eq for DefaultPolicyTargetComparator {}

        impl Hash for DefaultPolicyTargetComparator {
            fn hash<H: Hasher>(&self, state: &mut H) {
                self.host_id.hash(state);
            }
        }

        // Construct a fallback plan as a composition of:
        // - local rack alive replicas,
        // - local datacenter alive replicas (or all alive replicas is no DC is preferred),
        // - remote alive replicas (if DC failover is enabled),
        // - local rack alive nodes,
        // - local datacenter alive nodes (or all alive nodes is no DC is preferred),
        // - remote alive nodes (if DC failover is enabled),
        // - local datacenter nodes,
        // - remote nodes (if DC failover is enabled).
        let plan = maybe_replicas
            .chain(robinned_local_rack_nodes)
            .chain(robinned_local_nodes)
            .chain(maybe_remote_nodes)
            .chain(maybe_down_local_nodes)
            .chain(maybe_down_nodes)
            .unique_by(|(node, shard)| DefaultPolicyTargetComparator {
                host_id: node.host_id,
                shard: *shard,
            });

        // If latency awareness is enabled, wrap the plan by applying latency penalisation:
        // all penalised nodes are moved behind non-penalised nodes, in a stable fashion.
        if let Some(latency_awareness) = self.latency_awareness.as_ref() {
            Box::new(latency_awareness.wrap(plan))
        } else {
            Box::new(plan)
        }
    }

    fn name(&self) -> String {
        "DefaultPolicy".to_string()
    }

    fn on_query_success(&self, _routing_info: &RoutingInfo, latency: Duration, node: NodeRef<'_>) {
        if let Some(latency_awareness) = self.latency_awareness.as_ref() {
            latency_awareness.report_query(node, latency);
        }
    }

    fn on_query_failure(
        &self,
        _routing_info: &RoutingInfo,
        latency: Duration,
        node: NodeRef<'_>,
        error: &QueryError,
    ) {
        if let Some(latency_awareness) = self.latency_awareness.as_ref() {
            if LatencyAwareness::reliable_latency_measure(error) {
                latency_awareness.report_query(node, latency);
            }
        }
    }
}

impl DefaultPolicy {
    /// Creates a builder used to customise configuration of a new DefaultPolicy.
    pub fn builder() -> DefaultPolicyBuilder {
        DefaultPolicyBuilder::new()
    }

    /// Returns the given routing info processed based on given cluster data.
    fn routing_info<'a>(
        &'a self,
        query: &'a RoutingInfo,
        cluster: &'a ClusterData,
    ) -> ProcessedRoutingInfo<'a> {
        let mut routing_info = ProcessedRoutingInfo::new(query, cluster);

        if !self.is_token_aware {
            routing_info.token_with_strategy = None;
        }

        routing_info
    }

    /// Returns all nodes in the local datacenter if one is given,
    /// or else all nodes in the cluster.
    fn preferred_node_set<'a>(&'a self, cluster: &'a ClusterData) -> &'a [Arc<Node>] {
        if let Some(preferred_datacenter) = self.preferences.datacenter() {
            if let Some(nodes) = cluster
                .replica_locator()
                .unique_nodes_in_datacenter_ring(preferred_datacenter)
            {
                nodes
            } else {
                tracing::warn!(
                    "Datacenter specified as the preferred one ({}) does not exist!",
                    preferred_datacenter
                );
                // We won't guess any DC, as it could lead to possible violation of dc failover ban.
                &[]
            }
        } else {
            cluster.replica_locator().unique_nodes_in_global_ring()
        }
    }

    /// Returns a full replica set for given datacenter (if given, else for all DCs),
    /// cluster data and table spec.
    fn nonfiltered_replica_set<'a>(
        &'a self,
        ts: &TokenWithStrategy<'a>,
        replica_location: NodeLocationCriteria<'a>,
        cluster: &'a ClusterData,
        table_spec: &TableSpec,
    ) -> ReplicaSet<'a> {
        let datacenter = replica_location.datacenter();

        cluster
            .replica_locator()
            .replicas_for_token(ts.token, ts.strategy, datacenter, table_spec)
    }

    /// Wraps the provided predicate, adding the requirement for rack to match.
    fn make_rack_predicate<'a>(
        predicate: impl Fn(NodeRef<'a>) -> bool + 'a,
        replica_location: NodeLocationCriteria<'a>,
    ) -> impl Fn(NodeRef<'a>) -> bool {
        move |node| match replica_location {
            NodeLocationCriteria::Any | NodeLocationCriteria::Datacenter(_) => predicate(node),
            NodeLocationCriteria::DatacenterAndRack(_, rack) => {
                predicate(node) && node.rack.as_deref() == Some(rack)
            }
        }
    }

    /// Wraps the provided predicate, adding the requirement for rack to match.
    fn make_sharded_rack_predicate<'a>(
        predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a,
        replica_location: NodeLocationCriteria<'a>,
    ) -> impl Fn(NodeRef<'a>, Shard) -> bool {
        move |node, shard| match replica_location {
            NodeLocationCriteria::Any | NodeLocationCriteria::Datacenter(_) => {
                predicate(node, shard)
            }
            NodeLocationCriteria::DatacenterAndRack(_, rack) => {
                predicate(node, shard) && node.rack.as_deref() == Some(rack)
            }
        }
    }

    /// Returns iterator over replicas for given token and table spec, filtered
    /// by provided location criteria and predicate.
    /// Respects requested replica order, i.e. if requested, returns replicas ordered
    /// deterministically (i.e. by token ring order or by tablet definition order),
    /// else returns replicas in arbitrary order.
    fn filtered_replicas<'a>(
        &'a self,
        ts: &TokenWithStrategy<'a>,
        replica_location: NodeLocationCriteria<'a>,
        predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a,
        cluster: &'a ClusterData,
        order: ReplicaOrder,
        table_spec: &TableSpec,
    ) -> impl Iterator<Item = (NodeRef<'a>, Shard)> {
        let predicate = Self::make_sharded_rack_predicate(predicate, replica_location);

        let replica_iter = match order {
            ReplicaOrder::Arbitrary => Either::Left(
                self.nonfiltered_replica_set(ts, replica_location, cluster, table_spec)
                    .into_iter(),
            ),
            ReplicaOrder::Deterministic => Either::Right(
                self.nonfiltered_replica_set(ts, replica_location, cluster, table_spec)
                    .into_replicas_ordered()
                    .into_iter(),
            ),
        };
        replica_iter.filter(move |(node, shard): &(NodeRef<'a>, Shard)| predicate(node, *shard))
    }

    /// Picks a replica for given token and table spec which meets the provided location criteria
    /// and the predicate.
    /// The replica is chosen randomly over all candidates that meet the criteria
    /// unless the query is LWT; if so, the first replica meeting the criteria is chosen
    /// to avoid Paxos contention.
    fn pick_replica<'a>(
        &'a self,
        ts: &TokenWithStrategy<'a>,
        replica_location: NodeLocationCriteria<'a>,
        predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a,
        cluster: &'a ClusterData,
        statement_type: StatementType,
        table_spec: &TableSpec,
    ) -> Option<PickedReplica<'a>> {
        match statement_type {
            StatementType::Lwt => {
                self.pick_first_replica(ts, replica_location, predicate, cluster, table_spec)
            }
            StatementType::NonLwt => self
                .pick_random_replica(ts, replica_location, predicate, cluster, table_spec)
                .map(PickedReplica::Computed),
        }
    }

    /// Picks the first (wrt the deterministic order imposed on the keyspace, see comment below)
    /// replica for given token and table spec which meets the provided location criteria
    /// and the predicate.
    // This is to be used for LWT optimisation: in order to reduce contention
    // caused by Paxos conflicts, we always try to query replicas in the same,
    // deterministic order:
    // - ring order for token ring keyspaces,
    // - tablet definition order for tablet keyspaces.
    //
    // If preferred rack and DC are set, then the first (encountered on the ring) replica
    // that resides in that rack in that DC **and** satisfies the `predicate` is returned.
    //
    // If preferred DC is set, then the first (encountered on the ring) replica
    // that resides in that DC **and** satisfies the `predicate` is returned.
    //
    // If no DC/rack preferences are set, then the only possible replica to be returned
    // (due to expensive computation of the others, and we avoid expensive computation in `pick()`)
    // is the primary replica. If it exists, Some is returned, with either Computed(primary_replica)
    // **iff** it satisfies the predicate or ToBeComputedInFallback otherwise.
    fn pick_first_replica<'a>(
        &'a self,
        ts: &TokenWithStrategy<'a>,
        replica_location: NodeLocationCriteria<'a>,
        predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a,
        cluster: &'a ClusterData,
        table_spec: &TableSpec,
    ) -> Option<PickedReplica<'a>> {
        match replica_location {
            NodeLocationCriteria::Any => {
                // ReplicaSet returned by ReplicaLocator for this case:
                // 1) can be precomputed and later used cheaply,
                // 2) returns replicas in the **non-ring order** (this because ReplicaSet chains
                //    ring-ordered replicas sequences from different DCs, thus not preserving
                //    the global ring order).
                // Because of 2), we can't use a precomputed ReplicaSet, but instead we need ReplicasOrdered.
                // As ReplicasOrdered can compute cheaply only the primary global replica
                // (computation of the remaining ones is expensive), in case that the primary replica
                // does not satisfy the `predicate`, ToBeComputedInFallback is returned.
                // All expensive computation is to be done only when `fallback()` is called.
                self.nonfiltered_replica_set(ts, replica_location, cluster, table_spec)
                    .into_replicas_ordered()
                    .into_iter()
                    .next()
                    .map(|(primary_replica, shard)| {
                        if predicate(primary_replica, shard) {
                            PickedReplica::Computed((primary_replica, shard))
                        } else {
                            PickedReplica::ToBeComputedInFallback
                        }
                    })
            }
            NodeLocationCriteria::Datacenter(_) | NodeLocationCriteria::DatacenterAndRack(_, _) => {
                // ReplicaSet returned by ReplicaLocator for this case:
                // 1) can be precomputed and later used cheaply,
                // 2) returns replicas in the ring order (this is not true for the case
                //    when multiple DCs are allowed, because ReplicaSet chains replicas sequences
                //    from different DCs, thus not preserving the global ring order)
                self.filtered_replicas(
                    ts,
                    replica_location,
                    predicate,
                    cluster,
                    ReplicaOrder::Deterministic,
                    table_spec,
                )
                .next()
                .map(PickedReplica::Computed)
            }
        }
    }

    /// Picks a random replica for given token and table spec which meets the provided
    /// location criteria and the predicate.
    fn pick_random_replica<'a>(
        &'a self,
        ts: &TokenWithStrategy<'a>,
        replica_location: NodeLocationCriteria<'a>,
        predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a,
        cluster: &'a ClusterData,
        table_spec: &TableSpec,
    ) -> Option<(NodeRef<'a>, Shard)> {
        let predicate = Self::make_sharded_rack_predicate(predicate, replica_location);

        let replica_set = self.nonfiltered_replica_set(ts, replica_location, cluster, table_spec);

        if let Some(fixed) = self.fixed_seed {
            let mut gen = Pcg32::new(fixed, 0);
            replica_set.choose_filtered(&mut gen, |(node, shard)| predicate(node, *shard))
        } else {
            replica_set.choose_filtered(&mut thread_rng(), |(node, shard)| predicate(node, *shard))
        }
    }

    /// Returns iterator over replicas for given token and table spec, filtered
    /// by provided location criteria and predicate.
    /// By default, the replicas are shuffled.
    /// For LWTs, though, the replicas are instead returned in a deterministic order.
    fn maybe_shuffled_replicas<'a>(
        &'a self,
        ts: &TokenWithStrategy<'a>,
        replica_location: NodeLocationCriteria<'a>,
        predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a,
        cluster: &'a ClusterData,
        statement_type: StatementType,
        table_spec: &TableSpec,
    ) -> impl Iterator<Item = (NodeRef<'a>, Shard)> {
        let order = match statement_type {
            StatementType::Lwt => ReplicaOrder::Deterministic,
            StatementType::NonLwt => ReplicaOrder::Arbitrary,
        };

        let replicas =
            self.filtered_replicas(ts, replica_location, predicate, cluster, order, table_spec);

        match statement_type {
            // As an LWT optimisation: in order to reduce contention caused by Paxos conflicts,
            // we always try to query replicas in the same order.
            StatementType::Lwt => Either::Left(replicas),
            StatementType::NonLwt => Either::Right(self.shuffle(replicas)),
        }
    }

    /// Returns an iterator over the given slice of nodes, rotated by a random shift.
    fn randomly_rotated_nodes(nodes: &[Arc<Node>]) -> impl Iterator<Item = NodeRef<'_>> {
        // Create a randomly rotated slice view
        let nodes_len = nodes.len();
        if nodes_len > 0 {
            let index = rand::thread_rng().gen_range(0..nodes_len); // gen_range() panics when range is empty!
            Either::Left(
                nodes[index..]
                    .iter()
                    .chain(nodes[..index].iter())
                    .take(nodes.len()),
            )
        } else {
            Either::Right(std::iter::empty())
        }
    }

    /// Picks a random node from the slice of nodes. The node must satisfy the given predicate.
    fn pick_node<'a>(
        &'a self,
        nodes: &'a [Arc<Node>],
        predicate: impl Fn(NodeRef<'a>) -> bool,
    ) -> Option<NodeRef<'a>> {
        // Select the first node that matches the predicate
        Self::randomly_rotated_nodes(nodes).find(|&node| predicate(node))
    }

    /// Returns an iterator over the given slice of nodes, rotated by a random shift
    /// and filtered by given predicate.
    fn round_robin_nodes<'a>(
        &'a self,
        nodes: &'a [Arc<Node>],
        predicate: impl Fn(NodeRef<'a>) -> bool,
    ) -> impl Iterator<Item = NodeRef<'a>> {
        Self::randomly_rotated_nodes(nodes).filter(move |node| predicate(node))
    }

    /// Wraps a given iterator by shuffling its contents.
    fn shuffle<'a>(
        &self,
        iter: impl Iterator<Item = (NodeRef<'a>, Shard)>,
    ) -> impl Iterator<Item = (NodeRef<'a>, Shard)> {
        let mut vec: Vec<(NodeRef<'_>, Shard)> = iter.collect();

        if let Some(fixed) = self.fixed_seed {
            let mut gen = Pcg32::new(fixed, 0);
            vec.shuffle(&mut gen);
        } else {
            vec.shuffle(&mut thread_rng());
        }

        vec.into_iter()
    }

    /// Returns true iff the node should be considered to be alive.
    fn is_alive(node: NodeRef, _shard: Option<Shard>) -> bool {
        // For now, we leave this as stub, until we have time to improve node events.
        // node.is_enabled() && !node.is_down()
        node.is_enabled()
    }

    /// Returns true iff the datacenter failover is permitted for the statement being executed.
    fn is_datacenter_failover_possible(&self, routing_info: &ProcessedRoutingInfo) -> bool {
        self.preferences.datacenter().is_some()
            && self.permit_dc_failover
            && !routing_info.local_consistency
    }
}

impl Default for DefaultPolicy {
    fn default() -> Self {
        Self {
            preferences: NodeLocationPreference::Any,
            is_token_aware: true,
            permit_dc_failover: false,
            pick_predicate: Box::new(Self::is_alive),
            latency_awareness: None,
            fixed_seed: None,
        }
    }
}

/// The intended way to instantiate the DefaultPolicy.
///
/// # Example
/// ```
/// # async fn example() -> Result<(), Box<dyn std::error::Error>> {
/// use scylla::load_balancing::DefaultPolicy;
///
/// let default_policy = DefaultPolicy::builder()
///     .prefer_datacenter("dc1".to_string())
///     .token_aware(true)
///     .permit_dc_failover(true)
///     .build();
/// # Ok(())
/// # }
#[derive(Clone, Debug)]
pub struct DefaultPolicyBuilder {
    preferences: NodeLocationPreference,
    is_token_aware: bool,
    permit_dc_failover: bool,
    latency_awareness: Option<LatencyAwarenessBuilder>,
    enable_replica_shuffle: bool,
}

impl DefaultPolicyBuilder {
    /// Creates a builder used to customise configuration of a new DefaultPolicy.
    pub fn new() -> Self {
        Self {
            preferences: NodeLocationPreference::Any,
            is_token_aware: true,
            permit_dc_failover: false,
            latency_awareness: None,
            enable_replica_shuffle: true,
        }
    }

    /// Builds a new DefaultPolicy with the previously set configuration.
    pub fn build(self) -> Arc<dyn LoadBalancingPolicy> {
        let latency_awareness = self.latency_awareness.map(|builder| builder.build());
        let pick_predicate = if let Some(ref latency_awareness) = latency_awareness {
            let latency_predicate = latency_awareness.generate_predicate();
            Box::new(move |node: NodeRef<'_>, shard| {
                DefaultPolicy::is_alive(node, shard) && latency_predicate(node)
            })
                as Box<dyn Fn(NodeRef<'_>, Option<Shard>) -> bool + Send + Sync + 'static>
        } else {
            Box::new(DefaultPolicy::is_alive)
        };

        Arc::new(DefaultPolicy {
            preferences: self.preferences,
            is_token_aware: self.is_token_aware,
            permit_dc_failover: self.permit_dc_failover,
            pick_predicate,
            latency_awareness,
            fixed_seed: (!self.enable_replica_shuffle).then(|| {
                let seed = rand::random();
                debug!("DefaultPolicy: setting fixed seed to {}", seed);
                seed
            }),
        })
    }

    /// Sets the datacenter to be preferred by this policy.
    ///
    /// Allows the load balancing policy to prioritize nodes based on their location.
    /// When a preferred datacenter is set, the policy will treat nodes in that
    /// datacenter as "local" nodes, and nodes in other datacenters as "remote" nodes.
    /// This affects the order in which nodes are returned by the policy when
    /// selecting replicas for read or write operations. If no preferred datacenter
    /// is specified, the policy will treat all nodes as local nodes.
    ///
    /// When datacenter failover is disabled (`permit_dc_failover` is set to false),
    /// the default policy will only include local nodes in load balancing plans.
    /// Remote nodes will be excluded, even if they are alive and available
    /// to serve requests.
    pub fn prefer_datacenter(mut self, datacenter_name: String) -> Self {
        self.preferences = NodeLocationPreference::Datacenter(datacenter_name);
        self
    }

    /// Sets the datacenter and rack to be preferred by this policy.
    ///
    /// Allows the load balancing policy to prioritize nodes based on their location
    /// as well as their availability zones in the preferred datacenter.
    /// When a preferred datacenter is set, the policy will treat nodes in that
    /// datacenter as "local" nodes, and nodes in other datacenters as "remote" nodes.
    /// This affects the order in which nodes are returned by the policy when
    /// selecting replicas for read or write operations. If no preferred datacenter
    /// is specified, the policy will treat all nodes as local nodes.
    ///
    /// When datacenter failover is disabled (`permit_dc_failover` is set to false),
    /// the default policy will only include local nodes in load balancing plans.
    /// Remote nodes will be excluded, even if they are alive and available
    /// to serve requests.
    ///
    /// When a preferred rack is set, the policy will first return replicas in the local rack
    /// in the preferred datacenter, and then the other replicas in the datacenter.
    pub fn prefer_datacenter_and_rack(
        mut self,
        datacenter_name: String,
        rack_name: String,
    ) -> Self {
        self.preferences = NodeLocationPreference::DatacenterAndRack(datacenter_name, rack_name);
        self
    }

    /// Sets whether this policy is token-aware (balances load more consciously) or not.
    ///
    /// Token awareness refers to a mechanism by which the driver is aware
    /// of the token range assigned to each node in the cluster. Tokens
    /// are assigned to nodes to partition the data and distribute it
    /// across the cluster.
    ///
    /// When a user wants to read or write data, the driver can use token awareness
    /// to route the request to the correct node based on the token range of the data
    /// being accessed. This can help to minimize network traffic and improve
    /// performance by ensuring that the data is accessed locally as much as possible.
    ///
    /// In the case of `DefaultPolicy`, token awareness is enabled by default,
    /// meaning that the policy will prefer to return alive local replicas
    /// if the token is available. This means that if the client is requesting data
    /// that falls within the token range of a particular node, the policy will try
    /// to route the request to that node first, assuming it is alive and responsive.
    ///
    /// Token awareness can significantly improve the performance and scalability
    /// of applications built on Scylla. By using token awareness, users can ensure
    /// that data is accessed locally as much as possible, reducing network overhead
    /// and improving throughput.
    pub fn token_aware(mut self, is_token_aware: bool) -> Self {
        self.is_token_aware = is_token_aware;
        self
    }

    /// Sets whether this policy permits datacenter failover, i.e. ever attempts
    /// to send requests to nodes from a non-preferred datacenter.
    ///
    /// In the event of a datacenter outage or network failure, the nodes
    /// in that datacenter may become unavailable, and clients may no longer
    /// be able to access data stored on those nodes. To address this,
    /// the `DefaultPolicy` supports datacenter failover, which allows routing
    /// requests to nodes in other datacenters if the local nodes are unavailable.
    ///
    /// Datacenter failover can be enabled in `DefaultPolicy` setting this flag.
    /// When it is set, the policy will prefer to return alive remote replicas
    /// if datacenter failover is permitted and possible due to consistency
    /// constraints.
    pub fn permit_dc_failover(mut self, permit: bool) -> Self {
        self.permit_dc_failover = permit;
        self
    }

    /// Latency awareness is a mechanism that penalises nodes whose measured
    /// recent average latency classifies it as falling behind the others.
    ///
    /// Every `update_rate` the global minimum average latency is computed,
    /// and all nodes whose average latency is worse than `exclusion_threshold`
    /// times the global minimum average latency become penalised for
    /// `retry_period`. Penalisation involves putting those nodes at the very end
    /// of the query plan. As it is often not truly beneficial to prefer
    /// faster non-replica than replicas lagging behind the non-replicas,
    /// this mechanism may as well worsen latencies and/or throughput.
    ///
    /// ATTENTION: using latency awareness is NOT recommended, unless prior
    /// benchmarks prove its beneficial impact on the specific workload's
    /// performance. Use with caution.
    pub fn latency_awareness(mut self, latency_awareness_builder: LatencyAwarenessBuilder) -> Self {
        self.latency_awareness = Some(latency_awareness_builder);
        self
    }

    /// Sets whether this policy should shuffle replicas when token-awareness
    /// is enabled. Shuffling can help distribute the load over replicas, but
    /// can reduce the effectiveness of caching on the database side (e.g.
    /// for reads).
    ///
    /// This option is enabled by default. If disabled, replicas will be chosen
    /// in some random order that is chosen when the load balancing policy
    /// is created and will not change over its lifetime.
    pub fn enable_shuffling_replicas(mut self, enable: bool) -> Self {
        self.enable_replica_shuffle = enable;
        self
    }
}

impl Default for DefaultPolicyBuilder {
    fn default() -> Self {
        Self::new()
    }
}

struct ProcessedRoutingInfo<'a> {
    token_with_strategy: Option<TokenWithStrategy<'a>>,

    // True if one of LOCAL_ONE, LOCAL_QUORUM, LOCAL_SERIAL was requested
    local_consistency: bool,
}

impl<'a> ProcessedRoutingInfo<'a> {
    fn new(query: &'a RoutingInfo, cluster: &'a ClusterData) -> ProcessedRoutingInfo<'a> {
        let local_consistency = matches!(
            (query.consistency, query.serial_consistency),
            (Consistency::LocalQuorum, _)
                | (Consistency::LocalOne, _)
                | (_, Some(SerialConsistency::LocalSerial))
        );

        Self {
            token_with_strategy: TokenWithStrategy::new(query, cluster),
            local_consistency,
        }
    }
}

struct TokenWithStrategy<'a> {
    strategy: &'a Strategy,
    token: Token,
}

impl<'a> TokenWithStrategy<'a> {
    fn new(query: &'a RoutingInfo, cluster: &'a ClusterData) -> Option<TokenWithStrategy<'a>> {
        let token = query.token?;
        let keyspace_name = query.table?.ks_name();
        let keyspace = cluster.get_keyspace_info().get(keyspace_name)?;
        let strategy = &keyspace.strategy;
        Some(TokenWithStrategy { strategy, token })
    }
}

#[cfg(test)]
mod tests {
    use std::collections::HashMap;

    use scylla_cql::{frame::types::SerialConsistency, Consistency};
    use tracing::info;

    use self::framework::{
        get_plan_and_collect_node_identifiers, mock_cluster_data_for_token_unaware_tests,
        ExpectedGroups, ExpectedGroupsBuilder,
    };
    use crate::host_filter::HostFilter;
    use crate::transport::locator::tablets::TabletsInfo;
    use crate::transport::locator::test::{
        id_to_invalid_addr, mock_metadata_for_token_aware_tests, TABLE_NTS_RF_2, TABLE_NTS_RF_3,
        TABLE_SS_RF_2,
    };
    use crate::{
        load_balancing::{
            default::tests::framework::mock_cluster_data_for_token_aware_tests, Plan, RoutingInfo,
        },
        routing::Token,
        test_utils::setup_tracing,
        transport::ClusterData,
    };

    use super::{DefaultPolicy, NodeLocationPreference};

    pub(crate) mod framework {
        use std::collections::{HashMap, HashSet};

        use uuid::Uuid;

        use crate::{
            load_balancing::{LoadBalancingPolicy, Plan, RoutingInfo},
            routing::Token,
            test_utils::setup_tracing,
            transport::{
                locator::{
                    tablets::TabletsInfo,
                    test::{id_to_invalid_addr, mock_metadata_for_token_aware_tests},
                },
                topology::{Metadata, Peer},
                ClusterData,
            },
        };

        #[derive(Debug)]
        enum ExpectedGroup {
            NonDeterministic(HashSet<u16>),
            Deterministic(HashSet<u16>),
            Ordered(Vec<u16>),
        }

        impl ExpectedGroup {
            fn len(&self) -> usize {
                match self {
                    Self::NonDeterministic(s) => s.len(),
                    Self::Deterministic(s) => s.len(),
                    Self::Ordered(v) => v.len(),
                }
            }
        }

        pub(crate) struct ExpectedGroupsBuilder {
            groups: Vec<ExpectedGroup>,
        }

        impl ExpectedGroupsBuilder {
            pub(crate) fn new() -> Self {
                Self { groups: Vec::new() }
            }
            /// Expects that the next group in the plan will have a set of nodes
            /// that is equal to the provided one. The groups are assumed to be
            /// non deterministic, i.e. the policy is expected to shuffle
            /// the nodes within that group.
            pub(crate) fn group(mut self, group: impl IntoIterator<Item = u16>) -> Self {
                self.groups
                    .push(ExpectedGroup::NonDeterministic(group.into_iter().collect()));
                self
            }
            /// Expects that the next group in the plan will have a set of nodes
            /// that is equal to the provided one, but the order of nodes in
            /// that group must be stable over multiple plans.
            pub(crate) fn deterministic(mut self, group: impl IntoIterator<Item = u16>) -> Self {
                self.groups
                    .push(ExpectedGroup::Deterministic(group.into_iter().collect()));
                self
            }
            /// Expects that the next group in the plan will have a sequence of nodes
            /// that is equal to the provided one, including order.
            pub(crate) fn ordered(mut self, group: impl IntoIterator<Item = u16>) -> Self {
                self.groups
                    .push(ExpectedGroup::Ordered(group.into_iter().collect()));
                self
            }
            pub(crate) fn build(self) -> ExpectedGroups {
                ExpectedGroups {
                    groups: self.groups,
                }
            }
        }

        #[derive(Debug)]
        pub(crate) struct ExpectedGroups {
            groups: Vec<ExpectedGroup>,
        }

        impl ExpectedGroups {
            pub(crate) fn assert_proper_grouping_in_plans(&self, gots: &[Vec<u16>]) {
                // For simplicity, assume that there is at least one plan
                // in `gots`
                assert!(!gots.is_empty());

                // Each plan is assumed to have the same number of groups.
                // For group index `i`, the code below will go over all plans
                // and will collect their `i`-th groups and put them under
                // index `i` in `sets_of_groups`.
                let mut sets_of_groups: Vec<HashSet<Vec<u16>>> =
                    vec![HashSet::new(); self.groups.len()];

                for got in gots {
                    // First, make sure that `got` has the right number of items,
                    // equal to the sum of sizes of all expected groups
                    let combined_groups_len: usize = self.groups.iter().map(|s| s.len()).sum();
                    assert_eq!(
                        got.len(),
                        combined_groups_len,
                        "Plan length different than expected. Got plan {:?}, expected groups {:?}",
                        got,
                        self.groups,
                    );

                    // Now, split `got` into groups of expected sizes
                    // and just `assert_eq` them
                    let mut got = got.iter();
                    for (group_id, expected) in self.groups.iter().enumerate() {
                        // Collect the nodes that constitute the group
                        // in the actual plan
                        let got_group: Vec<_> = (&mut got).take(expected.len()).copied().collect();

                        match expected {
                            ExpectedGroup::NonDeterministic(expected_set)
                            | ExpectedGroup::Deterministic(expected_set) => {
                                // Verify that the group has the same nodes as the
                                // expected one
                                let got_set: HashSet<_> = got_group.iter().copied().collect();
                                assert_eq!(&got_set, expected_set, "Unordered group mismatch");
                            }
                            ExpectedGroup::Ordered(sequence) => {
                                assert_eq!(&got_group, sequence, "Ordered group mismatch");
                            }
                        }

                        // Put the group into sets_of_groups
                        sets_of_groups[group_id].insert(got_group);
                    }
                }

                // Verify that the groups are either deterministic
                // or non-deterministic
                for (sets, expected) in sets_of_groups.iter().zip(self.groups.iter()) {
                    match expected {
                        ExpectedGroup::NonDeterministic(s) => {
                            // The group is supposed to have non-deterministic
                            // ordering. If the group size is larger than one,
                            // then expect there to be more than one group
                            // in the set.
                            if gots.len() > 1 && s.len() > 1 {
                                assert!(
                                    sets.len() > 1,
                                    "Group {:?} is expected to be nondeterministic, but it appears to be deterministic",
                                    expected
                                );
                            }
                        }
                        ExpectedGroup::Deterministic(_) | ExpectedGroup::Ordered(_) => {
                            // The group is supposed to be deterministic,
                            // i.e. a given instance of the default policy
                            // must always return the nodes within it using
                            // the same order.
                            // There will only be one, unique ordering shared
                            // by all plans - check this
                            assert_eq!(
                                sets.len(),
                                1,
                                "Group {:?} is expected to be deterministic, but it appears to be nondeterministic",
                                expected
                            );
                        }
                    }
                }
            }
        }

        #[test]
        fn test_assert_proper_grouping_in_plan_good() {
            setup_tracing();
            let got = vec![1u16, 2, 3, 4, 5];
            let expected_groups = ExpectedGroupsBuilder::new()
                .group([1])
                .group([3, 2, 4])
                .group([5])
                .build();

            expected_groups.assert_proper_grouping_in_plans(&[got]);
        }

        #[test]
        #[should_panic]
        fn test_assert_proper_grouping_in_plan_too_many_nodes_in_the_end() {
            setup_tracing();
            let got = vec![1u16, 2, 3, 4, 5, 6];
            let expected_groups = ExpectedGroupsBuilder::new()
                .group([1])
                .group([3, 2, 4])
                .group([5])
                .build();

            expected_groups.assert_proper_grouping_in_plans(&[got]);
        }

        #[test]
        #[should_panic]
        fn test_assert_proper_grouping_in_plan_too_many_nodes_in_the_middle() {
            setup_tracing();
            let got = vec![1u16, 2, 6, 3, 4, 5];
            let expected_groups = ExpectedGroupsBuilder::new()
                .group([1])
                .group([3, 2, 4])
                .group([5])
                .build();

            expected_groups.assert_proper_grouping_in_plans(&[got]);
        }

        #[test]
        #[should_panic]
        fn test_assert_proper_grouping_in_plan_missing_node() {
            setup_tracing();
            let got = vec![1u16, 2, 3, 4];
            let expected_groups = ExpectedGroupsBuilder::new()
                .group([1])
                .group([3, 2, 4])
                .group([5])
                .build();

            expected_groups.assert_proper_grouping_in_plans(&[got]);
        }

        // based on locator mock cluster
        pub(crate) async fn mock_cluster_data_for_token_aware_tests() -> ClusterData {
            let metadata = mock_metadata_for_token_aware_tests();
            ClusterData::new(
                metadata,
                &Default::default(),
                &HashMap::new(),
                &None,
                None,
                TabletsInfo::new(),
            )
            .await
        }

        // creates ClusterData with info about 5 nodes living in 2 different datacenters
        // ring field is minimal, not intended to influence the tests
        pub(crate) async fn mock_cluster_data_for_token_unaware_tests() -> ClusterData {
            let peers = [("eu", 1), ("eu", 2), ("eu", 3), ("us", 4), ("us", 5)]
                .iter()
                .map(|(dc, id)| Peer {
                    datacenter: Some(dc.to_string()),
                    rack: None,
                    address: id_to_invalid_addr(*id),
                    tokens: vec![Token::new(*id as i64 * 100)],
                    host_id: Uuid::new_v4(),
                })
                .collect::<Vec<_>>();

            let info = Metadata {
                peers,
                keyspaces: HashMap::new(),
            };

            ClusterData::new(
                info,
                &Default::default(),
                &HashMap::new(),
                &None,
                None,
                TabletsInfo::new(),
            )
            .await
        }

        pub(crate) fn get_plan_and_collect_node_identifiers(
            policy: &impl LoadBalancingPolicy,
            query_info: &RoutingInfo,
            cluster: &ClusterData,
        ) -> Vec<u16> {
            let plan = Plan::new(policy, query_info, cluster);
            plan.map(|(node, _shard)| node.address.port())
                .collect::<Vec<_>>()
        }
    }

    pub(crate) const EMPTY_ROUTING_INFO: RoutingInfo = RoutingInfo {
        token: None,
        table: None,
        is_confirmed_lwt: false,
        consistency: Consistency::Quorum,
        serial_consistency: Some(SerialConsistency::Serial),
    };

    pub(super) async fn test_default_policy_with_given_cluster_and_routing_info(
        policy: &DefaultPolicy,
        cluster: &ClusterData,
        routing_info: &RoutingInfo<'_>,
        expected_groups: &ExpectedGroups,
    ) {
        let mut plans = Vec::new();
        for _ in 0..256 {
            let plan = get_plan_and_collect_node_identifiers(policy, routing_info, cluster);
            plans.push(plan);
        }
        let example_plan = Plan::new(policy, routing_info, cluster);
        info!("Example plan from policy:",);
        for (node, shard) in example_plan {
            info!(
                "Node port: {}, shard: {}, dc: {:?}, rack: {:?}, down: {:?}",
                node.address.port(),
                shard,
                node.datacenter,
                node.rack,
                node.is_down()
            );
        }

        expected_groups.assert_proper_grouping_in_plans(&plans);
    }

    async fn test_given_default_policy_with_token_unaware_statements(
        policy: DefaultPolicy,
        expected_groups: &ExpectedGroups,
    ) {
        let cluster = mock_cluster_data_for_token_unaware_tests().await;

        test_default_policy_with_given_cluster_and_routing_info(
            &policy,
            &cluster,
            &EMPTY_ROUTING_INFO,
            expected_groups,
        )
        .await;
    }

    #[tokio::test]
    async fn test_default_policy_with_token_unaware_statements() {
        setup_tracing();
        let local_dc = "eu".to_string();
        let policy_with_disabled_dc_failover = DefaultPolicy {
            preferences: NodeLocationPreference::Datacenter(local_dc.clone()),
            permit_dc_failover: false,
            ..Default::default()
        };
        let expected_groups = ExpectedGroupsBuilder::new()
            .group([1, 2, 3]) // pick + fallback local nodes
            .build();
        test_given_default_policy_with_token_unaware_statements(
            policy_with_disabled_dc_failover,
            &expected_groups,
        )
        .await;

        let policy_with_enabled_dc_failover = DefaultPolicy {
            preferences: NodeLocationPreference::Datacenter(local_dc.clone()),
            permit_dc_failover: true,
            ..Default::default()
        };
        let expected_groups = ExpectedGroupsBuilder::new()
            .group([1, 2, 3]) // pick + fallback local nodes
            .group([4, 5]) // fallback remote nodes
            .build();
        test_given_default_policy_with_token_unaware_statements(
            policy_with_enabled_dc_failover,
            &expected_groups,
        )
        .await;
    }

    #[tokio::test]
    async fn test_default_policy_with_token_aware_statements() {
        setup_tracing();

        use crate::transport::locator::test::{A, B, C, D, E, F, G};
        let cluster = mock_cluster_data_for_token_aware_tests().await;

        #[derive(Debug)]
        struct Test<'a> {
            policy: DefaultPolicy,
            routing_info: RoutingInfo<'a>,
            expected_groups: ExpectedGroups,
        }

        let tests = [
            // Keyspace NTS with RF=2 with enabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Two,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, G]) // pick + fallback local replicas
                    .group([F, D]) // remote replicas
                    .group([C, B]) // local nodes
                    .group([E]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=2 with enabled DC failover, shuffling replicas disabled
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    fixed_seed: Some(123),
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Two,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .deterministic([A, G]) // pick + fallback local replicas
                    .deterministic([F, D]) // remote replicas
                    .group([C, B]) // local nodes
                    .group([E]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=2 with DC failover forbidden by local Consistency
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::LocalOne, // local Consistency forbids datacenter failover
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, G]) // pick + fallback local replicas
                    .group([C, B]) // local nodes
                    .build(), // failover is forbidden by local Consistency
            },
            // Keyspace NTS with RF=2 with explicitly disabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::One,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, G]) // pick + fallback local replicas
                    .group([C, B]) // local nodes
                    .build(), // failover is explicitly forbidden
            },
            // Keyspace NTS with RF=3 with enabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, C, G]) // pick + fallback local replicas
                    .group([F, D, E]) // remote replicas
                    .group([B]) // local nodes
                    .group([]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=3 with enabled DC failover, shuffling replicas disabled
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    fixed_seed: Some(123),
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .deterministic([A, C, G]) // pick + fallback local replicas
                    .deterministic([F, D, E]) // remote replicas
                    .group([B]) // local nodes
                    .group([]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=3 with disabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, C, G]) // pick + fallback local replicas
                    .group([B]) // local nodes
                    .build(), // failover explicitly forbidden
            },
            // Keyspace SS with RF=2 with enabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_SS_RF_2),
                    consistency: Consistency::Two,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A]) // pick + fallback local replicas
                    .group([F]) // remote replicas
                    .group([C, G, B]) // local nodes
                    .group([D, E]) // remote nodes
                    .build(),
            },
            // Keyspace SS with RF=2 with DC failover forbidden by local Consistency
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_SS_RF_2),
                    consistency: Consistency::LocalOne, // local Consistency forbids datacenter failover
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A]) // pick + fallback local replicas
                    .group([C, G, B]) // local nodes
                    .build(), // failover is forbidden by local Consistency
            },
            // No token implies no token awareness
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: None, // no token
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, B, C, G]) // local nodes
                    .group([D, E, F]) // remote nodes
                    .build(),
            },
            // No keyspace implies no token awareness
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: None, // no keyspace
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, B, C, G]) // local nodes
                    .group([D, E, F]) // remote nodes
                    .build(),
            },
            // Unknown preferred DC, failover permitted
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("au".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, D, F, G]) // remote replicas
                    .group([B, C, E]) // remote nodes
                    .build(),
            },
            // Unknown preferred DC, failover forbidden
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("au".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new().build(), // empty plan, because all nodes are remote and failover is forbidden
            },
            // No preferred DC, failover permitted
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Any,
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, D, F, G]) // remote replicas
                    .group([B, C, E]) // remote nodes
                    .build(),
            },
            // No preferred DC, failover forbidden
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Any,
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Quorum,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, D, F, G]) // remote replicas
                    .group([B, C, E]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=3 with enabled DC failover and rack-awareness
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::DatacenterAndRack(
                        "eu".to_owned(),
                        "r1".to_owned(),
                    ),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::One,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, C]) // pick local rack replicas
                    .group([G]) // local DC replicas
                    .group([F, D, E]) // remote replicas
                    .group([B]) // local nodes
                    .build(),
            },
            // Keyspace SS with RF=2 with enabled rack-awareness, shuffling replicas disabled
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::DatacenterAndRack(
                        "eu".to_owned(),
                        "r1".to_owned(),
                    ),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    fixed_seed: Some(123),
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(560)),
                    table: Some(TABLE_SS_RF_2),
                    consistency: Consistency::Two,
                    ..Default::default()
                },
                // going through the ring, we get order: B , C , E , G , A , F , D
                //                                      eu  eu  us  eu  eu  us  us
                //                                      r1  r1  r1  r2  r1  r2  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .deterministic([B]) // pick local rack replicas
                    .deterministic([C]) // fallback replicas
                    .group([A]) // local rack nodes
                    .group([G]) // local DC nodes
                    .build(),
            },
            // Keyspace SS with RF=2 with enabled rack-awareness and no local-rack replica
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::DatacenterAndRack(
                        "eu".to_owned(),
                        "r2".to_owned(),
                    ),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_SS_RF_2),
                    consistency: Consistency::One,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A]) // pick local DC
                    .group([G]) // local rack nodes
                    .group([C, B]) // local DC nodes
                    .build(),
            },
            // Keyspace NTS with RF=3 with enabled DC failover and rack-awareness, no token provided
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::DatacenterAndRack(
                        "eu".to_owned(),
                        "r1".to_owned(),
                    ),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: None,
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::One,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, C, B]) // local rack nodes
                    .group([G]) // local DC nodes
                    .group([F, D, E]) // remote nodes
                    .build(),
            },
        ];

        for test in tests {
            info!("Test: {:?}", test);
            let Test {
                policy,
                routing_info,
                expected_groups,
            } = test;
            test_default_policy_with_given_cluster_and_routing_info(
                &policy,
                &cluster,
                &routing_info,
                &expected_groups,
            )
            .await;
        }
    }

    #[tokio::test]
    async fn test_default_policy_with_lwt_statements() {
        setup_tracing();
        use crate::transport::locator::test::{A, B, C, D, E, F, G};

        let cluster = mock_cluster_data_for_token_aware_tests().await;
        struct Test<'a> {
            policy: DefaultPolicy,
            routing_info: RoutingInfo<'a>,
            expected_groups: ExpectedGroups,
        }

        let tests = [
            // Keyspace NTS with RF=2 with enabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Two,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A, G]) // pick + fallback local replicas
                    .ordered([F, D]) // remote replicas
                    .group([C, B]) // local nodes
                    .group([E]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=2 with enabled DC failover, shuffling replicas disabled
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    fixed_seed: Some(123),
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Two,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A, G]) // pick + fallback local replicas
                    .ordered([F, D]) // remote replicas
                    .group([C, B]) // local nodes
                    .group([E]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=2 with DC failover forbidden by local Consistency
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::LocalOne, // local Consistency forbids datacenter failover
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A, G]) // pick + fallback local replicas
                    .group([C, B]) // local nodes
                    .build(), // failover is forbidden by local Consistency
            },
            // Keyspace NTS with RF=2 with explicitly disabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::One,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A, G]) // pick + fallback local replicas
                    .group([C, B]) // local nodes
                    .build(), // failover is explicitly forbidden
            },
            // Keyspace NTS with RF=3 with enabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A, C, G]) // pick + fallback local replicas
                    .ordered([F, D, E]) // remote replicas
                    .group([B]) // local nodes
                    .group([]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=3 with enabled DC failover, shuffling replicas disabled
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    fixed_seed: Some(123),
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A, C, G]) // pick + fallback local replicas
                    .ordered([F, D, E]) // remote replicas
                    .group([B]) // local nodes
                    .group([]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=3 with disabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A, C, G]) // pick + fallback local replicas
                    .group([B]) // local nodes
                    .build(), // failover explicitly forbidden
            },
            // Keyspace SS with RF=2 with enabled DC failover
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_SS_RF_2),
                    consistency: Consistency::Two,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A]) // pick + fallback local replicas
                    .ordered([F]) // remote replicas
                    .group([C, G, B]) // local nodes
                    .group([D, E]) // remote nodes
                    .build(),
            },
            // Keyspace SS with RF=2 with DC failover forbidden by local Consistency
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_SS_RF_2),
                    consistency: Consistency::LocalOne, // local Consistency forbids datacenter failover
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A]) // pick + fallback local replicas
                    .group([C, G, B]) // local nodes
                    .build(), // failover is forbidden by local Consistency
            },
            // No token implies no token awareness
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: None, // no token
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, B, C, G]) // local nodes
                    .group([D, E, F]) // remote nodes
                    .build(),
            },
            // No keyspace implies no token awareness
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: None, // no keyspace
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A, B, C, G]) // local nodes
                    .group([D, E, F]) // remote nodes
                    .build(),
            },
            // Unknown preferred DC, failover permitted
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("au".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([F, A, D, G]) // remote replicas
                    .group([B, C, E]) // remote nodes
                    .build(),
            },
            // Unknown preferred DC, failover forbidden
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Datacenter("au".to_owned()),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new().build(), // empty plan, because all nodes are remote and failover is forbidden
            },
            // No preferred DC, failover permitted
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Any,
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([F, A, D, G]) // remote replicas
                    .group([B, C, E]) // remote nodes
                    .build(),
            },
            // No preferred DC, failover forbidden
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Any,
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_2),
                    consistency: Consistency::Quorum,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([F, A, D, G]) // remote replicas
                    .group([B, C, E]) // remote nodes
                    .build(),
            },
            // Keyspace NTS with RF=3 with enabled DC failover and rack-awareness
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::DatacenterAndRack(
                        "eu".to_owned(),
                        "r1".to_owned(),
                    ),
                    is_token_aware: true,
                    permit_dc_failover: true,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::One,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([A, C]) // pick local rack replicas
                    .ordered([G]) // local DC replicas
                    .ordered([F, D, E]) // remote replicas
                    .group([B]) // local nodes
                    .build(),
            },
            // Keyspace SS with RF=2 with enabled rack-awareness, shuffling replicas disabled
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::DatacenterAndRack(
                        "eu".to_owned(),
                        "r1".to_owned(),
                    ),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    fixed_seed: Some(123),
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(760)),
                    table: Some(TABLE_SS_RF_2),
                    consistency: Consistency::Two,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: G , B , A , E , F , C , D
                //                                      eu  eu  eu  us  us  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .ordered([B]) // pick local rack replicas
                    .ordered([G]) // local DC replicas
                    .group([A, C]) // local nodes
                    .build(),
            },
            // Keyspace SS with RF=2 with enabled rack-awareness and no local-rack replica
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::DatacenterAndRack(
                        "eu".to_owned(),
                        "r2".to_owned(),
                    ),
                    is_token_aware: true,
                    permit_dc_failover: false,
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_SS_RF_2),
                    consistency: Consistency::One,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    .group([A]) // pick local DC
                    .group([C, G, B]) // local nodes
                    .build(),
            },
        ];

        for Test {
            policy,
            routing_info,
            expected_groups,
        } in tests
        {
            test_default_policy_with_given_cluster_and_routing_info(
                &policy,
                &cluster,
                &routing_info,
                &expected_groups,
            )
            .await;
        }

        let cluster_with_disabled_node_f = ClusterData::new(
            mock_metadata_for_token_aware_tests(),
            &Default::default(),
            &HashMap::new(),
            &None,
            {
                struct FHostFilter;
                impl HostFilter for FHostFilter {
                    fn accept(&self, peer: &crate::transport::topology::Peer) -> bool {
                        peer.address != id_to_invalid_addr(F)
                    }
                }

                Some(&FHostFilter)
            },
            TabletsInfo::new(),
        )
        .await;

        let tests_with_disabled_node_f = [
            // Keyspace NTS with RF=3 without preferred DC.
            // The primary replica does not satisfy the predicate (being disabled by HostFilter),
            // so pick() should return None and fallback should return A first.
            //
            // This is a regression test after a bug was fixed.
            Test {
                policy: DefaultPolicy {
                    preferences: NodeLocationPreference::Any,
                    is_token_aware: true,
                    permit_dc_failover: true,
                    pick_predicate: Box::new(|node, _shard| node.address != id_to_invalid_addr(F)),
                    ..Default::default()
                },
                routing_info: RoutingInfo {
                    token: Some(Token::new(160)),
                    table: Some(TABLE_NTS_RF_3),
                    consistency: Consistency::One,
                    is_confirmed_lwt: true,
                    ..Default::default()
                },
                // going through the ring, we get order: F , A , C , D , G , B , E
                //                                      us  eu  eu  us  eu  eu  us
                //                                      r2  r1  r1  r1  r2  r1  r1
                expected_groups: ExpectedGroupsBuilder::new()
                    // pick is empty, because the primary replica does not satisfy pick predicate,
                    // and with LWT we cannot compute other replicas for NTS without allocations.
                    .ordered([A, C, D, G, E]) // replicas
                    .group([B]) // nodes
                    .build(),
            },
        ];

        for Test {
            policy,
            routing_info,
            expected_groups,
        } in tests_with_disabled_node_f
        {
            test_default_policy_with_given_cluster_and_routing_info(
                &policy,
                &cluster_with_disabled_node_f,
                &routing_info,
                &expected_groups,
            )
            .await;
        }
    }
}

mod latency_awareness {
    use futures::{future::RemoteHandle, FutureExt};
    use itertools::Either;
    use tokio::time::{Duration, Instant};
    use tracing::{trace, warn};
    use uuid::Uuid;

    use crate::{
        load_balancing::NodeRef,
        routing::Shard,
        transport::errors::{DbError, QueryError},
        transport::node::Node,
    };
    use std::{
        collections::HashMap,
        ops::Deref,
        sync::{
            atomic::{AtomicU64, Ordering},
            Arc, RwLock,
        },
    };

    #[derive(Debug)]
    struct AtomicDuration(AtomicU64);

    impl AtomicDuration {
        fn new() -> Self {
            Self(AtomicU64::new(u64::MAX))
        }

        fn store(&self, duration: Duration) {
            self.0.store(duration.as_micros() as u64, Ordering::Relaxed)
        }

        fn load(&self) -> Option<Duration> {
            let micros = self.0.load(Ordering::Relaxed);
            if micros == u64::MAX {
                None
            } else {
                Some(Duration::from_micros(micros))
            }
        }
    }

    #[derive(Debug, Clone, Copy, PartialEq, Eq)]
    pub(super) struct TimestampedAverage {
        pub(super) timestamp: Instant,
        pub(super) average: Duration,
        pub(super) num_measures: usize,
    }

    impl TimestampedAverage {
        pub(crate) fn compute_next(
            previous: Option<Self>,
            last_latency: Duration,
            scale_secs: f64,
        ) -> Option<Self> {
            let now = Instant::now();
            match previous {
                prev if last_latency.is_zero() => prev,
                None => Some(Self {
                    num_measures: 1,
                    average: last_latency,
                    timestamp: now,
                }),
                Some(prev_avg) => Some({
                    let delay = now
                        .saturating_duration_since(prev_avg.timestamp)
                        .as_secs_f64();
                    let scaled_delay = delay / scale_secs;
                    let prev_weight = if scaled_delay <= 0. {
                        1.
                    } else {
                        (scaled_delay + 1.).ln() / scaled_delay
                    };

                    let last_latency_secs = last_latency.as_secs_f64();
                    let prev_avg_secs = prev_avg.average.as_secs_f64();
                    let average = match Duration::try_from_secs_f64(
                        (1. - prev_weight) * last_latency_secs + prev_weight * prev_avg_secs,
                    ) {
                        Ok(ts) => ts,
                        Err(e) => {
                            warn!(
                                "Error while calculating average: {e}. \
                                prev_avg_secs: {prev_avg_secs}, \
                                last_latency_secs: {last_latency_secs}, \
                                prev_weight: {prev_weight}, \
                                scaled_delay: {scaled_delay}, \
                                delay: {delay}, \
                                prev_avg.timestamp: {:?}, \
                                now: {now:?}",
                                prev_avg.timestamp
                            );

                            // Not sure when we could enter this branch,
                            // so I have no idea what would be a sensible value to return here,
                            // this does not seem like a very bad choice.
                            prev_avg.average
                        }
                    };
                    Self {
                        num_measures: prev_avg.num_measures + 1,
                        timestamp: now,
                        average,
                    }
                }),
            }
        }
    }

    /// A latency-aware load balancing policy module, which enables penalising nodes that are too slow.
    #[derive(Debug)]
    pub(super) struct LatencyAwareness {
        pub(super) exclusion_threshold: f64,
        pub(super) retry_period: Duration,
        pub(super) _update_rate: Duration,
        pub(super) minimum_measurements: usize,
        pub(super) scale_secs: f64,

        /// Last minimum average latency that was noted among the nodes. It is updated every
        /// [update_rate](Self::_update_rate).
        last_min_latency: Arc<AtomicDuration>,

        node_avgs: Arc<RwLock<HashMap<Uuid, RwLock<Option<TimestampedAverage>>>>>,

        // This is Some iff there is an associated updater running on a separate Tokio task
        // For some tests, not to rely on timing, this is None. The updater is then tick'ed
        // explicitly from outside this struct.
        _updater_handle: Option<RemoteHandle<()>>,
    }

    impl LatencyAwareness {
        pub(super) fn builder() -> LatencyAwarenessBuilder {
            LatencyAwarenessBuilder::new()
        }

        fn new_for_test(
            exclusion_threshold: f64,
            retry_period: Duration,
            update_rate: Duration,
            minimum_measurements: usize,
            scale: Duration,
        ) -> (Self, MinAvgUpdater) {
            let min_latency = Arc::new(AtomicDuration::new());

            let min_latency_clone = min_latency.clone();
            let node_avgs = Arc::new(RwLock::new(HashMap::new()));
            let node_avgs_clone = node_avgs.clone();

            let updater = MinAvgUpdater {
                node_avgs,
                min_latency,
                minimum_measurements,
            };

            (
                Self {
                    exclusion_threshold,
                    retry_period,
                    _update_rate: update_rate,
                    minimum_measurements,
                    scale_secs: scale.as_secs_f64(),
                    last_min_latency: min_latency_clone,
                    node_avgs: node_avgs_clone,
                    _updater_handle: None,
                },
                updater,
            )
        }

        fn new(
            exclusion_threshold: f64,
            retry_period: Duration,
            update_rate: Duration,
            minimum_measurements: usize,
            scale: Duration,
        ) -> Self {
            let (self_, updater) = Self::new_for_test(
                exclusion_threshold,
                retry_period,
                update_rate,
                minimum_measurements,
                scale,
            );

            let (updater_fut, updater_handle) = async move {
                let mut update_scheduler = tokio::time::interval(update_rate);
                loop {
                    update_scheduler.tick().await;
                    updater.tick().await;
                }
            }
            .remote_handle();
            tokio::task::spawn(updater_fut);

            Self {
                _updater_handle: Some(updater_handle),
                ..self_
            }
        }

        pub(super) fn generate_predicate(&self) -> impl Fn(&Node) -> bool {
            let last_min_latency = self.last_min_latency.clone();
            let node_avgs = self.node_avgs.clone();
            let exclusion_threshold = self.exclusion_threshold;
            let minimum_measurements = self.minimum_measurements;
            let retry_period = self.retry_period;

            move |node| {
                last_min_latency.load().map(|min_avg| match fast_enough(&node_avgs.read().unwrap(), node.host_id, exclusion_threshold, retry_period, minimum_measurements, min_avg) {
                    FastEnough::Yes => true,
                    FastEnough::No { average } => {
                        trace!("Latency awareness: Penalising node {{address={}, datacenter={:?}, rack={:?}}} for being on average at least {} times slower (latency: {}ms) than the fastest ({}ms).",
                                node.address, node.datacenter, node.rack, exclusion_threshold, average.as_millis(), min_avg.as_millis());
                        false
                    }
                }).unwrap_or(true)
            }
        }

        pub(super) fn wrap<'a>(
            &self,
            fallback: impl Iterator<Item = (NodeRef<'a>, Option<Shard>)>,
        ) -> impl Iterator<Item = (NodeRef<'a>, Option<Shard>)> {
            let min_avg_latency = match self.last_min_latency.load() {
                Some(min_avg) => min_avg,
                None => return Either::Left(fallback), // noop, as no latency data has been collected yet
            };

            let average_latencies = self.node_avgs.read().unwrap();
            let targets = fallback;

            let mut fast_targets = vec![];
            let mut penalised_targets = vec![];

            for node_and_shard @ (node, _shard) in targets {
                match fast_enough(
                    average_latencies.deref(),
                    node.host_id,
                    self.exclusion_threshold,
                    self.retry_period,
                    self.minimum_measurements,
                    min_avg_latency,
                ) {
                    FastEnough::Yes => fast_targets.push(node_and_shard),
                    FastEnough::No { average } => {
                        trace!("Latency awareness: Penalising node {{address={}, datacenter={:?}, rack={:?}}} for being on average at least {} times slower (latency: {}ms) than the fastest ({}ms).",
                                node.address, node.datacenter, node.rack, self.exclusion_threshold, average.as_millis(), min_avg_latency.as_millis());
                        penalised_targets.push(node_and_shard);
                    }
                }
            }

            let mut fast_targets = fast_targets.into_iter();
            let mut penalised_targets = penalised_targets.into_iter();

            let skipping_penalised_targets_iterator = std::iter::from_fn(move || {
                fast_targets.next().or_else(|| penalised_targets.next())
            });

            Either::Right(skipping_penalised_targets_iterator)
        }

        pub(super) fn report_query(&self, node: &Node, latency: Duration) {
            let node_avgs_guard = self.node_avgs.read().unwrap();
            if let Some(previous_node_avg) = node_avgs_guard.get(&node.host_id) {
                // The usual path, the node has been already noticed.
                let mut node_avg_guard = previous_node_avg.write().unwrap();
                let previous_node_avg = *node_avg_guard;
                *node_avg_guard =
                    TimestampedAverage::compute_next(previous_node_avg, latency, self.scale_secs);
            } else {
                // We drop the read lock not to deadlock while taking write lock.
                std::mem::drop(node_avgs_guard);
                let mut node_avgs_guard = self.node_avgs.write().unwrap();

                // We have to read this again, as other threads may race with us.
                let previous_node_avg = node_avgs_guard
                    .get(&node.host_id)
                    .and_then(|rwlock| *rwlock.read().unwrap());

                // We most probably need to add the node to the map.
                // (this will be Some only in an unlikely case that another thread raced with us and won)
                node_avgs_guard.insert(
                    node.host_id,
                    RwLock::new(TimestampedAverage::compute_next(
                        previous_node_avg,
                        latency,
                        self.scale_secs,
                    )),
                );
            }
        }

        pub(crate) fn reliable_latency_measure(error: &QueryError) -> bool {
            match error {
                // "fast" errors, i.e. ones that are returned quickly after the query begins
                QueryError::BadQuery(_)
                | QueryError::CqlRequestSerialization(_)
                | QueryError::BrokenConnection(_)
                | QueryError::ConnectionPoolError(_)
                | QueryError::EmptyPlan
                | QueryError::UnableToAllocStreamId
                | QueryError::DbError(DbError::IsBootstrapping, _)
                | QueryError::DbError(DbError::Unavailable { .. }, _)
                | QueryError::DbError(DbError::Unprepared { .. }, _)
                | QueryError::DbError(DbError::Overloaded { .. }, _)
                | QueryError::DbError(DbError::RateLimitReached { .. }, _) => false,

                // "slow" errors, i.e. ones that are returned after considerable time of query being run
                #[allow(deprecated)]
                QueryError::DbError(_, _)
                | QueryError::CqlResultParseError(_)
                | QueryError::CqlErrorParseError(_)
                | QueryError::BodyExtensionsParseError(_)
                | QueryError::MetadataError(_)
                | QueryError::ProtocolError(_)
                | QueryError::TimeoutError
                | QueryError::RequestTimeout(_)
                | QueryError::NextRowError(_)
                | QueryError::IntoLegacyQueryResultError(_) => true,
            }
        }
    }

    impl Default for LatencyAwareness {
        fn default() -> Self {
            Self::builder().build()
        }
    }

    /// Updates minimum average latency upon request each request to `tick()`.
    /// The said average is a crucial criterium for penalising "too slow" nodes.
    struct MinAvgUpdater {
        node_avgs: Arc<RwLock<HashMap<Uuid, RwLock<Option<TimestampedAverage>>>>>,
        min_latency: Arc<AtomicDuration>,
        minimum_measurements: usize,
    }

    impl MinAvgUpdater {
        async fn tick(&self) {
            let averages: &HashMap<Uuid, RwLock<Option<TimestampedAverage>>> =
                &self.node_avgs.read().unwrap();
            if averages.is_empty() {
                return; // No nodes queries registered to LAP performed yet.
            }

            let min_avg = averages
                .values()
                .filter_map(|avg| {
                    avg.read().unwrap().and_then(|timestamped_average| {
                        (timestamped_average.num_measures >= self.minimum_measurements)
                            .then_some(timestamped_average.average)
                    })
                })
                .min();
            if let Some(min_avg) = min_avg {
                self.min_latency.store(min_avg);
                trace!(
                    "Latency awareness: updated min average latency to {} ms",
                    min_avg.as_secs_f64() * 1000.
                );
            }
        }
    }

    /// The builder of LatencyAwareness module of DefaultPolicy.
    ///
    /// (For more information about latency awareness, see [DefaultPolicyBuilder::latency_awareness()](super::DefaultPolicyBuilder::latency_awareness)).
    /// It is intended to be created and configured by the user and then
    /// passed to DefaultPolicyBuilder, like this:
    ///
    /// # Example
    /// ```
    /// # fn example() {
    /// use scylla::load_balancing::{
    ///     LatencyAwarenessBuilder, DefaultPolicy
    /// };
    ///
    /// let latency_awareness_builder = LatencyAwarenessBuilder::new()
    ///     .exclusion_threshold(3.)
    ///     .minimum_measurements(200);
    ///
    /// let policy = DefaultPolicy::builder()
    ///     .latency_awareness(latency_awareness_builder)
    ///     .build();
    /// # }
    #[derive(Debug, Clone)]
    pub struct LatencyAwarenessBuilder {
        exclusion_threshold: f64,
        retry_period: Duration,
        update_rate: Duration,
        minimum_measurements: usize,
        scale: Duration,
    }

    impl LatencyAwarenessBuilder {
        /// Creates a builder of LatencyAwareness module of DefaultPolicy.
        pub fn new() -> Self {
            Self {
                exclusion_threshold: 2_f64,
                retry_period: Duration::from_secs(10),
                update_rate: Duration::from_millis(100),
                minimum_measurements: 50,
                scale: Duration::from_millis(100),
            }
        }

        /// Sets minimum measurements for latency awareness (if there have been fewer measurements taken for a node,
        /// the node will not be penalised).
        ///
        /// Penalising nodes is based on an average of their recently measured average latency.
        /// This average is only meaningful if a minimum of measurements have been collected.
        /// This is what this option controls. If fewer than [minimum_measurements](Self::minimum_measurements)
        /// data points have been collected for a given host, the policy will never penalise that host.
        /// Note that the number of collected measurements for a given host is reset if the node
        /// is restarted.
        /// The default for this option is **50**.
        pub fn minimum_measurements(self, minimum_measurements: usize) -> Self {
            Self {
                minimum_measurements,
                ..self
            }
        }

        /// Sets retry period for latency awareness (max time that a node is being penalised).
        ///
        /// The retry period defines how long a node may be penalised by the policy before it is given
        /// a 2nd chance. More precisely, a node is excluded from query plans if both his calculated
        /// average latency is [exclusion_threshold](Self::exclusion_threshold) times slower than
        /// the fastest node average latency (at the time the query plan is computed) **and** his
        /// calculated average latency has been updated since less than [retry_period](Self::retry_period).
        /// Since penalised nodes will likely not see their latency updated, this is basically how long
        /// the policy will exclude a node.
        pub fn retry_period(self, retry_period: Duration) -> Self {
            Self {
                retry_period,
                ..self
            }
        }

        /// Sets exclusion threshold for latency awareness (a threshold for a node to be penalised).
        ///
        /// The exclusion threshold controls how much worse the average latency of a node must be
        /// compared to the fastest performing node for it to be penalised by the policy.
        /// For example, if set to 2, the resulting policy excludes nodes that are more than twice
        /// slower than the fastest node.
        pub fn exclusion_threshold(self, exclusion_threshold: f64) -> Self {
            Self {
                exclusion_threshold,
                ..self
            }
        }

        /// Sets update rate for latency awareness (how often is the global minimal average latency updated).
        ///
        /// The update rate defines how often the minimum average latency is recomputed. While the
        /// average latency score of each node is computed iteratively (updated each time a new latency
        /// is collected), the minimum score needs to be recomputed from scratch every time, which is
        /// slightly more costly. For this reason, the minimum is only re-calculated at the given fixed
        /// rate and cached between re-calculation.
        /// The default update rate if **100 milliseconds**, which should be appropriate for most
        /// applications. In particular, note that while we want to avoid to recompute the minimum for
        /// every query, that computation is not particularly intensive either and there is no reason to
        /// use a very slow rate (more than second is probably unnecessarily slow for instance).
        pub fn update_rate(self, update_rate: Duration) -> Self {
            Self {
                update_rate,
                ..self
            }
        }

        /// Sets the scale to use for the resulting latency aware policy.
        ///
        /// The `scale` provides control on how the weight given to older latencies decreases
        /// over time. For a given host, if a new latency `l` is received at time `t`, and
        /// the previously calculated average is `prev` calculated at time `t'`, then the
        /// newly calculated average `avg` for that host is calculated thusly:
        ///
        /// ```text
        /// d = (t - t') / scale
        /// alpha = 1 - (ln(d+1) / d)
        /// avg = alpha * l + (1 - alpha) * prev
        /// ```
        ///
        /// Typically, with a `scale` of 100 milliseconds (the default), if a new latency is
        /// measured and the previous measure is 10 millisecond old (so `d=0.1`), then `alpha`
        /// will be around `0.05`. In other words, the new latency will weight 5% of the
        /// updated average. A bigger scale will get less weight to new measurements (compared to
        /// previous ones), a smaller one will give them more weight.
        ///
        /// The default scale (if this method is not used) is of **100 milliseconds**. If unsure,
        /// try this default scale first and experiment only if it doesn't provide acceptable results
        /// (hosts are excluded too quickly or not fast enough and tuning the exclusion threshold doesn't
        /// help).
        pub fn scale(self, scale: Duration) -> Self {
            Self { scale, ..self }
        }

        pub(super) fn build(self) -> LatencyAwareness {
            let Self {
                exclusion_threshold,
                retry_period,
                update_rate,
                minimum_measurements,
                scale,
            } = self;
            LatencyAwareness::new(
                exclusion_threshold,
                retry_period,
                update_rate,
                minimum_measurements,
                scale,
            )
        }

        #[cfg(test)]
        fn build_for_test(self) -> (LatencyAwareness, MinAvgUpdater) {
            let Self {
                exclusion_threshold,
                retry_period,
                update_rate,
                minimum_measurements,
                scale,
            } = self;
            LatencyAwareness::new_for_test(
                exclusion_threshold,
                retry_period,
                update_rate,
                minimum_measurements,
                scale,
            )
        }
    }

    impl Default for LatencyAwarenessBuilder {
        fn default() -> Self {
            Self::new()
        }
    }

    pub(super) enum FastEnough {
        Yes,
        No { average: Duration },
    }

    pub(super) fn fast_enough(
        average_latencies: &HashMap<Uuid, RwLock<Option<TimestampedAverage>>>,
        node: Uuid,
        exclusion_threshold: f64,
        retry_period: Duration,
        minimum_measurements: usize,
        min_avg: Duration,
    ) -> FastEnough {
        let avg = match average_latencies
            .get(&node)
            .and_then(|avgs| *avgs.read().unwrap())
        {
            Some(avg) => avg,
            None => return FastEnough::Yes,
        };
        if avg.num_measures >= minimum_measurements
            && avg.timestamp.elapsed() < retry_period
            && avg.average.as_micros() as f64 > exclusion_threshold * min_avg.as_micros() as f64
        {
            FastEnough::No {
                average: avg.average,
            }
        } else {
            FastEnough::Yes
        }
    }

    #[cfg(test)]
    mod tests {
        use scylla_cql::Consistency;

        use super::{
            super::tests::{framework::*, EMPTY_ROUTING_INFO},
            super::DefaultPolicy,
            *,
        };

        use crate::{
            load_balancing::default::NodeLocationPreference,
            routing::Shard,
            test_utils::setup_tracing,
            transport::locator::test::{TABLE_INVALID, TABLE_NTS_RF_2, TABLE_NTS_RF_3},
        };
        use crate::{
            load_balancing::{
                default::tests::test_default_policy_with_given_cluster_and_routing_info,
                RoutingInfo,
            },
            routing::Token,
            transport::{
                locator::test::{id_to_invalid_addr, A, B, C, D, E, F, G},
                ClusterData, NodeAddr,
            },
        };
        use tokio::time::Instant;

        trait DefaultPolicyTestExt {
            fn set_nodes_latency_stats(
                &self,
                cluster: &ClusterData,
                averages: &[(u16, Option<TimestampedAverage>)],
            );
        }

        impl DefaultPolicyTestExt for DefaultPolicy {
            fn set_nodes_latency_stats(
                &self,
                cluster: &ClusterData,
                averages: &[(u16, Option<TimestampedAverage>)],
            ) {
                let addr_to_host_id: HashMap<NodeAddr, Uuid> = cluster
                    .known_peers
                    .values()
                    .map(|node| (node.address, node.host_id))
                    .collect();

                for (id, average) in averages.iter().copied() {
                    let host_id = *addr_to_host_id.get(&id_to_invalid_addr(id)).unwrap();
                    let mut node_latencies = self
                        .latency_awareness
                        .as_ref()
                        .unwrap()
                        .node_avgs
                        .write()
                        .unwrap();
                    let mut node_latency =
                        node_latencies.entry(host_id).or_default().write().unwrap();
                    println!("Set latency: node {}, latency {:?}.", id, average);
                    *node_latency = average;
                }
                println!("Set node latency stats.")
            }
        }

        fn default_policy_with_given_latency_awareness(
            latency_awareness: LatencyAwareness,
        ) -> DefaultPolicy {
            let pick_predicate = {
                let latency_predicate = latency_awareness.generate_predicate();
                Box::new(move |node: NodeRef<'_>, shard| {
                    DefaultPolicy::is_alive(node, shard) && latency_predicate(node)
                })
                    as Box<dyn Fn(NodeRef<'_>, Option<Shard>) -> bool + Send + Sync + 'static>
            };

            DefaultPolicy {
                preferences: NodeLocationPreference::Datacenter("eu".to_owned()),
                permit_dc_failover: true,
                is_token_aware: true,
                pick_predicate,
                latency_awareness: Some(latency_awareness),
                fixed_seed: None,
            }
        }

        fn latency_aware_default_policy_customised(
            configurer: impl FnOnce(LatencyAwarenessBuilder) -> LatencyAwarenessBuilder,
        ) -> DefaultPolicy {
            let latency_awareness = configurer(LatencyAwareness::builder()).build();
            default_policy_with_given_latency_awareness(latency_awareness)
        }

        fn latency_aware_default_policy() -> DefaultPolicy {
            latency_aware_default_policy_customised(|b| b)
        }

        fn latency_aware_policy_with_explicit_updater_customised(
            configurer: impl FnOnce(LatencyAwarenessBuilder) -> LatencyAwarenessBuilder,
        ) -> (DefaultPolicy, MinAvgUpdater) {
            let (latency_awareness, updater) =
                configurer(LatencyAwareness::builder()).build_for_test();
            (
                default_policy_with_given_latency_awareness(latency_awareness),
                updater,
            )
        }

        fn latency_aware_default_policy_with_explicit_updater() -> (DefaultPolicy, MinAvgUpdater) {
            latency_aware_policy_with_explicit_updater_customised(|b| b)
        }

        #[tokio::test]
        async fn latency_aware_default_policy_does_not_penalise_if_no_latency_info_available_yet() {
            setup_tracing();
            let policy = latency_aware_default_policy();
            let cluster = tests::mock_cluster_data_for_token_unaware_tests().await;

            let expected_groups = ExpectedGroupsBuilder::new()
                .group([1, 2, 3]) // pick + fallback local nodes
                .group([4, 5]) // fallback remote nodes
                .build();

            test_default_policy_with_given_cluster_and_routing_info(
                &policy,
                &cluster,
                &EMPTY_ROUTING_INFO,
                &expected_groups,
            )
            .await;
        }

        #[tokio::test]
        async fn latency_aware_default_policy_does_not_penalise_if_not_enough_measurements() {
            setup_tracing();
            let policy = latency_aware_default_policy();
            let cluster = tests::mock_cluster_data_for_token_unaware_tests().await;

            let min_avg = Duration::from_millis(10);

            policy.set_nodes_latency_stats(
                &cluster,
                &[
                    (
                        1,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: Duration::from_secs_f64(
                                policy
                                    .latency_awareness
                                    .as_ref()
                                    .unwrap()
                                    .exclusion_threshold
                                    * 1.5
                                    * min_avg.as_secs_f64(),
                            ),
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements
                                - 1,
                        }),
                    ),
                    (
                        3,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: min_avg,
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                ],
            );

            let expected_groups = ExpectedGroupsBuilder::new()
                .group([1, 2, 3]) // pick + fallback local nodes
                .group([4, 5]) // fallback remote nodes
                .build();

            test_default_policy_with_given_cluster_and_routing_info(
                &policy,
                &cluster,
                &EMPTY_ROUTING_INFO,
                &expected_groups,
            )
            .await;
        }

        #[tokio::test]
        async fn latency_aware_default_policy_does_not_penalise_if_exclusion_threshold_not_crossed()
        {
            setup_tracing();
            let policy = latency_aware_default_policy();
            let cluster = tests::mock_cluster_data_for_token_unaware_tests().await;

            let min_avg = Duration::from_millis(10);

            policy.set_nodes_latency_stats(
                &cluster,
                &[
                    (
                        1,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: Duration::from_secs_f64(
                                policy
                                    .latency_awareness
                                    .as_ref()
                                    .unwrap()
                                    .exclusion_threshold
                                    * 0.95
                                    * min_avg.as_secs_f64(),
                            ),
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                    (
                        3,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: min_avg,
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                ],
            );

            let expected_groups = ExpectedGroupsBuilder::new()
                .group([1, 2, 3]) // pick + fallback local nodes
                .group([4, 5]) // fallback remote nodes
                .build();

            test_default_policy_with_given_cluster_and_routing_info(
                &policy,
                &cluster,
                &EMPTY_ROUTING_INFO,
                &expected_groups,
            )
            .await;
        }

        #[tokio::test]
        async fn latency_aware_default_policy_does_not_penalise_if_retry_period_expired() {
            setup_tracing();
            let policy = latency_aware_default_policy_customised(|b| {
                b.retry_period(Duration::from_millis(10))
            });

            let cluster = tests::mock_cluster_data_for_token_unaware_tests().await;

            let min_avg = Duration::from_millis(10);

            policy.set_nodes_latency_stats(
                &cluster,
                &[
                    (
                        1,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: Duration::from_secs_f64(
                                policy
                                    .latency_awareness
                                    .as_ref()
                                    .unwrap()
                                    .exclusion_threshold
                                    * 1.5
                                    * min_avg.as_secs_f64(),
                            ),
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                    (
                        3,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: min_avg,
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                ],
            );

            tokio::time::sleep(2 * policy.latency_awareness.as_ref().unwrap().retry_period).await;

            let expected_groups = ExpectedGroupsBuilder::new()
                .group([1, 2, 3]) // pick + fallback local nodes
                .group([4, 5]) // fallback remote nodes
                .build();

            test_default_policy_with_given_cluster_and_routing_info(
                &policy,
                &cluster,
                &EMPTY_ROUTING_INFO,
                &expected_groups,
            )
            .await;
        }

        #[tokio::test]
        async fn latency_aware_default_policy_penalises_if_conditions_met() {
            setup_tracing();
            let (policy, updater) = latency_aware_default_policy_with_explicit_updater();
            let cluster = tests::mock_cluster_data_for_token_unaware_tests().await;

            let min_avg = Duration::from_millis(10);

            policy.set_nodes_latency_stats(
                &cluster,
                &[
                    // 3 is fast enough to make 1 and 4 penalised.
                    (
                        1,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: Duration::from_secs_f64(
                                policy
                                    .latency_awareness
                                    .as_ref()
                                    .unwrap()
                                    .exclusion_threshold
                                    * 1.05
                                    * min_avg.as_secs_f64(),
                            ),
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                    (
                        3,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: min_avg,
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                    (
                        4,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: Duration::from_secs_f64(
                                policy
                                    .latency_awareness
                                    .as_ref()
                                    .unwrap()
                                    .exclusion_threshold
                                    * 1.05
                                    * min_avg.as_secs_f64(),
                            ),
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                ],
            );

            // Await last min average updater.
            updater.tick().await;

            let expected_groups = ExpectedGroupsBuilder::new()
                .group([2, 3]) // pick + fallback local nodes
                .group([5]) // fallback remote nodes
                .group([1]) // local node that was penalised due to high latency
                .group([4]) // remote node that was penalised due to high latency
                .build();

            test_default_policy_with_given_cluster_and_routing_info(
                &policy,
                &cluster,
                &EMPTY_ROUTING_INFO,
                &expected_groups,
            )
            .await;
        }

        #[tokio::test]
        async fn latency_aware_default_policy_stops_penalising_after_min_average_increases_enough_only_after_update_rate_elapses(
        ) {
            setup_tracing();

            let (policy, updater) = latency_aware_default_policy_with_explicit_updater();

            let cluster = tests::mock_cluster_data_for_token_unaware_tests().await;

            let min_avg = Duration::from_millis(10);

            policy.set_nodes_latency_stats(
                &cluster,
                &[
                    (
                        1,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: Duration::from_secs_f64(
                                policy
                                    .latency_awareness
                                    .as_ref()
                                    .unwrap()
                                    .exclusion_threshold
                                    * 1.05
                                    * min_avg.as_secs_f64(),
                            ),
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                    (
                        3,
                        Some(TimestampedAverage {
                            timestamp: Instant::now(),
                            average: min_avg,
                            num_measures: policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .minimum_measurements,
                        }),
                    ),
                ],
            );

            // Await last min average updater.
            updater.tick().await;
            {
                // min_avg is low enough to penalise node 1
                let expected_groups = ExpectedGroupsBuilder::new()
                    .group([2, 3]) // pick + fallback local nodes
                    .group([4, 5]) // fallback remote nodes
                    .group([1]) // local node that was penalised due to high latency
                    .build();

                test_default_policy_with_given_cluster_and_routing_info(
                    &policy,
                    &cluster,
                    &EMPTY_ROUTING_INFO,
                    &expected_groups,
                )
                .await;
            }

            // node 3 becomes as slow as node 1
            policy.set_nodes_latency_stats(
                &cluster,
                &[(
                    3,
                    Some(TimestampedAverage {
                        timestamp: Instant::now(),
                        average: Duration::from_secs_f64(
                            policy
                                .latency_awareness
                                .as_ref()
                                .unwrap()
                                .exclusion_threshold
                                * min_avg.as_secs_f64(),
                        ),
                        num_measures: policy
                            .latency_awareness
                            .as_ref()
                            .unwrap()
                            .minimum_measurements,
                    }),
                )],
            );
            {
                // min_avg has not yet been updated and so node 1 is still being penalised
                let expected_groups = ExpectedGroupsBuilder::new()
                    .group([2, 3]) // pick + fallback local nodes
                    .group([4, 5]) // fallback remote nodes
                    .group([1]) // local node that was penalised due to high latency
                    .build();

                test_default_policy_with_given_cluster_and_routing_info(
                    &policy,
                    &cluster,
                    &EMPTY_ROUTING_INFO,
                    &expected_groups,
                )
                .await;
            }

            updater.tick().await;
            {
                // min_avg has been updated and is already high enough to stop penalising node 1
                let expected_groups = ExpectedGroupsBuilder::new()
                    .group([1, 2, 3]) // pick + fallback local nodes
                    .group([4, 5]) // fallback remote nodes
                    .build();

                test_default_policy_with_given_cluster_and_routing_info(
                    &policy,
                    &cluster,
                    &EMPTY_ROUTING_INFO,
                    &expected_groups,
                )
                .await;
            }
        }

        #[tokio::test]
        async fn latency_aware_default_policy_is_correctly_token_aware() {
            setup_tracing();

            struct Test<'a, 'b> {
                // If Some, then the provided value is set as a min_avg.
                // Else, the min_avg is updated based on values provided to set_latency_stats().
                preset_min_avg: Option<Duration>,
                latency_stats: &'b [(u16, Option<TimestampedAverage>)],
                routing_info: RoutingInfo<'a>,
                expected_groups: ExpectedGroups,
            }

            let cluster = tests::mock_cluster_data_for_token_aware_tests().await;
            let latency_awareness_defaults =
                latency_aware_default_policy().latency_awareness.unwrap();
            let min_avg = Duration::from_millis(10);

            let fast_leader = || {
                Some(TimestampedAverage {
                    timestamp: Instant::now(),
                    average: min_avg,
                    num_measures: latency_awareness_defaults.minimum_measurements,
                })
            };

            let fast_enough = || {
                Some(TimestampedAverage {
                    timestamp: Instant::now(),
                    average: Duration::from_secs_f64(
                        latency_awareness_defaults.exclusion_threshold
                            * 0.95
                            * min_avg.as_secs_f64(),
                    ),
                    num_measures: latency_awareness_defaults.minimum_measurements,
                })
            };

            let slow_penalised = || {
                Some(TimestampedAverage {
                    timestamp: Instant::now(),
                    average: Duration::from_secs_f64(
                        latency_awareness_defaults.exclusion_threshold
                            * 1.05
                            * min_avg.as_secs_f64(),
                    ),
                    num_measures: latency_awareness_defaults.minimum_measurements,
                })
            };

            let too_few_measurements_slow = || {
                Some(TimestampedAverage {
                    timestamp: Instant::now(),
                    average: Duration::from_secs_f64(
                        latency_awareness_defaults.exclusion_threshold
                            * 1.05
                            * min_avg.as_secs_f64(),
                    ),
                    num_measures: latency_awareness_defaults.minimum_measurements - 1,
                })
            };

            let too_few_measurements_fast_leader = || {
                Some(TimestampedAverage {
                    timestamp: Instant::now(),
                    average: min_avg,
                    num_measures: 1,
                })
            };

            let tests = [
                Test {
                    // Latency-awareness penalisation fires up and moves C and D to the end.
                    preset_min_avg: None,
                    latency_stats: &[
                        (A, fast_leader()),
                        (C, slow_penalised()),
                        (D, slow_penalised()),
                        (E, too_few_measurements_slow()),
                    ],
                    routing_info: RoutingInfo {
                        token: Some(Token::new(160)),
                        table: Some(TABLE_NTS_RF_3),
                        consistency: Consistency::Quorum,
                        ..Default::default()
                    },
                    // going through the ring, we get order: F , A , C , D , G , B , E
                    //                                      us  eu  eu  us  eu  eu  us
                    //                                      r2  r1  r1  r1  r2  r1  r1
                    expected_groups: ExpectedGroupsBuilder::new()
                        .group([A, G]) // fast enough local replicas
                        .group([F, E]) // fast enough remote replicas
                        .group([B]) // fast enough local nodes
                        .group([C]) // penalised local replica
                        .group([D]) // penalised remote replica
                        .build(),
                },
                Test {
                    // Latency-awareness has old minimum average cached, so does not fire.
                    preset_min_avg: Some(100 * min_avg),
                    routing_info: RoutingInfo {
                        token: Some(Token::new(160)),
                        table: Some(TABLE_NTS_RF_3),
                        consistency: Consistency::Quorum,
                        ..Default::default()
                    },
                    latency_stats: &[
                        (A, fast_leader()),
                        (B, fast_enough()),
                        (C, slow_penalised()),
                        (D, slow_penalised()),
                    ],
                    // going through the ring, we get order: F , A , C , D , G , B , E
                    //                                      us  eu  eu  us  eu  eu  us
                    //                                      r2  r1  r1  r1  r2  r1  r1
                    expected_groups: ExpectedGroupsBuilder::new()
                        .group([A, C, G]) // fast enough local replicas
                        .group([F, D, E]) // fast enough remote replicas
                        .group([B]) // fast enough local nodes
                        .build(),
                },
                Test {
                    // Both A and B are slower than C, but only B has enough measurements collected.
                    preset_min_avg: None,
                    latency_stats: &[
                        (A, slow_penalised()), // not really penalised, because no fast leader here
                        (B, slow_penalised()), // ditto
                        (C, too_few_measurements_fast_leader()),
                    ],
                    routing_info: RoutingInfo {
                        token: Some(Token::new(160)),
                        table: Some(TABLE_NTS_RF_2),
                        consistency: Consistency::Quorum,
                        ..Default::default()
                    },
                    // going through the ring, we get order: F , A , C , D , G , B , E
                    //                                      us  eu  eu  us  eu  eu  us
                    //                                      r2  r1  r1  r1  r2  r1  r1
                    expected_groups: ExpectedGroupsBuilder::new()
                        .group([A, G]) // pick + fallback local replicas
                        .group([F, D]) // remote replicas
                        .group([C, B]) // local nodes
                        .group([E]) // remote nodes
                        .build(),
                },
                Test {
                    // No latency stats, so latency-awareness is a no-op.
                    preset_min_avg: None,
                    routing_info: RoutingInfo {
                        token: Some(Token::new(160)),
                        table: Some(TABLE_INVALID),
                        consistency: Consistency::Quorum,
                        ..Default::default()
                    },
                    latency_stats: &[],
                    expected_groups: ExpectedGroupsBuilder::new()
                        .group([A, B, C, G]) // local nodes
                        .group([D, E, F]) // remote nodes
                        .build(),
                },
            ];

            for test in &tests {
                let (policy, updater) = latency_aware_default_policy_with_explicit_updater();

                if let Some(preset_min_avg) = test.preset_min_avg {
                    policy.set_nodes_latency_stats(
                        &cluster,
                        &[(
                            1,
                            Some(TimestampedAverage {
                                timestamp: Instant::now(),
                                average: preset_min_avg,
                                num_measures: latency_awareness_defaults.minimum_measurements,
                            }),
                        )],
                    );
                    // Await last min average updater for update with a forged min_avg.
                    updater.tick().await;
                    policy.set_nodes_latency_stats(&cluster, &[(1, None)]);
                }
                policy.set_nodes_latency_stats(&cluster, test.latency_stats);

                if test.preset_min_avg.is_none() {
                    // Await last min average updater for update with None min_avg.
                    updater.tick().await;
                }

                test_default_policy_with_given_cluster_and_routing_info(
                    &policy,
                    &cluster,
                    &test.routing_info,
                    &test.expected_groups,
                )
                .await;
            }
        }

        #[tokio::test(start_paused = true)]
        async fn timestamped_average_works_when_clock_stops() {
            setup_tracing();
            let avg = Some(TimestampedAverage {
                timestamp: Instant::now(),
                average: Duration::from_secs(123),
                num_measures: 1,
            });
            let new_avg = TimestampedAverage::compute_next(avg, Duration::from_secs(456), 10.0);
            assert_eq!(
                new_avg,
                Some(TimestampedAverage {
                    timestamp: Instant::now(),
                    average: Duration::from_secs(123),
                    num_measures: 2,
                }),
            );
        }
    }
}