/
SQLQuerySuite.scala
3221 lines (2864 loc) · 112 KB
/
SQLQuerySuite.scala
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
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql
import java.io.File
import java.net.{MalformedURLException, URL}
import java.sql.{Date, Timestamp}
import java.util.concurrent.atomic.AtomicBoolean
import org.apache.spark.{AccumulatorSuite, SparkException}
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
import org.apache.spark.sql.catalyst.util.StringUtils
import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec}
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan
import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext}
import org.apache.spark.sql.test.SQLTestData._
import org.apache.spark.sql.types._
class SQLQuerySuite extends QueryTest with SharedSparkSession {
import testImplicits._
setupTestData()
test("SPARK-8010: promote numeric to string") {
val df = Seq((1, 1)).toDF("key", "value")
df.createOrReplaceTempView("src")
val queryCaseWhen = sql("select case when true then 1.0 else '1' end from src ")
val queryCoalesce = sql("select coalesce(null, 1, '1') from src ")
checkAnswer(queryCaseWhen, Row("1.0") :: Nil)
checkAnswer(queryCoalesce, Row("1") :: Nil)
}
test("show functions") {
def getFunctions(pattern: String): Seq[Row] = {
StringUtils.filterPattern(
spark.sessionState.catalog.listFunctions("default").map(_._1.funcName), pattern)
.map(Row(_))
}
def createFunction(names: Seq[String]): Unit = {
names.foreach { name =>
spark.udf.register(name, (arg1: Int, arg2: String) => arg2 + arg1)
}
}
def dropFunction(names: Seq[String]): Unit = {
names.foreach { name =>
spark.sessionState.catalog.dropTempFunction(name, false)
}
}
val functions = Array("ilog", "logi", "logii", "logiii", "crc32i", "cubei", "cume_disti",
"isize", "ispace", "to_datei", "date_addi", "current_datei")
createFunction(functions)
checkAnswer(sql("SHOW functions"), getFunctions("*"))
assert(sql("SHOW functions").collect().size > 200)
Seq("^c*", "*e$", "log*", "*date*").foreach { pattern =>
// For the pattern part, only '*' and '|' are allowed as wildcards.
// For '*', we need to replace it to '.*'.
checkAnswer(sql(s"SHOW FUNCTIONS '$pattern'"), getFunctions(pattern))
}
dropFunction(functions)
}
test("describe functions") {
checkKeywordsExist(sql("describe function extended upper"),
"Function: upper",
"Class: org.apache.spark.sql.catalyst.expressions.Upper",
"Usage: upper(str) - Returns `str` with all characters changed to uppercase",
"Extended Usage:",
"Examples:",
"> SELECT upper('SparkSql');",
"SPARKSQL")
checkKeywordsExist(sql("describe functioN Upper"),
"Function: upper",
"Class: org.apache.spark.sql.catalyst.expressions.Upper",
"Usage: upper(str) - Returns `str` with all characters changed to uppercase")
checkKeywordsNotExist(sql("describe functioN Upper"), "Extended Usage")
checkKeywordsExist(sql("describe functioN abcadf"), "Function: abcadf not found.")
}
test("SPARK-14415: All functions should have own descriptions") {
for (f <- spark.sessionState.functionRegistry.listFunction()) {
if (!Seq("cube", "grouping", "grouping_id", "rollup", "window").contains(f.unquotedString)) {
checkKeywordsNotExist(sql(s"describe function `$f`"), "N/A.")
}
}
}
test("using _FUNC_ instead of function names in examples") {
val exampleRe = "(>.*;)".r
val ignoreSet = Set(
// Examples for CaseWhen show simpler syntax:
// `CASE WHEN ... THEN ... WHEN ... THEN ... END`
"org.apache.spark.sql.catalyst.expressions.CaseWhen",
// _FUNC_ is replaced by `locate` but `locate(... IN ...)` is not supported
"org.apache.spark.sql.catalyst.expressions.StringLocate",
// _FUNC_ is replaced by `%` which causes a parsing error on `SELECT %(2, 1.8)`
"org.apache.spark.sql.catalyst.expressions.Remainder",
// Examples demonstrate alternative names, see SPARK-20749
"org.apache.spark.sql.catalyst.expressions.Length")
spark.sessionState.functionRegistry.listFunction().foreach { funcId =>
val info = spark.sessionState.catalog.lookupFunctionInfo(funcId)
val className = info.getClassName
withClue(s"Expression class '$className'") {
val exprExamples = info.getOriginalExamples
if (!exprExamples.isEmpty && !ignoreSet.contains(className)) {
assert(exampleRe.findAllIn(exprExamples).toSet.forall(_.contains("_FUNC_")))
}
}
}
}
test("SPARK-6743: no columns from cache") {
Seq(
(83, 0, 38),
(26, 0, 79),
(43, 81, 24)
).toDF("a", "b", "c").createOrReplaceTempView("cachedData")
spark.catalog.cacheTable("cachedData")
withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") {
checkAnswer(
sql("SELECT t1.b FROM cachedData, cachedData t1 GROUP BY t1.b"),
Row(0) :: Row(81) :: Nil)
}
}
test("self join with aliases") {
Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").createOrReplaceTempView("df")
checkAnswer(
sql(
"""
|SELECT x.str, COUNT(*)
|FROM df x JOIN df y ON x.str = y.str
|GROUP BY x.str
""".stripMargin),
Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil)
}
test("support table.star") {
checkAnswer(
sql(
"""
|SELECT r.*
|FROM testData l join testData2 r on (l.key = r.a)
""".stripMargin),
Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil)
}
test("self join with alias in agg") {
Seq(1, 2, 3)
.map(i => (i, i.toString))
.toDF("int", "str")
.groupBy("str")
.agg($"str", count("str").as("strCount"))
.createOrReplaceTempView("df")
checkAnswer(
sql(
"""
|SELECT x.str, SUM(x.strCount)
|FROM df x JOIN df y ON x.str = y.str
|GROUP BY x.str
""".stripMargin),
Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil)
}
test("SPARK-8668 expr function") {
checkAnswer(Seq((1, "Bobby G."))
.toDF("id", "name")
.select(expr("length(name)"), expr("abs(id)")), Row(8, 1))
checkAnswer(Seq((1, "building burrito tunnels"), (1, "major projects"))
.toDF("id", "saying")
.groupBy(expr("length(saying)"))
.count(), Row(24, 1) :: Row(14, 1) :: Nil)
}
test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") {
checkAnswer(
sql("SELECT a FROM testData2 SORT BY a"),
Seq(1, 1, 2, 2, 3, 3).map(Row(_))
)
}
test("SPARK-7158 collect and take return different results") {
import java.util.UUID
val df = Seq(Tuple1(1), Tuple1(2), Tuple1(3)).toDF("index")
// we except the id is materialized once
val idUDF = org.apache.spark.sql.functions.udf(() => UUID.randomUUID().toString)
val dfWithId = df.withColumn("id", idUDF())
// Make a new DataFrame (actually the same reference to the old one)
val cached = dfWithId.cache()
// Trigger the cache
val d0 = dfWithId.collect()
val d1 = cached.collect()
val d2 = cached.collect()
// Since the ID is only materialized once, then all of the records
// should come from the cache, not by re-computing. Otherwise, the ID
// will be different
assert(d0.map(_(0)) === d2.map(_(0)))
assert(d0.map(_(1)) === d2.map(_(1)))
assert(d1.map(_(0)) === d2.map(_(0)))
assert(d1.map(_(1)) === d2.map(_(1)))
}
test("grouping on nested fields") {
spark.read
.json(Seq("""{"nested": {"attribute": 1}, "value": 2}""").toDS())
.createOrReplaceTempView("rows")
checkAnswer(
sql(
"""
|select attribute, sum(cnt)
|from (
| select nested.attribute, count(*) as cnt
| from rows
| group by nested.attribute) a
|group by attribute
""".stripMargin),
Row(1, 1) :: Nil)
}
test("SPARK-6201 IN type conversion") {
spark.read
.json(Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}").toDS())
.createOrReplaceTempView("d")
checkAnswer(
sql("select * from d where d.a in (1,2)"),
Seq(Row("1"), Row("2")))
}
test("SPARK-11226 Skip empty line in json file") {
spark.read
.json(Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}", "").toDS())
.createOrReplaceTempView("d")
checkAnswer(
sql("select count(1) from d"),
Seq(Row(3)))
}
test("SPARK-8828 sum should return null if all input values are null") {
checkAnswer(
sql("select sum(a), avg(a) from allNulls"),
Seq(Row(null, null))
)
}
private def testCodeGen(sqlText: String, expectedResults: Seq[Row]): Unit = {
val df = sql(sqlText)
// First, check if we have GeneratedAggregate.
val hasGeneratedAgg = df.queryExecution.sparkPlan
.collect { case _: HashAggregateExec => true }
.nonEmpty
if (!hasGeneratedAgg) {
fail(
s"""
|Codegen is enabled, but query $sqlText does not have HashAggregate in the plan.
|${df.queryExecution.simpleString}
""".stripMargin)
}
// Then, check results.
checkAnswer(df, expectedResults)
}
test("aggregation with codegen") {
// Prepare a table that we can group some rows.
spark.table("testData")
.union(spark.table("testData"))
.union(spark.table("testData"))
.createOrReplaceTempView("testData3x")
try {
// Just to group rows.
testCodeGen(
"SELECT key FROM testData3x GROUP BY key",
(1 to 100).map(Row(_)))
// COUNT
testCodeGen(
"SELECT key, count(value) FROM testData3x GROUP BY key",
(1 to 100).map(i => Row(i, 3)))
testCodeGen(
"SELECT count(key) FROM testData3x",
Row(300) :: Nil)
// COUNT DISTINCT ON int
testCodeGen(
"SELECT value, count(distinct key) FROM testData3x GROUP BY value",
(1 to 100).map(i => Row(i.toString, 1)))
testCodeGen(
"SELECT count(distinct key) FROM testData3x",
Row(100) :: Nil)
// SUM
testCodeGen(
"SELECT value, sum(key) FROM testData3x GROUP BY value",
(1 to 100).map(i => Row(i.toString, 3 * i)))
testCodeGen(
"SELECT sum(key), SUM(CAST(key as Double)) FROM testData3x",
Row(5050 * 3, 5050 * 3.0) :: Nil)
// AVERAGE
testCodeGen(
"SELECT value, avg(key) FROM testData3x GROUP BY value",
(1 to 100).map(i => Row(i.toString, i)))
testCodeGen(
"SELECT avg(key) FROM testData3x",
Row(50.5) :: Nil)
// MAX
testCodeGen(
"SELECT value, max(key) FROM testData3x GROUP BY value",
(1 to 100).map(i => Row(i.toString, i)))
testCodeGen(
"SELECT max(key) FROM testData3x",
Row(100) :: Nil)
// MIN
testCodeGen(
"SELECT value, min(key) FROM testData3x GROUP BY value",
(1 to 100).map(i => Row(i.toString, i)))
testCodeGen(
"SELECT min(key) FROM testData3x",
Row(1) :: Nil)
// Some combinations.
testCodeGen(
"""
|SELECT
| value,
| sum(key),
| max(key),
| min(key),
| avg(key),
| count(key),
| count(distinct key)
|FROM testData3x
|GROUP BY value
""".stripMargin,
(1 to 100).map(i => Row(i.toString, i*3, i, i, i, 3, 1)))
testCodeGen(
"SELECT max(key), min(key), avg(key), count(key), count(distinct key) FROM testData3x",
Row(100, 1, 50.5, 300, 100) :: Nil)
// Aggregate with Code generation handling all null values
testCodeGen(
"SELECT sum('a'), avg('a'), count(null) FROM testData",
Row(null, null, 0) :: Nil)
} finally {
spark.catalog.dropTempView("testData3x")
}
}
test("Add Parser of SQL COALESCE()") {
checkAnswer(
sql("""SELECT COALESCE(1, 2)"""),
Row(1))
checkAnswer(
sql("SELECT COALESCE(null, 1, 1.5)"),
Row(BigDecimal(1)))
checkAnswer(
sql("SELECT COALESCE(null, null, null)"),
Row(null))
}
test("SPARK-3176 Added Parser of SQL LAST()") {
checkAnswer(
sql("SELECT LAST(n) FROM lowerCaseData"),
Row(4))
}
test("SPARK-2041 column name equals tablename") {
checkAnswer(
sql("SELECT tableName FROM tableName"),
Row("test"))
}
test("SQRT") {
checkAnswer(
sql("SELECT SQRT(key) FROM testData"),
(1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq
)
}
test("SQRT with automatic string casts") {
checkAnswer(
sql("SELECT SQRT(CAST(key AS STRING)) FROM testData"),
(1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq
)
}
test("SPARK-2407 Added Parser of SQL SUBSTR()") {
checkAnswer(
sql("SELECT substr(tableName, 1, 2) FROM tableName"),
Row("te"))
checkAnswer(
sql("SELECT substr(tableName, 3) FROM tableName"),
Row("st"))
checkAnswer(
sql("SELECT substring(tableName, 1, 2) FROM tableName"),
Row("te"))
checkAnswer(
sql("SELECT substring(tableName, 3) FROM tableName"),
Row("st"))
}
test("SPARK-3173 Timestamp support in the parser") {
(0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").createOrReplaceTempView("timestamps")
checkAnswer(sql(
"SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.0'"),
Row(Timestamp.valueOf("1969-12-31 16:00:00")))
checkAnswer(sql(
"SELECT time FROM timestamps WHERE time=CAST('1969-12-31 16:00:00.001' AS TIMESTAMP)"),
Row(Timestamp.valueOf("1969-12-31 16:00:00.001")))
checkAnswer(sql(
"SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.001'"),
Row(Timestamp.valueOf("1969-12-31 16:00:00.001")))
checkAnswer(sql(
"SELECT time FROM timestamps WHERE '1969-12-31 16:00:00.001'=time"),
Row(Timestamp.valueOf("1969-12-31 16:00:00.001")))
checkAnswer(sql(
"""SELECT time FROM timestamps WHERE time<'1969-12-31 16:00:00.003'
AND time>'1969-12-31 16:00:00.001'"""),
Row(Timestamp.valueOf("1969-12-31 16:00:00.002")))
checkAnswer(sql(
"""
|SELECT time FROM timestamps
|WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002')
""".stripMargin),
Seq(Row(Timestamp.valueOf("1969-12-31 16:00:00.001")),
Row(Timestamp.valueOf("1969-12-31 16:00:00.002"))))
checkAnswer(sql(
"SELECT time FROM timestamps WHERE time='123'"),
Nil)
}
test("left semi greater than predicate") {
withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") {
checkAnswer(
sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.a >= y.a + 2"),
Seq(Row(3, 1), Row(3, 2))
)
}
}
test("left semi greater than predicate and equal operator") {
checkAnswer(
sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.b = y.b and x.a >= y.a + 2"),
Seq(Row(3, 1), Row(3, 2))
)
checkAnswer(
sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.b = y.a and x.a >= y.b + 1"),
Seq(Row(2, 1), Row(2, 2), Row(3, 1), Row(3, 2))
)
}
test("select *") {
checkAnswer(
sql("SELECT * FROM testData"),
testData.collect().toSeq)
}
test("simple select") {
checkAnswer(
sql("SELECT value FROM testData WHERE key = 1"),
Row("1"))
}
def sortTest(): Unit = {
checkAnswer(
sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC"),
Seq(Row(1, 1), Row(1, 2), Row(2, 1), Row(2, 2), Row(3, 1), Row(3, 2)))
checkAnswer(
sql("SELECT * FROM testData2 ORDER BY a ASC, b DESC"),
Seq(Row(1, 2), Row(1, 1), Row(2, 2), Row(2, 1), Row(3, 2), Row(3, 1)))
checkAnswer(
sql("SELECT * FROM testData2 ORDER BY a DESC, b DESC"),
Seq(Row(3, 2), Row(3, 1), Row(2, 2), Row(2, 1), Row(1, 2), Row(1, 1)))
checkAnswer(
sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC"),
Seq(Row(3, 1), Row(3, 2), Row(2, 1), Row(2, 2), Row(1, 1), Row(1, 2)))
checkAnswer(
sql("SELECT b FROM binaryData ORDER BY a ASC"),
(1 to 5).map(Row(_)))
checkAnswer(
sql("SELECT b FROM binaryData ORDER BY a DESC"),
(1 to 5).map(Row(_)).toSeq.reverse)
checkAnswer(
sql("SELECT * FROM arrayData ORDER BY data[0] ASC"),
arrayData.collect().sortBy(_.data(0)).map(Row.fromTuple).toSeq)
checkAnswer(
sql("SELECT * FROM arrayData ORDER BY data[0] DESC"),
arrayData.collect().sortBy(_.data(0)).reverse.map(Row.fromTuple).toSeq)
checkAnswer(
sql("SELECT * FROM mapData ORDER BY data[1] ASC"),
mapData.collect().sortBy(_.data(1)).map(Row.fromTuple).toSeq)
checkAnswer(
sql("SELECT * FROM mapData ORDER BY data[1] DESC"),
mapData.collect().sortBy(_.data(1)).reverse.map(Row.fromTuple).toSeq)
}
test("external sorting") {
sortTest()
}
test("CTE feature") {
checkAnswer(
sql("with q1 as (select * from testData limit 10) select * from q1"),
testData.take(10).toSeq)
checkAnswer(
sql("""
|with q1 as (select * from testData where key= '5'),
|q2 as (select * from testData where key = '4')
|select * from q1 union all select * from q2""".stripMargin),
Row(5, "5") :: Row(4, "4") :: Nil)
}
test("Allow only a single WITH clause per query") {
intercept[AnalysisException] {
sql(
"with q1 as (select * from testData) with q2 as (select * from q1) select * from q2")
}
}
test("date row") {
checkAnswer(sql(
"""select cast("2015-01-28" as date) from testData limit 1"""),
Row(Date.valueOf("2015-01-28"))
)
}
test("from follow multiple brackets") {
checkAnswer(sql(
"""
|select key from ((select * from testData)
| union all (select * from testData)) x limit 1
""".stripMargin),
Row(1)
)
checkAnswer(sql(
"select key from (select * from testData) x limit 1"),
Row(1)
)
checkAnswer(sql(
"""
|select key from
| (select * from testData union all select * from testData) x
| limit 1
""".stripMargin),
Row(1)
)
}
test("average") {
checkAnswer(
sql("SELECT AVG(a) FROM testData2"),
Row(2.0))
}
test("average overflow") {
checkAnswer(
sql("SELECT AVG(a),b FROM largeAndSmallInts group by b"),
Seq(Row(2147483645.0, 1), Row(2.0, 2)))
}
test("count") {
checkAnswer(
sql("SELECT COUNT(*) FROM testData2"),
Row(testData2.count()))
}
test("count distinct") {
checkAnswer(
sql("SELECT COUNT(DISTINCT b) FROM testData2"),
Row(2))
}
test("approximate count distinct") {
checkAnswer(
sql("SELECT APPROX_COUNT_DISTINCT(a) FROM testData2"),
Row(3))
}
test("approximate count distinct with user provided standard deviation") {
checkAnswer(
sql("SELECT APPROX_COUNT_DISTINCT(a, 0.04) FROM testData2"),
Row(3))
}
test("null count") {
checkAnswer(
sql("SELECT a, COUNT(b) FROM testData3 GROUP BY a"),
Seq(Row(1, 0), Row(2, 1)))
checkAnswer(
sql(
"SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3"),
Row(2, 1, 2, 2, 1))
}
test("count of empty table") {
withTempView("t") {
Seq.empty[(Int, Int)].toDF("a", "b").createOrReplaceTempView("t")
checkAnswer(
sql("select count(a) from t"),
Row(0))
}
}
test("inner join where, one match per row") {
withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
checkAnswer(
sql("SELECT * FROM uppercasedata JOIN lowercasedata WHERE n = N"),
Seq(
Row(1, "A", 1, "a"),
Row(2, "B", 2, "b"),
Row(3, "C", 3, "c"),
Row(4, "D", 4, "d")))
}
}
test("inner join ON, one match per row") {
withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
checkAnswer(
sql("SELECT * FROM uppercasedata JOIN lowercasedata ON n = N"),
Seq(
Row(1, "A", 1, "a"),
Row(2, "B", 2, "b"),
Row(3, "C", 3, "c"),
Row(4, "D", 4, "d")))
}
}
test("inner join, where, multiple matches") {
withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
checkAnswer(
sql(
"""
|SELECT * FROM
| (SELECT * FROM testdata2 WHERE a = 1) x JOIN
| (SELECT * FROM testdata2 WHERE a = 1) y
|WHERE x.a = y.a""".stripMargin),
Row(1, 1, 1, 1) ::
Row(1, 1, 1, 2) ::
Row(1, 2, 1, 1) ::
Row(1, 2, 1, 2) :: Nil)
}
}
test("inner join, no matches") {
checkAnswer(
sql(
"""
|SELECT * FROM
| (SELECT * FROM testData2 WHERE a = 1) x JOIN
| (SELECT * FROM testData2 WHERE a = 2) y
|WHERE x.a = y.a""".stripMargin),
Nil)
}
test("big inner join, 4 matches per row") {
checkAnswer(
sql(
"""
|SELECT * FROM
| (SELECT * FROM testData UNION ALL
| SELECT * FROM testData UNION ALL
| SELECT * FROM testData UNION ALL
| SELECT * FROM testData) x JOIN
| (SELECT * FROM testData UNION ALL
| SELECT * FROM testData UNION ALL
| SELECT * FROM testData UNION ALL
| SELECT * FROM testData) y
|WHERE x.key = y.key""".stripMargin),
testData.rdd.flatMap(
row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq)
}
test("cartesian product join") {
withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") {
checkAnswer(
testData3.join(testData3),
Row(1, null, 1, null) ::
Row(1, null, 2, 2) ::
Row(2, 2, 1, null) ::
Row(2, 2, 2, 2) :: Nil)
}
}
test("left outer join") {
withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
checkAnswer(
sql("SELECT * FROM uppercasedata LEFT OUTER JOIN lowercasedata ON n = N"),
Row(1, "A", 1, "a") ::
Row(2, "B", 2, "b") ::
Row(3, "C", 3, "c") ::
Row(4, "D", 4, "d") ::
Row(5, "E", null, null) ::
Row(6, "F", null, null) :: Nil)
}
}
test("right outer join") {
withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
checkAnswer(
sql("SELECT * FROM lowercasedata RIGHT OUTER JOIN uppercasedata ON n = N"),
Row(1, "a", 1, "A") ::
Row(2, "b", 2, "B") ::
Row(3, "c", 3, "C") ::
Row(4, "d", 4, "D") ::
Row(null, null, 5, "E") ::
Row(null, null, 6, "F") :: Nil)
}
}
test("full outer join") {
checkAnswer(
sql(
"""
|SELECT * FROM
| (SELECT * FROM upperCaseData WHERE N <= 4) leftTable FULL OUTER JOIN
| (SELECT * FROM upperCaseData WHERE N >= 3) rightTable
| ON leftTable.N = rightTable.N
""".stripMargin),
Row(1, "A", null, null) ::
Row(2, "B", null, null) ::
Row(3, "C", 3, "C") ::
Row (4, "D", 4, "D") ::
Row(null, null, 5, "E") ::
Row(null, null, 6, "F") :: Nil)
}
test("SPARK-11111 null-safe join should not use cartesian product") {
val df = sql("select count(*) from testData a join testData b on (a.key <=> b.key)")
val cp = df.queryExecution.sparkPlan.collect {
case cp: CartesianProductExec => cp
}
assert(cp.isEmpty, "should not use CartesianProduct for null-safe join")
val smj = df.queryExecution.sparkPlan.collect {
case smj: SortMergeJoinExec => smj
case j: BroadcastHashJoinExec => j
}
assert(smj.size > 0, "should use SortMergeJoin or BroadcastHashJoin")
checkAnswer(df, Row(100) :: Nil)
}
test("SPARK-3349 partitioning after limit") {
sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC")
.limit(2)
.createOrReplaceTempView("subset1")
sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n ASC")
.limit(2)
.createOrReplaceTempView("subset2")
checkAnswer(
sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"),
Row(3, "c", 3) ::
Row(4, "d", 4) :: Nil)
checkAnswer(
sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"),
Row(1, "a", 1) ::
Row(2, "b", 2) :: Nil)
}
test("mixed-case keywords") {
checkAnswer(
sql(
"""
|SeleCT * from
| (select * from upperCaseData WherE N <= 4) leftTable fuLL OUtER joiN
| (sElEcT * FROM upperCaseData whERe N >= 3) rightTable
| oN leftTable.N = rightTable.N
""".stripMargin),
Row(1, "A", null, null) ::
Row(2, "B", null, null) ::
Row(3, "C", 3, "C") ::
Row(4, "D", 4, "D") ::
Row(null, null, 5, "E") ::
Row(null, null, 6, "F") :: Nil)
}
test("select with table name as qualifier") {
checkAnswer(
sql("SELECT testData.value FROM testData WHERE testData.key = 1"),
Row("1"))
}
test("inner join ON with table name as qualifier") {
checkAnswer(
sql("SELECT * FROM upperCaseData JOIN lowerCaseData ON lowerCaseData.n = upperCaseData.N"),
Seq(
Row(1, "A", 1, "a"),
Row(2, "B", 2, "b"),
Row(3, "C", 3, "c"),
Row(4, "D", 4, "d")))
}
test("qualified select with inner join ON with table name as qualifier") {
checkAnswer(
sql("SELECT upperCaseData.N, upperCaseData.L FROM upperCaseData JOIN lowerCaseData " +
"ON lowerCaseData.n = upperCaseData.N"),
Seq(
Row(1, "A"),
Row(2, "B"),
Row(3, "C"),
Row(4, "D")))
}
test("system function upper()") {
checkAnswer(
sql("SELECT n,UPPER(l) FROM lowerCaseData"),
Seq(
Row(1, "A"),
Row(2, "B"),
Row(3, "C"),
Row(4, "D")))
checkAnswer(
sql("SELECT n, UPPER(s) FROM nullStrings"),
Seq(
Row(1, "ABC"),
Row(2, "ABC"),
Row(3, null)))
}
test("system function lower()") {
checkAnswer(
sql("SELECT N,LOWER(L) FROM upperCaseData"),
Seq(
Row(1, "a"),
Row(2, "b"),
Row(3, "c"),
Row(4, "d"),
Row(5, "e"),
Row(6, "f")))
checkAnswer(
sql("SELECT n, LOWER(s) FROM nullStrings"),
Seq(
Row(1, "abc"),
Row(2, "abc"),
Row(3, null)))
}
test("UNION") {
checkAnswer(
sql("SELECT * FROM lowerCaseData UNION SELECT * FROM upperCaseData"),
Row(1, "A") :: Row(1, "a") :: Row(2, "B") :: Row(2, "b") :: Row(3, "C") :: Row(3, "c") ::
Row(4, "D") :: Row(4, "d") :: Row(5, "E") :: Row(6, "F") :: Nil)
checkAnswer(
sql("SELECT * FROM lowerCaseData UNION SELECT * FROM lowerCaseData"),
Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Row(4, "d") :: Nil)
checkAnswer(
sql("SELECT * FROM lowerCaseData UNION ALL SELECT * FROM lowerCaseData"),
Row(1, "a") :: Row(1, "a") :: Row(2, "b") :: Row(2, "b") :: Row(3, "c") :: Row(3, "c") ::
Row(4, "d") :: Row(4, "d") :: Nil)
}
test("UNION with column mismatches") {
// Column name mismatches are allowed.
checkAnswer(
sql("SELECT n,l FROM lowerCaseData UNION SELECT N as x1, L as x2 FROM upperCaseData"),
Row(1, "A") :: Row(1, "a") :: Row(2, "B") :: Row(2, "b") :: Row(3, "C") :: Row(3, "c") ::
Row(4, "D") :: Row(4, "d") :: Row(5, "E") :: Row(6, "F") :: Nil)
// Column type mismatches are not allowed, forcing a type coercion.
checkAnswer(
sql("SELECT n FROM lowerCaseData UNION SELECT L FROM upperCaseData"),
("1" :: "2" :: "3" :: "4" :: "A" :: "B" :: "C" :: "D" :: "E" :: "F" :: Nil).map(Row(_)))
// Column type mismatches where a coercion is not possible, in this case between integer
// and array types, trigger a TreeNodeException.
intercept[AnalysisException] {
sql("SELECT data FROM arrayData UNION SELECT 1 FROM arrayData").collect()
}
}
test("EXCEPT") {
checkAnswer(
sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM upperCaseData"),
Row(1, "a") ::
Row(2, "b") ::
Row(3, "c") ::
Row(4, "d") :: Nil)
checkAnswer(
sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM lowerCaseData"), Nil)
checkAnswer(
sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData"), Nil)
}
test("MINUS") {
checkAnswer(
sql("SELECT * FROM lowerCaseData MINUS SELECT * FROM upperCaseData"),
Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Row(4, "d") :: Nil)
checkAnswer(
sql("SELECT * FROM lowerCaseData MINUS SELECT * FROM lowerCaseData"), Nil)
checkAnswer(
sql("SELECT * FROM upperCaseData MINUS SELECT * FROM upperCaseData"), Nil)
}
test("INTERSECT") {
checkAnswer(
sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM lowerCaseData"),
Row(1, "a") ::
Row(2, "b") ::
Row(3, "c") ::
Row(4, "d") :: Nil)
checkAnswer(
sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM upperCaseData"), Nil)
}
test("SET commands semantics using sql()") {
spark.sessionState.conf.clear()
val testKey = "test.key.0"
val testVal = "test.val.0"
val nonexistentKey = "nonexistent"
// "set" itself returns all config variables currently specified in SQLConf.
assert(sql("SET").collect().size === TestSQLContext.overrideConfs.size)
sql("SET").collect().foreach { row =>
val key = row.getString(0)
val value = row.getString(1)
assert(
TestSQLContext.overrideConfs.contains(key),
s"$key should exist in SQLConf.")
assert(
TestSQLContext.overrideConfs(key) === value,
s"The value of $key should be ${TestSQLContext.overrideConfs(key)} instead of $value.")
}
val overrideConfs = sql("SET").collect()
// "set key=val"
sql(s"SET $testKey=$testVal")
checkAnswer(
sql("SET"),
overrideConfs ++ Seq(Row(testKey, testVal))
)
sql(s"SET ${testKey + testKey}=${testVal + testVal}")
checkAnswer(
sql("set"),
overrideConfs ++ Seq(Row(testKey, testVal), Row(testKey + testKey, testVal + testVal))
)
// "set key"
checkAnswer(
sql(s"SET $testKey"),
Row(testKey, testVal)
)
checkAnswer(
sql(s"SET $nonexistentKey"),
Row(nonexistentKey, "<undefined>")