aboutsummaryrefslogtreecommitdiff
path: root/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
blob: 2b945dbbe03dd549131af34f61104e9ff996ad16 (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
/*
 * 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.hive.execution

import java.io.File
import java.sql.Timestamp
import java.util.{Locale, TimeZone}

import scala.util.Try

import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.scalatest.BeforeAndAfter

import org.apache.spark.SparkFiles
import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession}
import org.apache.spark.sql.catalyst.expressions.Cast
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.plans.logical.Project
import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec
import org.apache.spark.sql.hive._
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SQLTestUtils

case class TestData(a: Int, b: String)

/**
 * A set of test cases expressed in Hive QL that are not covered by the tests
 * included in the hive distribution.
 */
class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAndAfter {
  private val originalTimeZone = TimeZone.getDefault
  private val originalLocale = Locale.getDefault

  import org.apache.spark.sql.hive.test.TestHive.implicits._

  private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled

  def spark: SparkSession = sparkSession

  override def beforeAll() {
    super.beforeAll()
    TestHive.setCacheTables(true)
    // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
    TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
    // Add Locale setting
    Locale.setDefault(Locale.US)
    // Ensures that cross joins are enabled so that we can test them
    TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true)
  }

  override def afterAll() {
    try {
      TestHive.setCacheTables(false)
      TimeZone.setDefault(originalTimeZone)
      Locale.setDefault(originalLocale)
      sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2")
      TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled)
    } finally {
      super.afterAll()
    }
  }

  private def assertUnsupportedFeature(body: => Unit): Unit = {
    val e = intercept[ParseException] { body }
    assert(e.getMessage.toLowerCase.contains("operation not allowed"))
  }

  // Testing the Broadcast based join for cartesian join (cross join)
  // We assume that the Broadcast Join Threshold will works since the src is a small table
  private val spark_10484_1 = """
                                | SELECT a.key, b.key
                                | FROM src a LEFT JOIN src b WHERE a.key > b.key + 300
                                | ORDER BY b.key, a.key
                                | LIMIT 20
                              """.stripMargin
  private val spark_10484_2 = """
                                | SELECT a.key, b.key
                                | FROM src a RIGHT JOIN src b WHERE a.key > b.key + 300
                                | ORDER BY a.key, b.key
                                | LIMIT 20
                              """.stripMargin
  private val spark_10484_3 = """
                                | SELECT a.key, b.key
                                | FROM src a FULL OUTER JOIN src b WHERE a.key > b.key + 300
                                | ORDER BY a.key, b.key
                                | LIMIT 20
                              """.stripMargin
  private val spark_10484_4 = """
                                | SELECT a.key, b.key
                                | FROM src a JOIN src b WHERE a.key > b.key + 300
                                | ORDER BY a.key, b.key
                                | LIMIT 20
                              """.stripMargin

  createQueryTest("SPARK-10484 Optimize the Cartesian (Cross) Join with broadcast based JOIN #1",
    spark_10484_1)

  createQueryTest("SPARK-10484 Optimize the Cartesian (Cross) Join with broadcast based JOIN #2",
    spark_10484_2)

  createQueryTest("SPARK-10484 Optimize the Cartesian (Cross) Join with broadcast based JOIN #3",
    spark_10484_3)

  createQueryTest("SPARK-10484 Optimize the Cartesian (Cross) Join with broadcast based JOIN #4",
    spark_10484_4)

  test("SPARK-10484 Optimize the Cartesian (Cross) Join with broadcast based JOIN") {
    def assertBroadcastNestedLoopJoin(sqlText: String): Unit = {
      assert(sql(sqlText).queryExecution.sparkPlan.collect {
        case _: BroadcastNestedLoopJoinExec => 1
      }.nonEmpty)
    }

    assertBroadcastNestedLoopJoin(spark_10484_1)
    assertBroadcastNestedLoopJoin(spark_10484_2)
    assertBroadcastNestedLoopJoin(spark_10484_3)
    assertBroadcastNestedLoopJoin(spark_10484_4)
  }

  createQueryTest("insert table with generator with column name",
    """
      |  CREATE TABLE gen_tmp (key Int);
      |  INSERT OVERWRITE TABLE gen_tmp
      |    SELECT explode(array(1,2,3)) AS val FROM src LIMIT 3;
      |  SELECT key FROM gen_tmp ORDER BY key ASC;
    """.stripMargin)

  createQueryTest("insert table with generator with multiple column names",
    """
      |  CREATE TABLE gen_tmp (key Int, value String);
      |  INSERT OVERWRITE TABLE gen_tmp
      |    SELECT explode(map(key, value)) as (k1, k2) FROM src LIMIT 3;
      |  SELECT key, value FROM gen_tmp ORDER BY key, value ASC;
    """.stripMargin)

  createQueryTest("insert table with generator without column name",
    """
      |  CREATE TABLE gen_tmp (key Int);
      |  INSERT OVERWRITE TABLE gen_tmp
      |    SELECT explode(array(1,2,3)) FROM src LIMIT 3;
      |  SELECT key FROM gen_tmp ORDER BY key ASC;
    """.stripMargin)

  test("multiple generators in projection") {
    intercept[AnalysisException] {
      sql("SELECT explode(array(key, key)), explode(array(key, key)) FROM src").collect()
    }

    intercept[AnalysisException] {
      sql("SELECT explode(array(key, key)) as k1, explode(array(key, key)) FROM src").collect()
    }
  }

  createQueryTest("! operator",
    """
      |SELECT a FROM (
      |  SELECT 1 AS a UNION ALL SELECT 2 AS a) t
      |WHERE !(a>1)
    """.stripMargin)

  createQueryTest("constant object inspector for generic udf",
    """SELECT named_struct(
      lower("AA"), "10",
      repeat(lower("AA"), 3), "11",
      lower(repeat("AA", 3)), "12",
      printf("bb%d", 12), "13",
      repeat(printf("s%d", 14), 2), "14") FROM src LIMIT 1""")

  createQueryTest("NaN to Decimal",
    "SELECT CAST(CAST('NaN' AS DOUBLE) AS DECIMAL(1,1)) FROM src LIMIT 1")

  createQueryTest("constant null testing",
    """SELECT
      |IF(FALSE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL1,
      |IF(TRUE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL2,
      |IF(FALSE, CAST(NULL AS INT), CAST(1 AS INT)) AS COL3,
      |IF(TRUE, CAST(NULL AS INT), CAST(1 AS INT)) AS COL4,
      |IF(FALSE, CAST(NULL AS DOUBLE), CAST(1 AS DOUBLE)) AS COL5,
      |IF(TRUE, CAST(NULL AS DOUBLE), CAST(1 AS DOUBLE)) AS COL6,
      |IF(FALSE, CAST(NULL AS BOOLEAN), CAST(1 AS BOOLEAN)) AS COL7,
      |IF(TRUE, CAST(NULL AS BOOLEAN), CAST(1 AS BOOLEAN)) AS COL8,
      |IF(FALSE, CAST(NULL AS BIGINT), CAST(1 AS BIGINT)) AS COL9,
      |IF(TRUE, CAST(NULL AS BIGINT), CAST(1 AS BIGINT)) AS COL10,
      |IF(FALSE, CAST(NULL AS FLOAT), CAST(1 AS FLOAT)) AS COL11,
      |IF(TRUE, CAST(NULL AS FLOAT), CAST(1 AS FLOAT)) AS COL12,
      |IF(FALSE, CAST(NULL AS SMALLINT), CAST(1 AS SMALLINT)) AS COL13,
      |IF(TRUE, CAST(NULL AS SMALLINT), CAST(1 AS SMALLINT)) AS COL14,
      |IF(FALSE, CAST(NULL AS TINYINT), CAST(1 AS TINYINT)) AS COL15,
      |IF(TRUE, CAST(NULL AS TINYINT), CAST(1 AS TINYINT)) AS COL16,
      |IF(FALSE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL17,
      |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18,
      |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19,
      |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20,
      |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21,
      |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22,
      |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23
      |FROM src LIMIT 1""".stripMargin)

  test("constant null testing timestamp") {
    val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL20")
      .collect().head
    assert(new Timestamp(1000) == r1.getTimestamp(0))
  }

  createQueryTest("null case",
    "SELECT case when(true) then 1 else null end FROM src LIMIT 1")

  createQueryTest("single case",
    """SELECT case when true then 1 else 2 end FROM src LIMIT 1""")

  createQueryTest("double case",
    """SELECT case when 1 = 2 then 1 when 2 = 2 then 3 else 2 end FROM src LIMIT 1""")

  createQueryTest("case else null",
    """SELECT case when 1 = 2 then 1 when 2 = 2 then 3 else null end FROM src LIMIT 1""")

  createQueryTest("having no references",
    "SELECT key FROM src GROUP BY key HAVING COUNT(*) > 1")

  createQueryTest("no from clause",
    "SELECT 1, +1, -1")

  createQueryTest("boolean = number",
    """
      |SELECT
      |  1 = true, 1L = true, 1Y = true, true = 1, true = 1L, true = 1Y,
      |  0 = true, 0L = true, 0Y = true, true = 0, true = 0L, true = 0Y,
      |  1 = false, 1L = false, 1Y = false, false = 1, false = 1L, false = 1Y,
      |  0 = false, 0L = false, 0Y = false, false = 0, false = 0L, false = 0Y,
      |  2 = true, 2L = true, 2Y = true, true = 2, true = 2L, true = 2Y,
      |  2 = false, 2L = false, 2Y = false, false = 2, false = 2L, false = 2Y
      |FROM src LIMIT 1
    """.stripMargin)

  test("CREATE TABLE AS runs once") {
    sql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect()
    assert(sql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1,
      "Incorrect number of rows in created table")
  }

  createQueryTest("between",
    "SELECT * FROM src WHERE key Between 1 and 2")

  createQueryTest("div",
    "SELECT 1 DIV 2, 1 div 2, 1 dIv 2, 100 DIV 51, 100 DIV 49 FROM src LIMIT 1")

  // Jdk version leads to different query output for double, so not use createQueryTest here
  test("division") {
    val res = sql("SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1").collect().head
    Seq(2.0, 0.5, 0.3333333333333333, 0.002).zip(res.toSeq).foreach( x =>
      assert(x._1 == x._2.asInstanceOf[Double]))
  }

  createQueryTest("modulus",
    "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), " +
      "(101 / 2) % 10 FROM src LIMIT 1")

  test("Query expressed in HiveQL") {
    sql("FROM src SELECT key").collect()
  }

  test("Query with constant folding the CAST") {
    sql("SELECT CAST(CAST('123' AS binary) AS binary) FROM src LIMIT 1").collect()
  }

  createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT",
    "SELECT AVG(0), SUM(0), COUNT(null), COUNT(value) FROM src GROUP BY key")

  createQueryTest("Cast Timestamp to Timestamp in UDF",
    """
      | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp))
      | FROM src LIMIT 1
    """.stripMargin)

  createQueryTest("Date comparison test 1",
    """
      | SELECT
      | CAST(CAST('1970-01-01 22:00:00' AS timestamp) AS date) ==
      | CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date)
      | FROM src LIMIT 1
    """.stripMargin)

  createQueryTest("Simple Average",
    "SELECT AVG(key) FROM src")

  createQueryTest("Simple Average + 1",
    "SELECT AVG(key) + 1.0 FROM src")

  createQueryTest("Simple Average + 1 with group",
    "SELECT AVG(key) + 1.0, value FROM src group by value")

  createQueryTest("string literal",
    "SELECT 'test' FROM src")

  createQueryTest("Escape sequences",
    """SELECT key, '\\\t\\' FROM src WHERE key = 86""")

  createQueryTest("IgnoreExplain",
    """EXPLAIN SELECT key FROM src""")

  createQueryTest("trivial join where clause",
    "SELECT * FROM src a JOIN src b WHERE a.key = b.key")

  createQueryTest("trivial join ON clause",
    "SELECT * FROM src a JOIN src b ON a.key = b.key")

  createQueryTest("length.udf",
    "SELECT length(\"test\") FROM src LIMIT 1")

  createQueryTest("partitioned table scan",
    "SELECT ds, hr, key, value FROM srcpart")

  createQueryTest("create table as",
    """
      |CREATE TABLE createdtable AS SELECT * FROM src;
      |SELECT * FROM createdtable
    """.stripMargin)

  createQueryTest("create table as with db name",
    """
      |CREATE DATABASE IF NOT EXISTS testdb;
      |CREATE TABLE testdb.createdtable AS SELECT * FROM default.src;
      |SELECT * FROM testdb.createdtable;
      |DROP DATABASE IF EXISTS testdb CASCADE
    """.stripMargin)

  createQueryTest("create table as with db name within backticks",
    """
      |CREATE DATABASE IF NOT EXISTS testdb;
      |CREATE TABLE `testdb`.`createdtable` AS SELECT * FROM default.src;
      |SELECT * FROM testdb.createdtable;
      |DROP DATABASE IF EXISTS testdb CASCADE
    """.stripMargin)

  createQueryTest("insert table with db name",
    """
      |CREATE DATABASE IF NOT EXISTS testdb;
      |CREATE TABLE testdb.createdtable like default.src;
      |INSERT INTO TABLE testdb.createdtable SELECT * FROM default.src;
      |SELECT * FROM testdb.createdtable;
      |DROP DATABASE IF EXISTS testdb CASCADE
    """.stripMargin)

  createQueryTest("insert into and insert overwrite",
    """
      |CREATE TABLE createdtable like src;
      |INSERT INTO TABLE createdtable SELECT * FROM src;
      |INSERT INTO TABLE createdtable SELECT * FROM src1;
      |SELECT * FROM createdtable;
      |INSERT OVERWRITE TABLE createdtable SELECT * FROM src WHERE key = 86;
      |SELECT * FROM createdtable;
    """.stripMargin)

  test("SPARK-7270: consider dynamic partition when comparing table output") {
    sql(s"CREATE TABLE test_partition (a STRING) PARTITIONED BY (b BIGINT, c STRING)")
    sql(s"CREATE TABLE ptest (a STRING, b BIGINT, c STRING)")

    val analyzedPlan = sql(
      """
        |INSERT OVERWRITE table test_partition PARTITION (b=1, c)
        |SELECT 'a', 'c' from ptest
      """.stripMargin).queryExecution.analyzed

    assertResult(false, "Incorrect cast detected\n" + analyzedPlan) {
      var hasCast = false
      analyzedPlan.collect {
        case p: Project => p.transformExpressionsUp { case c: Cast => hasCast = true; c }
      }
      hasCast
    }
  }

  createQueryTest("transform",
    "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src")

  createQueryTest("schema-less transform",
    """
      |SELECT TRANSFORM (key, value) USING 'cat' FROM src;
      |SELECT TRANSFORM (*) USING 'cat' FROM src;
    """.stripMargin)

  val delimiter = "'\t'"

  createQueryTest("transform with custom field delimiter",
    s"""
      |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter}
      |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src;
    """.stripMargin.replaceAll("\n", " "))

  createQueryTest("transform with custom field delimiter2",
    s"""
      |SELECT TRANSFORM (key, value) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter}
      |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src;
    """.stripMargin.replaceAll("\n", " "))

  createQueryTest("transform with custom field delimiter3",
    s"""
      |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter}
      |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src;
    """.stripMargin.replaceAll("\n", " "))

  createQueryTest("transform with SerDe",
    """
      |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 src;
    """.stripMargin.replaceAll(System.lineSeparator(), " "))

  test("transform with SerDe2") {

    sql("CREATE TABLE small_src(key INT, value STRING)")
    sql("INSERT OVERWRITE TABLE small_src SELECT key, value FROM src LIMIT 10")

    val expected = sql("SELECT key FROM small_src").collect().head
    val res = sql(
      """
        |SELECT TRANSFORM (key) ROW FORMAT SERDE
        |'org.apache.hadoop.hive.serde2.avro.AvroSerDe'
        |WITH SERDEPROPERTIES ('avro.schema.literal'='{"namespace":
        |"testing.hive.avro.serde","name": "src","type": "record","fields":
        |[{"name":"key","type":"int"}]}') USING 'cat' AS (tKey INT) ROW FORMAT SERDE
        |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' WITH SERDEPROPERTIES
        |('avro.schema.literal'='{"namespace": "testing.hive.avro.serde","name":
        |"src","type": "record","fields": [{"name":"key","type":"int"}]}')
        |FROM small_src
      """.stripMargin.replaceAll(System.lineSeparator(), " ")).collect().head

    assert(expected(0) === res(0))
  }

  createQueryTest("transform with SerDe3",
    """
      |SELECT TRANSFORM (*) ROW FORMAT SERDE
      |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES
      |('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue)
      |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
      |WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src;
    """.stripMargin.replaceAll(System.lineSeparator(), " "))

  createQueryTest("transform with SerDe4",
    """
      |SELECT TRANSFORM (*) ROW FORMAT SERDE
      |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES
      |('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE
      |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES
      |('serialization.last.column.takes.rest'='true') FROM src;
    """.stripMargin.replaceAll(System.lineSeparator(), " "))

  createQueryTest("LIKE",
    "SELECT * FROM src WHERE value LIKE '%1%'")

  createQueryTest("DISTINCT",
    "SELECT DISTINCT key, value FROM src")

  createQueryTest("empty aggregate input",
    "SELECT SUM(key) FROM (SELECT * FROM src LIMIT 0) a")

  createQueryTest("lateral view1",
    "SELECT tbl.* FROM src LATERAL VIEW explode(array(1,2)) tbl as a")

  createQueryTest("lateral view2",
    "SELECT * FROM src LATERAL VIEW explode(array(1,2)) tbl")

  createQueryTest("lateral view3",
    "FROM src SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX")

  // scalastyle:off
  createQueryTest("lateral view4",
    """
      |create table src_lv1 (key string, value string);
      |create table src_lv2 (key string, value string);
      |
      |FROM src
      |insert overwrite table src_lv1 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX
      |insert overwrite table src_lv2 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX
    """.stripMargin)
  // scalastyle:on

  createQueryTest("lateral view5",
    "FROM src SELECT explode(array(key+3, key+4))")

  createQueryTest("lateral view6",
    "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v")

  createQueryTest("Specify the udtf output",
    "SELECT d FROM (SELECT explode(array(1,1)) d FROM src LIMIT 1) t")

  createQueryTest("SPARK-9034 Reflect field names defined in GenericUDTF #1",
    "SELECT col FROM (SELECT explode(array(key,value)) FROM src LIMIT 1) t")

  createQueryTest("SPARK-9034 Reflect field names defined in GenericUDTF #2",
    "SELECT key,value FROM (SELECT explode(map(key,value)) FROM src LIMIT 1) t")

  test("sampling") {
    sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s")
    sql("SELECT * FROM src TABLESAMPLE(100 PERCENT) s")
  }

  test("DataFrame toString") {
    sql("SHOW TABLES").toString
    sql("SELECT * FROM src").toString
  }

  createQueryTest("case statements with key #1",
    "SELECT (CASE 1 WHEN 2 THEN 3 END) FROM src where key < 15")

  createQueryTest("case statements with key #2",
    "SELECT (CASE key WHEN 2 THEN 3 ELSE 0 END) FROM src WHERE key < 15")

  createQueryTest("case statements with key #3",
    "SELECT (CASE key WHEN 2 THEN 3 WHEN NULL THEN 4 END) FROM src WHERE key < 15")

  createQueryTest("case statements with key #4",
    "SELECT (CASE key WHEN 2 THEN 3 WHEN NULL THEN 4 ELSE 0 END) FROM src WHERE key < 15")

  createQueryTest("case statements WITHOUT key #1",
    "SELECT (CASE WHEN key > 2 THEN 3 END) FROM src WHERE key < 15")

  createQueryTest("case statements WITHOUT key #2",
    "SELECT (CASE WHEN key > 2 THEN 3 ELSE 4 END) FROM src WHERE key < 15")

  createQueryTest("case statements WITHOUT key #3",
    "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 END) FROM src WHERE key < 15")

  createQueryTest("case statements WITHOUT key #4",
    "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src WHERE key < 15")

  // Jdk version leads to different query output for double, so not use createQueryTest here
  test("timestamp cast #1") {
    val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head
    assert(1 == res.getDouble(0))
  }

  createQueryTest("timestamp cast #2",
    "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1")

  test("timestamp cast #3") {
    val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head
    assert(1200 == res.getInt(0))
  }

  createQueryTest("timestamp cast #4",
    "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1")

  test("timestamp cast #5") {
    val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head
    assert(-1 == res.get(0))
  }

  createQueryTest("timestamp cast #6",
    "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1")

  test("timestamp cast #7") {
    val res = sql("SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head
    assert(-1200 == res.getInt(0))
  }

  createQueryTest("timestamp cast #8",
    "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1")

  createQueryTest("select null from table",
    "SELECT null FROM src LIMIT 1")

  createQueryTest("CTE feature #1",
    "with q1 as (select key from src) select * from q1 where key = 5")

  createQueryTest("CTE feature #2",
    """with q1 as (select * from src where key= 5),
      |q2 as (select * from src s2 where key = 4)
      |select value from q1 union all select value from q2
    """.stripMargin)

  createQueryTest("CTE feature #3",
    """with q1 as (select key from src)
      |from q1
      |select * where key = 4
    """.stripMargin)

  // test get_json_object again Hive, because the HiveCompatibilitySuite cannot handle result
  // with newline in it.
  createQueryTest("get_json_object #1",
    "SELECT get_json_object(src_json.json, '$') FROM src_json")

  createQueryTest("get_json_object #2",
    "SELECT get_json_object(src_json.json, '$.owner'), get_json_object(src_json.json, '$.store')" +
      " FROM src_json")

  createQueryTest("get_json_object #3",
    "SELECT get_json_object(src_json.json, '$.store.bicycle'), " +
      "get_json_object(src_json.json, '$.store.book') FROM src_json")

  createQueryTest("get_json_object #4",
    "SELECT get_json_object(src_json.json, '$.store.book[0]'), " +
      "get_json_object(src_json.json, '$.store.book[*]') FROM src_json")

  createQueryTest("get_json_object #5",
    "SELECT get_json_object(src_json.json, '$.store.book[0].category'), " +
      "get_json_object(src_json.json, '$.store.book[*].category'), " +
      "get_json_object(src_json.json, '$.store.book[*].isbn'), " +
      "get_json_object(src_json.json, '$.store.book[*].reader') FROM src_json")

  createQueryTest("get_json_object #6",
    "SELECT get_json_object(src_json.json, '$.store.book[*].reader[0].age'), " +
      "get_json_object(src_json.json, '$.store.book[*].reader[*].age') FROM src_json")

  createQueryTest("get_json_object #7",
    "SELECT get_json_object(src_json.json, '$.store.basket[0][1]'), " +
      "get_json_object(src_json.json, '$.store.basket[*]'), " +
      // Hive returns wrong result with [*][0], so this expression is change to make test pass
      "get_json_object(src_json.json, '$.store.basket[0][0]'), " +
      "get_json_object(src_json.json, '$.store.basket[0][*]'), " +
      "get_json_object(src_json.json, '$.store.basket[*][*]'), " +
      "get_json_object(src_json.json, '$.store.basket[0][2].b'), " +
      "get_json_object(src_json.json, '$.store.basket[0][*].b') FROM src_json")

  createQueryTest("get_json_object #8",
    "SELECT get_json_object(src_json.json, '$.non_exist_key'), " +
      "get_json_object(src_json.json, '$..no_recursive'), " +
      "get_json_object(src_json.json, '$.store.book[10]'), " +
      "get_json_object(src_json.json, '$.store.book[0].non_exist_key'), " +
      "get_json_object(src_json.json, '$.store.basket[*].non_exist_key'), " +
      "get_json_object(src_json.json, '$.store.basket[0][*].non_exist_key') FROM src_json")

  createQueryTest("get_json_object #9",
    "SELECT get_json_object(src_json.json, '$.zip code') FROM src_json")

  createQueryTest("get_json_object #10",
    "SELECT get_json_object(src_json.json, '$.fb:testid') FROM src_json")

  test("predicates contains an empty AttributeSet() references") {
    sql(
      """
        |SELECT a FROM (
        |  SELECT 1 AS a FROM src LIMIT 1 ) t
        |WHERE abs(20141202) is not null
      """.stripMargin).collect()
  }

  test("implement identity function using case statement") {
    val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src")
      .rdd
      .map { case Row(i: Int) => i }
      .collect()
      .toSet

    val expected = sql("SELECT key FROM src")
      .rdd
      .map { case Row(i: Int) => i }
      .collect()
      .toSet

    assert(actual === expected)
  }

  // TODO: adopt this test when Spark SQL has the functionality / framework to report errors.
  // See https://github.com/apache/spark/pull/1055#issuecomment-45820167 for a discussion.
  ignore("non-boolean conditions in a CaseWhen are illegal") {
    intercept[Exception] {
      sql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM src").collect()
    }
  }

  createQueryTest("case sensitivity when query Hive table",
    "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15")

  test("case sensitivity: created temporary view") {
    val testData =
      TestHive.sparkContext.parallelize(
        TestData(1, "str1") ::
        TestData(2, "str2") :: Nil)
    testData.toDF().createOrReplaceTempView("REGisteredTABle")

    assertResult(Array(Row(2, "str2"))) {
      sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " +
        "WHERE TableAliaS.a > 1").collect()
    }
  }

  def isExplanation(result: DataFrame): Boolean = {
    val explanation = result.select('plan).collect().map { case Row(plan: String) => plan }
    explanation.head.startsWith("== Physical Plan ==")
  }

  test("SPARK-1704: Explain commands as a DataFrame") {
    sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")

    val df = sql("explain select key, count(value) from src group by key")
    assert(isExplanation(df))

    TestHive.reset()
  }

  test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") {
    val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3))
      .zipWithIndex.map {case ((value, attr), key) => HavingRow(key, value, attr)}
    TestHive.sparkContext.parallelize(fixture).toDF().createOrReplaceTempView("having_test")
    val results =
      sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3")
      .collect()
      .map(x => (x.getString(0), x.getInt(1)))

    assert(results === Array(("foo", 4)))
    TestHive.reset()
  }

  test("SPARK-2180: HAVING with non-boolean clause raises no exceptions") {
    sql("select key, count(*) c from src group by key having c").collect()
  }

  test("SPARK-2225: turn HAVING without GROUP BY into a simple filter") {
    assert(sql("select key from src having key > 490").collect().size < 100)
  }

  test("union/except/intersect") {
    assertResult(Array(Row(1), Row(1))) {
      sql("select 1 as a union all select 1 as a").collect()
    }
    assertResult(Array(Row(1))) {
      sql("select 1 as a union distinct select 1 as a").collect()
    }
    assertResult(Array(Row(1))) {
      sql("select 1 as a union select 1 as a").collect()
    }
    assertResult(Array()) {
      sql("select 1 as a except select 1 as a").collect()
    }
    assertResult(Array(Row(1))) {
      sql("select 1 as a intersect select 1 as a").collect()
    }
  }

  test("SPARK-5383 alias for udfs with multi output columns") {
    assert(
      sql("select stack(2, key, value, key, value) as (a, b) from src limit 5")
        .collect()
        .size == 5)

    assert(
      sql("select a, b from (select stack(2, key, value, key, value) as (a, b) from src) t limit 5")
        .collect()
        .size == 5)
  }

  test("SPARK-5367: resolve star expression in udf") {
    assert(sql("select concat(*) from src limit 5").collect().size == 5)
    assert(sql("select array(*) from src limit 5").collect().size == 5)
    assert(sql("select concat(key, *) from src limit 5").collect().size == 5)
    assert(sql("select array(key, *) from src limit 5").collect().size == 5)
  }

  test("Exactly once semantics for DDL and command statements") {
    val tableName = "test_exactly_once"
    val q0 = sql(s"CREATE TABLE $tableName(key INT, value STRING)")

    // If the table was not created, the following assertion would fail
    assert(Try(table(tableName)).isSuccess)

    // If the CREATE TABLE command got executed again, the following assertion would fail
    assert(Try(q0.count()).isSuccess)
  }

  test("DESCRIBE commands") {
    sql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) PARTITIONED BY (dt STRING)")

    sql(
      """FROM src INSERT OVERWRITE TABLE test_describe_commands1 PARTITION (dt='2008-06-08')
        |SELECT key, value
      """.stripMargin)

    // Describe a table
    assertResult(
      Array(
        Row("key", "int", null),
        Row("value", "string", null),
        Row("dt", "string", null),
        Row("# Partition Information", "", ""),
        Row("# col_name", "data_type", "comment"),
        Row("dt", "string", null))
    ) {
      sql("DESCRIBE test_describe_commands1")
        .select('col_name, 'data_type, 'comment)
        .collect()
    }

    // Describe a table with a fully qualified table name
    assertResult(
      Array(
        Row("key", "int", null),
        Row("value", "string", null),
        Row("dt", "string", null),
        Row("# Partition Information", "", ""),
        Row("# col_name", "data_type", "comment"),
        Row("dt", "string", null))
    ) {
      sql("DESCRIBE default.test_describe_commands1")
        .select('col_name, 'data_type, 'comment)
        .collect()
    }

    // Describe a temporary view.
    val testData =
      TestHive.sparkContext.parallelize(
        TestData(1, "str1") ::
        TestData(1, "str2") :: Nil)
    testData.toDF().createOrReplaceTempView("test_describe_commands2")

    assertResult(
      Array(
        Row("a", "int", null),
        Row("b", "string", null))
    ) {
      sql("DESCRIBE test_describe_commands2")
        .select('col_name, 'data_type, 'comment)
        .collect()
    }
  }

  test("SPARK-2263: Insert Map<K, V> values") {
    sql("CREATE TABLE m(value MAP<INT, STRING>)")
    sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10")
    sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).foreach {
      case (Row(map: Map[_, _]), Row(key: Int, value: String)) =>
        assert(map.size === 1)
        assert(map.head === (key, value))
    }
  }

  test("ADD JAR command") {
    val testJar = TestHive.getHiveFile("data/files/TestSerDe.jar").getCanonicalPath
    sql("CREATE TABLE alter1(a INT, b INT)")
    intercept[Exception] {
      sql(
        """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe'
          |WITH serdeproperties('s1'='9')
        """.stripMargin)
    }
    sql("DROP TABLE alter1")
  }

  test("ADD JAR command 2") {
    // this is a test case from mapjoin_addjar.q
    val testJar = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").getCanonicalPath
    val testData = TestHive.getHiveFile("data/files/sample.json").getCanonicalPath
    sql(s"ADD JAR $testJar")
    sql(
      """CREATE TABLE t1(a string, b string)
      |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin)
    sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""")
    sql("select * from src join t1 on src.key = t1.a")
    sql("DROP TABLE t1")
    assert(sql("list jars").
      filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0)
    assert(sql("list jar").
      filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0)
    val testJar2 = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath
    sql(s"ADD JAR $testJar2")
    assert(sql(s"list jar $testJar").count() == 1)
  }

  test("CREATE TEMPORARY FUNCTION") {
    val funcJar = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath
    val jarURL = s"file://$funcJar"
    sql(s"ADD JAR $jarURL")
    sql(
      """CREATE TEMPORARY FUNCTION udtf_count2 AS
        |'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
      """.stripMargin)
    assert(sql("DESCRIBE FUNCTION udtf_count2").count > 1)
    sql("DROP TEMPORARY FUNCTION udtf_count2")
  }

  test("ADD FILE command") {
    val testFile = TestHive.getHiveFile("data/files/v1.txt").getCanonicalFile
    sql(s"ADD FILE $testFile")

    val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { _ =>
      Iterator.single(new File(SparkFiles.get("v1.txt")).canRead)
    }

    assert(checkAddFileRDD.first())
    assert(sql("list files").
      filter(_.getString(0).contains("data/files/v1.txt")).count() > 0)
    assert(sql("list file").
      filter(_.getString(0).contains("data/files/v1.txt")).count() > 0)
    assert(sql(s"list file $testFile").count() == 1)
  }

  createQueryTest("dynamic_partition",
    """
      |DROP TABLE IF EXISTS dynamic_part_table;
      |CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT);
      |
      |SET hive.exec.dynamic.partition.mode=nonstrict;
      |
      |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2)
      |SELECT 1, 1, 1 FROM src WHERE key=150;
      |
      |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2)
      |SELECT 1, NULL, 1 FROM src WHERE key=150;
      |
      |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2)
      |SELECT 1, 1, NULL FROM src WHERE key=150;
      |
      |INSERT INTO TABLe dynamic_part_table PARTITION(partcol1, partcol2)
      |SELECT 1, NULL, NULL FROM src WHERE key=150;
      |
      |DROP TABLE IF EXISTS dynamic_part_table;
    """.stripMargin)

  ignore("Dynamic partition folder layout") {
    sql("DROP TABLE IF EXISTS dynamic_part_table")
    sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)")
    sql("SET hive.exec.dynamic.partition.mode=nonstrict")

    val data = Map(
      Seq("1", "1") -> 1,
      Seq("1", "NULL") -> 2,
      Seq("NULL", "1") -> 3,
      Seq("NULL", "NULL") -> 4)

    data.foreach { case (parts, value) =>
      sql(
        s"""INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2)
           |SELECT $value, ${parts.mkString(", ")} FROM src WHERE key=150
         """.stripMargin)

      val partFolder = Seq("partcol1", "partcol2")
        .zip(parts)
        .map { case (k, v) =>
          if (v == "NULL") {
            s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultStrVal}"
          } else {
            s"$k=$v"
          }
        }
        .mkString("/")

      // Loads partition data to a temporary table to verify contents
      val path = s"${sparkSession.getWarehousePath}/dynamic_part_table/$partFolder/part-00000"

      sql("DROP TABLE IF EXISTS dp_verify")
      sql("CREATE TABLE dp_verify(intcol INT)")
      sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify")

      assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value)))
    }
  }

  test("SPARK-5592: get java.net.URISyntaxException when dynamic partitioning") {
    sql("""
      |create table sc as select *
      |from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows)
      |union all
      |select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows)
      |union all
      |select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s
    """.stripMargin)
    sql("create table sc_part (key string) partitioned by (ts string) stored as rcfile")
    sql("set hive.exec.dynamic.partition=true")
    sql("set hive.exec.dynamic.partition.mode=nonstrict")
    sql("insert overwrite table sc_part partition(ts) select * from sc")
    sql("drop table sc_part")
  }

  test("Partition spec validation") {
    sql("DROP TABLE IF EXISTS dp_test")
    sql("CREATE TABLE dp_test(key INT, value STRING) PARTITIONED BY (dp INT, sp INT)")
    sql("SET hive.exec.dynamic.partition.mode=strict")

    // Should throw when using strict dynamic partition mode without any static partition
    intercept[AnalysisException] {
      sql(
        """INSERT INTO TABLE dp_test PARTITION(dp)
          |SELECT key, value, key % 5 FROM src
        """.stripMargin)
    }

    sql("SET hive.exec.dynamic.partition.mode=nonstrict")

    // Should throw when a static partition appears after a dynamic partition
    intercept[AnalysisException] {
      sql(
        """INSERT INTO TABLE dp_test PARTITION(dp, sp = 1)
          |SELECT key, value, key % 5 FROM src
        """.stripMargin)
    }
  }

  test("SPARK-3414 regression: should store analyzed logical plan when creating a temporary view") {
    sparkContext.makeRDD(Seq.empty[LogEntry]).toDF().createOrReplaceTempView("rawLogs")
    sparkContext.makeRDD(Seq.empty[LogFile]).toDF().createOrReplaceTempView("logFiles")

    sql(
      """
      SELECT name, message
      FROM rawLogs
      JOIN (
        SELECT name
        FROM logFiles
      ) files
      ON rawLogs.filename = files.name
      """).createOrReplaceTempView("boom")

    // This should be successfully analyzed
    sql("SELECT * FROM boom").queryExecution.analyzed
  }

  test("SPARK-3810: PreprocessTableInsertion static partitioning support") {
    val analyzedPlan = {
      loadTestTable("srcpart")
      sql("DROP TABLE IF EXISTS withparts")
      sql("CREATE TABLE withparts LIKE srcpart")
      sql("INSERT INTO TABLE withparts PARTITION(ds='1', hr='2') SELECT key, value FROM src")
        .queryExecution.analyzed
      }

    assertResult(1, "Duplicated project detected\n" + analyzedPlan) {
      analyzedPlan.collect {
        case _: Project => ()
      }.size
    }
  }

  test("SPARK-3810: PreprocessTableInsertion dynamic partitioning support") {
    val analyzedPlan = {
      loadTestTable("srcpart")
      sql("DROP TABLE IF EXISTS withparts")
      sql("CREATE TABLE withparts LIKE srcpart")
      sql("SET hive.exec.dynamic.partition.mode=nonstrict")

      sql("CREATE TABLE IF NOT EXISTS withparts LIKE srcpart")
      sql("INSERT INTO TABLE withparts PARTITION(ds, hr) SELECT key, value, '1', '2' FROM src")
        .queryExecution.analyzed
    }

    assertResult(2, "Duplicated project detected\n" + analyzedPlan) {
      analyzedPlan.collect {
        case _: Project => ()
      }.size
    }
  }

  test("parse HQL set commands") {
    // Adapted from its SQL counterpart.
    val testKey = "spark.sql.key.usedfortestonly"
    val testVal = "val0,val_1,val2.3,my_table"

    sql(s"set $testKey=$testVal")
    assert(getConf(testKey, testVal + "_") == testVal)

    sql("set some.property=20")
    assert(getConf("some.property", "0") == "20")
    sql("set some.property = 40")
    assert(getConf("some.property", "0") == "40")

    sql(s"set $testKey=$testVal")
    assert(getConf(testKey, "0") == testVal)

    sql(s"set $testKey=")
    assert(getConf(testKey, "0") == "")
  }

  test("current_database with multiple sessions") {
    sql("create database a")
    sql("use a")
    val s2 = newSession()
    s2.sql("create database b")
    s2.sql("use b")

    assert(sql("select current_database()").first() === Row("a"))
    assert(s2.sql("select current_database()").first() === Row("b"))

    try {
      sql("create table test_a(key INT, value STRING)")
      s2.sql("create table test_b(key INT, value STRING)")

      sql("select * from test_a")
      intercept[AnalysisException] {
        sql("select * from test_b")
      }
      sql("select * from b.test_b")

      s2.sql("select * from test_b")
      intercept[AnalysisException] {
        s2.sql("select * from test_a")
      }
      s2.sql("select * from a.test_a")
    } finally {
      sql("DROP TABLE IF EXISTS test_a")
      s2.sql("DROP TABLE IF EXISTS test_b")
    }

  }

  test("use database") {
    val currentDatabase = sql("select current_database()").first().getString(0)

    sql("CREATE DATABASE hive_test_db")
    sql("USE hive_test_db")
    assert("hive_test_db" == sql("select current_database()").first().getString(0))

    intercept[AnalysisException] {
      sql("USE not_existing_db")
    }

    sql(s"USE $currentDatabase")
    assert(currentDatabase == sql("select current_database()").first().getString(0))
  }

  test("lookup hive UDF in another thread") {
    val e = intercept[AnalysisException] {
      range(1).selectExpr("not_a_udf()")
    }
    assert(e.getMessage.contains("Undefined function"))
    assert(e.getMessage.contains("not_a_udf"))
    var success = false
    val t = new Thread("test") {
      override def run(): Unit = {
        val e = intercept[AnalysisException] {
          range(1).selectExpr("not_a_udf()")
        }
        assert(e.getMessage.contains("Undefined function"))
        assert(e.getMessage.contains("not_a_udf"))
        success = true
      }
    }
    t.start()
    t.join()
    assert(success)
  }

  createQueryTest("select from thrift based table",
    "SELECT * from src_thrift")

  // Put tests that depend on specific Hive settings before these last two test,
  // since they modify /clear stuff.

  test("role management commands are not supported") {
    assertUnsupportedFeature { sql("CREATE ROLE my_role") }
    assertUnsupportedFeature { sql("DROP ROLE my_role") }
    assertUnsupportedFeature { sql("SHOW CURRENT ROLES") }
    assertUnsupportedFeature { sql("SHOW ROLES") }
    assertUnsupportedFeature { sql("SHOW GRANT") }
    assertUnsupportedFeature { sql("SHOW ROLE GRANT USER my_principal") }
    assertUnsupportedFeature { sql("SHOW PRINCIPALS my_role") }
    assertUnsupportedFeature { sql("SET ROLE my_role") }
    assertUnsupportedFeature { sql("GRANT my_role TO USER my_user") }
    assertUnsupportedFeature { sql("GRANT ALL ON my_table TO USER my_user") }
    assertUnsupportedFeature { sql("REVOKE my_role FROM USER my_user") }
    assertUnsupportedFeature { sql("REVOKE ALL ON my_table FROM USER my_user") }
  }

  test("import/export commands are not supported") {
    assertUnsupportedFeature { sql("IMPORT TABLE my_table FROM 'my_path'") }
    assertUnsupportedFeature { sql("EXPORT TABLE my_table TO 'my_path'") }
  }

  test("some show commands are not supported") {
    assertUnsupportedFeature { sql("SHOW COMPACTIONS") }
    assertUnsupportedFeature { sql("SHOW TRANSACTIONS") }
    assertUnsupportedFeature { sql("SHOW INDEXES ON my_table") }
    assertUnsupportedFeature { sql("SHOW LOCKS my_table") }
  }

  test("lock/unlock table and database commands are not supported") {
    assertUnsupportedFeature { sql("LOCK TABLE my_table SHARED") }
    assertUnsupportedFeature { sql("UNLOCK TABLE my_table") }
    assertUnsupportedFeature { sql("LOCK DATABASE my_db SHARED") }
    assertUnsupportedFeature { sql("UNLOCK DATABASE my_db") }
  }

  test("create/drop/alter index commands are not supported") {
    assertUnsupportedFeature {
      sql("CREATE INDEX my_index ON TABLE my_table(a) as 'COMPACT' WITH DEFERRED REBUILD")}
    assertUnsupportedFeature { sql("DROP INDEX my_index ON my_table") }
    assertUnsupportedFeature { sql("ALTER INDEX my_index ON my_table REBUILD")}
    assertUnsupportedFeature {
      sql("ALTER INDEX my_index ON my_table set IDXPROPERTIES (\"prop1\"=\"val1_new\")")}
  }

  test("create/drop macro commands are not supported") {
    assertUnsupportedFeature {
      sql("CREATE TEMPORARY MACRO SIGMOID (x DOUBLE) 1.0 / (1.0 + EXP(-x))")
    }
    assertUnsupportedFeature { sql("DROP TEMPORARY MACRO SIGMOID") }
  }

  test("dynamic partitioning is allowed when hive.exec.dynamic.partition.mode is nonstrict") {
    val modeConfKey = "hive.exec.dynamic.partition.mode"
    withTable("with_parts") {
      sql("CREATE TABLE with_parts(key INT) PARTITIONED BY (p INT)")

      withSQLConf(modeConfKey -> "nonstrict") {
        sql("INSERT OVERWRITE TABLE with_parts partition(p) select 1, 2")
        assert(spark.table("with_parts").filter($"p" === 2).collect().head == Row(1, 2))
      }

      val originalValue = spark.sparkContext.hadoopConfiguration.get(modeConfKey, "nonstrict")
      try {
        spark.sparkContext.hadoopConfiguration.set(modeConfKey, "nonstrict")
        sql("INSERT OVERWRITE TABLE with_parts partition(p) select 3, 4")
        assert(spark.table("with_parts").filter($"p" === 4).collect().head == Row(3, 4))
      } finally {
        spark.sparkContext.hadoopConfiguration.set(modeConfKey, originalValue)
      }
    }
  }
}

// for SPARK-2180 test
case class HavingRow(key: Int, value: String, attr: Int)

case class LogEntry(filename: String, message: String)
case class LogFile(name: String)