aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
blob: c958eac266d615bedba621870c95343b1335abd0 (plain) (blame)
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
/*
 * 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.
 */

package org.apache.spark.sql

import java.math.MathContext
import java.sql.Timestamp

import org.apache.spark.AccumulatorSuite
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.expressions.SortOrder
import org.apache.spark.sql.catalyst.plans.logical.Aggregate
import org.apache.spark.sql.execution.aggregate
import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, CartesianProduct, SortMergeJoin}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext}
import org.apache.spark.sql.test.SQLTestData._
import org.apache.spark.sql.types._

class SQLQuerySuite extends QueryTest with SharedSQLContext {
  import testImplicits._

  setupTestData()

  test("having clause") {
    Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v").registerTempTable("hav")
    checkAnswer(
      sql("SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2"),
      Row("one", 6) :: Row("three", 3) :: Nil)
  }

  test("SPARK-8010: promote numeric to string") {
    val df = Seq((1, 1)).toDF("key", "value")
    df.registerTempTable("src")
    val queryCaseWhen = sql("select case when true then 1.0 else '1' end from src ")
    val queryCoalesce = sql("select coalesce(null, 1, '1') from src ")

    checkAnswer(queryCaseWhen, Row("1.0") :: Nil)
    checkAnswer(queryCoalesce, Row("1") :: Nil)
  }

  test("show functions") {
    def getFunctions(pattern: String): Seq[Row] = {
      val regex = java.util.regex.Pattern.compile(pattern)
      sqlContext.sessionState.functionRegistry.listFunction()
        .filter(regex.matcher(_).matches()).map(Row(_))
    }
    checkAnswer(sql("SHOW functions"), getFunctions(".*"))
    Seq("^c.*", ".*e$", "log.*", ".*date.*").foreach { pattern =>
      checkAnswer(sql(s"SHOW FUNCTIONS '$pattern'"), getFunctions(pattern))
    }
  }

  test("describe functions") {
    checkExistence(sql("describe function extended upper"), true,
      "Function: upper",
      "Class: org.apache.spark.sql.catalyst.expressions.Upper",
      "Usage: upper(str) - Returns str with all characters changed to uppercase",
      "Extended Usage:",
      "> SELECT upper('SparkSql');",
      "'SPARKSQL'")

    checkExistence(sql("describe functioN Upper"), true,
      "Function: upper",
      "Class: org.apache.spark.sql.catalyst.expressions.Upper",
      "Usage: upper(str) - Returns str with all characters changed to uppercase")

    checkExistence(sql("describe functioN Upper"), false,
      "Extended Usage")

    checkExistence(sql("describe functioN abcadf"), true,
      "Function: abcadf not found.")
  }

  test("SPARK-6743: no columns from cache") {
    Seq(
      (83, 0, 38),
      (26, 0, 79),
      (43, 81, 24)
    ).toDF("a", "b", "c").registerTempTable("cachedData")

    sqlContext.cacheTable("cachedData")
    checkAnswer(
      sql("SELECT t1.b FROM cachedData, cachedData t1 GROUP BY t1.b"),
      Row(0) :: Row(81) :: Nil)
  }

  test("self join with aliases") {
    Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").registerTempTable("df")

    checkAnswer(
      sql(
        """
          |SELECT x.str, COUNT(*)
          |FROM df x JOIN df y ON x.str = y.str
          |GROUP BY x.str
        """.stripMargin),
      Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil)
  }

  test("support table.star") {
    checkAnswer(
      sql(
        """
          |SELECT r.*
          |FROM testData l join testData2 r on (l.key = r.a)
        """.stripMargin),
      Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil)
  }

  test("self join with alias in agg") {
      Seq(1, 2, 3)
        .map(i => (i, i.toString))
        .toDF("int", "str")
        .groupBy("str")
        .agg($"str", count("str").as("strCount"))
        .registerTempTable("df")

    checkAnswer(
      sql(
        """
          |SELECT x.str, SUM(x.strCount)
          |FROM df x JOIN df y ON x.str = y.str
          |GROUP BY x.str
        """.stripMargin),
      Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil)
  }

  test("SPARK-8668 expr function") {
    checkAnswer(Seq((1, "Bobby G."))
      .toDF("id", "name")
      .select(expr("length(name)"), expr("abs(id)")), Row(8, 1))

    checkAnswer(Seq((1, "building burrito tunnels"), (1, "major projects"))
      .toDF("id", "saying")
      .groupBy(expr("length(saying)"))
      .count(), Row(24, 1) :: Row(14, 1) :: Nil)
  }

