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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

//! ExecutionContext contains methods for registering data sources and executing queries
use crate::{
    catalog::{
        catalog::{CatalogList, MemoryCatalogList},
        information_schema::CatalogWithInformationSchema,
    },
    datasource::listing::{ListingOptions, ListingTable},
    datasource::{
        file_format::{
            avro::{AvroFormat, DEFAULT_AVRO_EXTENSION},
            csv::{CsvFormat, DEFAULT_CSV_EXTENSION},
            parquet::{ParquetFormat, DEFAULT_PARQUET_EXTENSION},
            FileFormat,
        },
        MemTable,
    },
    logical_plan::{PlanType, ToStringifiedPlan},
    optimizer::eliminate_limit::EliminateLimit,
    physical_optimizer::{
        aggregate_statistics::AggregateStatistics,
        hash_build_probe_order::HashBuildProbeOrder, optimizer::PhysicalOptimizerRule,
    },
};
use log::{debug, trace};
use parking_lot::Mutex;
use std::collections::{HashMap, HashSet};
use std::path::Path;
use std::string::String;
use std::sync::Arc;
use std::{fs, path::PathBuf};

use futures::{StreamExt, TryStreamExt};
use tokio::task::{self, JoinHandle};

use arrow::{csv, datatypes::SchemaRef};

use crate::catalog::{
    catalog::{CatalogProvider, MemoryCatalogProvider},
    schema::{MemorySchemaProvider, SchemaProvider},
    ResolvedTableReference, TableReference,
};
use crate::datasource::listing::ListingTableConfig;
use crate::datasource::object_store::{ObjectStore, ObjectStoreRegistry};
use crate::datasource::TableProvider;
use crate::error::{DataFusionError, Result};
use crate::execution::dataframe_impl::DataFrameImpl;
use crate::logical_plan::{
    CreateExternalTable, CreateMemoryTable, DropTable, FunctionRegistry, LogicalPlan,
    LogicalPlanBuilder, UNNAMED_TABLE,
};
use crate::optimizer::common_subexpr_eliminate::CommonSubexprEliminate;
use crate::optimizer::filter_push_down::FilterPushDown;
use crate::optimizer::limit_push_down::LimitPushDown;
use crate::optimizer::optimizer::OptimizerRule;
use crate::optimizer::projection_push_down::ProjectionPushDown;
use crate::optimizer::simplify_expressions::SimplifyExpressions;
use crate::optimizer::single_distinct_to_groupby::SingleDistinctToGroupBy;
use crate::optimizer::to_approx_perc::ToApproxPerc;

use crate::physical_optimizer::coalesce_batches::CoalesceBatches;
use crate::physical_optimizer::merge_exec::AddCoalescePartitionsExec;
use crate::physical_optimizer::repartition::Repartition;

use crate::execution::runtime_env::{RuntimeConfig, RuntimeEnv};
use crate::logical_plan::plan::Explain;
use crate::physical_plan::planner::DefaultPhysicalPlanner;
use crate::physical_plan::udf::ScalarUDF;
use crate::physical_plan::ExecutionPlan;
use crate::physical_plan::PhysicalPlanner;
use crate::sql::{
    parser::{DFParser, FileType},
    planner::{ContextProvider, SqlToRel},
};
use crate::variable::{VarProvider, VarType};
use crate::{dataframe::DataFrame, physical_plan::udaf::AggregateUDF};
use async_trait::async_trait;
use chrono::{DateTime, Utc};
use parquet::arrow::ArrowWriter;
use parquet::file::properties::WriterProperties;

use super::{
    disk_manager::DiskManagerConfig,
    memory_manager::MemoryManagerConfig,
    options::{AvroReadOptions, CsvReadOptions},
    DiskManager, MemoryManager,
};

/// ExecutionContext is the main interface for executing queries with DataFusion. The context
/// provides the following functionality:
///
/// * Create DataFrame from a CSV or Parquet data source.
/// * Register a CSV or Parquet data source as a table that can be referenced from a SQL query.
/// * Register a custom data source that can be referenced from a SQL query.
/// * Execution a SQL query
///
/// The following example demonstrates how to use the context to execute a query against a CSV
/// data source using the DataFrame API:
///
/// ```
/// use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
/// let df = df.filter(col("a").lt_eq(col("b")))?
///            .aggregate(vec![col("a")], vec![min(col("b"))])?
///            .limit(100)?;
/// let results = df.collect();
/// # Ok(())
/// # }
/// ```
///
/// The following example demonstrates how to execute the same query using SQL:
///
/// ```
/// use datafusion::prelude::*;
///
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).await?;
/// let results = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100").await?;
/// # Ok(())
/// # }
/// ```
#[derive(Clone)]
pub struct ExecutionContext {
    /// Internal state for the context
    pub state: Arc<Mutex<ExecutionContextState>>,
}

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

impl ExecutionContext {
    /// Creates a new execution context using a default configuration.
    pub fn new() -> Self {
        Self::with_config(ExecutionConfig::new())
    }

    /// Creates a new execution context using the provided configuration.
    pub fn with_config(config: ExecutionConfig) -> Self {
        let catalog_list = Arc::new(MemoryCatalogList::new()) as Arc<dyn CatalogList>;

        if config.create_default_catalog_and_schema {
            let default_catalog = MemoryCatalogProvider::new();

            default_catalog.register_schema(
                config.default_schema.clone(),
                Arc::new(MemorySchemaProvider::new()),
            );

            let default_catalog: Arc<dyn CatalogProvider> = if config.information_schema {
                Arc::new(CatalogWithInformationSchema::new(
                    Arc::downgrade(&catalog_list),
                    Arc::new(default_catalog),
                ))
            } else {
                Arc::new(default_catalog)
            };

            catalog_list
                .register_catalog(config.default_catalog.clone(), default_catalog);
        }

        let runtime_env = Arc::new(RuntimeEnv::new(config.runtime.clone()).unwrap());

        Self {
            state: Arc::new(Mutex::new(ExecutionContextState {
                catalog_list,
                scalar_functions: HashMap::new(),
                aggregate_functions: HashMap::new(),
                config,
                execution_props: ExecutionProps::new(),
                object_store_registry: Arc::new(ObjectStoreRegistry::new()),
                runtime_env,
            })),
        }
    }

    /// Return the [RuntimeEnv] used to run queries with this [ExecutionContext]
    pub fn runtime_env(&self) -> Arc<RuntimeEnv> {
        self.state.lock().runtime_env.clone()
    }

