aboutsummaryrefslogtreecommitdiff
path: root/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
blob: 7fa5c29dc5b8f46382d6c9d30ae55803b97e1caf (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
/*
 * 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.catalyst

import java.nio.charset.StandardCharsets
import java.nio.file.{Files, NoSuchFileException, Paths}

import scala.util.control.NonFatal

import org.apache.spark.sql.Column
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.plans.logical.LeafNode
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SQLTestUtils

/**
 * A test suite for LogicalPlan-to-SQL conversion.
 *
 * Each query has a golden generated SQL file in test/resources/sqlgen. The test suite also has
 * built-in functionality to automatically generate these golden files.
 *
 * To re-generate golden files, run:
 *    SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "hive/test-only *LogicalPlanToSQLSuite"
 */
class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils {
  import testImplicits._

  // Used for generating new query answer files by saving
  private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1"
  private val goldenSQLPath = "src/test/resources/sqlgen/"

  protected override def beforeAll(): Unit = {
    super.beforeAll()
    (0 to 3).foreach { i =>
      sql(s"DROP TABLE IF EXISTS parquet_t$i")
    }
    sql("DROP TABLE IF EXISTS t0")

    spark.range(10).write.saveAsTable("parquet_t0")
    sql("CREATE TABLE t0 AS SELECT * FROM parquet_t0")

    spark
      .range(10)
      .select('id as 'key, concat(lit("val_"), 'id) as 'value)
      .write
      .saveAsTable("parquet_t1")

    spark
      .range(10)
      .select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd)
      .write
      .saveAsTable("parquet_t2")

    def createArray(id: Column): Column = {
      when(id % 3 === 0, lit(null)).otherwise(array('id, 'id + 1))
    }

    spark
      .range(10)
      .select(
        createArray('id).as("arr"),
        array(array('id), createArray('id)).as("arr2"),
        lit("""{"f1": "1", "f2": "2", "f3": 3}""").as("json"),
        'id
      )
      .write
      .saveAsTable("parquet_t3")
  }

  override protected def afterAll(): Unit = {
    try {
      (0 to 3).foreach { i =>
        sql(s"DROP TABLE IF EXISTS parquet_t$i")
      }
      sql("DROP TABLE IF EXISTS t0")
    } finally {
      super.afterAll()
    }
  }

  /**
   * Compare the generated SQL with the expected answer string.
   */
  private def checkSQLStructure(originalSQL: String, convertedSQL: String, answerFile: String) = {
    if (answerFile != null) {
      val separator = "-" * 80
      if (regenerateGoldenFiles) {
        val path = Paths.get(s"$goldenSQLPath/$answerFile.sql")
        val header = "-- This file is automatically generated by LogicalPlanToSQLSuite."
        val answerText = s"$header\n${originalSQL.trim()}\n${separator}\n$convertedSQL\n"
        Files.write(path, answerText.getBytes(StandardCharsets.UTF_8))
      } else {
        val goldenFileName = s"sqlgen/$answerFile.sql"
        val resourceFile = getClass.getClassLoader.getResource(goldenFileName)
        if (resourceFile == null) {
          throw new NoSuchFileException(goldenFileName)
        }
        val path = resourceFile.getPath
        val answerText = new String(Files.readAllBytes(Paths.get(path)), StandardCharsets.UTF_8)
        val sqls = answerText.split(separator)
        assert(sqls.length == 2, "Golden sql files should have a separator.")
        val expectedSQL = sqls(1).trim()
        assert(convertedSQL == expectedSQL)
      }
    }
  }

  /**
   * 1. Checks if SQL parsing succeeds.
   * 2. Checks if SQL generation succeeds.
   * 3. Checks the generated SQL against golden files.
   * 4. Verifies the execution result stays the same.
   */
  private def checkSQL(sqlString: String, answerFile: String = null): Unit = {
    val df = sql(sqlString)

    val convertedSQL = try new SQLBuilder(df).toSQL catch {
      case NonFatal(e) =>
        fail(
          s"""Cannot convert the following SQL query plan back to SQL query string:
             |
             |# Original SQL query string:
             |$sqlString
             |
             |# Resolved query plan:
             |${df.queryExecution.analyzed.treeString}
           """.stripMargin, e)
    }

    checkSQLStructure(sqlString, convertedSQL, answerFile)

    try {
      checkAnswer(sql(convertedSQL), df)
    } catch { case cause: Throwable =>
      fail(
        s"""Failed to execute converted SQL string or got wrong answer:
           |
           |# Converted SQL query string:
           |$convertedSQL
           |
           |# Original SQL query string:
           |$sqlString
           |
           |# Resolved query plan:
           |${df.queryExecution.analyzed.treeString}
         """.stripMargin, cause)
    }
  }

  // When saving golden files, these tests should be ignored to prevent making files.
  if (!regenerateGoldenFiles) {
    test("Test should fail if the SQL query cannot be parsed") {
      val m = intercept[ParseException] {
        checkSQL("SELE", "NOT_A_FILE")
      }.getMessage
      assert(m.contains("mismatched input"))
    }

    test("Test should fail if the golden file cannot be found") {
      val m2 = intercept[NoSuchFileException] {
        checkSQL("SELECT 1", "NOT_A_FILE")
      }.getMessage
      assert(m2.contains("NOT_A_FILE"))
    }

    test("Test should fail if the SQL query cannot be regenerated") {
      case class Unsupported() extends LeafNode with MultiInstanceRelation {
        override def newInstance(): Unsupported = copy()
        override def output: Seq[Attribute] = Nil
      }
      Unsupported().createOrReplaceTempView("not_sql_gen_supported_table_so_far")
      sql("select * from not_sql_gen_supported_table_so_far")
      val m3 = intercept[org.scalatest.exceptions.TestFailedException] {
        checkSQL("select * from not_sql_gen_supported_table_so_far", "in")
      }.getMessage
      assert(m3.contains("Cannot convert the following SQL query plan back to SQL query string"))
    }

    test("Test should fail if the SQL query did not equal to the golden SQL") {
      val m4 = intercept[org.scalatest.exceptions.TestFailedException] {
        checkSQL("SELECT 1", "in")
      }.getMessage
      assert(m4.contains("did not equal"))
    }
  }

  test("range") {
    checkSQL("select * from range(100)", "range")
    checkSQL("select * from range(1, 100, 20, 10)", "range_with_splits")
  }

  test("in") {
    checkSQL("SELECT id FROM parquet_t0 WHERE id IN (1, 2, 3)", "in")
  }

  test("not in") {
    checkSQL("SELECT id FROM t0 WHERE id NOT IN (1, 2, 3)", "not_in")
  }

  test("not like") {
    checkSQL("SELECT id FROM t0 WHERE id + 5 NOT LIKE '1%'", "not_like")
  }

  test("aggregate function in having clause") {
    checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0", "agg1")
  }

  test("aggregate function in order by clause") {
    checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY MAX(key)", "agg2")
  }

  // When there are multiple aggregate functions in ORDER BY clause, all of them are extracted into
  // Aggregate operator and aliased to the same name "aggOrder".  This is OK for normal query
  // execution since these aliases have different expression ID.  But this introduces name collision
  // when converting resolved plans back to SQL query strings as expression IDs are stripped.
  test("aggregate function in order by clause with multiple order keys") {
    checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key, MAX(key)", "agg3")
  }

  test("order by asc nulls last") {
    checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key nulls last, MAX(key)",
      "sort_asc_nulls_last")
  }

  test("order by desc nulls first") {
    checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key desc nulls first," +
      "MAX(key)", "sort_desc_nulls_first")
  }

  test("type widening in union") {
    checkSQL("SELECT id FROM parquet_t0 UNION ALL SELECT CAST(id AS INT) AS id FROM parquet_t0",
      "type_widening")
  }

  test("union distinct") {
    checkSQL("SELECT * FROM t0 UNION SELECT * FROM t0", "union_distinct")
  }

  test("three-child union") {
    checkSQL(
      """
        |SELECT id FROM parquet_t0
        |UNION ALL SELECT id FROM parquet_t0
        |UNION ALL SELECT id FROM parquet_t0
      """.stripMargin,
      "three_child_union")
  }

  test("intersect") {
    checkSQL("SELECT * FROM t0 INTERSECT SELECT * FROM t0", "intersect")
  }

  test("except") {
    checkSQL("SELECT * FROM t0 EXCEPT SELECT * FROM t0", "except")
  }

  test("self join") {
    checkSQL("SELECT x.key FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key", "self_join")
  }

  test("self join with group by") {
    checkSQL(
      "SELECT x.key, COUNT(*) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key group by x.key",
      "self_join_with_group_by")
  }

  test("case") {
    checkSQL("SELECT CASE WHEN id % 2 > 0 THEN 0 WHEN id % 2 = 0 THEN 1 END FROM parquet_t0",
      "case")
  }

  test("case with else") {
    checkSQL("SELECT CASE WHEN id % 2 > 0 THEN 0 ELSE 1 END FROM parquet_t0", "case_with_else")
  }

  test("case with key") {
    checkSQL("SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' END FROM parquet_t0",
      "case_with_key")
  }

  test("case with key and else") {
    checkSQL("SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' ELSE 'baz' END FROM parquet_t0",
      "case_with_key_and_else")
  }

  test("select distinct without aggregate functions") {
    checkSQL("SELECT DISTINCT id FROM parquet_t0", "select_distinct")
  }

  test("rollup/cube #1") {
    // Original logical plan:
    //   Aggregate [(key#17L % cast(5 as bigint))#47L,grouping__id#46],
    //             [(count(1),mode=Complete,isDistinct=false) AS cnt#43L,
    //              (key#17L % cast(5 as bigint))#47L AS _c1#45L,
    //              grouping__id#46 AS _c2#44]
    //   +- Expand [List(key#17L, value#18, (key#17L % cast(5 as bigint))#47L, 0),
    //              List(key#17L, value#18, null, 1)],
    //             [key#17L,value#18,(key#17L % cast(5 as bigint))#47L,grouping__id#46]
    //      +- Project [key#17L,
    //                  value#18,
    //                  (key#17L % cast(5 as bigint)) AS (key#17L % cast(5 as bigint))#47L]
    //         +- Subquery t1
    //            +- Relation[key#17L,value#18] ParquetRelation
    // Converted SQL:
    //   SELECT count( 1) AS `cnt`,
    //          (`t1`.`key` % CAST(5 AS BIGINT)),
    //          grouping_id() AS `_c2`
    //   FROM `default`.`t1`
    //   GROUP BY (`t1`.`key` % CAST(5 AS BIGINT))
    //   GROUPING SETS (((`t1`.`key` % CAST(5 AS BIGINT))), ())
    checkSQL(
      "SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH ROLLUP",
      "rollup_cube_1_1")

    checkSQL(
      "SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH CUBE",
      "rollup_cube_1_2")
  }

  test("rollup/cube #2") {
    checkSQL("SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH ROLLUP",
      "rollup_cube_2_1")

    checkSQL("SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH CUBE",
      "rollup_cube_2_2")
  }

  test("rollup/cube #3") {
    checkSQL(
      "SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH ROLLUP",
      "rollup_cube_3_1")

    checkSQL(
      "SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH CUBE",
      "rollup_cube_3_2")
  }

  test("rollup/cube #4") {
    checkSQL(
      s"""
        |SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1
        |GROUP BY key % 5, key - 5 WITH ROLLUP
      """.stripMargin,
      "rollup_cube_4_1")

    checkSQL(
      s"""
        |SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1
        |GROUP BY key % 5, key - 5 WITH CUBE
      """.stripMargin,
      "rollup_cube_4_2")
  }

  test("rollup/cube #5") {
    checkSQL(
      s"""
        |SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - 5) AS k3
        |FROM (SELECT key, key%2, key - 5 FROM parquet_t1) t GROUP BY key%5, key-5
        |WITH ROLLUP
      """.stripMargin,
      "rollup_cube_5_1")

    checkSQL(
      s"""
        |SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - 5) AS k3
        |FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5
        |WITH CUBE
      """.stripMargin,
      "rollup_cube_5_2")
  }

  test("rollup/cube #6") {
    checkSQL("SELECT a, b, sum(c) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b",
      "rollup_cube_6_1")

    checkSQL("SELECT a, b, sum(c) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b",
      "rollup_cube_6_2")

    checkSQL("SELECT a, b, sum(a) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b",
      "rollup_cube_6_3")

    checkSQL("SELECT a, b, sum(a) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b",
      "rollup_cube_6_4")

    checkSQL("SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH ROLLUP",
      "rollup_cube_6_5")

    checkSQL("SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH CUBE",
      "rollup_cube_6_6")
  }

  test("rollup/cube #7") {
    checkSQL("SELECT a, b, grouping_id(a, b) FROM parquet_t2 GROUP BY cube(a, b)",
      "rollup_cube_7_1")

    checkSQL("SELECT a, b, grouping(b) FROM parquet_t2 GROUP BY cube(a, b)",
      "rollup_cube_7_2")

    checkSQL("SELECT a, b, grouping(a) FROM parquet_t2 GROUP BY cube(a, b)",
      "rollup_cube_7_3")
  }

  test("rollup/cube #8") {
    // grouping_id() is part of another expression
    checkSQL(
      s"""
         |SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid
         |FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5
         |WITH ROLLUP
      """.stripMargin,
      "rollup_cube_8_1")

    checkSQL(
      s"""
         |SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid
         |FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5
         |WITH CUBE
      """.stripMargin,
      "rollup_cube_8_2")
  }

  test("rollup/cube #9") {
    // self join is used as the child node of ROLLUP/CUBE with replaced quantifiers
    checkSQL(
      s"""
         |SELECT t.key - 5, cnt, SUM(cnt)
         |FROM (SELECT x.key, COUNT(*) as cnt
         |FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t
         |GROUP BY cnt, t.key - 5
         |WITH ROLLUP
      """.stripMargin,
      "rollup_cube_9_1")

    checkSQL(
      s"""
         |SELECT t.key - 5, cnt, SUM(cnt)
         |FROM (SELECT x.key, COUNT(*) as cnt
         |FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t
         |GROUP BY cnt, t.key - 5
         |WITH CUBE
      """.stripMargin,
      "rollup_cube_9_2")
  }

  test("grouping sets #1") {
    checkSQL(
      s"""
         |SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id() AS k3
         |FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5
         |GROUPING SETS (key % 5, key - 5)
      """.stripMargin,
      "grouping_sets_1")
  }

  test("grouping sets #2") {
    checkSQL(
      "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a, b) ORDER BY a, b",
      "grouping_sets_2_1")

    checkSQL(
      "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a) ORDER BY a, b",
      "grouping_sets_2_2")

    checkSQL(
      "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (b) ORDER BY a, b",
      "grouping_sets_2_3")

    checkSQL(
      "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (()) ORDER BY a, b",
      "grouping_sets_2_4")

    checkSQL(
      s"""
         |SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b
         |GROUPING SETS ((), (a), (a, b)) ORDER BY a, b
      """.stripMargin,
      "grouping_sets_2_5")
  }

  test("cluster by") {
    checkSQL("SELECT id FROM parquet_t0 CLUSTER BY id", "cluster_by")
  }

  test("distribute by") {
    checkSQL("SELECT id FROM parquet_t0 DISTRIBUTE BY id", "distribute_by")
  }

  test("distribute by with sort by") {
    checkSQL("SELECT id FROM parquet_t0 DISTRIBUTE BY id SORT BY id",
      "distribute_by_with_sort_by")
  }

  test("SPARK-13720: sort by after having") {
    checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 SORT BY key",
      "sort_by_after_having")
  }

  test("distinct aggregation") {
    checkSQL("SELECT COUNT(DISTINCT id) FROM parquet_t0", "distinct_aggregation")
  }

  test("TABLESAMPLE") {
    // Project [id#2L]
    // +- Sample 0.0, 1.0, false, ...
    //    +- Subquery s
    //       +- Subquery parquet_t0
    //          +- Relation[id#2L] ParquetRelation
    checkSQL("SELECT s.id FROM parquet_t0 TABLESAMPLE(100 PERCENT) s", "tablesample_1")

    // Project [id#2L]
    // +- Sample 0.0, 1.0, false, ...
    //    +- Subquery parquet_t0
    //       +- Relation[id#2L] ParquetRelation
    checkSQL("SELECT * FROM parquet_t0 TABLESAMPLE(100 PERCENT)", "tablesample_2")

    // Project [id#21L]
    // +- Sample 0.0, 1.0, false, ...
    //    +- MetastoreRelation default, t0, Some(s)
    checkSQL("SELECT s.id FROM t0 TABLESAMPLE(100 PERCENT) s", "tablesample_3")

    // Project [id#24L]
    // +- Sample 0.0, 1.0, false, ...
    //    +- MetastoreRelation default, t0, None
    checkSQL("SELECT * FROM t0 TABLESAMPLE(100 PERCENT)", "tablesample_4")

    // When a sampling fraction is not 100%, the returned results are random.
    // Thus, added an always-false filter here to check if the generated plan can be successfully
    // executed.
    checkSQL("SELECT s.id FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) s WHERE 1=0", "tablesample_5")
    checkSQL("SELECT * FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) WHERE 1=0", "tablesample_6")
  }

  test("multi-distinct columns") {
    checkSQL("SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM parquet_t2 GROUP BY a",
      "multi_distinct")
  }

  test("persisted data source relations") {
    Seq("orc", "json", "parquet").foreach { format =>
      val tableName = s"${format}_parquet_t0"
      withTable(tableName) {
        spark.range(10).write.format(format).saveAsTable(tableName)
        checkSQL(s"SELECT id FROM $tableName", s"data_source_$tableName")
      }
    }
  }

  test("script transformation - schemaless") {
    checkSQL("SELECT TRANSFORM (a, b, c, d) USING 'cat' FROM parquet_t2",
      "script_transformation_1")
    checkSQL("SELECT TRANSFORM (*) USING 'cat' FROM parquet_t2",
      "script_transformation_2")
  }

  test("script transformation - alias list") {
    checkSQL("SELECT TRANSFORM (a, b, c, d) USING 'cat' AS (d1, d2, d3, d4) FROM parquet_t2",
      "script_transformation_alias_list")
  }

  test("script transformation - alias list with type") {
    checkSQL(
      """FROM
        |(FROM parquet_t1 SELECT TRANSFORM(key, value) USING 'cat' AS (thing1 int, thing2 string)) t
        |SELECT thing1 + 1
      """.stripMargin,
      "script_transformation_alias_list_with_type")
  }

  test("script transformation - row format delimited clause with only one format property") {
    checkSQL(
      """SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'
        |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'
        |FROM parquet_t1
      """.stripMargin,
      "script_transformation_row_format_one")
  }

  test("script transformation - row format delimited clause with multiple format properties") {
    checkSQL(
      """SELECT TRANSFORM (key)
        |ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t'
        |USING 'cat' AS (tKey)
        |ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t'
        |FROM parquet_t1
      """.stripMargin,
      "script_transformation_row_format_multiple")
  }

  test("script transformation - row format serde clauses with SERDEPROPERTIES") {
    checkSQL(
      """SELECT TRANSFORM (key, value)
        |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
        |WITH SERDEPROPERTIES('field.delim' = '|')
        |USING 'cat' AS (tKey, tValue)
        |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
        |WITH SERDEPROPERTIES('field.delim' = '|')
        |FROM parquet_t1
      """.stripMargin,
      "script_transformation_row_format_serde")
  }

  test("script transformation - row format serde clauses without SERDEPROPERTIES") {
    checkSQL(
      """SELECT TRANSFORM (key, value)
        |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
        |USING 'cat' AS (tKey, tValue)
        |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
        |FROM parquet_t1
      """.stripMargin,
      "script_transformation_row_format_without_serde")
  }

  test("plans with non-SQL expressions") {
    spark.udf.register("foo", (_: Int) * 2)
    intercept[UnsupportedOperationException](new SQLBuilder(sql("SELECT foo(id) FROM t0")).toSQL)
  }

  test("named expression in column names shouldn't be quoted") {
    def checkColumnNames(query: String, expectedColNames: String*): Unit = {
      checkSQL(query)
      assert(sql(query).columns === expectedColNames)
    }

    // Attributes
    checkColumnNames(
      """SELECT * FROM (
        |  SELECT 1 AS a, 2 AS b, 3 AS `we``ird`
        |) s
      """.stripMargin,
      "a", "b", "we`ird"
    )

    checkColumnNames(
      """SELECT x.a, y.a, x.b, y.b
        |FROM (SELECT 1 AS a, 2 AS b) x
        |CROSS JOIN (SELECT 1 AS a, 2 AS b) y
        |ON x.a = y.a
      """.stripMargin,
      "a", "a", "b", "b"
    )

    // String literal
    checkColumnNames(
      "SELECT 'foo', '\"bar\\''",
      "foo", "\"bar\'"
    )

    // Numeric literals (should have CAST or suffixes in column names)
    checkColumnNames(
      "SELECT 1Y, 2S, 3, 4L, 5.1, 6.1D",
      "1", "2", "3", "4", "5.1", "6.1"
    )

    // Aliases
    checkColumnNames(
      "SELECT 1 AS a",
      "a"
    )

    // Complex type extractors
    checkColumnNames(
      """SELECT
        |  a.f1, b[0].f1, b.f1, c["foo"], d[0]
        |FROM (
        |  SELECT
        |    NAMED_STRUCT("f1", 1, "f2", "foo") AS a,
        |    ARRAY(NAMED_STRUCT("f1", 1, "f2", "foo")) AS b,
        |    MAP("foo", 1) AS c,
        |    ARRAY(1) AS d
        |) s
      """.stripMargin,
      "f1", "b[0].f1", "f1", "c[foo]", "d[0]"
    )
  }

  test("window basic") {
    checkSQL("SELECT MAX(value) OVER (PARTITION BY key % 3) FROM parquet_t1", "window_basic_1")

    checkSQL(
      """
         |SELECT key, value, ROUND(AVG(key) OVER (), 2)
         |FROM parquet_t1 ORDER BY key
      """.stripMargin,
      "window_basic_2")

    checkSQL(
      """
         |SELECT value, MAX(key + 1) OVER (PARTITION BY key % 5 ORDER BY key % 7) AS max
         |FROM parquet_t1
      """.stripMargin,
      "window_basic_3")

    checkSQL(
      """
        |SELECT key, value, ROUND(AVG(key) OVER (), 2)
        |FROM parquet_t1 ORDER BY key nulls last
      """.stripMargin,
      "window_basic_asc_nulls_last")

    checkSQL(
      """
        |SELECT key, value, ROUND(AVG(key) OVER (), 2)
        |FROM parquet_t1 ORDER BY key desc nulls first
      """.stripMargin,
      "window_basic_desc_nulls_first")
  }

  test("multiple window functions in one expression") {
    checkSQL(
      """
        |SELECT
        |  MAX(key) OVER (ORDER BY key DESC, value) / MIN(key) OVER (PARTITION BY key % 3)
        |FROM parquet_t1
      """.stripMargin)
  }

  test("regular expressions and window functions in one expression") {
    checkSQL("SELECT MAX(key) OVER (PARTITION BY key % 3) + key FROM parquet_t1",
      "regular_expressions_and_window")
  }

  test("aggregate functions and window functions in one expression") {
    checkSQL("SELECT MAX(c) + COUNT(a) OVER () FROM parquet_t2 GROUP BY a, b",
      "aggregate_functions_and_window")
  }

  test("window with different window specification") {
    checkSQL(
      """
         |SELECT key, value,
         |DENSE_RANK() OVER (ORDER BY key, value) AS dr,
         |MAX(value) OVER (PARTITION BY key ORDER BY key ASC) AS max
         |FROM parquet_t1
      """.stripMargin)
  }

  test("window with the same window specification with aggregate + having") {
    checkSQL(
      """
         |SELECT key, value,
         |MAX(value) OVER (PARTITION BY key % 5 ORDER BY key DESC) AS max
         |FROM parquet_t1 GROUP BY key, value HAVING key > 5
      """.stripMargin,
      "window_with_the_same_window_with_agg_having")
  }

  test("window with the same window specification with aggregate functions") {
    checkSQL(
      """
         |SELECT key, value,
         |MAX(value) OVER (PARTITION BY key % 5 ORDER BY key) AS max
         |FROM parquet_t1 GROUP BY key, value
      """.stripMargin,
      "window_with_the_same_window_with_agg_functions")
  }

  test("window with the same window specification with aggregate") {
    checkSQL(
      """
         |SELECT key, value,
         |DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr,
         |COUNT(key)
         |FROM parquet_t1 GROUP BY key, value
      """.stripMargin,
      "window_with_the_same_window_with_agg")
  }

  test("window with the same window specification without aggregate and filter") {
    checkSQL(
      """
         |SELECT key, value,
         |DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr,
         |COUNT(key) OVER(DISTRIBUTE BY key SORT BY key, value) AS ca
         |FROM parquet_t1
      """.stripMargin,
      "window_with_the_same_window_with_agg_filter")
  }

  test("window clause") {
    checkSQL(
      """
         |SELECT key, MAX(value) OVER w1 AS MAX, MIN(value) OVER w2 AS min
         |FROM parquet_t1
         |WINDOW w1 AS (PARTITION BY key % 5 ORDER BY key), w2 AS (PARTITION BY key % 6)
      """.stripMargin)
  }

  test("special window functions") {
    checkSQL(
      """
        |SELECT
        |  RANK() OVER w,
        |  PERCENT_RANK() OVER w,
        |  DENSE_RANK() OVER w,
        |  ROW_NUMBER() OVER w,
        |  NTILE(10) OVER w,
        |  CUME_DIST() OVER w,
        |  LAG(key, 2) OVER w,
        |  LEAD(key, 2) OVER w
        |FROM parquet_t1
        |WINDOW w AS (PARTITION BY key % 5 ORDER BY key)
      """.stripMargin)
  }

  test("window with join") {
    checkSQL(
      """
        |SELECT x.key, MAX(y.key) OVER (PARTITION BY x.key % 5 ORDER BY x.key)
        |FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key
      """.stripMargin,
      "window_with_join")
  }

  test("join 2 tables and aggregate function in having clause") {
    checkSQL(
      """
        |SELECT COUNT(a.value), b.KEY, a.KEY
        |FROM parquet_t1 a CROSS JOIN parquet_t1 b
        |GROUP BY a.KEY, b.KEY
        |HAVING MAX(a.KEY) > 0
      """.stripMargin,
      "join_2_tables")
  }

  test("generator in project list without FROM clause") {
    checkSQL("SELECT EXPLODE(ARRAY(1,2,3))", "generator_without_from_1")
    checkSQL("SELECT EXPLODE(ARRAY(1,2,3)) AS val", "generator_without_from_2")
  }

  test("generator in project list with non-referenced table") {
    checkSQL("SELECT EXPLODE(ARRAY(1,2,3)) FROM t0", "generator_non_referenced_table_1")
    checkSQL("SELECT EXPLODE(ARRAY(1,2,3)) AS val FROM t0", "generator_non_referenced_table_2")
  }

  test("generator in project list with referenced table") {
    checkSQL("SELECT EXPLODE(arr) FROM parquet_t3", "generator_referenced_table_1")
    checkSQL("SELECT EXPLODE(arr) AS val FROM parquet_t3", "generator_referenced_table_2")
  }

  test("generator in project list with non-UDTF expressions") {
    checkSQL("SELECT EXPLODE(arr), id FROM parquet_t3", "generator_non_udtf_1")
    checkSQL("SELECT EXPLODE(arr) AS val, id as a FROM parquet_t3", "generator_non_udtf_2")
  }

  test("generator in lateral view") {
    checkSQL("SELECT val, id FROM parquet_t3 LATERAL VIEW EXPLODE(arr) exp AS val",
      "generator_in_lateral_view_1")
    checkSQL("SELECT val, id FROM parquet_t3 LATERAL VIEW OUTER EXPLODE(arr) exp AS val",
      "generator_in_lateral_view_2")
  }

  test("generator in lateral view with ambiguous names") {
    checkSQL(
      """
        |SELECT exp.id, parquet_t3.id
        |FROM parquet_t3
        |LATERAL VIEW EXPLODE(arr) exp AS id
      """.stripMargin,
      "generator_with_ambiguous_names_1")

    checkSQL(
      """
        |SELECT exp.id, parquet_t3.id
        |FROM parquet_t3
        |LATERAL VIEW OUTER EXPLODE(arr) exp AS id
      """.stripMargin,
      "generator_with_ambiguous_names_2")
  }

  test("use JSON_TUPLE as generator") {
    checkSQL(
      """
        |SELECT c0, c1, c2
        |FROM parquet_t3
        |LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt
      """.stripMargin,
      "json_tuple_generator_1")

    checkSQL(
      """
        |SELECT a, b, c
        |FROM parquet_t3
        |LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt AS a, b, c
      """.stripMargin,
      "json_tuple_generator_2")
  }

  test("nested generator in lateral view") {
    checkSQL(
      """
        |SELECT val, id
        |FROM parquet_t3
        |LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array
        |LATERAL VIEW EXPLODE(nested_array) exp1 AS val
      """.stripMargin,
      "nested_generator_in_lateral_view_1")

    checkSQL(
      """
        |SELECT val, id
        |FROM parquet_t3
        |LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array
        |LATERAL VIEW OUTER EXPLODE(nested_array) exp1 AS val
      """.stripMargin,
      "nested_generator_in_lateral_view_2")
  }

  test("generate with other operators") {
    checkSQL(
      """
        |SELECT EXPLODE(arr) AS val, id
        |FROM parquet_t3
        |WHERE id > 2
        |ORDER BY val, id
        |LIMIT 5
      """.stripMargin,
      "generate_with_other_1")

    checkSQL(
      """
        |SELECT val, id
        |FROM parquet_t3
        |LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array
        |LATERAL VIEW EXPLODE(nested_array) exp1 AS val
        |WHERE val > 2
        |ORDER BY val, id
        |LIMIT 5
      """.stripMargin,
      "generate_with_other_2")
  }

  test("filter after subquery") {
    checkSQL("SELECT a FROM (SELECT key + 1 AS a FROM parquet_t1) t WHERE a > 5",
      "filter_after_subquery")
  }

  test("SPARK-14933 - select parquet table") {
    withTable("parquet_t") {
      sql("create table parquet_t stored as parquet as select 1 as c1, 'abc' as c2")
      checkSQL("select * from parquet_t", "select_parquet_table")
    }
  }

  test("predicate subquery") {
    withTable("t1") {
      withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") {
        sql("CREATE TABLE t1(a int)")
        checkSQL("select * from t1 b where exists (select * from t1 a)", "predicate_subquery")
      }
    }
  }

  test("broadcast join") {
    checkSQL(
      """
        |SELECT /*+ MAPJOIN(srcpart) */ subq.key1, z.value
        |FROM (SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2
        |      FROM src1 x JOIN src y ON (x.key = y.key)) subq
        |JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11)
        |ORDER BY subq.key1, z.value
      """.stripMargin,
      "broadcast_join_subquery")
  }

  test("subquery using single table") {
    checkSQL(
      """
        |SELECT a.k, a.c
        |FROM (SELECT b.key as k, count(1) as c
        |      FROM src b
        |      GROUP BY b.key) a
        |WHERE a.k >= 90
      """.stripMargin,
      "subq2")
  }

  test("correlated subqueries using EXISTS on where clause") {
    checkSQL(
      """
        |select *
        |from src b
        |where exists (select a.key
        |              from src a
        |              where b.value = a.value and a.key = b.key and a.value > 'val_9')
      """.stripMargin,
      "subquery_exists_1")

    checkSQL(
      """
        |select *
        |from (select *
        |      from src b
        |      where exists (select a.key
        |                    from src a
        |                    where b.value = a.value and a.key = b.key and a.value > 'val_9')) a
      """.stripMargin,
      "subquery_exists_2")
  }

  test("correlated subqueries using EXISTS on having clause") {
    checkSQL(
      """
        |select b.key, count(*)
        |from src b
        |group by b.key
        |having exists (select a.key
        |               from src a
        |               where a.key = b.key and a.value > 'val_9')
      """.stripMargin,
      "subquery_exists_having_1")

    checkSQL(
      """
        |select *
        |from (select b.key, count(*)
        |      from src b
        |      group by b.key
        |      having exists (select a.key
        |                     from src a
        |                     where a.key = b.key and a.value > 'val_9')) a
      """.stripMargin,
      "subquery_exists_having_2")

    checkSQL(
      """
        |select b.key, min(b.value)
        |from src b
        |group by b.key
        |having exists (select a.key
        |               from src a
        |               where a.value > 'val_9' and a.value = min(b.value))
      """.stripMargin,
      "subquery_exists_having_3")
  }

  test("correlated subqueries using NOT EXISTS on where clause") {
    checkSQL(
      """
        |select *
        |from src b
        |where not exists (select a.key
        |                  from src a
        |                  where b.value = a.value  and a.key = b.key and a.value > 'val_2')
      """.stripMargin,
      "subquery_not_exists_1")

    checkSQL(
      """
        |select *
        |from src b
        |where not exists (select a.key
        |                  from src a
        |                  where b.value = a.value and a.value > 'val_2')
      """.stripMargin,
      "subquery_not_exists_2")
  }

  test("correlated subqueries using NOT EXISTS on having clause") {
    checkSQL(
      """
        |select *
        |from src b
        |group by key, value
        |having not exists (select a.key
        |                   from src a
        |                   where b.value = a.value  and a.key = b.key and a.value > 'val_12')
      """.stripMargin,
      "subquery_not_exists_having_1")

    checkSQL(
      """
        |select *
        |from src b
        |group by key, value
        |having not exists (select distinct a.key
        |                   from src a
        |                   where b.value = a.value and a.value > 'val_12')
      """.stripMargin,
      "subquery_not_exists_having_2")
  }

  test("subquery using IN on where clause") {
    checkSQL(
      """
        |SELECT key
        |FROM src
        |WHERE key in (SELECT max(key) FROM src)
      """.stripMargin,
      "subquery_in")
  }

  test("subquery using IN on having clause") {
    checkSQL(
      """
        |select key, count(*)
        |from src
        |group by key
        |having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key)
        |order by key
      """.stripMargin,
      "subquery_in_having_1")

    checkSQL(
      """
        |select b.key, min(b.value)
        |from src b
        |group by b.key
        |having b.key in (select a.key
        |                 from src a
        |                 where a.value > 'val_9' and a.value = min(b.value))
        |order by b.key
      """.stripMargin,
      "subquery_in_having_2")
  }

  test("SPARK-14933 - select orc table") {
    withTable("orc_t") {
      sql("create table orc_t stored as orc as select 1 as c1, 'abc' as c2")
      checkSQL("select * from orc_t", "select_orc_table")
    }
  }

  test("inline tables") {
    checkSQL(
      """
        |select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) where b > 1
      """.stripMargin,
      "inline_tables")
  }
}