  test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") {
    checkAnswer(
      sql("SELECT a FROM testData2 SORT BY a"),
      Seq(1, 1, 2, 2, 3, 3).map(Row(_))
    )
  }

  test("SPARK-7158 collect and take return different results") {
    import java.util.UUID

    val df = Seq(Tuple1(1), Tuple1(2), Tuple1(3)).toDF("index")
    // we except the id is materialized once
    val idUDF = org.apache.spark.sql.functions.udf(() => UUID.randomUUID().toString)

    val dfWithId = df.withColumn("id", idUDF())
    // Make a new DataFrame (actually the same reference to the old one)
    val cached = dfWithId.cache()
    // Trigger the cache
    val d0 = dfWithId.collect()
    val d1 = cached.collect()
    val d2 = cached.collect()

    // Since the ID is only materialized once, then all of the records
    // should come from the cache, not by re-computing. Otherwise, the ID
    // will be different
    assert(d0.map(_(0)) === d2.map(_(0)))
    assert(d0.map(_(1)) === d2.map(_(1)))

    assert(d1.map(_(0)) === d2.map(_(0)))
    assert(d1.map(_(1)) === d2.map(_(1)))
  }

  test("grouping on nested fields") {
    sqlContext.read.json(sparkContext.parallelize(
      """{"nested": {"attribute": 1}, "value": 2}""" :: Nil))
     .registerTempTable("rows")

    checkAnswer(
      sql(
        """
          |select attribute, sum(cnt)
          |from (
          |  select nested.attribute, count(*) as cnt
          |  from rows
          |  group by nested.attribute) a
          |group by attribute
        """.stripMargin),
      Row(1, 1) :: Nil)
  }

  test("SPARK-6201 IN type conversion") {
    sqlContext.read.json(
      sparkContext.parallelize(
        Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}")))
      .registerTempTable("d")

    checkAnswer(
      sql("select * from d where d.a in (1,2)"),
      Seq(Row("1"), Row("2")))
  }

  test("SPARK-11226 Skip empty line in json file") {
    sqlContext.read.json(
      sparkContext.parallelize(
        Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}", "")))
      .registerTempTable("d")

    checkAnswer(
      sql("select count(1) from d"),
      Seq(Row(3)))
  }

  test("SPARK-8828 sum should return null if all input values are null") {
    checkAnswer(
      sql("select sum(a), avg(a) from allNulls"),
      Seq(Row(null, null))
    )
  }

  private def testCodeGen(sqlText: String, expectedResults: Seq[Row]): Unit = {
    val df = sql(sqlText)
    // First, check if we have GeneratedAggregate.
    val hasGeneratedAgg = df.queryExecution.sparkPlan
      .collect { case _: aggregate.TungstenAggregate => true }
      .nonEmpty
    if (!hasGeneratedAgg) {
      fail(
        s"""
           |Codegen is enabled, but query $sqlText does not have TungstenAggregate in the plan.
           |${df.queryExecution.simpleString}
         """.stripMargin)
    }
    // Then, check results.
    checkAnswer(df, expectedResults)
  }

  test("aggregation with codegen") {
    // Prepare a table that we can group some rows.
    sqlContext.table("testData")
      .union(sqlContext.table("testData"))
      .union(sqlContext.table("testData"))
      .registerTempTable("testData3x")

    try {
      // Just to group rows.
      testCodeGen(
        "SELECT key FROM testData3x GROUP BY key",
        (1 to 100).map(Row(_)))
      // COUNT
      testCodeGen(
        "SELECT key, count(value) FROM testData3x GROUP BY key",
        (1 to 100).map(i => Row(i, 3)))
      testCodeGen(
        "SELECT count(key) FROM testData3x",
        Row(300) :: Nil)
      // COUNT DISTINCT ON int
      testCodeGen(
        "SELECT value, count(distinct key) FROM testData3x GROUP BY value",
        (1 to 100).map(i => Row(i.toString, 1)))
      testCodeGen(
        "SELECT count(distinct key) FROM testData3x",
        Row(100) :: Nil)
      // SUM
      testCodeGen(
        "SELECT value, sum(key) FROM testData3x GROUP BY value",
        (1 to 100).map(i => Row(i.toString, 3 * i)))
      testCodeGen(
        "SELECT sum(key), SUM(CAST(key as Double)) FROM testData3x",
        Row(5050 * 3, 5050 * 3.0) :: Nil)
      // AVERAGE
      testCodeGen(
        "SELECT value, avg(key) FROM testData3x GROUP BY value",
        (1 to 100).map(i => Row(i.toString, i)))
      testCodeGen(
        "SELECT avg(key) FROM testData3x",
        Row(50.5) :: Nil)
      // MAX
      testCodeGen(
        "SELECT value, max(key) FROM testData3x GROUP BY value",
        (1 to 100).map(i => Row(i.toString, i)))
      testCodeGen(
        "SELECT max(key) FROM testData3x",
        Row(100) :: Nil)
      // MIN
      testCodeGen(
        "SELECT value, min(key) FROM testData3x GROUP BY value",
        (1 to 100).map(i => Row(i.toString, i)))
      testCodeGen(
        "SELECT min(key) FROM testData3x",
        Row(1) :: Nil)
      // Some combinations.
      testCodeGen(
        """
          |SELECT
          |  value,
          |  sum(key),
          |  max(key),
          |  min(key),
          |  avg(key),
          |  count(key),
          |  count(distinct key)
          |FROM testData3x
          |GROUP BY value
        """.stripMargin,
        (1 to 100).map(i => Row(i.toString, i*3, i, i, i, 3, 1)))
      testCodeGen(
        "SELECT max(key), min(key), avg(key), count(key), count(distinct key) FROM testData3x",
        Row(100, 1, 50.5, 300, 100) :: Nil)
      // Aggregate with Code generation handling all null values
      testCodeGen(
        "SELECT  sum('a'), avg('a'), count(null) FROM testData",
        Row(null, null, 0) :: Nil)
    } finally {
      sqlContext.dropTempTable("testData3x")
    }
  }

  test("Add Parser of SQL COALESCE()") {
    checkAnswer(
      sql("""SELECT COALESCE(1, 2)"""),
      Row(1))
    checkAnswer(
      sql("SELECT COALESCE(null, 1, 1.5)"),
      Row(BigDecimal(1)))
    checkAnswer(
      sql("SELECT COALESCE(null, null, null)"),
      Row(null))
  }

  test("SPARK-3176 Added Parser of SQL LAST()") {
    checkAnswer(
      sql("SELECT LAST(n) FROM lowerCaseData"),
      Row(4))
  }

  test("SPARK-2041 column name equals tablename") {
    checkAnswer(
      sql("SELECT tableName FROM tableName"),
      Row("test"))
  }

  test("SQRT") {
    checkAnswer(
      sql("SELECT SQRT(key) FROM testData"),
      (1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq
    )
  }

  test("SQRT with automatic string casts") {
    checkAnswer(
      sql("SELECT SQRT(CAST(key AS STRING)) FROM testData"),
      (1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq
    )
  }

  test("SPARK-2407 Added Parser of SQL SUBSTR()") {
    checkAnswer(
      sql("SELECT substr(tableName, 1, 2) FROM tableName"),
      Row("te"))
    checkAnswer(
      sql("SELECT substr(tableName, 3) FROM tableName"),
      Row("st"))
    checkAnswer(
      sql("SELECT substring(tableName, 1, 2) FROM tableName"),
      Row("te"))
    checkAnswer(
      sql("SELECT substring(tableName, 3) FROM tableName"),
      Row("st"))
  }

  test("SPARK-3173 Timestamp support in the parser") {
    (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").registerTempTable("timestamps")

    checkAnswer(sql(
      "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.0'"),
      Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00")))

    checkAnswer(sql(
      "SELECT time FROM timestamps WHERE time=CAST('1969-12-31 16:00:00.001' AS TIMESTAMP)"),
      Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001")))

    checkAnswer(sql(
      "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.001'"),
      Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001")))

    checkAnswer(sql(
      "SELECT time FROM timestamps WHERE '1969-12-31 16:00:00.001'=time"),
      Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001")))

    checkAnswer(sql(
      """SELECT time FROM timestamps WHERE time<'1969-12-31 16:00:00.003'
          AND time>'1969-12-31 16:00:00.001'"""),
      Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002")))

    checkAnswer(sql(
      """
        |SELECT time FROM timestamps
        |WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002')
      """.stripMargin),
      Seq(Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001")),
        Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002"))))

    checkAnswer(sql(
      "SELECT time FROM timestamps WHERE time='123'"),
      Nil)
  }

  test("index into array") {
    checkAnswer(
      sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"),
      arrayData.map(d => Row(d.data, d.data(0), d.data(0) + d.data(1), d.data(1))).collect())
  }

  test("left semi greater than predicate") {
    checkAnswer(
      sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.a >= y.a + 2"),
      Seq(Row(3, 1), Row(3, 2))
    )
  }

  test("left semi greater than predicate and equal operator") {
    checkAnswer(
      sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.b = y.b and x.a >= y.a + 2"),
      Seq(Row(3, 1), Row(3, 2))
    )

    checkAnswer(
      sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.b = y.a and x.a >= y.b + 1"),
      Seq(Row(2, 1), Row(2, 2), Row(3, 1), Row(3, 2))
    )
  }

  test("index into array of arrays") {
    checkAnswer(
      sql(
        "SELECT nestedData, nestedData[0][0], nestedData[0][0] + nestedData[0][1] FROM arrayData"),
      arrayData.map(d =>
        Row(d.nestedData,
         d.nestedData(0)(0),
         d.nestedData(0)(0) + d.nestedData(0)(1))).collect().toSeq)
  }

  test("agg") {
    checkAnswer(
      sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"),
      Seq(Row(1, 3), Row(2, 3), Row(3, 3)))
  }

  test("Group By Ordinal - basic") {
    checkAnswer(
      sql("SELECT a, sum(b) FROM testData2 GROUP BY 1"),
      sql("SELECT a, sum(b) FROM testData2 GROUP BY a"))

    // duplicate group-by columns
    checkAnswer(
      sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a, 1"),
      sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a"))

    checkAnswer(
      sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY 1, 2"),
      sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a"))
  }

  test("Group By Ordinal - non aggregate expressions") {
    checkAnswer(
      sql("SELECT a, b + 2, count(2) FROM testData2 GROUP BY a, 2"),
      sql("SELECT a, b + 2, count(2) FROM testData2 GROUP BY a, b + 2"))

    checkAnswer(
      sql("SELECT a, b + 2 as c, count(2) FROM testData2 GROUP BY a, 2"),
      sql("SELECT a, b + 2, count(2) FROM testData2 GROUP BY a, b + 2"))
  }

  test("Group By Ordinal - non-foldable constant expression") {
    checkAnswer(
      sql("SELECT a, b, sum(b) FROM testData2 GROUP BY a, b, 1 + 0"),
      sql("SELECT a, b, sum(b) FROM testData2 GROUP BY a, b"))

    checkAnswer(
      sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a, 1 + 2"),
      sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a"))
  }

  test("Group By Ordinal - alias") {
    checkAnswer(
      sql("SELECT a, (b + 2) as c, count(2) FROM testData2 GROUP BY a, 2"),
      sql("SELECT a, b + 2, count(2) FROM testData2 GROUP BY a, b + 2"))

    checkAnswer(
      sql("SELECT a as b, b as a, sum(b) FROM testData2 GROUP BY 1, 2"),
      sql("SELECT a, b, sum(b) FROM testData2 GROUP BY a, b"))
  }

  test("Group By Ordinal - constants") {
    checkAnswer(
      sql("SELECT 1, 2, sum(b) FROM testData2 GROUP BY 1, 2"),
      sql("SELECT 1, 2, sum(b) FROM testData2"))
  }

  test("Group By Ordinal - negative cases") {
    intercept[UnresolvedException[Aggregate]] {
      sql("SELECT a, b FROM testData2 GROUP BY -1")
    }

    intercept[UnresolvedException[Aggregate]] {
      sql("SELECT a, b FROM testData2 GROUP BY 3")
    }

    var e = intercept[UnresolvedException[Aggregate]](
      sql("SELECT SUM(a) FROM testData2 GROUP BY 1"))
    assert(e.getMessage contains
      "Invalid call to Group by position: the '1'th column in the select contains " +
        "an aggregate function")

    e = intercept[UnresolvedException[Aggregate]](
      sql("SELECT SUM(a) + 1 FROM testData2 GROUP BY 1"))
    assert(e.getMessage contains
      "Invalid call to Group by position: the '1'th column in the select contains " +
        "an aggregate function")

    var ae = intercept[AnalysisException](
      sql("SELECT a, rand(0), sum(b) FROM testData2 GROUP BY a, 2"))
    assert(ae.getMessage contains
      "nondeterministic expression rand(0) should not appear in grouping expression")

    ae = intercept[AnalysisException](
      sql("SELECT * FROM testData2 GROUP BY a, b, 1"))
    assert(ae.getMessage contains
      "Group by position: star is not allowed to use in the select list " +
        "when using ordinals in group by")
  }

  test("Group By Ordinal: spark.sql.groupByOrdinal=false") {
    withSQLConf(SQLConf.GROUP_BY_ORDINAL.key -> "false") {
      // If spark.sql.groupByOrdinal=false, ignore the position number.
      intercept[AnalysisException] {
        sql("SELECT a, sum(b) FROM testData2 GROUP BY 1")
      }
      // '*' is not allowed to use in the select list when users specify ordinals in group by
      checkAnswer(
        sql("SELECT * FROM testData2 GROUP BY a, b, 1"),
        sql("SELECT * FROM testData2 GROUP BY a, b"))
    }
  }

  test("aggregates with nulls") {
    checkAnswer(
      sql("SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a)," +
        "AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM nullInts"),
      Row(0, -1.5, 1, 3, 2, 1.0, 1, 6, 3)
    )
  }

  test("select *") {
    checkAnswer(
      sql("SELECT * FROM testData"),
      testData.collect().toSeq)
  }

  test("simple select") {
    checkAnswer(
      sql("SELECT value FROM testData WHERE key = 1"),
      Row("1"))
  }

  def sortTest(): Unit = {
    checkAnswer(
      sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC"),
      Seq(Row(1, 1), Row(1, 2), Row(2, 1), Row(2, 2), Row(3, 1), Row(3, 2)))

    checkAnswer(
      sql("SELECT * FROM testData2 ORDER BY a ASC, b DESC"),
      Seq(Row(1, 2), Row(1, 1), Row(2, 2), Row(2, 1), Row(3, 2), Row(3, 1)))

    checkAnswer(
      sql("SELECT * FROM testData2 ORDER BY a DESC, b DESC"),
      Seq(Row(3, 2), Row(3, 1), Row(2, 2), Row(2, 1), Row(1, 2), Row(1, 1)))

    checkAnswer(
      sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC"),
      Seq(Row(3, 1), Row(3, 2), Row(2, 1), Row(2, 2), Row(1, 1), Row(1, 2)))

    checkAnswer(
      sql("SELECT b FROM binaryData ORDER BY a ASC"),
      (1 to 5).map(Row(_)))

    checkAnswer(
      sql("SELECT b FROM binaryData ORDER BY a DESC"),
      (1 to 5).map(Row(_)).toSeq.reverse)

    checkAnswer(
      sql("SELECT * FROM arrayData ORDER BY data[0] ASC"),
      arrayData.collect().sortBy(_.data(0)).map(Row.fromTuple).toSeq)

    checkAnswer(
      sql("SELECT * FROM arrayData ORDER BY data[0] DESC"),
      arrayData.collect().sortBy(_.data(0)).reverse.map(Row.fromTuple).toSeq)

    checkAnswer(
      sql("SELECT * FROM mapData ORDER BY data[1] ASC"),
      mapData.collect().sortBy(_.data(1)).map(Row.fromTuple).toSeq)

    checkAnswer(
      sql("SELECT * FROM mapData ORDER BY data[1] DESC"),
      mapData.collect().sortBy(_.data(1)).reverse.map(Row.fromTuple).toSeq)
  }

  test("external sorting") {
    sortTest()
  }

  test("limit") {
    checkAnswer(
      sql("SELECT * FROM testData LIMIT 10"),
      testData.take(10).toSeq)

    checkAnswer(
      sql("SELECT * FROM arrayData LIMIT 1"),
      arrayData.collect().take(1).map(Row.fromTuple).toSeq)

    checkAnswer(
      sql("SELECT * FROM mapData LIMIT 1"),
      mapData.collect().take(1).map(Row.fromTuple).toSeq)
  }

  test("CTE feature") {
    checkAnswer(
      sql("with q1 as (select * from testData limit 10) select * from q1"),
      testData.take(10).toSeq)

    checkAnswer(
      sql("""
        |with q1 as (select * from testData where key= '5'),
        |q2 as (select * from testData where key = '4')
        |select * from q1 union all select * from q2""".stripMargin),
      Row(5, "5") :: Row(4, "4") :: Nil)

  }

  test("Allow only a single WITH clause per query") {
    intercept[AnalysisException] {
      sql(
        "with q1 as (select * from testData) with q2 as (select * from q1) select * from q2")
    }
  }

  test("date row") {
    checkAnswer(sql(
      """select cast("2015-01-28" as date) from testData limit 1"""),
      Row(java.sql.Date.valueOf("2015-01-28"))
    )
  }

  test("from follow multiple brackets") {
    checkAnswer(sql(
      """
        |select key from ((select * from testData)
        |  union all (select * from testData)) x limit 1
      """.stripMargin),
      Row(1)
    )

    checkAnswer(sql(
      "select key from (select * from testData) x limit 1"),
      Row(1)
    )

    checkAnswer(sql(
      """
        |select key from
        |  (select * from testData union all select * from testData) x
        |  limit 1
      """.stripMargin),
      Row(1)
    )
  }

  test("average") {
    checkAnswer(
      sql("SELECT AVG(a) FROM testData2"),
      Row(2.0))
  }

  test("average overflow") {
    checkAnswer(
      sql("SELECT AVG(a),b FROM largeAndSmallInts group by b"),
      Seq(Row(2147483645.0, 1), Row(2.0, 2)))
  }

  test("count") {
    checkAnswer(
      sql("SELECT COUNT(*) FROM testData2"),
      Row(testData2.count()))
  }

  test("count distinct") {
    checkAnswer(
      sql("SELECT COUNT(DISTINCT b) FROM testData2"),
      Row(2))
  }

  test("approximate count distinct") {
    checkAnswer(
      sql("SELECT APPROX_COUNT_DISTINCT(a) FROM testData2"),
      Row(3))
  }

  test("approximate count distinct with user provided standard deviation") {
    checkAnswer(
      sql("SELECT APPROX_COUNT_DISTINCT(a, 0.04) FROM testData2"),
      Row(3))
  }

  test("null count") {
    checkAnswer(
      sql("SELECT a, COUNT(b) FROM testData3 GROUP BY a"),
      Seq(Row(1, 0), Row(2, 1)))

    checkAnswer(
      sql(
        "SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3"),
      Row(2, 1, 2, 2, 1))
  }

  test("count of empty table") {
    withTempTable("t") {
      Seq.empty[(Int, Int)].toDF("a", "b").registerTempTable("t")
      checkAnswer(
        sql("select count(a) from t"),
        Row(0))
    }
  }

  test("inner join where, one match per row") {
    checkAnswer(
      sql("SELECT * FROM upperCaseData JOIN lowerCaseData WHERE n = N"),
      Seq(
        Row(1, "A", 1, "a"),
        Row(2, "B", 2, "b"),
        Row(3, "C", 3, "c"),
        Row(4, "D", 4, "d")))
  }

  test("inner join ON, one match per row") {
    checkAnswer(
      sql("SELECT * FROM upperCaseData JOIN lowerCaseData ON n = N"),
      Seq(
        Row(1, "A", 1, "a"),
        Row(2, "B", 2, "b"),
        Row(3, "C", 3, "c"),
        Row(4, "D", 4, "d")))
  }

  test("inner join, where, multiple matches") {
    checkAnswer(
      sql("""
        |SELECT * FROM
        |  (SELECT * FROM testData2 WHERE a = 1) x JOIN
        |  (SELECT * FROM testData2 WHERE a = 1) y
        |WHERE x.a = y.a""".stripMargin),
      Row(1, 1, 1, 1) ::
      Row(1, 1, 1, 2) ::
      Row(1, 2, 1, 1) ::
      Row(1, 2, 1, 2) :: Nil)
  }

  test("inner join, no matches") {
    checkAnswer(
      sql(
        """
          |SELECT * FROM
          |  (SELECT * FROM testData2 WHERE a = 1) x JOIN
          |  (SELECT * FROM testData2 WHERE a = 2) y
          |WHERE x.a = y.a""".stripMargin),
      Nil)
  }

  test("big inner join, 4 matches per row") {
    checkAnswer(
      sql(
        """
          |SELECT * FROM
          |  (SELECT * FROM testData UNION ALL
          |   SELECT * FROM testData UNION ALL
          |   SELECT * FROM testData UNION ALL
          |   SELECT * FROM testData) x JOIN
          |  (SELECT * FROM testData UNION ALL
          |   SELECT * FROM testData UNION ALL
          |   SELECT * FROM testData UNION ALL
          |   SELECT * FROM testData) y
          |WHERE x.key = y.key""".stripMargin),
      testData.rdd.flatMap(
        row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq)
  }

  test("cartesian product join") {
    checkAnswer(
      testData3.join(testData3),
      Row(1, null, 1, null) ::
      Row(1, null, 2, 2) ::
      Row(2, 2, 1, null) ::
      Row(2, 2, 2, 2) :: Nil)
  }

  test("left outer join") {
    checkAnswer(
      sql("SELECT * FROM upperCaseData LEFT OUTER JOIN lowerCaseData ON n = N"),
      Row(1, "A", 1, "a") ::
      Row(2, "B", 2, "b") ::
      Row(3, "C", 3, "c") ::
      Row(4, "D", 4, "d") ::
      Row(5, "E", null, null) ::
      Row(6, "F", null, null) :: Nil)
  }

  test("right outer join") {
    checkAnswer(
      sql("SELECT * FROM lowerCaseData RIGHT OUTER JOIN upperCaseData ON n = N"),
      Row(1, "a", 1, "A") ::
      Row(2, "b", 2, "B") ::
      Row(3, "c", 3, "C") ::
      Row(4, "d", 4, "D") ::
      Row(null, null, 5, "E") ::
      Row(null, null, 6, "F") :: Nil)
  }

  test("full outer join") {
    checkAnswer(
      sql(
        """
          |SELECT * FROM
          |  (SELECT * FROM upperCaseData WHERE N <= 4) leftTable FULL OUTER JOIN
          |  (SELECT * FROM upperCaseData WHERE N >= 3) rightTable
          |    ON leftTable.N = rightTable.N
        """.stripMargin),
      Row(1, "A", null, null) ::
      Row(2, "B", null, null) ::
      Row(3, "C", 3, "C") ::
      Row (4, "D", 4, "D") ::
      Row(null, null, 5, "E") ::
      Row(null, null, 6, "F") :: Nil)
  }

  test("SPARK-11111 null-safe join should not use cartesian product") {
    val df = sql("select count(*) from testData a join testData b on (a.key <=> b.key)")
    val cp = df.queryExecution.sparkPlan.collect {
      case cp: CartesianProduct => cp
    }
    assert(cp.isEmpty, "should not use CartesianProduct for null-safe join")
    val smj = df.queryExecution.sparkPlan.collect {
      case smj: SortMergeJoin => smj
      case j: BroadcastHashJoin => j
    }
    assert(smj.size > 0, "should use SortMergeJoin or BroadcastHashJoin")
    checkAnswer(df, Row(100) :: Nil)
  }

  test("SPARK-3349 partitioning after limit") {
    sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC")
      .limit(2)
      .registerTempTable("subset1")
    sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n ASC")
      .limit(2)
      .registerTempTable("subset2")
    checkAnswer(
      sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"),
      Row(3, "c", 3) ::
      Row(4, "d", 4) :: Nil)
    checkAnswer(
      sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"),
      Row(1, "a", 1) ::
      Row(2, "b", 2) :: Nil)
  }

  test("mixed-case keywords") {
    checkAnswer(
      sql(
        """
          |SeleCT * from
          |  (select * from upperCaseData WherE N <= 4) leftTable fuLL OUtER joiN
          |  (sElEcT * FROM upperCaseData whERe N >= 3) rightTable
          |    oN leftTable.N = rightTable.N
        """.stripMargin),
      Row(1, "A", null, null) ::
      Row(2, "B", null, null) ::
      Row(3, "C", 3, "C") ::
      Row(4, "D", 4, "D") ::
      Row(null, null, 5, "E") ::
      Row(null, null, 6, "F") :: Nil)
  }

  test("select with table name as qualifier") {
    checkAnswer(
      sql("SELECT testData.value FROM testData WHERE testData.key = 1"),
      Row("1"))
  }

  test("inner join ON with table name as qualifier") {
    checkAnswer(
      sql("SELECT * FROM upperCaseData JOIN lowerCaseData ON lowerCaseData.n = upperCaseData.N"),
      Seq(
        Row(1, "A", 1, "a"),
        Row(2, "B", 2, "b"),
        Row(3, "C", 3, "c"),
        Row(4, "D", 4, "d")))
  }

  test("qualified select with inner join ON with table name as qualifier") {
    checkAnswer(
      sql("SELECT upperCaseData.N, upperCaseData.L FROM upperCaseData JOIN lowerCaseData " +
        "ON lowerCaseData.n = upperCaseData.N"),
      Seq(
        Row(1, "A"),
        Row(2, "B"),
        Row(3, "C"),
        Row(4, "D")))
  }

  test("system function upper()") {
    checkAnswer(
      sql("SELECT n,UPPER(l) FROM lowerCaseData"),
      Seq(
        Row(1, "A"),
        Row(2, "B"),
        Row(3, "C"),
        Row(4, "D")))

    checkAnswer(
      sql("SELECT n, UPPER(s) FROM nullStrings"),
      Seq(
        Row(1, "ABC"),
        Row(2, "ABC"),
        Row(3, null)))
  }

  test("system function lower()") {
    checkAnswer(
      sql("SELECT N,LOWER(L) FROM upperCaseData"),
      Seq(
        Row(1, "a"),
        Row(2, "b"),
        Row(3, "c"),
        Row(4, "d"),
        Row(5, "e"),
        Row(6, "f")))

    checkAnswer(
      sql("SELECT n, LOWER(s) FROM nullStrings"),
      Seq(
        Row(1, "abc"),
        Row(2, "abc"),
        Row(3, null)))
  }

  test("UNION") {
    checkAnswer(
      sql("SELECT * FROM lowerCaseData UNION SELECT * FROM upperCaseData"),
      Row(1, "A") :: Row(1, "a") :: Row(2, "B") :: Row(2, "b") :: Row(3, "C") :: Row(3, "c") ::
      Row(4, "D") :: Row(4, "d") :: Row(5, "E") :: Row(6, "F") :: Nil)
    checkAnswer(
      sql("SELECT * FROM lowerCaseData UNION SELECT * FROM lowerCaseData"),
      Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Row(4, "d") :: Nil)
    checkAnswer(
      sql("SELECT * FROM lowerCaseData UNION ALL SELECT * FROM lowerCaseData"),
      Row(1, "a") :: Row(1, "a") :: Row(2, "b") :: Row(2, "b") :: Row(3, "c") :: Row(3, "c") ::
      Row(4, "d") :: Row(4, "d") :: Nil)
  }

  test("UNION with column mismatches") {
    // Column name mismatches are allowed.
    checkAnswer(
      sql("SELECT n,l FROM lowerCaseData UNION SELECT N as x1, L as x2 FROM upperCaseData"),
      Row(1, "A") :: Row(1, "a") :: Row(2, "B") :: Row(2, "b") :: Row(3, "C") :: Row(3, "c") ::
      Row(4, "D") :: Row(4, "d") :: Row(5, "E") :: Row(6, "F") :: Nil)
    // Column type mismatches are not allowed, forcing a type coercion.
    checkAnswer(
      sql("SELECT n FROM lowerCaseData UNION SELECT L FROM upperCaseData"),
      ("1" :: "2" :: "3" :: "4" :: "A" :: "B" :: "C" :: "D" :: "E" :: "F" :: Nil).map(Row(_)))
    // Column type mismatches where a coercion is not possible, in this case between integer
    // and array types, trigger a TreeNodeException.
    intercept[AnalysisException] {
      sql("SELECT data FROM arrayData UNION SELECT 1 FROM arrayData").collect()
    }
  }

  test("EXCEPT") {
    checkAnswer(
      sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM upperCaseData"),
      Row(1, "a") ::
      Row(2, "b") ::
      Row(3, "c") ::
      Row(4, "d") :: Nil)
    checkAnswer(
      sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM lowerCaseData"), Nil)
    checkAnswer(
      sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData"), Nil)
  }

  test("INTERSECT") {
    checkAnswer(
      sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM lowerCaseData"),
      Row(1, "a") ::
      Row(2, "b") ::
      Row(3, "c") ::
      Row(4, "d") :: Nil)
    checkAnswer(
      sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM upperCaseData"), Nil)
  }

  test("SET commands semantics using sql()") {
    sqlContext.conf.clear()
    val testKey = "test.key.0"
    val testVal = "test.val.0"
    val nonexistentKey = "nonexistent"

    // "set" itself returns all config variables currently specified in SQLConf.
    assert(sql("SET").collect().size === TestSQLContext.overrideConfs.size)
    sql("SET").collect().foreach { row =>
      val key = row.getString(0)
      val value = row.getString(1)
      assert(
        TestSQLContext.overrideConfs.contains(key),
        s"$key should exist in SQLConf.")
      assert(
        TestSQLContext.overrideConfs(key) === value,
        s"The value of $key should be ${TestSQLContext.overrideConfs(key)} instead of $value.")
    }
    val overrideConfs = sql("SET").collect()

    // "set key=val"
    sql(s"SET $testKey=$testVal")
    checkAnswer(
      sql("SET"),
      overrideConfs ++ Seq(Row(testKey, testVal))
    )

    sql(s"SET ${testKey + testKey}=${testVal + testVal}")
    checkAnswer(
      sql("set"),
      overrideConfs ++ Seq(Row(testKey, testVal), Row(testKey + testKey, testVal + testVal))
    )

    // "set key"
    checkAnswer(
      sql(s"SET $testKey"),
      Row(testKey, testVal)
    )
    checkAnswer(
      sql(s"SET $nonexistentKey"),
      Row(nonexistentKey, "<undefined>")
    )
    sqlContext.conf.clear()
  }

  test("SET commands with illegal or inappropriate argument") {
    sqlContext.conf.clear()
    // Set negative mapred.reduce.tasks for automatically determining
    // the number of reducers is not supported
    intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1"))
    intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01"))
    intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-2"))
    sqlContext.conf.clear()
  }

  test("apply schema") {
    val schema1 = StructType(
      StructField("f1", IntegerType, false) ::
      StructField("f2", StringType, false) ::
      StructField("f3", BooleanType, false) ::
      StructField("f4", IntegerType, true) :: Nil)

    val rowRDD1 = unparsedStrings.map { r =>
      val values = r.split(",").map(_.trim)
      val v4 = try values(3).toInt catch {
        case _: NumberFormatException => null
      }
      Row(values(0).toInt, values(1), values(2).toBoolean, v4)
    }

    val df1 = sqlContext.createDataFrame(rowRDD1, schema1)
    df1.registerTempTable("applySchema1")
    checkAnswer(
      sql("SELECT * FROM applySchema1"),
      Row(1, "A1", true, null) ::
      Row(2, "B2", false, null) ::
      Row(3, "C3", true, null) ::
      Row(4, "D4", true, 2147483644) :: Nil)

    checkAnswer(
      sql("SELECT f1, f4 FROM applySchema1"),
      Row(1, null) ::
      Row(2, null) ::
      Row(3, null) ::
      Row(4, 2147483644) :: Nil)

    val schema2 = StructType(
      StructField("f1", StructType(
        StructField("f11", IntegerType, false) ::
        StructField("f12", BooleanType, false) :: Nil), false) ::
      StructField("f2", MapType(StringType, IntegerType, true), false) :: Nil)

    val rowRDD2 = unparsedStrings.map { r =>
      val values = r.split(",").map(_.trim)
      val v4 = try values(3).toInt catch {
        case _: NumberFormatException => null
      }
      Row(Row(values(0).toInt, values(2).toBoolean), Map(values(1) -> v4))
    }

    val df2 = sqlContext.createDataFrame(rowRDD2, schema2)
    df2.registerTempTable("applySchema2")
    checkAnswer(
      sql("SELECT * FROM applySchema2"),
      Row(Row(1, true), Map("A1" -> null)) ::
      Row(Row(2, false), Map("B2" -> null)) ::
      Row(Row(3, true), Map("C3" -> null)) ::
      Row(Row(4, true), Map("D4" -> 2147483644)) :: Nil)

    checkAnswer(
      sql("SELECT f1.f11, f2['D4'] FROM applySchema2"),
      Row(1, null) ::
      Row(2, null) ::
      Row(3, null) ::
      Row(4, 2147483644) :: Nil)

    // The value of a MapType column can be a mutable map.
    val rowRDD3 = unparsedStrings.map { r =>
      val values = r.split(",").map(_.trim)
      val v4 = try values(3).toInt catch {
        case _: NumberFormatException => null
      }
      Row(Row(values(0).toInt, values(2).toBoolean), scala.collection.mutable.Map(values(1) -> v4))
    }

    val df3 = sqlContext.createDataFrame(rowRDD3, schema2)
    df3.registerTempTable("applySchema3")

    checkAnswer(
      sql("SELECT f1.f11, f2['D4'] FROM applySchema3"),
      Row(1, null) ::
      Row(2, null) ::
      Row(3, null) ::
      Row(4, 2147483644) :: Nil)
  }

  test("SPARK-3423 BETWEEN") {
    checkAnswer(
      sql("SELECT key, value FROM testData WHERE key BETWEEN 5 and 7"),
      Seq(Row(5, "5"), Row(6, "6"), Row(7, "7"))
    )

    checkAnswer(
      sql("SELECT key, value FROM testData WHERE key BETWEEN 7 and 7"),
      Row(7, "7")
    )

    checkAnswer(
      sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"),
      Nil
    )
  }

  test("cast boolean to string") {
    // TODO Ensure true/false string letter casing is consistent with Hive in all cases.
    checkAnswer(
      sql("SELECT CAST(TRUE AS STRING), CAST(FALSE AS STRING) FROM testData LIMIT 1"),
      Row("true", "false"))
  }

  test("metadata is propagated correctly") {
    val person: DataFrame = sql("SELECT * FROM person")
    val schema = person.schema
    val docKey = "doc"
    val docValue = "first name"
    val metadata = new MetadataBuilder()
      .putString(docKey, docValue)
      .build()
    val schemaWithMeta = new StructType(Array(
      schema("id"), schema("name").copy(metadata = metadata), schema("age")))
    val personWithMeta = sqlContext.createDataFrame(person.rdd, schemaWithMeta)
    def validateMetadata(rdd: DataFrame): Unit = {
      assert(rdd.schema("name").metadata.getString(docKey) == docValue)
    }
    personWithMeta.registerTempTable("personWithMeta")
    validateMetadata(personWithMeta.select($"name"))
    validateMetadata(personWithMeta.select($"name"))
    validateMetadata(personWithMeta.select($"id", $"name"))
    validateMetadata(sql("SELECT * FROM personWithMeta"))
    validateMetadata(sql("SELECT id, name FROM personWithMeta"))
    validateMetadata(sql("SELECT * FROM personWithMeta JOIN salary ON id = personId"))
    validateMetadata(sql(
      "SELECT name, salary FROM personWithMeta JOIN salary ON id = personId"))
  }

  test("SPARK-3371 Renaming a function expression with group by gives error") {
    sqlContext.udf.register("len", (s: String) => s.length)
    checkAnswer(
      sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"),
      Row(1))
  }

  test("SPARK-3813 CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END") {
    checkAnswer(
      sql("SELECT CASE key WHEN 1 THEN 1 ELSE 0 END FROM testData WHERE key = 1 group by key"),
      Row(1))
  }

  test("SPARK-3813 CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END") {
    checkAnswer(
      sql("SELECT CASE WHEN key = 1 THEN 1 ELSE 2 END FROM testData WHERE key = 1 group by key"),
      Row(1))
  }

  test("throw errors for non-aggregate attributes with aggregation") {
    def checkAggregation(query: String, isInvalidQuery: Boolean = true) {
      if (isInvalidQuery) {
        val e = intercept[AnalysisException](sql(query).queryExecution.analyzed)
        assert(e.getMessage contains "group by")
      } else {
        // Should not throw
        sql(query).queryExecution.analyzed
      }
    }

    checkAggregation("SELECT key, COUNT(*) FROM testData")
    checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", isInvalidQuery = false)

    checkAggregation("SELECT value, COUNT(*) FROM testData GROUP BY key")
    checkAggregation("SELECT COUNT(value), SUM(key) FROM testData GROUP BY key", false)

    checkAggregation("SELECT key + 2, COUNT(*) FROM testData GROUP BY key + 1")
    checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false)
  }

  test("Test to check we can use Long.MinValue") {
    checkAnswer(
      sql(s"SELECT ${Long.MinValue} FROM testData ORDER BY key LIMIT 1"), Row(Long.MinValue)
    )

    checkAnswer(
      sql(s"SELECT key FROM testData WHERE key > ${Long.MinValue}"),
      (1 to 100).map(Row(_)).toSeq
    )
  }

  test("Floating point number format") {
    checkAnswer(
      sql("SELECT 0.3"), Row(BigDecimal(0.3))
    )

    checkAnswer(
      sql("SELECT -0.8"), Row(BigDecimal(-0.8))
    )

    checkAnswer(
      sql("SELECT .5"), Row(BigDecimal(0.5))
    )

    checkAnswer(
      sql("SELECT -.18"), Row(BigDecimal(-0.18))
    )
  }

  test("Auto cast integer type") {
    checkAnswer(
      sql(s"SELECT ${Int.MaxValue + 1L}"), Row(Int.MaxValue + 1L)
    )

    checkAnswer(
      sql(s"SELECT ${Int.MinValue - 1L}"), Row(Int.MinValue - 1L)
    )

    checkAnswer(
      sql("SELECT 9223372036854775808"), Row(new java.math.BigDecimal("9223372036854775808"))
    )

    checkAnswer(
      sql("SELECT -9223372036854775809"), Row(new java.math.BigDecimal("-9223372036854775809"))
    )
  }

  test("Test to check we can apply sign to expression") {

    checkAnswer(
      sql("SELECT -100"), Row(-100)
    )

    checkAnswer(
      sql("SELECT +230"), Row(230)
    )

    checkAnswer(
      sql("SELECT -5.2"), Row(BigDecimal(-5.2))
    )

    checkAnswer(
      sql("SELECT +6.8e0"), Row(6.8d)
    )

    checkAnswer(
      sql("SELECT -key FROM testData WHERE key = 2"), Row(-2)
    )

    checkAnswer(
      sql("SELECT +key FROM testData WHERE key = 3"), Row(3)
    )

    checkAnswer(
      sql("SELECT -(key + 1) FROM testData WHERE key = 1"), Row(-2)
    )

    checkAnswer(
      sql("SELECT - key + 1 FROM testData WHERE key = 10"), Row(-9)
    )

    checkAnswer(
      sql("SELECT +(key + 5) FROM testData WHERE key = 5"), Row(10)
    )

    checkAnswer(
      sql("SELECT -MAX(key) FROM testData"), Row(-100)
    )

    checkAnswer(
      sql("SELECT +MAX(key) FROM testData"), Row(100)
    )

    checkAnswer(
      sql("SELECT - (-10)"), Row(10)
    )

    checkAnswer(
      sql("SELECT + (-key) FROM testData WHERE key = 32"), Row(-32)
    )

    checkAnswer(
      sql("SELECT - (+Max(key)) FROM testData"), Row(-100)
    )

    checkAnswer(
      sql("SELECT - - 3"), Row(3)
    )

    checkAnswer(
      sql("SELECT - + 20"), Row(-20)
    )

    checkAnswer(
      sql("SELEcT - + 45"), Row(-45)
    )

    checkAnswer(
      sql("SELECT + + 100"), Row(100)
    )

    checkAnswer(
      sql("SELECT - - Max(key) FROM testData"), Row(100)
    )

    checkAnswer(
      sql("SELECT + - key FROM testData WHERE key = 33"), Row(-33)
    )
  }

  test("Multiple join") {
    checkAnswer(
      sql(
        """SELECT a.key, b.key, c.key
          |FROM testData a
          |JOIN testData b ON a.key = b.key
          |JOIN testData c ON a.key = c.key
        """.stripMargin),
      (1 to 100).map(i => Row(i, i, i)))
  }

  test("SPARK-3483 Special chars in column names") {
    val data = sparkContext.parallelize(
      Seq("""{"key?number1": "value1", "key.number2": "value2"}"""))
    sqlContext.read.json(data).registerTempTable("records")
    sql("SELECT `key?number1`, `key.number2` FROM records")
  }

  test("SPARK-3814 Support Bitwise & operator") {
    checkAnswer(sql("SELECT key&1 FROM testData WHERE key = 1 "), Row(1))
  }

  test("SPARK-3814 Support Bitwise | operator") {
    checkAnswer(sql("SELECT key|0 FROM testData WHERE key = 1 "), Row(1))
  }

  test("SPARK-3814 Support Bitwise ^ operator") {
    checkAnswer(sql("SELECT key^0 FROM testData WHERE key = 1 "), Row(1))
  }

  test("SPARK-3814 Support Bitwise ~ operator") {
    checkAnswer(sql("SELECT ~key FROM testData WHERE key = 1 "), Row(-2))
  }

  test("SPARK-4120 Join of multiple tables does not work in SparkSQL") {
    checkAnswer(
      sql(
        """SELECT a.key, b.key, c.key
          |FROM testData a,testData b,testData c
          |where a.key = b.key and a.key = c.key
        """.stripMargin),
      (1 to 100).map(i => Row(i, i, i)))
  }

  test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") {
    checkAnswer(sql("SELECT key FROM testData WHERE key not between 0 and 10 order by key"),
        (11 to 100).map(i => Row(i)))
  }

  test("SPARK-4207 Query which has syntax like 'not like' is not working in Spark SQL") {
    checkAnswer(sql("SELECT key FROM testData WHERE value not like '100%' order by key"),
        (1 to 99).map(i => Row(i)))
  }

  test("SPARK-4322 Grouping field with struct field as sub expression") {
    sqlContext.read.json(sparkContext.makeRDD("""{"a": {"b": [{"c": 1}]}}""" :: Nil))
      .registerTempTable("data")
    checkAnswer(sql("SELECT a.b[0].c FROM data GROUP BY a.b[0].c"), Row(1))
    sqlContext.dropTempTable("data")

    sqlContext.read.json(
      sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).registerTempTable("data")
    checkAnswer(sql("SELECT a.b + 1 FROM data GROUP BY a.b + 1"), Row(2))
    sqlContext.dropTempTable("data")
  }

  test("SPARK-4432 Fix attribute reference resolution error when using ORDER BY") {
    checkAnswer(
      sql("SELECT a + b FROM testData2 ORDER BY a"),
      Seq(2, 3, 3, 4, 4, 5).map(Row(_))
    )
  }

  test("oder by asc by default when not specify ascending and descending") {
    checkAnswer(
      sql("SELECT a, b FROM testData2 ORDER BY a desc, b"),
      Seq(Row(3, 1), Row(3, 2), Row(2, 1), Row(2, 2), Row(1, 1), Row(1, 2))
    )
  }

  test("Supporting relational operator '<=>' in Spark SQL") {
    val nullCheckData1 = TestData(1, "1") :: TestData(2, null) :: Nil
    val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i)))
    rdd1.toDF().registerTempTable("nulldata1")
    val nullCheckData2 = TestData(1, "1") :: TestData(2, null) :: Nil
    val rdd2 = sparkContext.parallelize((0 to 1).map(i => nullCheckData2(i)))
    rdd2.toDF().registerTempTable("nulldata2")
    checkAnswer(sql("SELECT nulldata1.key FROM nulldata1 join " +
      "nulldata2 on nulldata1.value <=> nulldata2.value"),
        (1 to 2).map(i => Row(i)))
  }

  test("Multi-column COUNT(DISTINCT ...)") {
    val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil
    val rdd = sparkContext.parallelize((0 to 1).map(i => data(i)))
    rdd.toDF().registerTempTable("distinctData")
    checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2))
  }

  test("SPARK-4699 case sensitivity SQL query") {
    val orig = sqlContext.getConf(SQLConf.CASE_SENSITIVE)
    try {
      sqlContext.setConf(SQLConf.CASE_SENSITIVE, false)
      val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil
      val rdd = sparkContext.parallelize((0 to 1).map(i => data(i)))
      rdd.toDF().registerTempTable("testTable1")
      checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1"))
    } finally {
      sqlContext.setConf(SQLConf.CASE_SENSITIVE, orig)
    }
  }

  test("SPARK-6145: ORDER BY test for nested fields") {
    sqlContext.read.json(sparkContext.makeRDD(
        """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil))
      .registerTempTable("nestedOrder")

    checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.b"), Row(1))
    checkAnswer(sql("SELECT a.b FROM nestedOrder ORDER BY a.b"), Row(1))
    checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.a.a"), Row(1))
    checkAnswer(sql("SELECT a.a.a FROM nestedOrder ORDER BY a.a.a"), Row(1))
    checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY c[0].d"), Row(1))
    checkAnswer(sql("SELECT c[0].d FROM nestedOrder ORDER BY c[0].d"), Row(1))
  }

  test("SPARK-6145: special cases") {
    sqlContext.read.json(sparkContext.makeRDD(
      """{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""" :: Nil)).registerTempTable("t")
    checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY _c0.a"), Row(1))
    checkAnswer(sql("SELECT b[0].a FROM t ORDER BY _c0.a"), Row(1))
  }

  test("SPARK-6898: complete support for special chars in column names") {
    sqlContext.read.json(sparkContext.makeRDD(
      """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil))
      .registerTempTable("t")

    checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1))
  }

  test("SPARK-6583 order by aggregated function") {
    Seq("1" -> 3, "1" -> 4, "2" -> 7, "2" -> 8, "3" -> 5, "3" -> 6, "4" -> 1, "4" -> 2)
      .toDF("a", "b").registerTempTable("orderByData")

    checkAnswer(
      sql(
        """
          |SELECT a
          |FROM orderByData
          |GROUP BY a
          |ORDER BY sum(b)
        """.stripMargin),
      Row("4") :: Row("1") :: Row("3") :: Row("2") :: Nil)

    checkAnswer(
      sql(
        """
          |SELECT sum(b)
          |FROM orderByData
          |GROUP BY a
          |ORDER BY sum(b)
        """.stripMargin),
      Row(3) :: Row(7) :: Row(11) :: Row(15) :: Nil)

    checkAnswer(
      sql(
        """
          |SELECT sum(b)
          |FROM orderByData
          |GROUP BY a
          |ORDER BY sum(b), max(b)
        """.stripMargin),
      Row(3) :: Row(7) :: Row(11) :: Row(15) :: Nil)

    checkAnswer(
      sql(
        """
          |SELECT a, sum(b)
          |FROM orderByData
          |GROUP BY a
          |ORDER BY sum(b)
        """.stripMargin),
      Row("4", 3) :: Row("1", 7) :: Row("3", 11) :: Row("2", 15) :: Nil)

    checkAnswer(
      sql(
        """
            |SELECT a, sum(b)
            |FROM orderByData
            |GROUP BY a
            |ORDER BY sum(b) + 1
          """.stripMargin),
      Row("4", 3) :: Row("1", 7) :: Row("3", 11) :: Row("2", 15) :: Nil)

    checkAnswer(
      sql(
        """
            |SELECT count(*)
            |FROM orderByData
            |GROUP BY a
            |ORDER BY count(*)
          """.stripMargin),
      Row(2) :: Row(2) :: Row(2) :: Row(2) :: Nil)

    checkAnswer(
      sql(
        """
            |SELECT a
            |FROM orderByData
            |GROUP BY a
            |ORDER BY a, count(*), sum(b)
          """.stripMargin),
      Row("1") :: Row("2") :: Row("3") :: Row("4") :: Nil)
  }

  test("SPARK-7952: fix the equality check between boolean and numeric types") {
    withTempTable("t") {
      // numeric field i, boolean field j, result of i = j, result of i <=> j
      Seq[(Integer, java.lang.Boolean, java.lang.Boolean, java.lang.Boolean)](
        (1, true, true, true),
        (0, false, true, true),
        (2, true, false, false),
        (2, false, false, false),
        (null, true, null, false),
        (null, false, null, false),
        (0, null, null, false),
        (1, null, null, false),
        (null, null, null, true)
      ).toDF("i", "b", "r1", "r2").registerTempTable("t")

      checkAnswer(sql("select i = b from t"), sql("select r1 from t"))
      checkAnswer(sql("select i <=> b from t"), sql("select r2 from t"))
    }
  }

  test("SPARK-7067: order by queries for complex ExtractValue chain") {
    withTempTable("t") {
      sqlContext.read.json(sparkContext.makeRDD(
        """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).registerTempTable("t")
      checkAnswer(sql("SELECT a.b FROM t ORDER BY b[0].d"), Row(Seq(Row(1))))
    }
  }

  test("SPARK-8782: ORDER BY NULL") {
    withTempTable("t") {
      Seq((1, 2), (1, 2)).toDF("a", "b").registerTempTable("t")
      checkAnswer(sql("SELECT * FROM t ORDER BY NULL"), Seq(Row(1, 2), Row(1, 2)))
    }
  }

  test("SPARK-8837: use keyword in column name") {
    withTempTable("t") {
      val df = Seq(1 -> "a").toDF("count", "sort")
      checkAnswer(df.filter("count > 0"), Row(1, "a"))
      df.registerTempTable("t")
      checkAnswer(sql("select count, sort from t"), Row(1, "a"))
    }
  }

  test("SPARK-8753: add interval type") {
    import org.apache.spark.unsafe.types.CalendarInterval

    val df = sql("select interval 3 years -3 month 7 week 123 microseconds")
    checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * 1000 * 1000 * 3600 * 24 * 7 + 123 )))
    withTempPath(f => {
      // Currently we don't yet support saving out values of interval data type.
      val e = intercept[AnalysisException] {
        df.write.json(f.getCanonicalPath)
      }
      e.message.contains("Cannot save interval data type into external storage")
    })

    val e1 = intercept[AnalysisException] {
      sql("select interval")
    }
    assert(e1.message.contains("at least one time unit should be given for interval literal"))
    // Currently we don't yet support nanosecond
    val e2 = intercept[AnalysisException] {
      sql("select interval 23 nanosecond")
    }
    assert(e2.message.contains("cannot recognize input near"))
  }

  test("SPARK-8945: add and subtract expressions for interval type") {
    import org.apache.spark.unsafe.types.CalendarInterval
    import org.apache.spark.unsafe.types.CalendarInterval.MICROS_PER_WEEK

    val df = sql("select interval 3 years -3 month 7 week 123 microseconds as i")
    checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * MICROS_PER_WEEK + 123)))

    checkAnswer(df.select(df("i") + new CalendarInterval(2, 123)),
      Row(new CalendarInterval(12 * 3 - 3 + 2, 7L * MICROS_PER_WEEK + 123 + 123)))

    checkAnswer(df.select(df("i") - new CalendarInterval(2, 123)),
      Row(new CalendarInterval(12 * 3 - 3 - 2, 7L * MICROS_PER_WEEK + 123 - 123)))

    // unary minus
    checkAnswer(df.select(-df("i")),
      Row(new CalendarInterval(-(12 * 3 - 3), -(7L * MICROS_PER_WEEK + 123))))
  }

  test("aggregation with codegen updates peak execution memory") {
    AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "aggregation with codegen") {
      testCodeGen(
        "SELECT key, count(value) FROM testData GROUP BY key",
        (1 to 100).map(i => Row(i, 1)))
    }
  }

  test("decimal precision with multiply/division") {
    checkAnswer(sql("select 10.3 * 3.0"), Row(BigDecimal("30.90")))
    checkAnswer(sql("select 10.3000 * 3.0"), Row(BigDecimal("30.90000")))
    checkAnswer(sql("select 10.30000 * 30.0"), Row(BigDecimal("309.000000")))
    checkAnswer(sql("select 10.300000000000000000 * 3.000000000000000000"),
      Row(BigDecimal("30.900000000000000000000000000000000000", new MathContext(38))))
    checkAnswer(sql("select 10.300000000000000000 * 3.0000000000000000000"),
      Row(null))

    checkAnswer(sql("select 10.3 / 3.0"), Row(BigDecimal("3.433333")))
    checkAnswer(sql("select 10.3000 / 3.0"), Row(BigDecimal("3.4333333")))
    checkAnswer(sql("select 10.30000 / 30.0"), Row(BigDecimal("0.343333333")))
    checkAnswer(sql("select 10.300000000000000000 / 3.00000000000000000"),
      Row(BigDecimal("3.433333333333333333333333333", new MathContext(38))))
    checkAnswer(sql("select 10.3000000000000000000 / 3.00000000000000000"),
      Row(BigDecimal("3.4333333333333333333333333333", new MathContext(38))))
  }

  test("SPARK-10215 Div of Decimal returns null") {
    val d = Decimal(1.12321).toBigDecimal
    val df = Seq((d, 1)).toDF("a", "b")

    checkAnswer(
      df.selectExpr("b * a / b"),
      Seq(Row(d)))
    checkAnswer(
      df.selectExpr("b * a / b / b"),
      Seq(Row(d)))
    checkAnswer(
      df.selectExpr("b * a + b"),
      Seq(Row(BigDecimal(2.12321))))
    checkAnswer(
      df.selectExpr("b * a - b"),
      Seq(Row(BigDecimal(0.12321))))
    checkAnswer(
      df.selectExpr("b * a * b"),
      Seq(Row(d)))
  }

  test("precision smaller than scale") {
    checkAnswer(sql("select 10.00"), Row(BigDecimal("10.00")))
    checkAnswer(sql("select 1.00"), Row(BigDecimal("1.00")))
    checkAnswer(sql("select 0.10"), Row(BigDecimal("0.10")))
    checkAnswer(sql("select 0.01"), Row(BigDecimal("0.01")))
    checkAnswer(sql("select 0.001"), Row(BigDecimal("0.001")))
    checkAnswer(sql("select -0.01"), Row(BigDecimal("-0.01")))
    checkAnswer(sql("select -0.001"), Row(BigDecimal("-0.001")))
  }

  test("external sorting updates peak execution memory") {
    AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") {
      sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect()
    }
  }

  test("SPARK-9511: error with table starting with number") {
    withTempTable("1one") {
      sparkContext.parallelize(1 to 10).map(i => (i, i.toString))
        .toDF("num", "str")
        .registerTempTable("1one")
      checkAnswer(sql("select count(num) from 1one"), Row(10))
    }
  }

  test("specifying database name for a temporary table is not allowed") {
    withTempPath { dir =>
      val path = dir.getCanonicalPath
      val df =
        sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str")
      df
        .write
        .format("parquet")
        .save(path)

      // We don't support creating a temporary table while specifying a database
      intercept[AnalysisException] {
        sqlContext.sql(
          s"""
          |CREATE TEMPORARY TABLE db.t
          |USING parquet
          |OPTIONS (
          |  path '$path'
          |)
        """.stripMargin)
      }.getMessage

      // If you use backticks to quote the name then it's OK.
      sqlContext.sql(
        s"""
          |CREATE TEMPORARY TABLE `db.t`
          |USING parquet
          |OPTIONS (
          |  path '$path'
          |)
        """.stripMargin)
      checkAnswer(sqlContext.table("`db.t`"), df)
    }
  }

  test("SPARK-10130 type coercion for IF should have children resolved first") {
    withTempTable("src") {
      Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src")
      checkAnswer(
        sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0)))
    }
  }

  test("SPARK-10389: order by non-attribute grouping expression on Aggregate") {
    withTempTable("src") {
      Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src")
      checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1"),
        Seq(Row(1), Row(1)))
      checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY (key + 1) * 2"),
        Seq(Row(1), Row(1)))
    }
  }

  test("run sql directly on files") {
    val df = sqlContext.range(100).toDF()
    withTempPath(f => {
      df.write.json(f.getCanonicalPath)
      checkAnswer(sql(s"select id from json.`${f.getCanonicalPath}`"),
        df)
      checkAnswer(sql(s"select id from `org.apache.spark.sql.json`.`${f.getCanonicalPath}`"),
        df)
      checkAnswer(sql(s"select a.id from json.`${f.getCanonicalPath}` as a"),
        df)
    })

    val e1 = intercept[AnalysisException] {
      sql("select * from in_valid_table")
    }
    assert(e1.message.contains("Table not found"))

    val e2 = intercept[AnalysisException] {
      sql("select * from no_db.no_table").show()
    }
    assert(e2.message.contains("Table not found"))

    val e3 = intercept[AnalysisException] {
      sql("select * from json.invalid_file")
    }
    assert(e3.message.contains("Path does not exist"))
  }

  test("SortMergeJoin returns wrong results when using UnsafeRows") {
    // This test is for the fix of https://issues.apache.org/jira/browse/SPARK-10737.
    // This bug will be triggered when Tungsten is enabled and there are multiple
    // SortMergeJoin operators executed in the same task.
    val confs = SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1" :: Nil
    withSQLConf(confs: _*) {
      val df1 = (1 to 50).map(i => (s"str_$i", i)).toDF("i", "j")
      val df2 =
        df1
          .join(df1.select(df1("i")), "i")
          .select(df1("i"), df1("j"))

      val df3 = df2.withColumnRenamed("i", "i1").withColumnRenamed("j", "j1")
      val df4 =
        df2
          .join(df3, df2("i") === df3("i1"))
          .withColumn("diff", $"j" - $"j1")
          .select(df2("i"), df2("j"), $"diff")

      checkAnswer(
        df4,
        df1.withColumn("diff", lit(0)))
    }
  }

  test("SPARK-11032: resolve having correctly") {
    withTempTable("src") {
      Seq(1 -> "a").toDF("i", "j").registerTempTable("src")
      checkAnswer(
        sql("SELECT MIN(t.i) FROM (SELECT * FROM src WHERE i > 0) t HAVING(COUNT(1) > 0)"),
        Row(1))
    }
  }

  test("SPARK-11303: filter should not be pushed down into sample") {
    val df = sqlContext.range(100)
    List(true, false).foreach { withReplacement =>
      val sampled = df.sample(withReplacement, 0.1, 1)
      val sampledOdd = sampled.filter("id % 2 != 0")
      val sampledEven = sampled.filter("id % 2 = 0")
      assert(sampled.count() == sampledOdd.count() + sampledEven.count())
    }
  }

  test("Struct Star Expansion") {
    val structDf = testData2.select("a", "b").as("record")

    checkAnswer(
      structDf.select($"record.a", $"record.b"),
      Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil)

    checkAnswer(
      structDf.select($"record.*"),
      Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil)

    checkAnswer(
      structDf.select($"record.*", $"record.*"),
      Row(1, 1, 1, 1) :: Row(1, 2, 1, 2) :: Row(2, 1, 2, 1) :: Row(2, 2, 2, 2) ::
        Row(3, 1, 3, 1) :: Row(3, 2, 3, 2) :: Nil)

    checkAnswer(
      sql("select struct(a, b) as r1, struct(b, a) as r2 from testData2").select($"r1.*", $"r2.*"),
      Row(1, 1, 1, 1) :: Row(1, 2, 2, 1) :: Row(2, 1, 1, 2) :: Row(2, 2, 2, 2) ::
        Row(3, 1, 1, 3) :: Row(3, 2, 2, 3) :: Nil)

    // Try with a registered table.
    sql("select struct(a, b) as record from testData2").registerTempTable("structTable")
    checkAnswer(
      sql("SELECT record.* FROM structTable"),
      Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil)

    checkAnswer(sql(
      """
        | SELECT min(struct(record.*)) FROM
        |   (select struct(a,b) as record from testData2) tmp
      """.stripMargin),
      Row(Row(1, 1)) :: Nil)

    // Try with an alias on the select list
    checkAnswer(sql(
      """
        | SELECT max(struct(record.*)) as r FROM
        |   (select struct(a,b) as record from testData2) tmp
      """.stripMargin).select($"r.*"),
      Row(3, 2) :: Nil)

    // With GROUP BY
    checkAnswer(sql(
      """
        | SELECT min(struct(record.*)) FROM
        |   (select a as a, struct(a,b) as record from testData2) tmp
        | GROUP BY a
      """.stripMargin),
      Row(Row(1, 1)) :: Row(Row(2, 1)) :: Row(Row(3, 1)) :: Nil)

    // With GROUP BY and alias
    checkAnswer(sql(
      """
        | SELECT max(struct(record.*)) as r FROM
        |   (select a as a, struct(a,b) as record from testData2) tmp
        | GROUP BY a
      """.stripMargin).select($"r.*"),
      Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil)

    // With GROUP BY and alias and additional fields in the struct
    checkAnswer(sql(
      """
        | SELECT max(struct(a, record.*, b)) as r FROM
        |   (select a as a, b as b, struct(a,b) as record from testData2) tmp
        | GROUP BY a
      """.stripMargin).select($"r.*"),
      Row(1, 1, 2, 2) :: Row(2, 2, 2, 2) :: Row(3, 3, 2, 2) :: Nil)

    // Create a data set that contains nested structs.
    val nestedStructData = sql(
      """
        | SELECT struct(r1, r2) as record FROM
        |   (SELECT struct(a, b) as r1, struct(b, a) as r2 FROM testData2) tmp
      """.stripMargin)

    checkAnswer(nestedStructData.select($"record.*"),
      Row(Row(1, 1), Row(1, 1)) :: Row(Row(1, 2), Row(2, 1)) :: Row(Row(2, 1), Row(1, 2)) ::
        Row(Row(2, 2), Row(2, 2)) :: Row(Row(3, 1), Row(1, 3)) :: Row(Row(3, 2), Row(2, 3)) :: Nil)
    checkAnswer(nestedStructData.select($"record.r1"),
      Row(Row(1, 1)) :: Row(Row(1, 2)) :: Row(Row(2, 1)) :: Row(Row(2, 2)) ::
        Row(Row(3, 1)) :: Row(Row(3, 2)) :: Nil)
    checkAnswer(
      nestedStructData.select($"record.r1.*"),
      Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil)

    // Try with a registered table
    withTempTable("nestedStructTable") {
      nestedStructData.registerTempTable("nestedStructTable")
      checkAnswer(
        sql("SELECT record.* FROM nestedStructTable"),
        nestedStructData.select($"record.*"))
      checkAnswer(
        sql("SELECT record.r1 FROM nestedStructTable"),
        nestedStructData.select($"record.r1"))
      checkAnswer(
        sql("SELECT record.r1.* FROM nestedStructTable"),
        nestedStructData.select($"record.r1.*"))

      // Try resolving something not there.
      assert(intercept[AnalysisException](sql("SELECT abc.* FROM nestedStructTable"))
        .getMessage.contains("cannot resolve"))
    }

    // Create paths with unusual characters
    val specialCharacterPath = sql(
      """
        | SELECT struct(`col$.a_`, `a.b.c.`) as `r&&b.c` FROM
        |   (SELECT struct(a, b) as `col$.a_`, struct(b, a) as `a.b.c.` FROM testData2) tmp
      """.stripMargin)
    withTempTable("specialCharacterTable") {
      specialCharacterPath.registerTempTable("specialCharacterTable")
      checkAnswer(
        specialCharacterPath.select($"`r&&b.c`.*"),
        nestedStructData.select($"record.*"))
      checkAnswer(
        sql("SELECT `r&&b.c`.`col$.a_` FROM specialCharacterTable"),
        nestedStructData.select($"record.r1"))
      checkAnswer(
        sql("SELECT `r&&b.c`.`a.b.c.` FROM specialCharacterTable"),
        nestedStructData.select($"record.r2"))
      checkAnswer(
        sql("SELECT `r&&b.c`.`col$.a_`.* FROM specialCharacterTable"),
        nestedStructData.select($"record.r1.*"))
    }

    // Try star expanding a scalar. This should fail.
    assert(intercept[AnalysisException](sql("select a.* from testData2")).getMessage.contains(
      "Can only star expand struct data types."))
  }

  test("Struct Star Expansion - Name conflict") {
    // Create a data set that contains a naming conflict
    val nameConflict = sql("SELECT struct(a, b) as nameConflict, a as a FROM testData2")
    withTempTable("nameConflict") {
      nameConflict.registerTempTable("nameConflict")
      // Unqualified should resolve to table.
      checkAnswer(sql("SELECT nameConflict.* FROM nameConflict"),
        Row(Row(1, 1), 1) :: Row(Row(1, 2), 1) :: Row(Row(2, 1), 2) :: Row(Row(2, 2), 2) ::
          Row(Row(3, 1), 3) :: Row(Row(3, 2), 3) :: Nil)
      // Qualify the struct type with the table name.
      checkAnswer(sql("SELECT nameConflict.nameConflict.* FROM nameConflict"),
        Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil)
    }
  }

  test("Star Expansion - group by") {
    withSQLConf("spark.sql.retainGroupColumns" -> "false") {
      checkAnswer(
        testData2.groupBy($"a", $"b").agg($"*"),
        sql("SELECT * FROM testData2 group by a, b"))
    }
  }

  test("Common subexpression elimination") {
    // TODO: support subexpression elimination in whole stage codegen
    withSQLConf("spark.sql.codegen.wholeStage" -> "false") {
      // select from a table to prevent constant folding.
      val df = sql("SELECT a, b from testData2 limit 1")
      checkAnswer(df, Row(1, 1))

      checkAnswer(df.selectExpr("a + 1", "a + 1"), Row(2, 2))
      checkAnswer(df.selectExpr("a + 1", "a + 1 + 1"), Row(2, 3))

      // This does not work because the expressions get grouped like (a + a) + 1
      checkAnswer(df.selectExpr("a + 1", "a + a + 1"), Row(2, 3))
      checkAnswer(df.selectExpr("a + 1", "a + (a + 1)"), Row(2, 3))

      // Identity udf that tracks the number of times it is called.
      val countAcc = sparkContext.accumulator(0, "CallCount")
      sqlContext.udf.register("testUdf", (x: Int) => {
        countAcc.++=(1)
        x
      })

      // Evaluates df, verifying it is equal to the expectedResult and the accumulator's value
      // is correct.
      def verifyCallCount(df: DataFrame, expectedResult: Row, expectedCount: Int): Unit = {
        countAcc.setValue(0)
        checkAnswer(df, expectedResult)
        assert(countAcc.value == expectedCount)
      }

      verifyCallCount(df.selectExpr("testUdf(a)"), Row(1), 1)
      verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 1)
      verifyCallCount(df.selectExpr("testUdf(a + 1)", "testUdf(a + 1)"), Row(2, 2), 1)
      verifyCallCount(df.selectExpr("testUdf(a + 1)", "testUdf(a)"), Row(2, 1), 2)
      verifyCallCount(
        df.selectExpr("testUdf(a + 1) + testUdf(a + 1)", "testUdf(a + 1)"), Row(4, 2), 1)

      verifyCallCount(
        df.selectExpr("testUdf(a + 1) + testUdf(1 + b)", "testUdf(a + 1)"), Row(4, 2), 2)

      val testUdf = functions.udf((x: Int) => {
        countAcc.++=(1)
        x
      })
      verifyCallCount(
        df.groupBy().agg(sum(testUdf($"b") + testUdf($"b") + testUdf($"b"))), Row(3.0), 1)

      verifyCallCount(
        df.selectExpr("testUdf(a + 1) + testUdf(1 + a)", "testUdf(a + 1)"), Row(4, 2), 1)

      // Try disabling it via configuration.
      sqlContext.setConf("spark.sql.subexpressionElimination.enabled", "false")
      verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 2)
      sqlContext.setConf("spark.sql.subexpressionElimination.enabled", "true")
      verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 1)
    }
  }

  test("SPARK-10707: nullability should be correctly propagated through set operations (1)") {
    // This test produced an incorrect result of 1 before the SPARK-10707 fix because of the
    // NullPropagation rule: COUNT(v) got replaced with COUNT(1) because the output column of
    // UNION was incorrectly considered non-nullable:
    checkAnswer(
      sql("""SELECT count(v) FROM (
            |  SELECT v FROM (
            |    SELECT 'foo' AS v UNION ALL
            |    SELECT NULL AS v
            |  ) my_union WHERE isnull(v)
            |) my_subview""".stripMargin),
      Seq(Row(0)))
  }

  test("SPARK-10707: nullability should be correctly propagated through set operations (2)") {
    // This test uses RAND() to stop column pruning for Union and checks the resulting isnull
    // value. This would produce an incorrect result before the fix in SPARK-10707 because the "v"
    // column of the union was considered non-nullable.
    checkAnswer(
      sql(
        """
          |SELECT a FROM (
          |  SELECT ISNULL(v) AS a, RAND() FROM (
          |    SELECT 'foo' AS v UNION ALL SELECT null AS v
          |  ) my_union
          |) my_view
        """.stripMargin),
      Row(false) :: Row(true) :: Nil)
  }

  test("rollup") {
    checkAnswer(
      sql("select course, year, sum(earnings) from courseSales group by rollup(course, year)" +
        " order by course, year"),
      Row(null, null, 113000.0) ::
        Row("Java", null, 50000.0) ::
        Row("Java", 2012, 20000.0) ::
        Row("Java", 2013, 30000.0) ::
        Row("dotNET", null, 63000.0) ::
        Row("dotNET", 2012, 15000.0) ::
        Row("dotNET", 2013, 48000.0) :: Nil
    )
  }

  test("grouping sets when aggregate functions containing groupBy columns") {
    checkAnswer(
      sql("select course, sum(earnings) as sum from courseSales group by course, earnings " +
        "grouping sets((), (course), (course, earnings)) " +
        "order by course, sum"),
      Row(null, 113000.0) ::
        Row("Java", 20000.0) ::
        Row("Java", 30000.0) ::
        Row("Java", 50000.0) ::
        Row("dotNET", 5000.0) ::
        Row("dotNET", 10000.0) ::
        Row("dotNET", 48000.0) ::
        Row("dotNET", 63000.0) :: Nil
    )

    checkAnswer(
      sql("select course, sum(earnings) as sum, grouping_id(course, earnings) from courseSales " +
        "group by course, earnings grouping sets((), (course), (course, earnings)) " +
        "order by course, sum"),
      Row(null, 113000.0, 3) ::
        Row("Java", 20000.0, 0) ::
        Row("Java", 30000.0, 0) ::
        Row("Java", 50000.0, 1) ::
        Row("dotNET", 5000.0, 0) ::
        Row("dotNET", 10000.0, 0) ::
        Row("dotNET", 48000.0, 0) ::
        Row("dotNET", 63000.0, 1) :: Nil
    )
  }

  test("cube") {
    checkAnswer(
      sql("select course, year, sum(earnings) from courseSales group by cube(course, year)"),
      Row("Java", 2012, 20000.0) ::
        Row("Java", 2013, 30000.0) ::
        Row("Java", null, 50000.0) ::
        Row("dotNET", 2012, 15000.0) ::
        Row("dotNET", 2013, 48000.0) ::
        Row("dotNET", null, 63000.0) ::
        Row(null, 2012, 35000.0) ::
        Row(null, 2013, 78000.0) ::
        Row(null, null, 113000.0) :: Nil
    )
  }

  test("grouping sets") {
    checkAnswer(
      sql("select course, year, sum(earnings) from courseSales group by course, year " +
        "grouping sets(course, year)"),
      Row("Java", null, 50000.0) ::
        Row("dotNET", null, 63000.0) ::
        Row(null, 2012, 35000.0) ::
        Row(null, 2013, 78000.0) :: Nil
    )

    checkAnswer(
      sql("select course, year, sum(earnings) from courseSales group by course, year " +
        "grouping sets(course)"),
      Row("Java", null, 50000.0) ::
        Row("dotNET", null, 63000.0) :: Nil
    )

    checkAnswer(
      sql("select course, year, sum(earnings) from courseSales group by course, year " +
        "grouping sets(year)"),
      Row(null, 2012, 35000.0) ::
        Row(null, 2013, 78000.0) :: Nil
    )
  }

  test("grouping and grouping_id") {
    checkAnswer(
      sql("select course, year, grouping(course), grouping(year), grouping_id(course, year)" +
        " from courseSales group by cube(course, year)"),
      Row("Java", 2012, 0, 0, 0) ::
        Row("Java", 2013, 0, 0, 0) ::
        Row("Java", null, 0, 1, 1) ::
        Row("dotNET", 2012, 0, 0, 0) ::
        Row("dotNET", 2013, 0, 0, 0) ::
        Row("dotNET", null, 0, 1, 1) ::
        Row(null, 2012, 1, 0, 2) ::
        Row(null, 2013, 1, 0, 2) ::
        Row(null, null, 1, 1, 3) :: Nil
    )

    var error = intercept[AnalysisException] {
      sql("select course, year, grouping(course) from courseSales group by course, year")
    }
    assert(error.getMessage contains "grouping() can only be used with GroupingSets/Cube/Rollup")
    error = intercept[AnalysisException] {
      sql("select course, year, grouping_id(course, year) from courseSales group by course, year")
    }
    assert(error.getMessage contains "grouping_id() can only be used with GroupingSets/Cube/Rollup")
    error = intercept[AnalysisException] {
      sql("select course, year, grouping__id from courseSales group by cube(course, year)")
    }
    assert(error.getMessage contains "grouping__id is deprecated; use grouping_id() instead")
  }

  test("SPARK-13056: Null in map value causes NPE") {
    val df = Seq(1 -> Map("abc" -> "somestring", "cba" -> null)).toDF("key", "value")
    withTempTable("maptest") {
      df.registerTempTable("maptest")
      // local optimization will by pass codegen code, so we should keep the filter `key=1`
      checkAnswer(sql("SELECT value['abc'] FROM maptest where key = 1"), Row("somestring"))
      checkAnswer(sql("SELECT value['cba'] FROM maptest where key = 1"), Row(null))
    }
  }

  test("hash function") {
    val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j")
    withTempTable("tbl") {
      df.registerTempTable("tbl")
      checkAnswer(
        df.select(hash($"i", $"j")),
        sql("SELECT hash(i, j) from tbl")
      )
    }
  }

  test("order by ordinal number") {
    checkAnswer(
      sql("SELECT * FROM testData2 ORDER BY 1 DESC"),
      sql("SELECT * FROM testData2 ORDER BY a DESC"))
    // If the position is not an integer, ignore it.
    checkAnswer(
      sql("SELECT * FROM testData2 ORDER BY 1 + 0 DESC, b ASC"),
      sql("SELECT * FROM testData2 ORDER BY b ASC"))
    checkAnswer(
      sql("SELECT * FROM testData2 ORDER BY 1 DESC, b ASC"),
      sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC"))
    checkAnswer(
      sql("SELECT * FROM testData2 SORT BY 1 DESC, 2"),
      sql("SELECT * FROM testData2 SORT BY a DESC, b ASC"))
    checkAnswer(
      sql("SELECT * FROM testData2 ORDER BY 1 ASC, b ASC"),
      Seq(Row(1, 1), Row(1, 2), Row(2, 1), Row(2, 2), Row(3, 1), Row(3, 2)))
  }

  test("order by ordinal number - negative cases") {
    intercept[UnresolvedException[SortOrder]] {
      sql("SELECT * FROM testData2 ORDER BY 0")
    }
    intercept[UnresolvedException[SortOrder]] {
      sql("SELECT * FROM testData2 ORDER BY -1 DESC, b ASC")
    }
    intercept[UnresolvedException[SortOrder]] {
      sql("SELECT * FROM testData2 ORDER BY 3 DESC, b ASC")
    }
  }

  test("order by ordinal number with conf spark.sql.orderByOrdinal=false") {
    withSQLConf(SQLConf.ORDER_BY_ORDINAL.key -> "false") {
      // If spark.sql.orderByOrdinal=false, ignore the position number.
      checkAnswer(
        sql("SELECT * FROM testData2 ORDER BY 1 DESC, b ASC"),
        sql("SELECT * FROM testData2 ORDER BY b ASC"))
    }
  }

  test("natural join") {
    val df1 = Seq(("one", 1), ("two", 2), ("three", 3)).toDF("k", "v1")
    val df2 = Seq(("one", 1), ("two", 22), ("one", 5)).toDF("k", "v2")
    withTempTable("nt1", "nt2") {
      df1.registerTempTable("nt1")
      df2.registerTempTable("nt2")
      checkAnswer(
        sql("SELECT * FROM nt1 natural join nt2 where k = \"one\""),
        Row("one", 1, 1) :: Row("one", 1, 5) :: Nil)

      checkAnswer(
        sql("SELECT * FROM nt1 natural left join nt2 order by v1, v2"),
        Row("one", 1, 1) :: Row("one", 1, 5) :: Row("two", 2, 22) :: Row("three", 3, null) :: Nil)

      checkAnswer(
        sql("SELECT * FROM nt1 natural right join nt2 order by v1, v2"),
        Row("one", 1, 1) :: Row("one", 1, 5) :: Row("two", 2, 22) :: Nil)

      checkAnswer(
        sql("SELECT count(*) FROM nt1 natural full outer join nt2"),
        Row(4) :: Nil)
    }
  }

  test("join with using clause") {
    val df1 = Seq(("r1c1", "r1c2", "t1r1c3"),
      ("r2c1", "r2c2", "t1r2c3"), ("r3c1x", "r3c2", "t1r3c3")).toDF("c1", "c2", "c3")
    val df2 = Seq(("r1c1", "r1c2", "t2r1c3"),
      ("r2c1", "r2c2", "t2r2c3"), ("r3c1y", "r3c2", "t2r3c3")).toDF("c1", "c2", "c3")
    val df3 = Seq((null, "r1c2", "t3r1c3"),
      ("r2c1", "r2c2", "t3r2c3"), ("r3c1y", "r3c2", "t3r3c3")).toDF("c1", "c2", "c3")
    withTempTable("t1", "t2", "t3") {
      df1.registerTempTable("t1")
      df2.registerTempTable("t2")
      df3.registerTempTable("t3")
      // inner join with one using column
      checkAnswer(
        sql("SELECT * FROM t1 join t2 using (c1)"),
        Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t2r1c3") ::
          Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t2r2c3") :: Nil)

      // inner join with two using columns
      checkAnswer(
        sql("SELECT * FROM t1 join t2 using (c1, c2)"),
        Row("r1c1", "r1c2", "t1r1c3", "t2r1c3") ::
          Row("r2c1", "r2c2", "t1r2c3", "t2r2c3") :: Nil)

      // Left outer join with one using column.
      checkAnswer(
        sql("SELECT * FROM t1 left join t2 using (c1)"),
        Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t2r1c3") ::
          Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t2r2c3") ::
          Row("r3c1x", "r3c2", "t1r3c3", null, null) :: Nil)

      // Right outer join with one using column.
      checkAnswer(
        sql("SELECT * FROM t1 right join t2 using (c1)"),
        Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t2r1c3") ::
          Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t2r2c3") ::
          Row("r3c1y", null, null, "r3c2", "t2r3c3") :: Nil)

      // Full outer join with one using column.
      checkAnswer(
        sql("SELECT * FROM t1 full outer join t2 using (c1)"),
        Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t2r1c3") ::
          Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t2r2c3") ::
          Row("r3c1x", "r3c2", "t1r3c3", null, null) ::
          Row("r3c1y", null,
            null, "r3c2", "t2r3c3") :: Nil)

      // Full outer join with null value in join column.
      checkAnswer(
        sql("SELECT * FROM t1 full outer join t3 using (c1)"),
        Row("r1c1", "r1c2", "t1r1c3", null, null) ::
          Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t3r2c3") ::
          Row("r3c1x", "r3c2", "t1r3c3", null, null) ::
          Row("r3c1y", null, null, "r3c2", "t3r3c3") ::
          Row(null, null, null, "r1c2", "t3r1c3") :: Nil)

      // Self join with using columns.
      checkAnswer(
        sql("SELECT * FROM t1 join t1 using (c1)"),
        Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t1r1c3") ::
          Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t1r2c3") ::
          Row("r3c1x", "r3c2", "t1r3c3", "r3c2", "t1r3c3") :: Nil)
    }
  }
}