    /// Creates a dataframe that will execute a SQL query.
    ///
    /// This method is `async` because queries of type `CREATE EXTERNAL TABLE`
    /// might require the schema to be inferred.
    pub async fn sql(&mut self, sql: &str) -> Result<Arc<dyn DataFrame>> {
        let plan = self.create_logical_plan(sql)?;
        match plan {
            LogicalPlan::CreateExternalTable(CreateExternalTable {
                ref schema,
                ref name,
                ref location,
                ref file_type,
                ref has_header,
            }) => {
                let (file_format, file_extension) = match file_type {
                    FileType::CSV => Ok((
                        Arc::new(CsvFormat::default().with_has_header(*has_header))
                            as Arc<dyn FileFormat>,
                        DEFAULT_CSV_EXTENSION,
                    )),
                    FileType::Parquet => Ok((
                        Arc::new(ParquetFormat::default()) as Arc<dyn FileFormat>,
                        DEFAULT_PARQUET_EXTENSION,
                    )),
                    FileType::Avro => Ok((
                        Arc::new(AvroFormat::default()) as Arc<dyn FileFormat>,
                        DEFAULT_AVRO_EXTENSION,
                    )),
                    _ => Err(DataFusionError::NotImplemented(format!(
                        "Unsupported file type {:?}.",
                        file_type
                    ))),
                }?;

                let options = ListingOptions {
                    format: file_format,
                    collect_stat: false,
                    file_extension: file_extension.to_owned(),
                    target_partitions: self.state.lock().config.target_partitions,
                    table_partition_cols: vec![],
                };

                // TODO make schema in CreateExternalTable optional instead of empty
                let provided_schema = if schema.fields().is_empty() {
                    None
                } else {
                    Some(Arc::new(schema.as_ref().to_owned().into()))
                };

                self.register_listing_table(name, location, options, provided_schema)
                    .await?;
                let plan = LogicalPlanBuilder::empty(false).build()?;
                Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan)))
            }

            LogicalPlan::CreateMemoryTable(CreateMemoryTable { name, input }) => {
                let plan = self.optimize(&input)?;
                let physical = Arc::new(DataFrameImpl::new(self.state.clone(), &plan));

                let batches: Vec<_> = physical.collect_partitioned().await?;
                let table = Arc::new(MemTable::try_new(
                    Arc::new(plan.schema().as_ref().into()),
                    batches,
                )?);
                self.register_table(name.as_str(), table)?;

                let plan = LogicalPlanBuilder::empty(false).build()?;
                Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan)))
            }

            LogicalPlan::DropTable(DropTable { name, if_exist, .. }) => {
                let returned = self.deregister_table(name.as_str())?;
                if !if_exist && returned.is_none() {
                    Err(DataFusionError::Execution(format!(
                        "Memory table {:?} doesn't exist.",
                        name
                    )))
                } else {
                    let plan = LogicalPlanBuilder::empty(false).build()?;
                    Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan)))
                }
            }

            plan => Ok(Arc::new(DataFrameImpl::new(
                self.state.clone(),
                &self.optimize(&plan)?,
            ))),
        }
    }

    /// Creates a logical plan.
    ///
    /// This function is intended for internal use and should not be called directly.
    pub fn create_logical_plan(&self, sql: &str) -> Result<LogicalPlan> {
        let statements = DFParser::parse_sql(sql)?;

        if statements.len() != 1 {
            return Err(DataFusionError::NotImplemented(
                "The context currently only supports a single SQL statement".to_string(),
            ));
        }

        // create a query planner
        let state = self.state.lock().clone();
        let query_planner = SqlToRel::new(&state);
        query_planner.statement_to_plan(&statements[0])
    }

    /// Registers a variable provider within this context.
    pub fn register_variable(
        &mut self,
        variable_type: VarType,
        provider: Arc<dyn VarProvider + Send + Sync>,
    ) {
        self.state
            .lock()
            .execution_props
            .add_var_provider(variable_type, provider);
    }

    /// Registers a scalar UDF within this context.
    ///
    /// Note in SQL queries, function names are looked up using
    /// lowercase unless the query uses quotes. For example,
    ///
    /// `SELECT MY_FUNC(x)...` will look for a function named `"my_func"`
    /// `SELECT "my_FUNC"(x)` will look for a function named `"my_FUNC"`
    pub fn register_udf(&mut self, f: ScalarUDF) {
        self.state
            .lock()
            .scalar_functions
            .insert(f.name.clone(), Arc::new(f));
    }

    /// Registers an aggregate UDF within this context.
    ///
    /// Note in SQL queries, aggregate names are looked up using
    /// lowercase unless the query uses quotes. For example,
    ///
    /// `SELECT MY_UDAF(x)...` will look for an aggregate named `"my_udaf"`
    /// `SELECT "my_UDAF"(x)` will look for an aggregate named `"my_UDAF"`
    pub fn register_udaf(&mut self, f: AggregateUDF) {
        self.state
            .lock()
            .aggregate_functions
            .insert(f.name.clone(), Arc::new(f));
    }

    /// Creates a DataFrame for reading an Avro data source.

    pub async fn read_avro(
        &mut self,
        uri: impl Into<String>,
        options: AvroReadOptions<'_>,
    ) -> Result<Arc<dyn DataFrame>> {
        let uri: String = uri.into();
        let (object_store, path) = self.object_store(&uri)?;
        let target_partitions = self.state.lock().config.target_partitions;
        Ok(Arc::new(DataFrameImpl::new(
            self.state.clone(),
            &LogicalPlanBuilder::scan_avro(
                object_store,
                path,
                options,
                None,
                target_partitions,
            )
            .await?
            .build()?,
        )))
    }

    /// Creates an empty DataFrame.
    pub fn read_empty(&self) -> Result<Arc<dyn DataFrame>> {
        Ok(Arc::new(DataFrameImpl::new(
            self.state.clone(),
            &LogicalPlanBuilder::empty(true).build()?,
        )))
    }

    /// Creates a DataFrame for reading a CSV data source.
    pub async fn read_csv(
        &mut self,
        uri: impl Into<String>,
        options: CsvReadOptions<'_>,
    ) -> Result<Arc<dyn DataFrame>> {
        let uri: String = uri.into();
        let (object_store, path) = self.object_store(&uri)?;
        let target_partitions = self.state.lock().config.target_partitions;
        Ok(Arc::new(DataFrameImpl::new(
            self.state.clone(),
            &LogicalPlanBuilder::scan_csv(
                object_store,
                path,
                options,
                None,
                target_partitions,
            )
            .await?
            .build()?,
        )))
    }

    /// Creates a DataFrame for reading a Parquet data source.
    pub async fn read_parquet(
        &mut self,
        uri: impl Into<String>,
    ) -> Result<Arc<dyn DataFrame>> {
        let uri: String = uri.into();
        let (object_store, path) = self.object_store(&uri)?;
        let target_partitions = self.state.lock().config.target_partitions;
        let logical_plan =
            LogicalPlanBuilder::scan_parquet(object_store, path, None, target_partitions)
                .await?
                .build()?;
        Ok(Arc::new(DataFrameImpl::new(
            self.state.clone(),
            &logical_plan,
        )))
    }

    /// Creates a DataFrame for reading a custom TableProvider.
    pub fn read_table(
        &mut self,
        provider: Arc<dyn TableProvider>,
    ) -> Result<Arc<dyn DataFrame>> {
        Ok(Arc::new(DataFrameImpl::new(
            self.state.clone(),
            &LogicalPlanBuilder::scan(UNNAMED_TABLE, provider, None)?.build()?,
        )))
    }

    /// Registers a table that uses the listing feature of the object store to
    /// find the files to be processed
    /// This is async because it might need to resolve the schema.
    pub async fn register_listing_table<'a>(
        &'a mut self,
        name: &'a str,
        uri: &'a str,
        options: ListingOptions,
        provided_schema: Option<SchemaRef>,
    ) -> Result<()> {
        let (object_store, path) = self.object_store(uri)?;
        let resolved_schema = match provided_schema {
            None => {
                options
                    .infer_schema(Arc::clone(&object_store), path)
                    .await?
            }
            Some(s) => s,
        };
        let config = ListingTableConfig::new(object_store, path)
            .with_listing_options(options)
            .with_schema(resolved_schema);
        let table = ListingTable::try_new(config)?;
        self.register_table(name, Arc::new(table))?;
        Ok(())
    }

    /// Registers a CSV data source so that it can be referenced from SQL statements
    /// executed against this context.
    pub async fn register_csv(
        &mut self,
        name: &str,
        uri: &str,
        options: CsvReadOptions<'_>,
    ) -> Result<()> {
        let listing_options =
            options.to_listing_options(self.state.lock().config.target_partitions);

        self.register_listing_table(
            name,
            uri,
            listing_options,
            options.schema.map(|s| Arc::new(s.to_owned())),
        )
        .await?;

        Ok(())
    }

    /// Registers a Parquet data source so that it can be referenced from SQL statements
    /// executed against this context.
    pub async fn register_parquet(&mut self, name: &str, uri: &str) -> Result<()> {
        let (target_partitions, enable_pruning) = {
            let m = self.state.lock();
            (m.config.target_partitions, m.config.parquet_pruning)
        };
        let file_format = ParquetFormat::default().with_enable_pruning(enable_pruning);

        let listing_options = ListingOptions {
            format: Arc::new(file_format),
            collect_stat: true,
            file_extension: DEFAULT_PARQUET_EXTENSION.to_owned(),
            target_partitions,
            table_partition_cols: vec![],
        };

        self.register_listing_table(name, uri, listing_options, None)
            .await?;
        Ok(())
    }

    /// Registers an Avro data source so that it can be referenced from SQL statements
    /// executed against this context.
    pub async fn register_avro(
        &mut self,
        name: &str,
        uri: &str,
        options: AvroReadOptions<'_>,
    ) -> Result<()> {
        let listing_options =
            options.to_listing_options(self.state.lock().config.target_partitions);

        self.register_listing_table(name, uri, listing_options, options.schema)
            .await?;
        Ok(())
    }

    /// Registers a named catalog using a custom `CatalogProvider` so that
    /// it can be referenced from SQL statements executed against this
    /// context.
    ///
    /// Returns the `CatalogProvider` previously registered for this
    /// name, if any
    pub fn register_catalog(
        &self,
        name: impl Into<String>,
        catalog: Arc<dyn CatalogProvider>,
    ) -> Option<Arc<dyn CatalogProvider>> {
        let name = name.into();

        let state = self.state.lock();
        let catalog = if state.config.information_schema {
            Arc::new(CatalogWithInformationSchema::new(
                Arc::downgrade(&state.catalog_list),
                catalog,
            ))
        } else {
            catalog
        };

        state.catalog_list.register_catalog(name, catalog)
    }

    /// Retrieves a `CatalogProvider` instance by name
    pub fn catalog(&self, name: &str) -> Option<Arc<dyn CatalogProvider>> {
        self.state.lock().catalog_list.catalog(name)
    }

    /// Registers a object store with scheme using a custom `ObjectStore` so that
    /// an external file system or object storage system could be used against this context.
    ///
    /// Returns the `ObjectStore` previously registered for this scheme, if any
    pub fn register_object_store(
        &self,
        scheme: impl Into<String>,
        object_store: Arc<dyn ObjectStore>,
    ) -> Option<Arc<dyn ObjectStore>> {
        let scheme = scheme.into();

        self.state
            .lock()
            .object_store_registry
            .register_store(scheme, object_store)
    }

    /// Retrieves a `ObjectStore` instance by scheme
    pub fn object_store<'a>(
        &self,
        uri: &'a str,
    ) -> Result<(Arc<dyn ObjectStore>, &'a str)> {
        self.state
            .lock()
            .object_store_registry
            .get_by_uri(uri)
            .map_err(DataFusionError::from)
    }

    /// Registers a table using a custom `TableProvider` so that
    /// it can be referenced from SQL statements executed against this
    /// context.
    ///
    /// Returns the `TableProvider` previously registered for this
    /// reference, if any
    pub fn register_table<'a>(
        &'a mut self,
        table_ref: impl Into<TableReference<'a>>,
        provider: Arc<dyn TableProvider>,
    ) -> Result<Option<Arc<dyn TableProvider>>> {
        let table_ref = table_ref.into();
        self.state
            .lock()
            .schema_for_ref(table_ref)?
            .register_table(table_ref.table().to_owned(), provider)
    }

    /// Deregisters the given table.
    ///
    /// Returns the registered provider, if any
    pub fn deregister_table<'a>(
        &'a mut self,
        table_ref: impl Into<TableReference<'a>>,
    ) -> Result<Option<Arc<dyn TableProvider>>> {
        let table_ref = table_ref.into();
        self.state
            .lock()
            .schema_for_ref(table_ref)?
            .deregister_table(table_ref.table())
    }

    /// Retrieves a DataFrame representing a table previously registered by calling the
    /// register_table function.
    ///
    /// Returns an error if no table has been registered with the provided reference.
    pub fn table<'a>(
        &self,
        table_ref: impl Into<TableReference<'a>>,
    ) -> Result<Arc<dyn DataFrame>> {
        let table_ref = table_ref.into();
        let schema = self.state.lock().schema_for_ref(table_ref)?;
        match schema.table(table_ref.table()) {
            Some(ref provider) => {
                let plan = LogicalPlanBuilder::scan(
                    table_ref.table(),
                    Arc::clone(provider),
                    None,
                )?
                .build()?;
                Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan)))
            }
            _ => Err(DataFusionError::Plan(format!(
                "No table named '{}'",
                table_ref.table()
            ))),
        }
    }

    /// Returns the set of available tables in the default catalog and schema.
    ///
    /// Use [`table`] to get a specific table.
    ///
    /// [`table`]: ExecutionContext::table
    #[deprecated(
        note = "Please use the catalog provider interface (`ExecutionContext::catalog`) to examine available catalogs, schemas, and tables"
    )]
    pub fn tables(&self) -> Result<HashSet<String>> {
        Ok(self
            .state
            .lock()
            // a bare reference will always resolve to the default catalog and schema
            .schema_for_ref(TableReference::Bare { table: "" })?
            .table_names()
            .iter()
            .cloned()
            .collect())
    }

    /// Optimizes the logical plan by applying optimizer rules.
    pub fn optimize(&self, plan: &LogicalPlan) -> Result<LogicalPlan> {
        if let LogicalPlan::Explain(e) = plan {
            let mut stringified_plans = e.stringified_plans.clone();

            // optimize the child plan, capturing the output of each optimizer
            let plan =
                self.optimize_internal(e.plan.as_ref(), |optimized_plan, optimizer| {
                    let optimizer_name = optimizer.name().to_string();
                    let plan_type = PlanType::OptimizedLogicalPlan { optimizer_name };
                    stringified_plans.push(optimized_plan.to_stringified(plan_type));
                })?;

            Ok(LogicalPlan::Explain(Explain {
                verbose: e.verbose,
                plan: Arc::new(plan),
                stringified_plans,
                schema: e.schema.clone(),
            }))
        } else {
            self.optimize_internal(plan, |_, _| {})
        }
    }

    /// Creates a physical plan from a logical plan.
    pub async fn create_physical_plan(
        &self,
        logical_plan: &LogicalPlan,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        let (state, planner) = {
            let mut state = self.state.lock();
            state.execution_props.start_execution();

            // We need to clone `state` to release the lock that is not `Send`. We could
            // make the lock `Send` by using `tokio::sync::Mutex`, but that would require to
            // propagate async even to the `LogicalPlan` building methods.
            // Cloning `state` here is fine as we then pass it as immutable `&state`, which
            // means that we avoid write consistency issues as the cloned version will not
            // be written to. As for eventual modifications that would be applied to the
            // original state after it has been cloned, they will not be picked up by the
            // clone but that is okay, as it is equivalent to postponing the state update
            // by keeping the lock until the end of the function scope.
            (state.clone(), Arc::clone(&state.config.query_planner))
        };

        planner.create_physical_plan(logical_plan, &state).await
    }

    /// Executes a query and writes the results to a partitioned CSV file.
    pub async fn write_csv(
        &self,
        plan: Arc<dyn ExecutionPlan>,
        path: impl AsRef<str>,
    ) -> Result<()> {
        let path = path.as_ref();
        // create directory to contain the CSV files (one per partition)
        let fs_path = Path::new(path);
        let runtime = self.runtime_env();
        match fs::create_dir(fs_path) {
            Ok(()) => {
                let mut tasks = vec![];
                for i in 0..plan.output_partitioning().partition_count() {
                    let plan = plan.clone();
                    let filename = format!("part-{}.csv", i);
                    let path = fs_path.join(&filename);
                    let file = fs::File::create(path)?;
                    let mut writer = csv::Writer::new(file);
                    let stream = plan.execute(i, runtime.clone()).await?;
                    let handle: JoinHandle<Result<()>> = task::spawn(async move {
                        stream
                            .map(|batch| writer.write(&batch?))
                            .try_collect()
                            .await
                            .map_err(DataFusionError::from)
                    });
                    tasks.push(handle);
                }
                futures::future::join_all(tasks).await;
                Ok(())
            }
            Err(e) => Err(DataFusionError::Execution(format!(
                "Could not create directory {}: {:?}",
                path, e
            ))),
        }
    }

    /// Executes a query and writes the results to a partitioned Parquet file.
    pub async fn write_parquet(
        &self,
        plan: Arc<dyn ExecutionPlan>,
        path: impl AsRef<str>,
        writer_properties: Option<WriterProperties>,
    ) -> Result<()> {
        let path = path.as_ref();
        // create directory to contain the Parquet files (one per partition)
        let fs_path = Path::new(path);
        let runtime = self.runtime_env();
        match fs::create_dir(fs_path) {
            Ok(()) => {
                let mut tasks = vec![];
                for i in 0..plan.output_partitioning().partition_count() {
                    let plan = plan.clone();
                    let filename = format!("part-{}.parquet", i);
                    let path = fs_path.join(&filename);
                    let file = fs::File::create(path)?;
                    let mut writer = ArrowWriter::try_new(
                        file.try_clone().unwrap(),
                        plan.schema(),
                        writer_properties.clone(),
                    )?;
                    let stream = plan.execute(i, runtime.clone()).await?;
                    let handle: JoinHandle<Result<()>> = task::spawn(async move {
                        stream
                            .map(|batch| writer.write(&batch?))
                            .try_collect()
                            .await
                            .map_err(DataFusionError::from)?;
                        writer.close().map_err(DataFusionError::from).map(|_| ())
                    });
                    tasks.push(handle);
                }
                futures::future::join_all(tasks).await;
                Ok(())
            }
            Err(e) => Err(DataFusionError::Execution(format!(
                "Could not create directory {}: {:?}",
                path, e
            ))),
        }
    }

    /// Optimizes the logical plan by applying optimizer rules, and
    /// invoking observer function after each call
    fn optimize_internal<F>(
        &self,
        plan: &LogicalPlan,
        mut observer: F,
    ) -> Result<LogicalPlan>
    where
        F: FnMut(&LogicalPlan, &dyn OptimizerRule),
    {
        let state = &mut self.state.lock();
        let execution_props = &mut state.execution_props.clone();
        let optimizers = &state.config.optimizers;

        let execution_props = execution_props.start_execution();

        let mut new_plan = plan.clone();
        debug!("Input logical plan:\n{}\n", plan.display_indent());
        trace!("Full input logical plan:\n{:?}", plan);
        for optimizer in optimizers {
            new_plan = optimizer.optimize(&new_plan, execution_props)?;
            observer(&new_plan, optimizer.as_ref());
        }
        debug!("Optimized logical plan:\n{}\n", new_plan.display_indent());
        trace!("Full Optimized logical plan:\n {:?}", plan);
        Ok(new_plan)
    }
}

impl From<Arc<Mutex<ExecutionContextState>>> for ExecutionContext {
    fn from(state: Arc<Mutex<ExecutionContextState>>) -> Self {
        ExecutionContext { state }
    }
}

impl FunctionRegistry for ExecutionContext {
    fn udfs(&self) -> HashSet<String> {
        self.state.lock().udfs()
    }

    fn udf(&self, name: &str) -> Result<Arc<ScalarUDF>> {
        self.state.lock().udf(name)
    }

    fn udaf(&self, name: &str) -> Result<Arc<AggregateUDF>> {
        self.state.lock().udaf(name)
    }
}

/// A planner used to add extensions to DataFusion logical and physical plans.
#[async_trait]
pub trait QueryPlanner {
    /// Given a `LogicalPlan`, create an `ExecutionPlan` suitable for execution
    async fn create_physical_plan(
        &self,
        logical_plan: &LogicalPlan,
        ctx_state: &ExecutionContextState,
    ) -> Result<Arc<dyn ExecutionPlan>>;
}

/// The query planner used if no user defined planner is provided
struct DefaultQueryPlanner {}

#[async_trait]
impl QueryPlanner for DefaultQueryPlanner {
    /// Given a `LogicalPlan`, create an `ExecutionPlan` suitable for execution
    async fn create_physical_plan(
        &self,
        logical_plan: &LogicalPlan,
        ctx_state: &ExecutionContextState,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        let planner = DefaultPhysicalPlanner::default();
        planner.create_physical_plan(logical_plan, ctx_state).await
    }
}

/// Configuration options for execution context
#[derive(Clone)]
pub struct ExecutionConfig {
    /// Number of partitions for query execution. Increasing partitions can increase concurrency.
    pub target_partitions: usize,
    /// Responsible for optimizing a logical plan
    optimizers: Vec<Arc<dyn OptimizerRule + Send + Sync>>,
    /// Responsible for optimizing a physical execution plan
    pub physical_optimizers: Vec<Arc<dyn PhysicalOptimizerRule + Send + Sync>>,
    /// Responsible for planning `LogicalPlan`s, and `ExecutionPlan`
    query_planner: Arc<dyn QueryPlanner + Send + Sync>,
    /// Default catalog name for table resolution
    default_catalog: String,
    /// Default schema name for table resolution
    default_schema: String,
    /// Whether the default catalog and schema should be created automatically
    create_default_catalog_and_schema: bool,
    /// Should DataFusion provide access to `information_schema`
    /// virtual tables for displaying schema information
    information_schema: bool,
    /// Should DataFusion repartition data using the join keys to execute joins in parallel
    /// using the provided `target_partitions` level
    pub repartition_joins: bool,
    /// Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel
    /// using the provided `target_partitions` level
    pub repartition_aggregations: bool,
    /// Should DataFusion repartition data using the partition keys to execute window functions in
    /// parallel using the provided `target_partitions` level
    pub repartition_windows: bool,
    /// Should Datafusion parquet reader using the predicate to prune data
    parquet_pruning: bool,
    /// Runtime configurations such as memory threshold and local disk for spill
    pub runtime: RuntimeConfig,
}

impl Default for ExecutionConfig {
    fn default() -> Self {
        Self {
            target_partitions: num_cpus::get(),
            optimizers: vec![
                // Simplify expressions first to maximize the chance
                // of applying other optimizations
                Arc::new(SimplifyExpressions::new()),
                Arc::new(CommonSubexprEliminate::new()),
                Arc::new(EliminateLimit::new()),
                Arc::new(ProjectionPushDown::new()),
                Arc::new(FilterPushDown::new()),
                Arc::new(LimitPushDown::new()),
                Arc::new(SingleDistinctToGroupBy::new()),
                // ToApproxPerc must be applied last because
                // it rewrites only the function and may interfere with
                // other rules
                Arc::new(ToApproxPerc::new()),
            ],
            physical_optimizers: vec![
                Arc::new(AggregateStatistics::new()),
                Arc::new(HashBuildProbeOrder::new()),
                Arc::new(CoalesceBatches::new()),
                Arc::new(Repartition::new()),
                Arc::new(AddCoalescePartitionsExec::new()),
            ],
            query_planner: Arc::new(DefaultQueryPlanner {}),
            default_catalog: "datafusion".to_owned(),
            default_schema: "public".to_owned(),
            create_default_catalog_and_schema: true,
            information_schema: false,
            repartition_joins: true,
            repartition_aggregations: true,
            repartition_windows: true,
            parquet_pruning: true,
            runtime: RuntimeConfig::default(),
        }
    }
}

impl ExecutionConfig {
    /// Create an execution config with default setting
    pub fn new() -> Self {
        Default::default()
    }

    /// Customize target_partitions
    pub fn with_target_partitions(mut self, n: usize) -> Self {
        // partition count must be greater than zero
        assert!(n > 0);
        self.target_partitions = n;
        self
    }

    /// Customize batch size
    pub fn with_batch_size(mut self, n: usize) -> Self {
        // batch size must be greater than zero
        assert!(n > 0);
        self.runtime.batch_size = n;
        self
    }

    /// Replace the default query planner
    pub fn with_query_planner(
        mut self,
        query_planner: Arc<dyn QueryPlanner + Send + Sync>,
    ) -> Self {
        self.query_planner = query_planner;
        self
    }

    /// Replace the optimizer rules
    pub fn with_optimizer_rules(
        mut self,
        optimizers: Vec<Arc<dyn OptimizerRule + Send + Sync>>,
    ) -> Self {
        self.optimizers = optimizers;
        self
    }

    /// Replace the physical optimizer rules
    pub fn with_physical_optimizer_rules(
        mut self,
        physical_optimizers: Vec<Arc<dyn PhysicalOptimizerRule + Send + Sync>>,
    ) -> Self {
        self.physical_optimizers = physical_optimizers;
        self
    }

    /// Adds a new [`OptimizerRule`]
    pub fn add_optimizer_rule(
        mut self,
        optimizer_rule: Arc<dyn OptimizerRule + Send + Sync>,
    ) -> Self {
        self.optimizers.push(optimizer_rule);
        self
    }

    /// Adds a new [`PhysicalOptimizerRule`]
    pub fn add_physical_optimizer_rule(
        mut self,
        optimizer_rule: Arc<dyn PhysicalOptimizerRule + Send + Sync>,
    ) -> Self {
        self.physical_optimizers.push(optimizer_rule);
        self
    }

    /// Selects a name for the default catalog and schema
    pub fn with_default_catalog_and_schema(
        mut self,
        catalog: impl Into<String>,
        schema: impl Into<String>,
    ) -> Self {
        self.default_catalog = catalog.into();
        self.default_schema = schema.into();
        self
    }

    /// Controls whether the default catalog and schema will be automatically created
    pub fn create_default_catalog_and_schema(mut self, create: bool) -> Self {
        self.create_default_catalog_and_schema = create;
        self
    }

    /// Enables or disables the inclusion of `information_schema` virtual tables
    pub fn with_information_schema(mut self, enabled: bool) -> Self {
        self.information_schema = enabled;
        self
    }

    /// Enables or disables the use of repartitioning for joins to improve parallelism
    pub fn with_repartition_joins(mut self, enabled: bool) -> Self {
        self.repartition_joins = enabled;
        self
    }

    /// Enables or disables the use of repartitioning for aggregations to improve parallelism
    pub fn with_repartition_aggregations(mut self, enabled: bool) -> Self {
        self.repartition_aggregations = enabled;
        self
    }

    /// Enables or disables the use of repartitioning for window functions to improve parallelism
    pub fn with_repartition_windows(mut self, enabled: bool) -> Self {
        self.repartition_windows = enabled;
        self
    }

    /// Enables or disables the use of pruning predicate for parquet readers to skip row groups
    pub fn with_parquet_pruning(mut self, enabled: bool) -> Self {
        self.parquet_pruning = enabled;
        self
    }

    /// Customize runtime config
    pub fn with_runtime_config(mut self, config: RuntimeConfig) -> Self {
        self.runtime = config;
        self
    }

    /// Use an an existing [MemoryManager]
    pub fn with_existing_memory_manager(mut self, existing: Arc<MemoryManager>) -> Self {
        self.runtime = self
            .runtime
            .with_memory_manager(MemoryManagerConfig::new_existing(existing));
        self
    }

    /// Specify the total memory to use while running the DataFusion
    /// plan to `max_memory * memory_fraction` in bytes.
    ///
    /// Note DataFusion does not yet respect this limit in all cases.
    pub fn with_memory_limit(
        mut self,
        max_memory: usize,
        memory_fraction: f64,
    ) -> Result<Self> {
        self.runtime =
            self.runtime
                .with_memory_manager(MemoryManagerConfig::try_new_limit(
                    max_memory,
                    memory_fraction,
                )?);
        Ok(self)
    }

    /// Use an an existing [DiskManager]
    pub fn with_existing_disk_manager(mut self, existing: Arc<DiskManager>) -> Self {
        self.runtime = self
            .runtime
            .with_disk_manager(DiskManagerConfig::new_existing(existing));
        self
    }

    /// Use the specified path to create any needed temporary files
    pub fn with_temp_file_path(mut self, path: impl Into<PathBuf>) -> Self {
        self.runtime = self
            .runtime
            .with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()]));
        self
    }
}

/// Holds per-execution properties and data (such as starting timestamps, etc).
/// An instance of this struct is created each time a [`LogicalPlan`] is prepared for
/// execution (optimized). If the same plan is optimized multiple times, a new
/// `ExecutionProps` is created each time.
///
/// It is important that this structure be cheap to create as it is
/// done so during predicate pruning and expression simplification
#[derive(Clone)]
pub struct ExecutionProps {
    pub(crate) query_execution_start_time: DateTime<Utc>,
    /// providers for scalar variables
    pub var_providers: Option<HashMap<VarType, Arc<dyn VarProvider + Send + Sync>>>,
}

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

impl ExecutionProps {
    /// Creates a new execution props
    pub fn new() -> Self {
        ExecutionProps {
            query_execution_start_time: chrono::Utc::now(),
            var_providers: None,
        }
    }

    /// Marks the execution of query started timestamp
    pub fn start_execution(&mut self) -> &Self {
        self.query_execution_start_time = chrono::Utc::now();
        &*self
    }

    /// Registers a variable provider, returning the existing
    /// provider, if any
    pub fn add_var_provider(
        &mut self,
        var_type: VarType,
        provider: Arc<dyn VarProvider + Send + Sync>,
    ) -> Option<Arc<dyn VarProvider + Send + Sync>> {
        let mut var_providers = self.var_providers.take().unwrap_or_default();

        let old_provider = var_providers.insert(var_type, provider);

        self.var_providers = Some(var_providers);

        old_provider
    }

    /// Returns the provider for the var_type, if any
    pub fn get_var_provider(
        &self,
        var_type: VarType,
    ) -> Option<Arc<dyn VarProvider + Send + Sync>> {
        self.var_providers
            .as_ref()
            .and_then(|var_providers| var_providers.get(&var_type).map(Arc::clone))
    }
}

/// Execution context for registering data sources and executing queries
#[derive(Clone)]
pub struct ExecutionContextState {
    /// Collection of catalogs containing schemas and ultimately TableProviders
    pub catalog_list: Arc<dyn CatalogList>,
    /// Scalar functions that are registered with the context
    pub scalar_functions: HashMap<String, Arc<ScalarUDF>>,
    /// Aggregate functions registered in the context
    pub aggregate_functions: HashMap<String, Arc<AggregateUDF>>,
    /// Context configuration
    pub config: ExecutionConfig,
    /// Execution properties
    pub execution_props: ExecutionProps,
    /// Object Store that are registered with the context
    pub object_store_registry: Arc<ObjectStoreRegistry>,
    /// Runtime environment
    pub runtime_env: Arc<RuntimeEnv>,
}

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

impl ExecutionContextState {
    /// Returns new ExecutionContextState
    pub fn new() -> Self {
        ExecutionContextState {
            catalog_list: Arc::new(MemoryCatalogList::new()),
            scalar_functions: HashMap::new(),
            aggregate_functions: HashMap::new(),
            config: ExecutionConfig::new(),
            execution_props: ExecutionProps::new(),
            object_store_registry: Arc::new(ObjectStoreRegistry::new()),
            runtime_env: Arc::new(RuntimeEnv::default()),
        }
    }

    fn resolve_table_ref<'a>(
        &'a self,
        table_ref: impl Into<TableReference<'a>>,
    ) -> ResolvedTableReference<'a> {
        table_ref
            .into()
            .resolve(&self.config.default_catalog, &self.config.default_schema)
    }

    fn schema_for_ref<'a>(
        &'a self,
        table_ref: impl Into<TableReference<'a>>,
    ) -> Result<Arc<dyn SchemaProvider>> {
        let resolved_ref = self.resolve_table_ref(table_ref.into());

        self.catalog_list
            .catalog(resolved_ref.catalog)
            .ok_or_else(|| {
                DataFusionError::Plan(format!(
                    "failed to resolve catalog: {}",
                    resolved_ref.catalog
                ))
            })?
            .schema(resolved_ref.schema)
            .ok_or_else(|| {
                DataFusionError::Plan(format!(
                    "failed to resolve schema: {}",
                    resolved_ref.schema
                ))
            })
    }
}

impl ContextProvider for ExecutionContextState {
    fn get_table_provider(&self, name: TableReference) -> Option<Arc<dyn TableProvider>> {
        let resolved_ref = self.resolve_table_ref(name);
        let schema = self.schema_for_ref(resolved_ref).ok()?;
        schema.table(resolved_ref.table)
    }

    fn get_function_meta(&self, name: &str) -> Option<Arc<ScalarUDF>> {
        self.scalar_functions.get(name).cloned()
    }

    fn get_aggregate_meta(&self, name: &str) -> Option<Arc<AggregateUDF>> {
        self.aggregate_functions.get(name).cloned()
    }
}

impl FunctionRegistry for ExecutionContextState {
    fn udfs(&self) -> HashSet<String> {
        self.scalar_functions.keys().cloned().collect()
    }

    fn udf(&self, name: &str) -> Result<Arc<ScalarUDF>> {
        let result = self.scalar_functions.get(name);

        result.cloned().ok_or_else(|| {
            DataFusionError::Plan(format!(
                "There is no UDF named \"{}\" in the registry",
                name
            ))
        })
    }

    fn udaf(&self, name: &str) -> Result<Arc<AggregateUDF>> {
        let result = self.aggregate_functions.get(name);

        result.cloned().ok_or_else(|| {
            DataFusionError::Plan(format!(
                "There is no UDAF named \"{}\" in the registry",
                name
            ))
        })
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::execution::context::QueryPlanner;
    use crate::from_slice::FromSlice;
    use crate::logical_plan::{binary_expr, lit, Operator};
    use crate::physical_plan::collect;
    use crate::physical_plan::functions::{make_scalar_function, Volatility};
    use crate::test;
    use crate::variable::VarType;
    use crate::{
        assert_batches_eq, assert_batches_sorted_eq,
        logical_plan::{col, create_udf, sum, Expr},
    };
    use crate::{
        datasource::{empty::EmptyTable, MemTable},
        logical_plan::create_udaf,
        physical_plan::expressions::AvgAccumulator,
    };
    use arrow::array::{
        Array, ArrayRef, DictionaryArray, Float32Array, Float64Array, Int16Array,
        Int32Array, Int64Array, Int8Array, LargeStringArray, StringArray, UInt16Array,
        UInt32Array, UInt64Array, UInt8Array,
    };
    use arrow::compute::add;
    use arrow::datatypes::*;
    use arrow::record_batch::RecordBatch;
    use async_trait::async_trait;
    use std::fs::File;
    use std::sync::Weak;
    use std::thread::{self, JoinHandle};
    use std::{io::prelude::*, sync::Mutex};
    use tempfile::TempDir;

    #[tokio::test]
    async fn shared_memory_and_disk_manager() {
        // Demonstrate the ability to share DiskManager and
        // MemoryManager between two different executions.
        let ctx1 = ExecutionContext::new();

        // configure with same memory / disk manager
        let memory_manager = ctx1.runtime_env().memory_manager.clone();
        let disk_manager = ctx1.runtime_env().disk_manager.clone();
        let config = ExecutionConfig::new()
            .with_existing_memory_manager(memory_manager.clone())
            .with_existing_disk_manager(disk_manager.clone());

        let ctx2 = ExecutionContext::with_config(config);

        assert!(std::ptr::eq(
            Arc::as_ptr(&memory_manager),
            Arc::as_ptr(&ctx1.runtime_env().memory_manager)
        ));
        assert!(std::ptr::eq(
            Arc::as_ptr(&memory_manager),
            Arc::as_ptr(&ctx2.runtime_env().memory_manager)
        ));

        assert!(std::ptr::eq(
            Arc::as_ptr(&disk_manager),
            Arc::as_ptr(&ctx1.runtime_env().disk_manager)
        ));
        assert!(std::ptr::eq(
            Arc::as_ptr(&disk_manager),
            Arc::as_ptr(&ctx2.runtime_env().disk_manager)
        ));
    }

    #[tokio::test]
    async fn create_variable_expr() -> Result<()> {
        let tmp_dir = TempDir::new()?;
        let partition_count = 4;
        let mut ctx = create_ctx(&tmp_dir, partition_count).await?;

        let variable_provider = test::variable::SystemVar::new();
        ctx.register_variable(VarType::System, Arc::new(variable_provider));
        let variable_provider = test::variable::UserDefinedVar::new();
        ctx.register_variable(VarType::UserDefined, Arc::new(variable_provider));

        let provider = test::create_table_dual();
        ctx.register_table("dual", provider)?;

        let results =
            plan_and_collect(&mut ctx, "SELECT @@version, @name FROM dual").await?;

        let expected = vec![
            "+----------------------+------------------------+",
            "| @@version            | @name                  |",
            "+----------------------+------------------------+",
            "| system-var-@@version | user-defined-var-@name |",
            "+----------------------+------------------------+",
        ];
        assert_batches_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn register_deregister() -> Result<()> {
        let tmp_dir = TempDir::new()?;
        let partition_count = 4;
        let mut ctx = create_ctx(&tmp_dir, partition_count).await?;

        let provider = test::create_table_dual();
        ctx.register_table("dual", provider)?;

        assert!(ctx.deregister_table("dual")?.is_some());
        assert!(ctx.deregister_table("dual")?.is_none());

        Ok(())
    }

    #[tokio::test]
    async fn sort() -> Result<()> {
        let results =
            execute("SELECT c1, c2 FROM test ORDER BY c1 DESC, c2 ASC", 4).await?;
        assert_eq!(results.len(), 1);

        let expected: Vec<&str> = vec![
            "+----+----+",
            "| c1 | c2 |",
            "+----+----+",
            "| 3  | 1  |",
            "| 3  | 2  |",
            "| 3  | 3  |",
            "| 3  | 4  |",
            "| 3  | 5  |",
            "| 3  | 6  |",
            "| 3  | 7  |",
            "| 3  | 8  |",
            "| 3  | 9  |",
            "| 3  | 10 |",
            "| 2  | 1  |",
            "| 2  | 2  |",
            "| 2  | 3  |",
            "| 2  | 4  |",
            "| 2  | 5  |",
            "| 2  | 6  |",
            "| 2  | 7  |",
            "| 2  | 8  |",
            "| 2  | 9  |",
            "| 2  | 10 |",
            "| 1  | 1  |",
            "| 1  | 2  |",
            "| 1  | 3  |",
            "| 1  | 4  |",
            "| 1  | 5  |",
            "| 1  | 6  |",
            "| 1  | 7  |",
            "| 1  | 8  |",
            "| 1  | 9  |",
            "| 1  | 10 |",
            "| 0  | 1  |",
            "| 0  | 2  |",
            "| 0  | 3  |",
            "| 0  | 4  |",
            "| 0  | 5  |",
            "| 0  | 6  |",
            "| 0  | 7  |",
            "| 0  | 8  |",
            "| 0  | 9  |",
            "| 0  | 10 |",
            "+----+----+",
        ];

        // Note it is important to NOT use assert_batches_sorted_eq
        // here as we are testing the sortedness of the output
        assert_batches_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn sort_empty() -> Result<()> {
        // The predicate on this query purposely generates no results
        let results = execute(
            "SELECT c1, c2 FROM test WHERE c1 > 100000 ORDER BY c1 DESC, c2 ASC",
            4,
        )
        .await
        .unwrap();
        assert_eq!(results.len(), 0);
        Ok(())
    }

    #[tokio::test]
    async fn left_join_using() -> Result<()> {
        let results = execute(
            "SELECT t1.c1, t2.c2 FROM test t1 JOIN test t2 USING (c2) ORDER BY t2.c2",
            1,
        )
        .await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+----+----+",
            "| c1 | c2 |",
            "+----+----+",
            "| 0  | 1  |",
            "| 0  | 2  |",
            "| 0  | 3  |",
            "| 0  | 4  |",
            "| 0  | 5  |",
            "| 0  | 6  |",
            "| 0  | 7  |",
            "| 0  | 8  |",
            "| 0  | 9  |",
            "| 0  | 10 |",
            "+----+----+",
        ];

        assert_batches_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn left_join_using_join_key_projection() -> Result<()> {
        let results = execute(
            "SELECT t1.c1, t1.c2, t2.c2 FROM test t1 JOIN test t2 USING (c2) ORDER BY t2.c2",
            1,
        )
            .await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+----+----+----+",
            "| c1 | c2 | c2 |",
            "+----+----+----+",
            "| 0  | 1  | 1  |",
            "| 0  | 2  | 2  |",
            "| 0  | 3  | 3  |",
            "| 0  | 4  | 4  |",
            "| 0  | 5  | 5  |",
            "| 0  | 6  | 6  |",
            "| 0  | 7  | 7  |",
            "| 0  | 8  | 8  |",
            "| 0  | 9  | 9  |",
            "| 0  | 10 | 10 |",
            "+----+----+----+",
        ];

        assert_batches_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn left_join() -> Result<()> {
        let results = execute(
            "SELECT t1.c1, t1.c2, t2.c2 FROM test t1 JOIN test t2 ON t1.c2 = t2.c2 ORDER BY t1.c2",
            1,
        )
            .await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+----+----+----+",
            "| c1 | c2 | c2 |",
            "+----+----+----+",
            "| 0  | 1  | 1  |",
            "| 0  | 2  | 2  |",
            "| 0  | 3  | 3  |",
            "| 0  | 4  | 4  |",
            "| 0  | 5  | 5  |",
            "| 0  | 6  | 6  |",
            "| 0  | 7  | 7  |",
            "| 0  | 8  | 8  |",
            "| 0  | 9  | 9  |",
            "| 0  | 10 | 10 |",
            "+----+----+----+",
        ];

        assert_batches_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn window() -> Result<()> {
        let results = execute(
            "SELECT \
            c1, \
            c2, \
            SUM(c2) OVER (), \
            COUNT(c2) OVER (), \
            MAX(c2) OVER (), \
            MIN(c2) OVER (), \
            AVG(c2) OVER () \
            FROM test \
            ORDER BY c1, c2 \
            LIMIT 5",
            4,
        )
        .await?;
        // result in one batch, although e.g. having 2 batches do not change
        // result semantics, having a len=1 assertion upfront keeps surprises
        // at bay
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+----+----+--------------+----------------+--------------+--------------+--------------+",
            "| c1 | c2 | SUM(test.c2) | COUNT(test.c2) | MAX(test.c2) | MIN(test.c2) | AVG(test.c2) |",
            "+----+----+--------------+----------------+--------------+--------------+--------------+",
            "| 0  | 1  | 220          | 40             | 10           | 1            | 5.5          |",
            "| 0  | 2  | 220          | 40             | 10           | 1            | 5.5          |",
            "| 0  | 3  | 220          | 40             | 10           | 1            | 5.5          |",
            "| 0  | 4  | 220          | 40             | 10           | 1            | 5.5          |",
            "| 0  | 5  | 220          | 40             | 10           | 1            | 5.5          |",
            "+----+----+--------------+----------------+--------------+--------------+--------------+",
        ];

        // window function shall respect ordering
        assert_batches_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn window_order_by() -> Result<()> {
        let results = execute(
            "SELECT \
            c1, \
            c2, \
            ROW_NUMBER() OVER (ORDER BY c1, c2), \
            FIRST_VALUE(c2) OVER (ORDER BY c1, c2), \
            LAST_VALUE(c2) OVER (ORDER BY c1, c2), \
            NTH_VALUE(c2, 2) OVER (ORDER BY c1, c2), \
            SUM(c2) OVER (ORDER BY c1, c2), \
            COUNT(c2) OVER (ORDER BY c1, c2), \
            MAX(c2) OVER (ORDER BY c1, c2), \
            MIN(c2) OVER (ORDER BY c1, c2), \
            AVG(c2) OVER (ORDER BY c1, c2) \
            FROM test \
            ORDER BY c1, c2 \
            LIMIT 5",
            4,
        )
        .await?;
        // result in one batch, although e.g. having 2 batches do not change
        // result semantics, having a len=1 assertion upfront keeps surprises
        // at bay
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+----+----+--------------+----------------------+---------------------+-----------------------------+--------------+----------------+--------------+--------------+--------------+",
            "| c1 | c2 | ROW_NUMBER() | FIRST_VALUE(test.c2) | LAST_VALUE(test.c2) | NTH_VALUE(test.c2,Int64(2)) | SUM(test.c2) | COUNT(test.c2) | MAX(test.c2) | MIN(test.c2) | AVG(test.c2) |",
            "+----+----+--------------+----------------------+---------------------+-----------------------------+--------------+----------------+--------------+--------------+--------------+",
            "| 0  | 1  | 1            | 1                    | 1                   |                             | 1            | 1              | 1            | 1            | 1            |",
            "| 0  | 2  | 2            | 1                    | 2                   | 2                           | 3            | 2              | 2            | 1            | 1.5          |",
            "| 0  | 3  | 3            | 1                    | 3                   | 2                           | 6            | 3              | 3            | 1            | 2            |",
            "| 0  | 4  | 4            | 1                    | 4                   | 2                           | 10           | 4              | 4            | 1            | 2.5          |",
            "| 0  | 5  | 5            | 1                    | 5                   | 2                           | 15           | 5              | 5            | 1            | 3            |",
            "+----+----+--------------+----------------------+---------------------+-----------------------------+--------------+----------------+--------------+--------------+--------------+",
        ];

        // window function shall respect ordering
        assert_batches_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn window_partition_by() -> Result<()> {
        let results = execute(
            "SELECT \
            c1, \
            c2, \
            SUM(c2) OVER (PARTITION BY c2), \
            COUNT(c2) OVER (PARTITION BY c2), \
            MAX(c2) OVER (PARTITION BY c2), \
            MIN(c2) OVER (PARTITION BY c2), \
            AVG(c2) OVER (PARTITION BY c2) \
            FROM test \
            ORDER BY c1, c2 \
            LIMIT 5",
            4,
        )
        .await?;

        let expected = vec![
            "+----+----+--------------+----------------+--------------+--------------+--------------+",
            "| c1 | c2 | SUM(test.c2) | COUNT(test.c2) | MAX(test.c2) | MIN(test.c2) | AVG(test.c2) |",
            "+----+----+--------------+----------------+--------------+--------------+--------------+",
            "| 0  | 1  | 4            | 4              | 1            | 1            | 1            |",
            "| 0  | 2  | 8            | 4              | 2            | 2            | 2            |",
            "| 0  | 3  | 12           | 4              | 3            | 3            | 3            |",
            "| 0  | 4  | 16           | 4              | 4            | 4            | 4            |",
            "| 0  | 5  | 20           | 4              | 5            | 5            | 5            |",
            "+----+----+--------------+----------------+--------------+--------------+--------------+",
        ];

        // window function shall respect ordering
        assert_batches_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn window_partition_by_order_by() -> Result<()> {
        let results = execute(
            "SELECT \
            c1, \
            c2, \
            ROW_NUMBER() OVER (PARTITION BY c2 ORDER BY c1), \
            FIRST_VALUE(c2 + c1) OVER (PARTITION BY c2 ORDER BY c1), \
            LAST_VALUE(c2 + c1) OVER (PARTITION BY c2 ORDER BY c1), \
            NTH_VALUE(c2 + c1, 1) OVER (PARTITION BY c2 ORDER BY c1), \
            SUM(c2) OVER (PARTITION BY c2 ORDER BY c1), \
            COUNT(c2) OVER (PARTITION BY c2 ORDER BY c1), \
            MAX(c2) OVER (PARTITION BY c2 ORDER BY c1), \
            MIN(c2) OVER (PARTITION BY c2 ORDER BY c1), \
            AVG(c2) OVER (PARTITION BY c2 ORDER BY c1) \
            FROM test \
            ORDER BY c1, c2 \
            LIMIT 5",
            4,
        )
        .await?;

        let expected = vec![
            "+----+----+--------------+--------------------------------+-------------------------------+---------------------------------------+--------------+----------------+--------------+--------------+--------------+",
            "| c1 | c2 | ROW_NUMBER() | FIRST_VALUE(test.c2 + test.c1) | LAST_VALUE(test.c2 + test.c1) | NTH_VALUE(test.c2 + test.c1,Int64(1)) | SUM(test.c2) | COUNT(test.c2) | MAX(test.c2) | MIN(test.c2) | AVG(test.c2) |",
            "+----+----+--------------+--------------------------------+-------------------------------+---------------------------------------+--------------+----------------+--------------+--------------+--------------+",
            "| 0  | 1  | 1            | 1                              | 1                             | 1                                     | 1            | 1              | 1            | 1            | 1            |",
            "| 0  | 2  | 1            | 2                              | 2                             | 2                                     | 2            | 1              | 2            | 2            | 2            |",
            "| 0  | 3  | 1            | 3                              | 3                             | 3                                     | 3            | 1              | 3            | 3            | 3            |",
            "| 0  | 4  | 1            | 4                              | 4                             | 4                                     | 4            | 1              | 4            | 4            | 4            |",
            "| 0  | 5  | 1            | 5                              | 5                             | 5                                     | 5            | 1              | 5            | 5            | 5            |",
            "+----+----+--------------+--------------------------------+-------------------------------+---------------------------------------+--------------+----------------+--------------+--------------+--------------+",
        ];

        // window function shall respect ordering
        assert_batches_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn aggregate_decimal_min() -> Result<()> {
        let mut ctx = ExecutionContext::new();
        // the data type of c1 is decimal(10,3)
        ctx.register_table("d_table", test::table_with_decimal())
            .unwrap();
        let result = plan_and_collect(&mut ctx, "select min(c1) from d_table")
            .await
            .unwrap();
        let expected = vec![
            "+-----------------+",
            "| MIN(d_table.c1) |",
            "+-----------------+",
            "| -100.009        |",
            "+-----------------+",
        ];
        assert_eq!(
            &DataType::Decimal(10, 3),
            result[0].schema().field(0).data_type()
        );
        assert_batches_sorted_eq!(expected, &result);
        Ok(())
    }

    #[tokio::test]
    async fn aggregate_decimal_max() -> Result<()> {
        let mut ctx = ExecutionContext::new();
        // the data type of c1 is decimal(10,3)
        ctx.register_table("d_table", test::table_with_decimal())
            .unwrap();

        let result = plan_and_collect(&mut ctx, "select max(c1) from d_table")
            .await
            .unwrap();
        let expected = vec![
            "+-----------------+",
            "| MAX(d_table.c1) |",
            "+-----------------+",
            "| 110.009         |",
            "+-----------------+",
        ];
        assert_eq!(
            &DataType::Decimal(10, 3),
            result[0].schema().field(0).data_type()
        );
        assert_batches_sorted_eq!(expected, &result);
        Ok(())
    }

    #[tokio::test]
    async fn aggregate_decimal_sum() -> Result<()> {
        let mut ctx = ExecutionContext::new();
        // the data type of c1 is decimal(10,3)
        ctx.register_table("d_table", test::table_with_decimal())
            .unwrap();
        let result = plan_and_collect(&mut ctx, "select sum(c1) from d_table")
            .await
            .unwrap();
        let expected = vec![
            "+-----------------+",
            "| SUM(d_table.c1) |",
            "+-----------------+",
            "| 100.000         |",
            "+-----------------+",
        ];
        assert_eq!(
            &DataType::Decimal(20, 3),
            result[0].schema().field(0).data_type()
        );
        assert_batches_sorted_eq!(expected, &result);
        Ok(())
    }

    #[tokio::test]
    async fn aggregate_decimal_avg() -> Result<()> {
        let mut ctx = ExecutionContext::new();
        // the data type of c1 is decimal(10,3)
        ctx.register_table("d_table", test::table_with_decimal())
            .unwrap();
        let result = plan_and_collect(&mut ctx, "select avg(c1) from d_table")
            .await
            .unwrap();
        let expected = vec![
            "+-----------------+",
            "| AVG(d_table.c1) |",
            "+-----------------+",
            "| 5.0000000       |",
            "+-----------------+",
        ];
        assert_eq!(
            &DataType::Decimal(14, 7),
            result[0].schema().field(0).data_type()
        );
        assert_batches_sorted_eq!(expected, &result);
        Ok(())
    }

    #[tokio::test]
    async fn aggregate() -> Result<()> {
        let results = execute("SELECT SUM(c1), SUM(c2) FROM test", 4).await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+--------------+--------------+",
            "| SUM(test.c1) | SUM(test.c2) |",
            "+--------------+--------------+",
            "| 60           | 220          |",
            "+--------------+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_empty() -> Result<()> {
        // The predicate on this query purposely generates no results
        let results = execute("SELECT SUM(c1), SUM(c2) FROM test where c1 > 100000", 4)
            .await
            .unwrap();

        assert_eq!(results.len(), 1);

        let expected = vec![
            "+--------------+--------------+",
            "| SUM(test.c1) | SUM(test.c2) |",
            "+--------------+--------------+",
            "|              |              |",
            "+--------------+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_avg() -> Result<()> {
        let results = execute("SELECT AVG(c1), AVG(c2) FROM test", 4).await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+--------------+--------------+",
            "| AVG(test.c1) | AVG(test.c2) |",
            "+--------------+--------------+",
            "| 1.5          | 5.5          |",
            "+--------------+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_max() -> Result<()> {
        let results = execute("SELECT MAX(c1), MAX(c2) FROM test", 4).await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+--------------+--------------+",
            "| MAX(test.c1) | MAX(test.c2) |",
            "+--------------+--------------+",
            "| 3            | 10           |",
            "+--------------+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_min() -> Result<()> {
        let results = execute("SELECT MIN(c1), MIN(c2) FROM test", 4).await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+--------------+--------------+",
            "| MIN(test.c1) | MIN(test.c2) |",
            "+--------------+--------------+",
            "| 0            | 1            |",
            "+--------------+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_grouped() -> Result<()> {
        let results = execute("SELECT c1, SUM(c2) FROM test GROUP BY c1", 4).await?;

        let expected = vec![
            "+----+--------------+",
            "| c1 | SUM(test.c2) |",
            "+----+--------------+",
            "| 0  | 55           |",
            "| 1  | 55           |",
            "| 2  | 55           |",
            "| 3  | 55           |",
            "+----+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_grouped_avg() -> Result<()> {
        let results = execute("SELECT c1, AVG(c2) FROM test GROUP BY c1", 4).await?;

        let expected = vec![
            "+----+--------------+",
            "| c1 | AVG(test.c2) |",
            "+----+--------------+",
            "| 0  | 5.5          |",
            "| 1  | 5.5          |",
            "| 2  | 5.5          |",
            "| 3  | 5.5          |",
            "+----+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn boolean_literal() -> Result<()> {
        let results =
            execute("SELECT c1, c3 FROM test WHERE c1 > 2 AND c3 = true", 4).await?;

        let expected = vec![
            "+----+------+",
            "| c1 | c3   |",
            "+----+------+",
            "| 3  | true |",
            "| 3  | true |",
            "| 3  | true |",
            "| 3  | true |",
            "| 3  | true |",
            "+----+------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_grouped_empty() -> Result<()> {
        let results =
            execute("SELECT c1, AVG(c2) FROM test WHERE c1 = 123 GROUP BY c1", 4).await?;

        let expected = vec![
            "+----+--------------+",
            "| c1 | AVG(test.c2) |",
            "+----+--------------+",
            "+----+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_grouped_max() -> Result<()> {
        let results = execute("SELECT c1, MAX(c2) FROM test GROUP BY c1", 4).await?;

        let expected = vec![
            "+----+--------------+",
            "| c1 | MAX(test.c2) |",
            "+----+--------------+",
            "| 0  | 10           |",
            "| 1  | 10           |",
            "| 2  | 10           |",
            "| 3  | 10           |",
            "+----+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_grouped_min() -> Result<()> {
        let results = execute("SELECT c1, MIN(c2) FROM test GROUP BY c1", 4).await?;

        let expected = vec![
            "+----+--------------+",
            "| c1 | MIN(test.c2) |",
            "+----+--------------+",
            "| 0  | 1            |",
            "| 1  | 1            |",
            "| 2  | 1            |",
            "| 3  | 1            |",
            "+----+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_avg_add() -> Result<()> {
        let results = execute(
            "SELECT AVG(c1), AVG(c1) + 1, AVG(c1) + 2, 1 + AVG(c1) FROM test",
            4,
        )
        .await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+--------------+----------------------------+----------------------------+----------------------------+",
            "| AVG(test.c1) | AVG(test.c1) Plus Int64(1) | AVG(test.c1) Plus Int64(2) | Int64(1) Plus AVG(test.c1) |",
            "+--------------+----------------------------+----------------------------+----------------------------+",
            "| 1.5          | 2.5                        | 3.5                        | 2.5                        |",
            "+--------------+----------------------------+----------------------------+----------------------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn join_partitioned() -> Result<()> {
        // self join on partition id (workaround for duplicate column name)
        let results = execute(
            "SELECT 1 FROM test JOIN (SELECT c1 AS id1 FROM test) AS a ON c1=id1",
            4,
        )
        .await?;

        assert_eq!(
            results.iter().map(|b| b.num_rows()).sum::<usize>(),
            4 * 10 * 10
        );

        Ok(())
    }

    #[tokio::test]
    async fn count_basic() -> Result<()> {
        let results = execute("SELECT COUNT(c1), COUNT(c2) FROM test", 1).await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+----------------+----------------+",
            "| COUNT(test.c1) | COUNT(test.c2) |",
            "+----------------+----------------+",
            "| 10             | 10             |",
            "+----------------+----------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn count_partitioned() -> Result<()> {
        let results = execute("SELECT COUNT(c1), COUNT(c2) FROM test", 4).await?;
        assert_eq!(results.len(), 1);

        let expected = vec![
            "+----------------+----------------+",
            "| COUNT(test.c1) | COUNT(test.c2) |",
            "+----------------+----------------+",
            "| 40             | 40             |",
            "+----------------+----------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn count_aggregated() -> Result<()> {
        let results = execute("SELECT c1, COUNT(c2) FROM test GROUP BY c1", 4).await?;

        let expected = vec![
            "+----+----------------+",
            "| c1 | COUNT(test.c2) |",
            "+----+----------------+",
            "| 0  | 10             |",
            "| 1  | 10             |",
            "| 2  | 10             |",
            "| 3  | 10             |",
            "+----+----------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);
        Ok(())
    }

    #[tokio::test]
    async fn group_by_date_trunc() -> Result<()> {
        let tmp_dir = TempDir::new()?;
        let mut ctx = ExecutionContext::new();
        let schema = Arc::new(Schema::new(vec![
            Field::new("c2", DataType::UInt64, false),
            Field::new(
                "t1",
                DataType::Timestamp(TimeUnit::Microsecond, None),
                false,
            ),
        ]));

        // generate a partitioned file
        for partition in 0..4 {
            let filename = format!("partition-{}.{}", partition, "csv");
            let file_path = tmp_dir.path().join(&filename);
            let mut file = File::create(file_path)?;

            // generate some data
            for i in 0..10 {
                let data = format!("{},2020-12-{}T00:00:00.000Z\n", i, i + 10);
                file.write_all(data.as_bytes())?;
            }
        }

        ctx.register_csv(
            "test",
            tmp_dir.path().to_str().unwrap(),
            CsvReadOptions::new().schema(&schema).has_header(false),
        )
        .await?;

        let results = plan_and_collect(
            &mut ctx,
            "SELECT date_trunc('week', t1) as week, SUM(c2) FROM test GROUP BY date_trunc('week', t1)",
        ).await?;

        let expected = vec![
            "+---------------------+--------------+",
            "| week                | SUM(test.c2) |",
            "+---------------------+--------------+",
            "| 2020-12-07 00:00:00 | 24           |",
            "| 2020-12-14 00:00:00 | 156          |",
            "+---------------------+--------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn group_by_largeutf8() {
        {
            let mut ctx = ExecutionContext::new();

            // input data looks like:
            // A, 1
            // B, 2
            // A, 2
            // A, 4
            // C, 1
            // A, 1

            let str_array: LargeStringArray = vec!["A", "B", "A", "A", "C", "A"]
                .into_iter()
                .map(Some)
                .collect();
            let str_array = Arc::new(str_array);

            let val_array: Int64Array = vec![1, 2, 2, 4, 1, 1].into();
            let val_array = Arc::new(val_array);

            let schema = Arc::new(Schema::new(vec![
                Field::new("str", str_array.data_type().clone(), false),
                Field::new("val", val_array.data_type().clone(), false),
            ]));

            let batch =
                RecordBatch::try_new(schema.clone(), vec![str_array, val_array]).unwrap();

            let provider = MemTable::try_new(schema.clone(), vec![vec![batch]]).unwrap();
            ctx.register_table("t", Arc::new(provider)).unwrap();

            let results =
                plan_and_collect(&mut ctx, "SELECT str, count(val) FROM t GROUP BY str")
                    .await
                    .expect("ran plan correctly");

            let expected = vec![
                "+-----+--------------+",
                "| str | COUNT(t.val) |",
                "+-----+--------------+",
                "| A   | 4            |",
                "| B   | 1            |",
                "| C   | 1            |",
                "+-----+--------------+",
            ];
            assert_batches_sorted_eq!(expected, &results);
        }
    }

    #[tokio::test]
    async fn unprojected_filter() {
        let mut ctx = ExecutionContext::new();
        let df = ctx
            .read_table(test::table_with_sequence(1, 3).unwrap())
            .unwrap();

        let df = df
            .select(vec![binary_expr(col("i"), Operator::Plus, col("i"))])
            .unwrap()
            .filter(col("i").gt(lit(2)))
            .unwrap();
        let results = df.collect().await.unwrap();

        let expected = vec![
            "+--------------------------+",
            "| ?table?.i Plus ?table?.i |",
            "+--------------------------+",
            "| 6                        |",
            "+--------------------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);
    }

    #[tokio::test]
    async fn group_by_dictionary() {
        async fn run_test_case<K: ArrowDictionaryKeyType>() {
            let mut ctx = ExecutionContext::new();

            // input data looks like:
            // A, 1
            // B, 2
            // A, 2
            // A, 4
            // C, 1
            // A, 1

            let dict_array: DictionaryArray<K> =
                vec!["A", "B", "A", "A", "C", "A"].into_iter().collect();
            let dict_array = Arc::new(dict_array);

            let val_array: Int64Array = vec![1, 2, 2, 4, 1, 1].into();
            let val_array = Arc::new(val_array);

            let schema = Arc::new(Schema::new(vec![
                Field::new("dict", dict_array.data_type().clone(), false),
                Field::new("val", val_array.data_type().clone(), false),
            ]));

            let batch = RecordBatch::try_new(schema.clone(), vec![dict_array, val_array])
                .unwrap();

            let provider = MemTable::try_new(schema.clone(), vec![vec![batch]]).unwrap();
            ctx.register_table("t", Arc::new(provider)).unwrap();

            let results = plan_and_collect(
                &mut ctx,
                "SELECT dict, count(val) FROM t GROUP BY dict",
            )
            .await
            .expect("ran plan correctly");

            let expected = vec![
                "+------+--------------+",
                "| dict | COUNT(t.val) |",
                "+------+--------------+",
                "| A    | 4            |",
                "| B    | 1            |",
                "| C    | 1            |",
                "+------+--------------+",
            ];
            assert_batches_sorted_eq!(expected, &results);

            // Now, use dict as an aggregate
            let results =
                plan_and_collect(&mut ctx, "SELECT val, count(dict) FROM t GROUP BY val")
                    .await
                    .expect("ran plan correctly");

            let expected = vec![
                "+-----+---------------+",
                "| val | COUNT(t.dict) |",
                "+-----+---------------+",
                "| 1   | 3             |",
                "| 2   | 2             |",
                "| 4   | 1             |",
                "+-----+---------------+",
            ];
            assert_batches_sorted_eq!(expected, &results);

            // Now, use dict as an aggregate
            let results = plan_and_collect(
                &mut ctx,
                "SELECT val, count(distinct dict) FROM t GROUP BY val",
            )
            .await
            .expect("ran plan correctly");

            let expected = vec![
                "+-----+------------------------+",
                "| val | COUNT(DISTINCT t.dict) |",
                "+-----+------------------------+",
                "| 1   | 2                      |",
                "| 2   | 2                      |",
                "| 4   | 1                      |",
                "+-----+------------------------+",
            ];
            assert_batches_sorted_eq!(expected, &results);
        }

        run_test_case::<Int8Type>().await;
        run_test_case::<Int16Type>().await;
        run_test_case::<Int32Type>().await;
        run_test_case::<Int64Type>().await;
        run_test_case::<UInt8Type>().await;
        run_test_case::<UInt16Type>().await;
        run_test_case::<UInt32Type>().await;
        run_test_case::<UInt64Type>().await;
    }

    async fn run_count_distinct_integers_aggregated_scenario(
        partitions: Vec<Vec<(&str, u64)>>,
    ) -> Result<Vec<RecordBatch>> {
        let tmp_dir = TempDir::new()?;
        let mut ctx = ExecutionContext::new();
        let schema = Arc::new(Schema::new(vec![
            Field::new("c_group", DataType::Utf8, false),
            Field::new("c_int8", DataType::Int8, false),
            Field::new("c_int16", DataType::Int16, false),
            Field::new("c_int32", DataType::Int32, false),
            Field::new("c_int64", DataType::Int64, false),
            Field::new("c_uint8", DataType::UInt8, false),
            Field::new("c_uint16", DataType::UInt16, false),
            Field::new("c_uint32", DataType::UInt32, false),
            Field::new("c_uint64", DataType::UInt64, false),
        ]));

        for (i, partition) in partitions.iter().enumerate() {
            let filename = format!("partition-{}.csv", i);
            let file_path = tmp_dir.path().join(&filename);
            let mut file = File::create(file_path)?;
            for row in partition {
                let row_str = format!(
                    "{},{}\n",
                    row.0,
                    // Populate values for each of the integer fields in the
                    // schema.
                    (0..8)
                        .map(|_| { row.1.to_string() })
                        .collect::<Vec<_>>()
                        .join(","),
                );
                file.write_all(row_str.as_bytes())?;
            }
        }
        ctx.register_csv(
            "test",
            tmp_dir.path().to_str().unwrap(),
            CsvReadOptions::new().schema(&schema).has_header(false),
        )
        .await?;

        let results = plan_and_collect(
            &mut ctx,
            "
              SELECT
                c_group,
                COUNT(c_uint64),
                COUNT(DISTINCT c_int8),
                COUNT(DISTINCT c_int16),
                COUNT(DISTINCT c_int32),
                COUNT(DISTINCT c_int64),
                COUNT(DISTINCT c_uint8),
                COUNT(DISTINCT c_uint16),
                COUNT(DISTINCT c_uint32),
                COUNT(DISTINCT c_uint64)
              FROM test
              GROUP BY c_group
            ",
        )
        .await?;

        Ok(results)
    }

    #[tokio::test]
    async fn count_distinct_integers_aggregated_single_partition() -> Result<()> {
        let partitions = vec![
            // The first member of each tuple will be the value for the
            // `c_group` column, and the second member will be the value for
            // each of the int/uint fields.
            vec![
                ("a", 1),
                ("a", 1),
                ("a", 2),
                ("b", 9),
                ("c", 9),
                ("c", 10),
                ("c", 9),
            ],
        ];

        let results = run_count_distinct_integers_aggregated_scenario(partitions).await?;

        let expected = vec![
            "+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+",
            "| c_group | COUNT(test.c_uint64) | COUNT(DISTINCT test.c_int8) | COUNT(DISTINCT test.c_int16) | COUNT(DISTINCT test.c_int32) | COUNT(DISTINCT test.c_int64) | COUNT(DISTINCT test.c_uint8) | COUNT(DISTINCT test.c_uint16) | COUNT(DISTINCT test.c_uint32) | COUNT(DISTINCT test.c_uint64) |",
            "+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+",
            "| a       | 3                    | 2                           | 2                            | 2                            | 2                            | 2                            | 2                             | 2                             | 2                             |",
            "| b       | 1                    | 1                           | 1                            | 1                            | 1                            | 1                            | 1                             | 1                             | 1                             |",
            "| c       | 3                    | 2                           | 2                            | 2                            | 2                            | 2                            | 2                             | 2                             | 2                             |",
            "+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn count_distinct_integers_aggregated_multiple_partitions() -> Result<()> {
        let partitions = vec![
            // The first member of each tuple will be the value for the
            // `c_group` column, and the second member will be the value for
            // each of the int/uint fields.
            vec![("a", 1), ("a", 1), ("a", 2), ("b", 9), ("c", 9)],
            vec![("a", 1), ("a", 3), ("b", 8), ("b", 9), ("b", 10), ("b", 11)],
        ];

        let results = run_count_distinct_integers_aggregated_scenario(partitions).await?;

        let expected = vec![
            "+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+",
            "| c_group | COUNT(test.c_uint64) | COUNT(DISTINCT test.c_int8) | COUNT(DISTINCT test.c_int16) | COUNT(DISTINCT test.c_int32) | COUNT(DISTINCT test.c_int64) | COUNT(DISTINCT test.c_uint8) | COUNT(DISTINCT test.c_uint16) | COUNT(DISTINCT test.c_uint32) | COUNT(DISTINCT test.c_uint64) |",
            "+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+",
            "| a       | 5                    | 3                           | 3                            | 3                            | 3                            | 3                            | 3                             | 3                             | 3                             |",
            "| b       | 5                    | 4                           | 4                            | 4                            | 4                            | 4                            | 4                             | 4                             | 4                             |",
            "| c       | 1                    | 1                           | 1                            | 1                            | 1                            | 1                            | 1                             | 1                             | 1                             |",
            "+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+",
        ];
        assert_batches_sorted_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn aggregate_with_alias() -> Result<()> {
        let tmp_dir = TempDir::new()?;
        let ctx = create_ctx(&tmp_dir, 1).await?;

        let schema = Arc::new(Schema::new(vec![
            Field::new("c1", DataType::Utf8, false),
            Field::new("c2", DataType::UInt32, false),
        ]));

        let plan = LogicalPlanBuilder::scan_empty(None, schema.as_ref(), None)?
            .aggregate(vec![col("c1")], vec![sum(col("c2"))])?
            .project(vec![col("c1"), sum(col("c2")).alias("total_salary")])?
            .build()?;

        let plan = ctx.optimize(&plan)?;

        let physical_plan = ctx.create_physical_plan(&Arc::new(plan)).await?;
        assert_eq!("c1", physical_plan.schema().field(0).name().as_str());
        assert_eq!(
            "total_salary",
            physical_plan.schema().field(1).name().as_str()
        );
        Ok(())
    }

    #[tokio::test]
    async fn limit() -> Result<()> {
        let tmp_dir = TempDir::new()?;
        let mut ctx = create_ctx(&tmp_dir, 1).await?;
        ctx.register_table("t", test::table_with_sequence(1, 1000).unwrap())
            .unwrap();

        let results =
            plan_and_collect(&mut ctx, "SELECT i FROM t ORDER BY i DESC limit 3")
                .await
                .unwrap();

        let expected = vec![
            "+------+", "| i    |", "+------+", "| 1000 |", "| 999  |", "| 998  |",
            "+------+",
        ];

        assert_batches_eq!(expected, &results);

        let results = plan_and_collect(&mut ctx, "SELECT i FROM t ORDER BY i limit 3")
            .await
            .unwrap();

        let expected = vec![
            "+---+", "| i |", "+---+", "| 1 |", "| 2 |", "| 3 |", "+---+",
        ];

        assert_batches_eq!(expected, &results);

        let results = plan_and_collect(&mut ctx, "SELECT i FROM t limit 3")
            .await
            .unwrap();

        // the actual rows are not guaranteed, so only check the count (should be 3)
        let num_rows: usize = results.into_iter().map(|b| b.num_rows()).sum();
        assert_eq!(num_rows, 3);

        Ok(())
    }

    #[tokio::test]
    async fn limit_multi_partitions() -> Result<()> {
        let tmp_dir = TempDir::new()?;
        let mut ctx = create_ctx(&tmp_dir, 1).await?;

        let partitions = vec![
            vec![test::make_partition(0)],
            vec![test::make_partition(1)],
            vec![test::make_partition(2)],
            vec![test::make_partition(3)],
            vec![test::make_partition(4)],
            vec![test::make_partition(5)],
        ];
        let schema = partitions[0][0].schema();
        let provider = Arc::new(MemTable::try_new(schema, partitions).unwrap());

        ctx.register_table("t", provider).unwrap();

        // select all rows
        let results = plan_and_collect(&mut ctx, "SELECT i FROM t").await.unwrap();

        let num_rows: usize = results.into_iter().map(|b| b.num_rows()).sum();
        assert_eq!(num_rows, 15);

        for limit in 1..10 {
            let query = format!("SELECT i FROM t limit {}", limit);
            let results = plan_and_collect(&mut ctx, &query).await.unwrap();

            let num_rows: usize = results.into_iter().map(|b| b.num_rows()).sum();
            assert_eq!(num_rows, limit, "mismatch with query {}", query);
        }

        Ok(())
    }

    #[tokio::test]
    async fn case_sensitive_identifiers_functions() {
        let mut ctx = ExecutionContext::new();
        ctx.register_table("t", test::table_with_sequence(1, 1).unwrap())
            .unwrap();

        let expected = vec![
            "+-----------+",
            "| sqrt(t.i) |",
            "+-----------+",
            "| 1         |",
            "+-----------+",
        ];

        let results = plan_and_collect(&mut ctx, "SELECT sqrt(i) FROM t")
            .await
            .unwrap();

        assert_batches_sorted_eq!(expected, &results);

        let results = plan_and_collect(&mut ctx, "SELECT SQRT(i) FROM t")
            .await
            .unwrap();
        assert_batches_sorted_eq!(expected, &results);

        // Using double quotes allows specifying the function name with capitalization
        let err = plan_and_collect(&mut ctx, "SELECT \"SQRT\"(i) FROM t")
            .await
            .unwrap_err();
        assert_eq!(
            err.to_string(),
            "Error during planning: Invalid function 'SQRT'"
        );

        let results = plan_and_collect(&mut ctx, "SELECT \"sqrt\"(i) FROM t")
            .await
            .unwrap();
        assert_batches_sorted_eq!(expected, &results);
    }

    #[tokio::test]
    async fn case_builtin_math_expression() {
        let mut ctx = ExecutionContext::new();

        let type_values = vec![
            (
                DataType::Int8,
                Arc::new(Int8Array::from_slice(&[1])) as ArrayRef,
            ),
            (
                DataType::Int16,
                Arc::new(Int16Array::from_slice(&[1])) as ArrayRef,
            ),
            (
                DataType::Int32,
                Arc::new(Int32Array::from_slice(&[1])) as ArrayRef,
            ),
            (
                DataType::Int64,
                Arc::new(Int64Array::from_slice(&[1])) as ArrayRef,
            ),
            (
                DataType::UInt8,
                Arc::new(UInt8Array::from_slice(&[1])) as ArrayRef,
            ),
            (
                DataType::UInt16,
                Arc::new(UInt16Array::from_slice(&[1])) as ArrayRef,
            ),
            (
                DataType::UInt32,
                Arc::new(UInt32Array::from_slice(&[1])) as ArrayRef,
            ),
            (
                DataType::UInt64,
                Arc::new(UInt64Array::from_slice(&[1])) as ArrayRef,
            ),
            (
                DataType::Float32,
                Arc::new(Float32Array::from_slice(&[1.0_f32])) as ArrayRef,
            ),
            (
                DataType::Float64,
                Arc::new(Float64Array::from_slice(&[1.0_f64])) as ArrayRef,
            ),
        ];

        for (data_type, array) in type_values.iter() {
            let schema =
                Arc::new(Schema::new(vec![Field::new("v", data_type.clone(), false)]));
            let batch =
                RecordBatch::try_new(schema.clone(), vec![array.clone()]).unwrap();
            let provider = MemTable::try_new(schema, vec![vec![batch]]).unwrap();
            ctx.deregister_table("t").unwrap();
            ctx.register_table("t", Arc::new(provider)).unwrap();
            let expected = vec![
                "+-----------+",
                "| sqrt(t.v) |",
                "+-----------+",
                "| 1         |",
                "+-----------+",
            ];
            let results = plan_and_collect(&mut ctx, "SELECT sqrt(v) FROM t")
                .await
                .unwrap();

            assert_batches_sorted_eq!(expected, &results);
        }
    }

    #[tokio::test]
    async fn case_sensitive_identifiers_user_defined_functions() -> Result<()> {
        let mut ctx = ExecutionContext::new();
        ctx.register_table("t", test::table_with_sequence(1, 1).unwrap())
            .unwrap();

        let myfunc = |args: &[ArrayRef]| Ok(Arc::clone(&args[0]));
        let myfunc = make_scalar_function(myfunc);

        ctx.register_udf(create_udf(
            "MY_FUNC",
            vec![DataType::Int32],
            Arc::new(DataType::Int32),
            Volatility::Immutable,
            myfunc,
        ));

        // doesn't work as it was registered with non lowercase
        let err = plan_and_collect(&mut ctx, "SELECT MY_FUNC(i) FROM t")
            .await
            .unwrap_err();
        assert_eq!(
            err.to_string(),
            "Error during planning: Invalid function \'my_func\'"
        );

        // Can call it if you put quotes
        let result = plan_and_collect(&mut ctx, "SELECT \"MY_FUNC\"(i) FROM t").await?;

        let expected = vec![
            "+--------------+",
            "| MY_FUNC(t.i) |",
            "+--------------+",
            "| 1            |",
            "+--------------+",
        ];
        assert_batches_eq!(expected, &result);

        Ok(())
    }

    #[tokio::test]
    async fn case_sensitive_identifiers_aggregates() {
        let mut ctx = ExecutionContext::new();
        ctx.register_table("t", test::table_with_sequence(1, 1).unwrap())
            .unwrap();

        let expected = vec![
            "+----------+",
            "| MAX(t.i) |",
            "+----------+",
            "| 1        |",
            "+----------+",
        ];

        let results = plan_and_collect(&mut ctx, "SELECT max(i) FROM t")
            .await
            .unwrap();

        assert_batches_sorted_eq!(expected, &results);

        let results = plan_and_collect(&mut ctx, "SELECT MAX(i) FROM t")
            .await
            .unwrap();
        assert_batches_sorted_eq!(expected, &results);

        // Using double quotes allows specifying the function name with capitalization
        let err = plan_and_collect(&mut ctx, "SELECT \"MAX\"(i) FROM t")
            .await
            .unwrap_err();
        assert_eq!(
            err.to_string(),
            "Error during planning: Invalid function 'MAX'"
        );

        let results = plan_and_collect(&mut ctx, "SELECT \"max\"(i) FROM t")
            .await
            .unwrap();
        assert_batches_sorted_eq!(expected, &results);
    }

    #[tokio::test]
    async fn case_sensitive_identifiers_user_defined_aggregates() -> Result<()> {
        let mut ctx = ExecutionContext::new();
        ctx.register_table("t", test::table_with_sequence(1, 1).unwrap())
            .unwrap();

        // Note capitalizaton
        let my_avg = create_udaf(
            "MY_AVG",
            DataType::Float64,
            Arc::new(DataType::Float64),
            Volatility::Immutable,
            Arc::new(|| Ok(Box::new(AvgAccumulator::try_new(&DataType::Float64)?))),
            Arc::new(vec![DataType::UInt64, DataType::Float64]),
        );

        ctx.register_udaf(my_avg);

        // doesn't work as it was registered as non lowercase
        let err = plan_and_collect(&mut ctx, "SELECT MY_AVG(i) FROM t")
            .await
            .unwrap_err();
        assert_eq!(
            err.to_string(),
            "Error during planning: Invalid function \'my_avg\'"
        );

        // Can call it if you put quotes
        let result = plan_and_collect(&mut ctx, "SELECT \"MY_AVG\"(i) FROM t").await?;

        let expected = vec![
            "+-------------+",
            "| MY_AVG(t.i) |",
            "+-------------+",
            "| 1           |",
            "+-------------+",
        ];
        assert_batches_eq!(expected, &result);

        Ok(())
    }

    #[tokio::test]
    async fn write_csv_results() -> Result<()> {
        // create partitioned input file and context
        let tmp_dir = TempDir::new()?;
        let mut ctx = create_ctx(&tmp_dir, 4).await?;

        // execute a simple query and write the results to CSV
        let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out";
        write_csv(&mut ctx, "SELECT c1, c2 FROM test", &out_dir).await?;

        // create a new context and verify that the results were saved to a partitioned csv file
        let mut ctx = ExecutionContext::new();

        let schema = Arc::new(Schema::new(vec![
            Field::new("c1", DataType::UInt32, false),
            Field::new("c2", DataType::UInt64, false),
        ]));

        // register each partition as well as the top level dir
        let csv_read_option = CsvReadOptions::new().schema(&schema);
        ctx.register_csv("part0", &format!("{}/part-0.csv", out_dir), csv_read_option)
            .await?;
        ctx.register_csv("allparts", &out_dir, csv_read_option)
            .await?;

        let part0 = plan_and_collect(&mut ctx, "SELECT c1, c2 FROM part0").await?;
        let allparts = plan_and_collect(&mut ctx, "SELECT c1, c2 FROM allparts").await?;

        let allparts_count: usize = allparts.iter().map(|batch| batch.num_rows()).sum();

        assert_eq!(part0[0].schema(), allparts[0].schema());

        assert_eq!(allparts_count, 40);

        Ok(())
    }

    #[tokio::test]
    async fn write_parquet_results() -> Result<()> {
        // create partitioned input file and context
        let tmp_dir = TempDir::new()?;
        let mut ctx = create_ctx(&tmp_dir, 4).await?;

        // execute a simple query and write the results to CSV
        let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out";
        write_parquet(&mut ctx, "SELECT c1, c2 FROM test", &out_dir, None).await?;

        // create a new context and verify that the results were saved to a partitioned csv file
        let mut ctx = ExecutionContext::new();

        // register each partition as well as the top level dir
        ctx.register_parquet("part0", &format!("{}/part-0.parquet", out_dir))
            .await?;
        ctx.register_parquet("part1", &format!("{}/part-1.parquet", out_dir))
            .await?;
        ctx.register_parquet("part2", &format!("{}/part-2.parquet", out_dir))
            .await?;
        ctx.register_parquet("part3", &format!("{}/part-3.parquet", out_dir))
            .await?;
        ctx.register_parquet("allparts", &out_dir).await?;

        let part0 = plan_and_collect(&mut ctx, "SELECT c1, c2 FROM part0").await?;
        let allparts = plan_and_collect(&mut ctx, "SELECT c1, c2 FROM allparts").await?;

        let allparts_count: usize = allparts.iter().map(|batch| batch.num_rows()).sum();

        assert_eq!(part0[0].schema(), allparts[0].schema());

        assert_eq!(allparts_count, 40);

        Ok(())
    }

    #[tokio::test]
    async fn query_csv_with_custom_partition_extension() -> Result<()> {
        let tmp_dir = TempDir::new()?;

        // The main stipulation of this test: use a file extension that isn't .csv.
        let file_extension = ".tst";

        let mut ctx = ExecutionContext::new();
        let schema = populate_csv_partitions(&tmp_dir, 2, file_extension)?;
        ctx.register_csv(
            "test",
            tmp_dir.path().to_str().unwrap(),
            CsvReadOptions::new()
                .schema(&schema)
                .file_extension(file_extension),
        )
        .await?;
        let results =
            plan_and_collect(&mut ctx, "SELECT SUM(c1), SUM(c2), COUNT(*) FROM test")
                .await?;

        assert_eq!(results.len(), 1);
        let expected = vec![
            "+--------------+--------------+-----------------+",
            "| SUM(test.c1) | SUM(test.c2) | COUNT(UInt8(1)) |",
            "+--------------+--------------+-----------------+",
            "| 10           | 110          | 20              |",
            "+--------------+--------------+-----------------+",
        ];
        assert_batches_eq!(expected, &results);

        Ok(())
    }

    #[tokio::test]
    async fn send_context_to_threads() -> Result<()> {
        // ensure ExecutionContexts can be used in a multi-threaded
        // environment. Usecase is for concurrent planing.
        let tmp_dir = TempDir::new()?;
        let partition_count = 4;
        let ctx = Arc::new(Mutex::new(create_ctx(&tmp_dir, partition_count).await?));

        let threads: Vec<JoinHandle<Result<_>>> = (0..2)
            .map(|_| ctx.clone())
            .map(|ctx_clone| {
                thread::spawn(move || {
                    let ctx = ctx_clone.lock().expect("Locked context");
                    // Ensure we can create logical plan code on a separate thread.
                    ctx.create_logical_plan(
                        "SELECT c1, c2 FROM test WHERE c1 > 0 AND c1 < 3",
                    )
                })
            })
            .collect();

        for thread in threads {
            thread.join().expect("Failed to join thread")?;
        }
        Ok(())
    }

    #[tokio::test]
    async fn ctx_sql_should_optimize_plan() -> Result<()> {
        let mut ctx = ExecutionContext::new();
        let plan1 = ctx
            .create_logical_plan("SELECT * FROM (SELECT 1) AS one WHERE TRUE AND TRUE")?;

        let opt_plan1 = ctx.optimize(&plan1)?;

        let plan2 = ctx
            .sql("SELECT * FROM (SELECT 1) AS one WHERE TRUE AND TRUE")
            .await?;

        assert_eq!(
            format!("{:?}", opt_plan1),
            format!("{:?}", plan2.to_logical_plan())
        );

        Ok(())
    }

    #[tokio::test]
    async fn scalar_udf() -> Result<()> {
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]);

        let batch = RecordBatch::try_new(
            Arc::new(schema.clone()),
            vec![
                Arc::new(Int32Array::from_slice(&[1, 10, 10, 100])),
                Arc::new(Int32Array::from_slice(&[2, 12, 12, 120])),
            ],
        )?;

        let mut ctx = ExecutionContext::new();

        let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch]])?;
        ctx.register_table("t", Arc::new(provider))?;

        let myfunc = |args: &[ArrayRef]| {
            let l = &args[0]
                .as_any()
                .downcast_ref::<Int32Array>()
                .expect("cast failed");
            let r = &args[1]
                .as_any()
                .downcast_ref::<Int32Array>()
                .expect("cast failed");
            Ok(Arc::new(add(l, r)?) as ArrayRef)
        };
        let myfunc = make_scalar_function(myfunc);

        ctx.register_udf(create_udf(
            "my_add",
            vec![DataType::Int32, DataType::Int32],
            Arc::new(DataType::Int32),
            Volatility::Immutable,
            myfunc,
        ));

        // from here on, we may be in a different scope. We would still like to be able
        // to call UDFs.

        let t = ctx.table("t")?;

        let plan = LogicalPlanBuilder::from(t.to_logical_plan())
            .project(vec![
                col("a"),
                col("b"),
                ctx.udf("my_add")?.call(vec![col("a"), col("b")]),
            ])?
            .build()?;

        assert_eq!(
            format!("{:?}", plan),
            "Projection: #t.a, #t.b, my_add(#t.a, #t.b)\n  TableScan: t projection=None"
        );

        let plan = ctx.optimize(&plan)?;
        let plan = ctx.create_physical_plan(&plan).await?;
        let runtime = ctx.state.lock().runtime_env.clone();
        let result = collect(plan, runtime).await?;

        let expected = vec![
            "+-----+-----+-----------------+",
            "| a   | b   | my_add(t.a,t.b) |",
            "+-----+-----+-----------------+",
            "| 1   | 2   | 3               |",
            "| 10  | 12  | 22              |",
            "| 10  | 12  | 22              |",
            "| 100 | 120 | 220             |",
            "+-----+-----+-----------------+",
        ];
        assert_batches_eq!(expected, &result);

        let batch = &result[0];
        let a = batch
            .column(0)
            .as_any()
            .downcast_ref::<Int32Array>()
            .expect("failed to cast a");
        let b = batch
            .column(1)
            .as_any()
            .downcast_ref::<Int32Array>()
            .expect("failed to cast b");
        let sum = batch
            .column(2)
            .as_any()
            .downcast_ref::<Int32Array>()
            .expect("failed to cast sum");

        assert_eq!(4, a.len());
        assert_eq!(4, b.len());
        assert_eq!(4, sum.len());
        for i in 0..sum.len() {
            assert_eq!(a.value(i) + b.value(i), sum.value(i));
        }

        ctx.deregister_table("t")?;

        Ok(())
    }

    #[tokio::test]
    async fn simple_avg() -> Result<()> {
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);

        let batch1 = RecordBatch::try_new(
            Arc::new(schema.clone()),
            vec![Arc::new(Int32Array::from_slice(&[1, 2, 3]))],
        )?;
        let batch2 = RecordBatch::try_new(
            Arc::new(schema.clone()),
            vec![Arc::new(Int32Array::from_slice(&[4, 5]))],
        )?;

        let mut ctx = ExecutionContext::new();

        let provider =
            MemTable::try_new(Arc::new(schema), vec![vec![batch1], vec![batch2]])?;
        ctx.register_table("t", Arc::new(provider))?;

        let result = plan_and_collect(&mut ctx, "SELECT AVG(a) FROM t").await?;

        let batch = &result[0];
        assert_eq!(1, batch.num_columns());
        assert_eq!(1, batch.num_rows());

        let values = batch
            .column(0)
            .as_any()
            .downcast_ref::<Float64Array>()
            .expect("failed to cast version");
        assert_eq!(values.len(), 1);
        // avg(1,2,3,4,5) = 3.0
        assert_eq!(values.value(0), 3.0_f64);
        Ok(())
    }

    /// tests the creation, registration and usage of a UDAF
    #[tokio::test]
    async fn simple_udaf() -> Result<()> {
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);

        let batch1 = RecordBatch::try_new(
            Arc::new(schema.clone()),
            vec![Arc::new(Int32Array::from_slice(&[1, 2, 3]))],
        )?;
        let batch2 = RecordBatch::try_new(
            Arc::new(schema.clone()),
            vec![Arc::new(Int32Array::from_slice(&[4, 5]))],
        )?;

        let mut ctx = ExecutionContext::new();

        let provider =
            MemTable::try_new(Arc::new(schema), vec![vec![batch1], vec![batch2]])?;
        ctx.register_table("t", Arc::new(provider))?;

        // define a udaf, using a DataFusion's accumulator
        let my_avg = create_udaf(
            "my_avg",
            DataType::Float64,
            Arc::new(DataType::Float64),
            Volatility::Immutable,
            Arc::new(|| Ok(Box::new(AvgAccumulator::try_new(&DataType::Float64)?))),
            Arc::new(vec![DataType::UInt64, DataType::Float64]),
        );

        ctx.register_udaf(my_avg);

        let result = plan_and_collect(&mut ctx, "SELECT MY_AVG(a) FROM t").await?;

        let expected = vec![
            "+-------------+",
            "| my_avg(t.a) |",
            "+-------------+",
            "| 3           |",
            "+-------------+",
        ];
        assert_batches_eq!(expected, &result);

        Ok(())
    }

    #[tokio::test]
    async fn custom_query_planner() -> Result<()> {
        let mut ctx = ExecutionContext::with_config(
            ExecutionConfig::new().with_query_planner(Arc::new(MyQueryPlanner {})),
        );

        let df = ctx.sql("SELECT 1").await?;
        df.collect().await.expect_err("query not supported");
        Ok(())
    }

    #[tokio::test]
    async fn disabled_default_catalog_and_schema() -> Result<()> {
        let mut ctx = ExecutionContext::with_config(
            ExecutionConfig::new().create_default_catalog_and_schema(false),
        );

        assert!(matches!(
            ctx.register_table("test", test::table_with_sequence(1, 1)?),
            Err(DataFusionError::Plan(_))
        ));

        assert!(matches!(
            ctx.sql("select * from datafusion.public.test").await,
            Err(DataFusionError::Plan(_))
        ));

        Ok(())
    }

    #[tokio::test]
    async fn custom_catalog_and_schema() -> Result<()> {
        let mut ctx = ExecutionContext::with_config(
            ExecutionConfig::new()
                .create_default_catalog_and_schema(false)
                .with_default_catalog_and_schema("my_catalog", "my_schema"),
        );

        let catalog = MemoryCatalogProvider::new();
        let schema = MemorySchemaProvider::new();
        schema.register_table("test".to_owned(), test::table_with_sequence(1, 1)?)?;
        catalog.register_schema("my_schema", Arc::new(schema));
        ctx.register_catalog("my_catalog", Arc::new(catalog));

        for table_ref in &["my_catalog.my_schema.test", "my_schema.test", "test"] {
            let result = plan_and_collect(
                &mut ctx,
                &format!("SELECT COUNT(*) AS count FROM {}", table_ref),
            )
            .await?;

            let expected = vec![
                "+-------+",
                "| count |",
                "+-------+",
                "| 1     |",
                "+-------+",
            ];
            assert_batches_eq!(expected, &result);
        }

        Ok(())
    }

    #[tokio::test]
    async fn cross_catalog_access() -> Result<()> {
        let mut ctx = ExecutionContext::new();

        let catalog_a = MemoryCatalogProvider::new();
        let schema_a = MemorySchemaProvider::new();
        schema_a
            .register_table("table_a".to_owned(), test::table_with_sequence(1, 1)?)?;
        catalog_a.register_schema("schema_a", Arc::new(schema_a));
        ctx.register_catalog("catalog_a", Arc::new(catalog_a));

        let catalog_b = MemoryCatalogProvider::new();
        let schema_b = MemorySchemaProvider::new();
        schema_b
            .register_table("table_b".to_owned(), test::table_with_sequence(1, 2)?)?;
        catalog_b.register_schema("schema_b", Arc::new(schema_b));
        ctx.register_catalog("catalog_b", Arc::new(catalog_b));

        let result = plan_and_collect(
            &mut ctx,
            "SELECT cat, SUM(i) AS total FROM (
                    SELECT i, 'a' AS cat FROM catalog_a.schema_a.table_a
                    UNION ALL
                    SELECT i, 'b' AS cat FROM catalog_b.schema_b.table_b
                ) AS all
                GROUP BY cat
                ORDER BY cat
                ",
        )
        .await?;

        let expected = vec![
            "+-----+-------+",
            "| cat | total |",
            "+-----+-------+",
            "| a   | 1     |",
            "| b   | 3     |",
            "+-----+-------+",
        ];
        assert_batches_eq!(expected, &result);

        Ok(())
    }

    #[tokio::test]
    async fn create_external_table_with_timestamps() {
        let mut ctx = ExecutionContext::new();

        let data = "Jorge,2018-12-13T12:12:10.011Z\n\
                    Andrew,2018-11-13T17:11:10.011Z";

        let tmp_dir = TempDir::new().unwrap();
        let file_path = tmp_dir.path().join("timestamps.csv");

        // scope to ensure the file is closed and written
        {
            File::create(&file_path)
                .expect("creating temp file")
                .write_all(data.as_bytes())
                .expect("writing data");
        }

        let sql = format!(
            "CREATE EXTERNAL TABLE csv_with_timestamps (
                  name VARCHAR,
                  ts TIMESTAMP
              )
              STORED AS CSV
              LOCATION '{}'
              ",
            file_path.to_str().expect("path is utf8")
        );

        plan_and_collect(&mut ctx, &sql)
            .await
            .expect("Executing CREATE EXTERNAL TABLE");

        let sql = "SELECT * from csv_with_timestamps";
        let result = plan_and_collect(&mut ctx, sql).await.unwrap();
        let expected = vec![
            "+--------+-------------------------+",
            "| name   | ts                      |",
            "+--------+-------------------------+",
            "| Andrew | 2018-11-13 17:11:10.011 |",
            "| Jorge  | 2018-12-13 12:12:10.011 |",
            "+--------+-------------------------+",
        ];
        assert_batches_sorted_eq!(expected, &result);
    }

    #[tokio::test]
    async fn query_empty_table() {
        let mut ctx = ExecutionContext::new();
        let empty_table = Arc::new(EmptyTable::new(Arc::new(Schema::empty())));
        ctx.register_table("test_tbl", empty_table).unwrap();
        let sql = "SELECT * FROM test_tbl";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("Query empty table");
        let expected = vec!["++", "++"];
        assert_batches_sorted_eq!(expected, &result);
    }

    #[tokio::test]
    async fn catalogs_not_leaked() {
        // the information schema used to introduce cyclic Arcs
        let ctx = ExecutionContext::with_config(
            ExecutionConfig::new().with_information_schema(true),
        );

        // register a single catalog
        let catalog = Arc::new(MemoryCatalogProvider::new());
        let catalog_weak = Arc::downgrade(&catalog);
        ctx.register_catalog("my_catalog", catalog);

        let catalog_list_weak = {
            let state = ctx.state.lock();
            Arc::downgrade(&state.catalog_list)
        };

        drop(ctx);

        assert_eq!(Weak::strong_count(&catalog_list_weak), 0);
        assert_eq!(Weak::strong_count(&catalog_weak), 0);
    }

    #[tokio::test]
    async fn schema_merge_ignores_metadata() {
        // Create two parquet files in same table with same schema but different metadata
        let tmp_dir = TempDir::new().unwrap();
        let table_dir = tmp_dir.path().join("parquet_test");
        let table_path = Path::new(&table_dir);

        let mut non_empty_metadata: HashMap<String, String> = HashMap::new();
        non_empty_metadata.insert("testing".to_string(), "metadata".to_string());

        let fields = vec![
            Field::new("id", DataType::Int32, true),
            Field::new("name", DataType::Utf8, true),
        ];
        let schemas = vec![
            Arc::new(Schema::new_with_metadata(
                fields.clone(),
                non_empty_metadata.clone(),
            )),
            Arc::new(Schema::new(fields.clone())),
        ];

        if let Ok(()) = fs::create_dir(table_path) {
            for (i, schema) in schemas.iter().enumerate().take(2) {
                let filename = format!("part-{}.parquet", i);
                let path = table_path.join(&filename);
                let file = fs::File::create(path).unwrap();
                let mut writer =
                    ArrowWriter::try_new(file.try_clone().unwrap(), schema.clone(), None)
                        .unwrap();

                // create mock record batch
                let ids = Arc::new(Int32Array::from_slice(&[i as i32]));
                let names = Arc::new(StringArray::from_slice(&["test"]));
                let rec_batch =
                    RecordBatch::try_new(schema.clone(), vec![ids, names]).unwrap();

                writer.write(&rec_batch).unwrap();
                writer.close().unwrap();
            }
        }

        // Read the parquet files into a dataframe to confirm results
        // (no errors)
        let mut ctx = ExecutionContext::new();
        let df = ctx
            .read_parquet(table_dir.to_str().unwrap().to_string())
            .await
            .unwrap();
        let result = df.collect().await.unwrap();

        assert_eq!(result[0].schema().metadata(), result[1].schema().metadata());
    }

    #[tokio::test]
    async fn normalized_column_identifiers() {
        // create local execution context
        let mut ctx = ExecutionContext::new();

        // register csv file with the execution context
        ctx.register_csv(
            "case_insensitive_test",
            "tests/example.csv",
            CsvReadOptions::new(),
        )
        .await
        .unwrap();

        let sql = "SELECT A, b FROM case_insensitive_test";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| a | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        let sql = "SELECT t.A, b FROM case_insensitive_test AS t";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| a | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        // Aliases

        let sql = "SELECT t.A as x, b FROM case_insensitive_test AS t";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| x | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        let sql = "SELECT t.A AS X, b FROM case_insensitive_test AS t";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| x | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        let sql = r#"SELECT t.A AS "X", b FROM case_insensitive_test AS t"#;
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| X | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        // Order by

        let sql = "SELECT t.A AS x, b FROM case_insensitive_test AS t ORDER BY x";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| x | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        let sql = "SELECT t.A AS x, b FROM case_insensitive_test AS t ORDER BY X";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| x | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        let sql = r#"SELECT t.A AS "X", b FROM case_insensitive_test AS t ORDER BY "X""#;
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| X | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        // Where

        let sql = "SELECT a, b FROM case_insensitive_test where A IS NOT null";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| a | b |",
            "+---+---+",
            "| 1 | 2 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        // Group by

        let sql = "SELECT a as x, count(*) as c FROM case_insensitive_test GROUP BY X";
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| x | c |",
            "+---+---+",
            "| 1 | 1 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);

        let sql =
            r#"SELECT a as "X", count(*) as c FROM case_insensitive_test GROUP BY "X""#;
        let result = plan_and_collect(&mut ctx, sql)
            .await
            .expect("ran plan correctly");
        let expected = vec![
            "+---+---+",
            "| X | c |",
            "+---+---+",
            "| 1 | 1 |",
            "+---+---+",
        ];
        assert_batches_sorted_eq!(expected, &result);
    }

    struct MyPhysicalPlanner {}

    #[async_trait]
    impl PhysicalPlanner for MyPhysicalPlanner {
        async fn create_physical_plan(
            &self,
            _logical_plan: &LogicalPlan,
            _ctx_state: &ExecutionContextState,
        ) -> Result<Arc<dyn ExecutionPlan>> {
            Err(DataFusionError::NotImplemented(
                "query not supported".to_string(),
            ))
        }

        fn create_physical_expr(
            &self,
            _expr: &Expr,
            _input_dfschema: &crate::logical_plan::DFSchema,
            _input_schema: &Schema,
            _ctx_state: &ExecutionContextState,
        ) -> Result<Arc<dyn crate::physical_plan::PhysicalExpr>> {
            unimplemented!()
        }
    }

    struct MyQueryPlanner {}

    #[async_trait]
    impl QueryPlanner for MyQueryPlanner {
        async fn create_physical_plan(
            &self,
            logical_plan: &LogicalPlan,
            ctx_state: &ExecutionContextState,
        ) -> Result<Arc<dyn ExecutionPlan>> {
            let physical_planner = MyPhysicalPlanner {};
            physical_planner
                .create_physical_plan(logical_plan, ctx_state)
                .await
        }
    }

    /// Execute SQL and return results
    async fn plan_and_collect(
        ctx: &mut ExecutionContext,
        sql: &str,
    ) -> Result<Vec<RecordBatch>> {
        ctx.sql(sql).await?.collect().await
    }

    /// Execute SQL and return results
    async fn execute(sql: &str, partition_count: usize) -> Result<Vec<RecordBatch>> {
        let tmp_dir = TempDir::new()?;
        let mut ctx = create_ctx(&tmp_dir, partition_count).await?;
        plan_and_collect(&mut ctx, sql).await
    }

    /// Execute SQL and write results to partitioned csv files
    async fn write_csv(
        ctx: &mut ExecutionContext,
        sql: &str,
        out_dir: &str,
    ) -> Result<()> {
        let logical_plan = ctx.create_logical_plan(sql)?;
        let logical_plan = ctx.optimize(&logical_plan)?;
        let physical_plan = ctx.create_physical_plan(&logical_plan).await?;
        ctx.write_csv(physical_plan, out_dir).await
    }

    /// Execute SQL and write results to partitioned parquet files
    async fn write_parquet(
        ctx: &mut ExecutionContext,
        sql: &str,
        out_dir: &str,
        writer_properties: Option<WriterProperties>,
    ) -> Result<()> {
        let logical_plan = ctx.create_logical_plan(sql)?;
        let logical_plan = ctx.optimize(&logical_plan)?;
        let physical_plan = ctx.create_physical_plan(&logical_plan).await?;
        ctx.write_parquet(physical_plan, out_dir, writer_properties)
            .await
    }

    /// Generate CSV partitions within the supplied directory
    fn populate_csv_partitions(
        tmp_dir: &TempDir,
        partition_count: usize,
        file_extension: &str,
    ) -> Result<SchemaRef> {
        // define schema for data source (csv file)
        let schema = Arc::new(Schema::new(vec![
            Field::new("c1", DataType::UInt32, false),
            Field::new("c2", DataType::UInt64, false),
            Field::new("c3", DataType::Boolean, false),
        ]));

        // generate a partitioned file
        for partition in 0..partition_count {
            let filename = format!("partition-{}.{}", partition, file_extension);
            let file_path = tmp_dir.path().join(&filename);
            let mut file = File::create(file_path)?;

            // generate some data
            for i in 0..=10 {
                let data = format!("{},{},{}\n", partition, i, i % 2 == 0);
                file.write_all(data.as_bytes())?;
            }
        }

        Ok(schema)
    }

    /// Generate a partitioned CSV file and register it with an execution context
    async fn create_ctx(
        tmp_dir: &TempDir,
        partition_count: usize,
    ) -> Result<ExecutionContext> {
        let mut ctx = ExecutionContext::with_config(
            ExecutionConfig::new().with_target_partitions(8),
        );

        let schema = populate_csv_partitions(tmp_dir, partition_count, ".csv")?;

        // register csv file with the execution context
        ctx.register_csv(
            "test",
            tmp_dir.path().to_str().unwrap(),
            CsvReadOptions::new().schema(&schema),
        )
        .await?;

        Ok(ctx)
    }

    // Test for compilation error when calling read_* functions from an #[async_trait] function.
    // See https://github.com/apache/arrow-datafusion/issues/1154
    #[async_trait]
    trait CallReadTrait {
        async fn call_read_csv(&self) -> Arc<dyn DataFrame>;
        async fn call_read_avro(&self) -> Arc<dyn DataFrame>;
        async fn call_read_parquet(&self) -> Arc<dyn DataFrame>;
    }

    struct CallRead {}

    #[async_trait]
    impl CallReadTrait for CallRead {
        async fn call_read_csv(&self) -> Arc<dyn DataFrame> {
            let mut ctx = ExecutionContext::new();
            ctx.read_csv("dummy", CsvReadOptions::new()).await.unwrap()
        }

        async fn call_read_avro(&self) -> Arc<dyn DataFrame> {
            let mut ctx = ExecutionContext::new();
            ctx.read_avro("dummy", AvroReadOptions::default())
                .await
                .unwrap()
        }

        async fn call_read_parquet(&self) -> Arc<dyn DataFrame> {
            let mut ctx = ExecutionContext::new();
            ctx.read_parquet("dummy").await.unwrap()
        }
    }
}