aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
blob: e17340c70b7e666e832f21dc70154d4d9d8173a6 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
/*
 * 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.execution.datasources.json

import java.io.{File, StringWriter}
import java.nio.charset.StandardCharsets
import java.sql.{Date, Timestamp}

import scala.collection.JavaConverters._

import com.fasterxml.jackson.core.JsonFactory
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, PathFilter}
import org.apache.hadoop.io.SequenceFile.CompressionType
import org.apache.hadoop.io.compress.GzipCodec

import org.apache.spark.SparkException
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.execution.datasources.json.InferSchema.compatibleType
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils

class TestFileFilter extends PathFilter {
  override def accept(path: Path): Boolean = path.getParent.getName != "p=2"
}

class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
  import testImplicits._

  test("Type promotion") {
    def checkTypePromotion(expected: Any, actual: Any) {
      assert(expected.getClass == actual.getClass,
        s"Failed to promote ${actual.getClass} to ${expected.getClass}.")
      assert(expected == actual,
        s"Promoted value ${actual}(${actual.getClass}) does not equal the expected value " +
          s"${expected}(${expected.getClass}).")
    }

    val factory = new JsonFactory()
    def enforceCorrectType(value: Any, dataType: DataType): Any = {
      val writer = new StringWriter()
      Utils.tryWithResource(factory.createGenerator(writer)) { generator =>
        generator.writeObject(value)
        generator.flush()
      }

      Utils.tryWithResource(factory.createParser(writer.toString)) { parser =>
        parser.nextToken()
        JacksonParser.convertRootField(factory, parser, dataType)
      }
    }

    val intNumber: Int = 2147483647
    checkTypePromotion(intNumber, enforceCorrectType(intNumber, IntegerType))
    checkTypePromotion(intNumber.toLong, enforceCorrectType(intNumber, LongType))
    checkTypePromotion(intNumber.toDouble, enforceCorrectType(intNumber, DoubleType))
    checkTypePromotion(
      Decimal(intNumber), enforceCorrectType(intNumber, DecimalType.SYSTEM_DEFAULT))

    val longNumber: Long = 9223372036854775807L
    checkTypePromotion(longNumber, enforceCorrectType(longNumber, LongType))
    checkTypePromotion(longNumber.toDouble, enforceCorrectType(longNumber, DoubleType))
    checkTypePromotion(
      Decimal(longNumber), enforceCorrectType(longNumber, DecimalType.SYSTEM_DEFAULT))

    val doubleNumber: Double = 1.7976931348623157E308d
    checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType))

    checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber * 1000L)),
        enforceCorrectType(intNumber, TimestampType))
    checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong * 1000L)),
        enforceCorrectType(intNumber.toLong, TimestampType))
    val strTime = "2014-09-30 12:34:56"
    checkTypePromotion(DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)),
        enforceCorrectType(strTime, TimestampType))

    val strDate = "2014-10-15"
    checkTypePromotion(
      DateTimeUtils.fromJavaDate(Date.valueOf(strDate)), enforceCorrectType(strDate, DateType))

    val ISO8601Time1 = "1970-01-01T01:00:01.0Z"
    checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(3601000)),
        enforceCorrectType(ISO8601Time1, TimestampType))
    checkTypePromotion(DateTimeUtils.millisToDays(3601000),
      enforceCorrectType(ISO8601Time1, DateType))
    val ISO8601Time2 = "1970-01-01T02:00:01-01:00"
    checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(10801000)),
        enforceCorrectType(ISO8601Time2, TimestampType))
    checkTypePromotion(DateTimeUtils.millisToDays(10801000),
      enforceCorrectType(ISO8601Time2, DateType))
  }

  test("Get compatible type") {
    def checkDataType(t1: DataType, t2: DataType, expected: DataType) {
      var actual = compatibleType(t1, t2)
      assert(actual == expected,
        s"Expected $expected as the most general data type for $t1 and $t2, found $actual")
      actual = compatibleType(t2, t1)
      assert(actual == expected,
        s"Expected $expected as the most general data type for $t1 and $t2, found $actual")
    }

    // NullType
    checkDataType(NullType, BooleanType, BooleanType)
    checkDataType(NullType, IntegerType, IntegerType)
    checkDataType(NullType, LongType, LongType)
    checkDataType(NullType, DoubleType, DoubleType)
    checkDataType(NullType, DecimalType.SYSTEM_DEFAULT, DecimalType.SYSTEM_DEFAULT)
    checkDataType(NullType, StringType, StringType)
    checkDataType(NullType, ArrayType(IntegerType), ArrayType(IntegerType))
    checkDataType(NullType, StructType(Nil), StructType(Nil))
    checkDataType(NullType, NullType, NullType)

    // BooleanType
    checkDataType(BooleanType, BooleanType, BooleanType)
    checkDataType(BooleanType, IntegerType, StringType)
    checkDataType(BooleanType, LongType, StringType)
    checkDataType(BooleanType, DoubleType, StringType)
    checkDataType(BooleanType, DecimalType.SYSTEM_DEFAULT, StringType)
    checkDataType(BooleanType, StringType, StringType)
    checkDataType(BooleanType, ArrayType(IntegerType), StringType)
    checkDataType(BooleanType, StructType(Nil), StringType)

    // IntegerType
    checkDataType(IntegerType, IntegerType, IntegerType)
    checkDataType(IntegerType, LongType, LongType)
    checkDataType(IntegerType, DoubleType, DoubleType)
    checkDataType(IntegerType, DecimalType.SYSTEM_DEFAULT, DecimalType.SYSTEM_DEFAULT)
    checkDataType(IntegerType, StringType, StringType)
    checkDataType(IntegerType, ArrayType(IntegerType), StringType)
    checkDataType(IntegerType, StructType(Nil), StringType)

    // LongType
    checkDataType(LongType, LongType, LongType)
    checkDataType(LongType, DoubleType, DoubleType)
    checkDataType(LongType, DecimalType.SYSTEM_DEFAULT, DecimalType.SYSTEM_DEFAULT)
    checkDataType(LongType, StringType, StringType)
    checkDataType(LongType, ArrayType(IntegerType), StringType)
    checkDataType(LongType, StructType(Nil), StringType)

    // DoubleType
    checkDataType(DoubleType, DoubleType, DoubleType)
    checkDataType(DoubleType, DecimalType.SYSTEM_DEFAULT, DoubleType)
    checkDataType(DoubleType, StringType, StringType)
    checkDataType(DoubleType, ArrayType(IntegerType), StringType)
    checkDataType(DoubleType, StructType(Nil), StringType)

    // DecimalType
    checkDataType(DecimalType.SYSTEM_DEFAULT, DecimalType.SYSTEM_DEFAULT,
      DecimalType.SYSTEM_DEFAULT)
    checkDataType(DecimalType.SYSTEM_DEFAULT, StringType, StringType)
    checkDataType(DecimalType.SYSTEM_DEFAULT, ArrayType(IntegerType), StringType)
    checkDataType(DecimalType.SYSTEM_DEFAULT, StructType(Nil), StringType)

    // StringType
    checkDataType(StringType, StringType, StringType)
    checkDataType(StringType, ArrayType(IntegerType), StringType)
    checkDataType(StringType, StructType(Nil), StringType)

    // ArrayType
    checkDataType(ArrayType(IntegerType), ArrayType(IntegerType), ArrayType(IntegerType))
    checkDataType(ArrayType(IntegerType), ArrayType(LongType), ArrayType(LongType))
    checkDataType(ArrayType(IntegerType), ArrayType(StringType), ArrayType(StringType))
    checkDataType(ArrayType(IntegerType), StructType(Nil), StringType)
    checkDataType(
      ArrayType(IntegerType, true), ArrayType(IntegerType), ArrayType(IntegerType, true))
    checkDataType(
      ArrayType(IntegerType, true), ArrayType(IntegerType, false), ArrayType(IntegerType, true))
    checkDataType(
      ArrayType(IntegerType, true), ArrayType(IntegerType, true), ArrayType(IntegerType, true))
    checkDataType(
      ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, true))
    checkDataType(
      ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType, false))
    checkDataType(
      ArrayType(IntegerType, false), ArrayType(IntegerType, true), ArrayType(IntegerType, true))

    // StructType
    checkDataType(StructType(Nil), StructType(Nil), StructType(Nil))
    checkDataType(
      StructType(StructField("f1", IntegerType, true) :: Nil),
      StructType(StructField("f1", IntegerType, true) :: Nil),
      StructType(StructField("f1", IntegerType, true) :: Nil))
    checkDataType(
      StructType(StructField("f1", IntegerType, true) :: Nil),
      StructType(Nil),
      StructType(StructField("f1", IntegerType, true) :: Nil))
    checkDataType(
      StructType(
        StructField("f1", IntegerType, true) ::
        StructField("f2", IntegerType, true) :: Nil),
      StructType(StructField("f1", LongType, true) :: Nil),
      StructType(
        StructField("f1", LongType, true) ::
        StructField("f2", IntegerType, true) :: Nil))
    checkDataType(
      StructType(
        StructField("f1", IntegerType, true) :: Nil),
      StructType(
        StructField("f2", IntegerType, true) :: Nil),
      StructType(
        StructField("f1", IntegerType, true) ::
        StructField("f2", IntegerType, true) :: Nil))
    checkDataType(
      StructType(
        StructField("f1", IntegerType, true) :: Nil),
      DecimalType.SYSTEM_DEFAULT,
      StringType)
  }

  test("Complex field and type inferring with null in sampling") {
    val jsonDF = sqlContext.read.json(jsonNullStruct)
    val expectedSchema = StructType(
      StructField("headers", StructType(
        StructField("Charset", StringType, true) ::
          StructField("Host", StringType, true) :: Nil)
        , true) ::
        StructField("ip", StringType, true) ::
        StructField("nullstr", StringType, true):: Nil)

    assert(expectedSchema === jsonDF.schema)
    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select nullstr, headers.Host from jsonTable"),
      Seq(Row("", "1.abc.com"), Row("", null), Row("", null), Row(null, null))
    )
  }

  test("Primitive field and type inferring") {
    val jsonDF = sqlContext.read.json(primitiveFieldAndType)

    val expectedSchema = StructType(
      StructField("bigInteger", DecimalType(20, 0), true) ::
      StructField("boolean", BooleanType, true) ::
      StructField("double", DoubleType, true) ::
      StructField("integer", LongType, true) ::
      StructField("long", LongType, true) ::
      StructField("null", StringType, true) ::
      StructField("string", StringType, true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select * from jsonTable"),
      Row(new java.math.BigDecimal("92233720368547758070"),
        true,
        1.7976931348623157E308,
        10,
        21474836470L,
        null,
        "this is a simple string.")
    )
  }

  test("Complex field and type inferring") {
    val jsonDF = sqlContext.read.json(complexFieldAndType1)

    val expectedSchema = StructType(
      StructField("arrayOfArray1", ArrayType(ArrayType(StringType, true), true), true) ::
      StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType, true), true), true) ::
      StructField("arrayOfBigInteger", ArrayType(DecimalType(21, 0), true), true) ::
      StructField("arrayOfBoolean", ArrayType(BooleanType, true), true) ::
      StructField("arrayOfDouble", ArrayType(DoubleType, true), true) ::
      StructField("arrayOfInteger", ArrayType(LongType, true), true) ::
      StructField("arrayOfLong", ArrayType(LongType, true), true) ::
      StructField("arrayOfNull", ArrayType(StringType, true), true) ::
      StructField("arrayOfString", ArrayType(StringType, true), true) ::
      StructField("arrayOfStruct", ArrayType(
        StructType(
          StructField("field1", BooleanType, true) ::
          StructField("field2", StringType, true) ::
          StructField("field3", StringType, true) :: Nil), true), true) ::
      StructField("struct", StructType(
        StructField("field1", BooleanType, true) ::
        StructField("field2", DecimalType(20, 0), true) :: Nil), true) ::
      StructField("structWithArrayFields", StructType(
        StructField("field1", ArrayType(LongType, true), true) ::
        StructField("field2", ArrayType(StringType, true), true) :: Nil), true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    // Access elements of a primitive array.
    checkAnswer(
      sql("select arrayOfString[0], arrayOfString[1], arrayOfString[2] from jsonTable"),
      Row("str1", "str2", null)
    )

    // Access an array of null values.
    checkAnswer(
      sql("select arrayOfNull from jsonTable"),
      Row(Seq(null, null, null, null))
    )

    // Access elements of a BigInteger array (we use DecimalType internally).
    checkAnswer(
      sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] from jsonTable"),
      Row(new java.math.BigDecimal("922337203685477580700"),
        new java.math.BigDecimal("-922337203685477580800"), null)
    )

    // Access elements of an array of arrays.
    checkAnswer(
      sql("select arrayOfArray1[0], arrayOfArray1[1] from jsonTable"),
      Row(Seq("1", "2", "3"), Seq("str1", "str2"))
    )

    // Access elements of an array of arrays.
    checkAnswer(
      sql("select arrayOfArray2[0], arrayOfArray2[1] from jsonTable"),
      Row(Seq(1.0, 2.0, 3.0), Seq(1.1, 2.1, 3.1))
    )

    // Access elements of an array inside a filed with the type of ArrayType(ArrayType).
    checkAnswer(
      sql("select arrayOfArray1[1][1], arrayOfArray2[1][1] from jsonTable"),
      Row("str2", 2.1)
    )

    // Access elements of an array of structs.
    checkAnswer(
      sql("select arrayOfStruct[0], arrayOfStruct[1], arrayOfStruct[2], arrayOfStruct[3] " +
        "from jsonTable"),
      Row(
        Row(true, "str1", null),
        Row(false, null, null),
        Row(null, null, null),
        null)
    )

    // Access a struct and fields inside of it.
    checkAnswer(
      sql("select struct, struct.field1, struct.field2 from jsonTable"),
      Row(
        Row(true, new java.math.BigDecimal("92233720368547758070")),
        true,
        new java.math.BigDecimal("92233720368547758070")) :: Nil
    )

    // Access an array field of a struct.
    checkAnswer(
      sql("select structWithArrayFields.field1, structWithArrayFields.field2 from jsonTable"),
      Row(Seq(4, 5, 6), Seq("str1", "str2"))
    )

    // Access elements of an array field of a struct.
    checkAnswer(
      sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] from jsonTable"),
      Row(5, null)
    )
  }

  test("GetField operation on complex data type") {
    val jsonDF = sqlContext.read.json(complexFieldAndType1)
    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"),
      Row(true, "str1")
    )

    // Getting all values of a specific field from an array of structs.
    checkAnswer(
      sql("select arrayOfStruct.field1, arrayOfStruct.field2 from jsonTable"),
      Row(Seq(true, false, null), Seq("str1", null, null))
    )
  }

  test("Type conflict in primitive field values") {
    val jsonDF = sqlContext.read.json(primitiveFieldValueTypeConflict)

    val expectedSchema = StructType(
      StructField("num_bool", StringType, true) ::
      StructField("num_num_1", LongType, true) ::
      StructField("num_num_2", DoubleType, true) ::
      StructField("num_num_3", DoubleType, true) ::
      StructField("num_str", StringType, true) ::
      StructField("str_bool", StringType, true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select * from jsonTable"),
      Row("true", 11L, null, 1.1, "13.1", "str1") ::
        Row("12", null, 21474836470.9, null, null, "true") ::
        Row("false", 21474836470L, 92233720368547758070d, 100, "str1", "false") ::
        Row(null, 21474836570L, 1.1, 21474836470L, "92233720368547758070", null) :: Nil
    )

    // Number and Boolean conflict: resolve the type as number in this query.
    checkAnswer(
      sql("select num_bool - 10 from jsonTable where num_bool > 11"),
      Row(2)
    )

    // Widening to LongType
    checkAnswer(
      sql("select num_num_1 - 100 from jsonTable where num_num_1 > 11"),
      Row(21474836370L) :: Row(21474836470L) :: Nil
    )

    checkAnswer(
      sql("select num_num_1 - 100 from jsonTable where num_num_1 > 10"),
      Row(-89) :: Row(21474836370L) :: Row(21474836470L) :: Nil
    )

    // Widening to DecimalType
    checkAnswer(
      sql("select num_num_2 + 1.3 from jsonTable where num_num_2 > 1.1"),
      Row(21474836472.2) ::
        Row(92233720368547758071.3) :: Nil
    )

    // Widening to Double
    checkAnswer(
      sql("select num_num_3 + 1.2 from jsonTable where num_num_3 > 1.1"),
      Row(101.2) :: Row(21474836471.2) :: Nil
    )

    // Number and String conflict: resolve the type as number in this query.
    checkAnswer(
      sql("select num_str + 1.2 from jsonTable where num_str > 14"),
      Row(BigDecimal("92233720368547758071.2"))
    )

    // Number and String conflict: resolve the type as number in this query.
    checkAnswer(
      sql("select num_str + 1.2 from jsonTable where num_str >= 92233720368547758060"),
      Row(new java.math.BigDecimal("92233720368547758071.2"))
    )

    // String and Boolean conflict: resolve the type as string.
    checkAnswer(
      sql("select * from jsonTable where str_bool = 'str1'"),
      Row("true", 11L, null, 1.1, "13.1", "str1")
    )
  }

  ignore("Type conflict in primitive field values (Ignored)") {
    val jsonDF = sqlContext.read.json(primitiveFieldValueTypeConflict)
    jsonDF.registerTempTable("jsonTable")

    // Right now, the analyzer does not promote strings in a boolean expression.
    // Number and Boolean conflict: resolve the type as boolean in this query.
    checkAnswer(
      sql("select num_bool from jsonTable where NOT num_bool"),
      Row(false)
    )

    checkAnswer(
      sql("select str_bool from jsonTable where NOT str_bool"),
      Row(false)
    )

    // Right now, the analyzer does not know that num_bool should be treated as a boolean.
    // Number and Boolean conflict: resolve the type as boolean in this query.
    checkAnswer(
      sql("select num_bool from jsonTable where num_bool"),
      Row(true)
    )

    checkAnswer(
      sql("select str_bool from jsonTable where str_bool"),
      Row(false)
    )

    // The plan of the following DSL is
    // Project [(CAST(num_str#65:4, DoubleType) + 1.2) AS num#78]
    //  Filter (CAST(CAST(num_str#65:4, DoubleType), DecimalType) > 92233720368547758060)
    //    ExistingRdd [num_bool#61,num_num_1#62L,num_num_2#63,num_num_3#64,num_str#65,str_bool#66]
    // We should directly cast num_str to DecimalType and also need to do the right type promotion
    // in the Project.
    checkAnswer(
      jsonDF.
        where('num_str >= BigDecimal("92233720368547758060")).
        select(('num_str + 1.2).as("num")),
      Row(new java.math.BigDecimal("92233720368547758071.2").doubleValue())
    )

    // The following test will fail. The type of num_str is StringType.
    // So, to evaluate num_str + 1.2, we first need to use Cast to convert the type.
    // In our test data, one value of num_str is 13.1.
    // The result of (CAST(num_str#65:4, DoubleType) + 1.2) for this value is 14.299999999999999,
    // which is not 14.3.
    // Number and String conflict: resolve the type as number in this query.
    checkAnswer(
      sql("select num_str + 1.2 from jsonTable where num_str > 13"),
      Row(BigDecimal("14.3")) :: Row(BigDecimal("92233720368547758071.2")) :: Nil
    )
  }

  test("Type conflict in complex field values") {
    val jsonDF = sqlContext.read.json(complexFieldValueTypeConflict)

    val expectedSchema = StructType(
      StructField("array", ArrayType(LongType, true), true) ::
      StructField("num_struct", StringType, true) ::
      StructField("str_array", StringType, true) ::
      StructField("struct", StructType(
        StructField("field", StringType, true) :: Nil), true) ::
      StructField("struct_array", StringType, true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select * from jsonTable"),
      Row(Seq(), "11", "[1,2,3]", Row(null), "[]") ::
        Row(null, """{"field":false}""", null, null, "{}") ::
        Row(Seq(4, 5, 6), null, "str", Row(null), "[7,8,9]") ::
        Row(Seq(7), "{}", """["str1","str2",33]""", Row("str"), """{"field":true}""") :: Nil
    )
  }

  test("Type conflict in array elements") {
    val jsonDF = sqlContext.read.json(arrayElementTypeConflict)

    val expectedSchema = StructType(
      StructField("array1", ArrayType(StringType, true), true) ::
      StructField("array2", ArrayType(StructType(
        StructField("field", LongType, true) :: Nil), true), true) ::
      StructField("array3", ArrayType(StringType, true), true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select * from jsonTable"),
      Row(Seq("1", "1.1", "true", null, "[]", "{}", "[2,3,4]",
        """{"field":"str"}"""), Seq(Row(214748364700L), Row(1)), null) ::
      Row(null, null, Seq("""{"field":"str"}""", """{"field":1}""")) ::
      Row(null, null, Seq("1", "2", "3")) :: Nil
    )

    // Treat an element as a number.
    checkAnswer(
      sql("select array1[0] + 1 from jsonTable where array1 is not null"),
      Row(2)
    )
  }

  test("Handling missing fields") {
    val jsonDF = sqlContext.read.json(missingFields)

    val expectedSchema = StructType(
      StructField("a", BooleanType, true) ::
      StructField("b", LongType, true) ::
      StructField("c", ArrayType(LongType, true), true) ::
      StructField("d", StructType(
        StructField("field", BooleanType, true) :: Nil), true) ::
      StructField("e", StringType, true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")
  }

  test("Loading a JSON dataset from a text file") {
    val dir = Utils.createTempDir()
    dir.delete()
    val path = dir.getCanonicalPath
    primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)
    val jsonDF = sqlContext.read.json(path)

    val expectedSchema = StructType(
      StructField("bigInteger", DecimalType(20, 0), true) ::
      StructField("boolean", BooleanType, true) ::
      StructField("double", DoubleType, true) ::
      StructField("integer", LongType, true) ::
      StructField("long", LongType, true) ::
      StructField("null", StringType, true) ::
      StructField("string", StringType, true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select * from jsonTable"),
      Row(new java.math.BigDecimal("92233720368547758070"),
      true,
      1.7976931348623157E308,
      10,
      21474836470L,
      null,
      "this is a simple string.")
    )
  }

  test("Loading a JSON dataset primitivesAsString returns schema with primitive types as strings") {
    val dir = Utils.createTempDir()
    dir.delete()
    val path = dir.getCanonicalPath
    primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)
    val jsonDF = sqlContext.read.option("primitivesAsString", "true").json(path)

    val expectedSchema = StructType(
      StructField("bigInteger", StringType, true) ::
      StructField("boolean", StringType, true) ::
      StructField("double", StringType, true) ::
      StructField("integer", StringType, true) ::
      StructField("long", StringType, true) ::
      StructField("null", StringType, true) ::
      StructField("string", StringType, true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select * from jsonTable"),
      Row("92233720368547758070",
      "true",
      "1.7976931348623157E308",
      "10",
      "21474836470",
      null,
      "this is a simple string.")
    )
  }

  test("Loading a JSON dataset primitivesAsString returns complex fields as strings") {
    val jsonDF = sqlContext.read.option("primitivesAsString", "true").json(complexFieldAndType1)

    val expectedSchema = StructType(
      StructField("arrayOfArray1", ArrayType(ArrayType(StringType, true), true), true) ::
      StructField("arrayOfArray2", ArrayType(ArrayType(StringType, true), true), true) ::
      StructField("arrayOfBigInteger", ArrayType(StringType, true), true) ::
      StructField("arrayOfBoolean", ArrayType(StringType, true), true) ::
      StructField("arrayOfDouble", ArrayType(StringType, true), true) ::
      StructField("arrayOfInteger", ArrayType(StringType, true), true) ::
      StructField("arrayOfLong", ArrayType(StringType, true), true) ::
      StructField("arrayOfNull", ArrayType(StringType, true), true) ::
      StructField("arrayOfString", ArrayType(StringType, true), true) ::
      StructField("arrayOfStruct", ArrayType(
        StructType(
          StructField("field1", StringType, true) ::
          StructField("field2", StringType, true) ::
          StructField("field3", StringType, true) :: Nil), true), true) ::
      StructField("struct", StructType(
        StructField("field1", StringType, true) ::
        StructField("field2", StringType, true) :: Nil), true) ::
      StructField("structWithArrayFields", StructType(
        StructField("field1", ArrayType(StringType, true), true) ::
        StructField("field2", ArrayType(StringType, true), true) :: Nil), true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    // Access elements of a primitive array.
    checkAnswer(
      sql("select arrayOfString[0], arrayOfString[1], arrayOfString[2] from jsonTable"),
      Row("str1", "str2", null)
    )

    // Access an array of null values.
    checkAnswer(
      sql("select arrayOfNull from jsonTable"),
      Row(Seq(null, null, null, null))
    )

    // Access elements of a BigInteger array (we use DecimalType internally).
    checkAnswer(
      sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] from jsonTable"),
      Row("922337203685477580700", "-922337203685477580800", null)
    )

    // Access elements of an array of arrays.
    checkAnswer(
      sql("select arrayOfArray1[0], arrayOfArray1[1] from jsonTable"),
      Row(Seq("1", "2", "3"), Seq("str1", "str2"))
    )

    // Access elements of an array of arrays.
    checkAnswer(
      sql("select arrayOfArray2[0], arrayOfArray2[1] from jsonTable"),
      Row(Seq("1", "2", "3"), Seq("1.1", "2.1", "3.1"))
    )

    // Access elements of an array inside a filed with the type of ArrayType(ArrayType).
    checkAnswer(
      sql("select arrayOfArray1[1][1], arrayOfArray2[1][1] from jsonTable"),
      Row("str2", "2.1")
    )

    // Access elements of an array of structs.
    checkAnswer(
      sql("select arrayOfStruct[0], arrayOfStruct[1], arrayOfStruct[2], arrayOfStruct[3] " +
        "from jsonTable"),
      Row(
        Row("true", "str1", null),
        Row("false", null, null),
        Row(null, null, null),
        null)
    )

    // Access a struct and fields inside of it.
    checkAnswer(
      sql("select struct, struct.field1, struct.field2 from jsonTable"),
      Row(
        Row("true", "92233720368547758070"),
        "true",
        "92233720368547758070") :: Nil
    )

    // Access an array field of a struct.
    checkAnswer(
      sql("select structWithArrayFields.field1, structWithArrayFields.field2 from jsonTable"),
      Row(Seq("4", "5", "6"), Seq("str1", "str2"))
    )

    // Access elements of an array field of a struct.
    checkAnswer(
      sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] from jsonTable"),
      Row("5", null)
    )
  }

  test("Loading a JSON dataset prefersDecimal returns schema with float types as BigDecimal") {
    val jsonDF = sqlContext.read.option("prefersDecimal", "true").json(primitiveFieldAndType)

    val expectedSchema = StructType(
      StructField("bigInteger", DecimalType(20, 0), true) ::
        StructField("boolean", BooleanType, true) ::
        StructField("double", DecimalType(17, -292), true) ::
        StructField("integer", LongType, true) ::
        StructField("long", LongType, true) ::
        StructField("null", StringType, true) ::
        StructField("string", StringType, true) :: Nil)

    assert(expectedSchema === jsonDF.schema)

    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select * from jsonTable"),
      Row(BigDecimal("92233720368547758070"),
        true,
        BigDecimal("1.7976931348623157E308"),
        10,
        21474836470L,
        null,
        "this is a simple string.")
    )
  }

  test("Find compatible types even if inferred DecimalType is not capable of other IntegralType") {
    val mixedIntegerAndDoubleRecords = sparkContext.parallelize(
      """{"a": 3, "b": 1.1}""" ::
      s"""{"a": 3.1, "b": 0.${"0" * 38}1}""" :: Nil)
    val jsonDF = sqlContext.read
      .option("prefersDecimal", "true")
      .json(mixedIntegerAndDoubleRecords)

    // The values in `a` field will be decimals as they fit in decimal. For `b` field,
    // they will be doubles as `1.0E-39D` does not fit.
    val expectedSchema = StructType(
      StructField("a", DecimalType(21, 1), true) ::
      StructField("b", DoubleType, true) :: Nil)

    assert(expectedSchema === jsonDF.schema)
    checkAnswer(
      jsonDF,
      Row(BigDecimal("3"), 1.1D) ::
      Row(BigDecimal("3.1"), 1.0E-39D) :: Nil
    )
  }

  test("Infer big integers correctly even when it does not fit in decimal") {
    val jsonDF = sqlContext.read
      .json(bigIntegerRecords)

    // The value in `a` field will be a double as it does not fit in decimal. For `b` field,
    // it will be a decimal as `92233720368547758070`.
    val expectedSchema = StructType(
      StructField("a", DoubleType, true) ::
      StructField("b", DecimalType(20, 0), true) :: Nil)

    assert(expectedSchema === jsonDF.schema)
    checkAnswer(jsonDF, Row(1.0E38D, BigDecimal("92233720368547758070")))
  }

  test("Infer floating-point values correctly even when it does not fit in decimal") {
    val jsonDF = sqlContext.read
      .option("prefersDecimal", "true")
      .json(floatingValueRecords)

    // The value in `a` field will be a double as it does not fit in decimal. For `b` field,
    // it will be a decimal as `0.01` by having a precision equal to the scale.
    val expectedSchema = StructType(
      StructField("a", DoubleType, true) ::
      StructField("b", DecimalType(2, 2), true):: Nil)

    assert(expectedSchema === jsonDF.schema)
    checkAnswer(jsonDF, Row(1.0E-39D, BigDecimal(0.01)))

    val mergedJsonDF = sqlContext.read
      .option("prefersDecimal", "true")
      .json(floatingValueRecords ++ bigIntegerRecords)

    val expectedMergedSchema = StructType(
      StructField("a", DoubleType, true) ::
      StructField("b", DecimalType(22, 2), true):: Nil)

    assert(expectedMergedSchema === mergedJsonDF.schema)
    checkAnswer(
      mergedJsonDF,
      Row(1.0E-39D, BigDecimal(0.01)) ::
      Row(1.0E38D, BigDecimal("92233720368547758070")) :: Nil
    )
  }

  test("Loading a JSON dataset from a text file with SQL") {
    val dir = Utils.createTempDir()
    dir.delete()
    val path = dir.getCanonicalPath
    primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)

    sql(
      s"""
        |CREATE TEMPORARY TABLE jsonTableSQL
        |USING org.apache.spark.sql.json
        |OPTIONS (
        |  path '$path'
        |)
      """.stripMargin)

    checkAnswer(
      sql("select * from jsonTableSQL"),
      Row(new java.math.BigDecimal("92233720368547758070"),
        true,
        1.7976931348623157E308,
        10,
        21474836470L,
        null,
        "this is a simple string.")
    )
  }

  test("Applying schemas") {
    val dir = Utils.createTempDir()
    dir.delete()
    val path = dir.getCanonicalPath
    primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)

    val schema = StructType(
      StructField("bigInteger", DecimalType.SYSTEM_DEFAULT, true) ::
      StructField("boolean", BooleanType, true) ::
      StructField("double", DoubleType, true) ::
      StructField("integer", IntegerType, true) ::
      StructField("long", LongType, true) ::
      StructField("null", StringType, true) ::
      StructField("string", StringType, true) :: Nil)

    val jsonDF1 = sqlContext.read.schema(schema).json(path)

    assert(schema === jsonDF1.schema)

    jsonDF1.registerTempTable("jsonTable1")

    checkAnswer(
      sql("select * from jsonTable1"),
      Row(new java.math.BigDecimal("92233720368547758070"),
      true,
      1.7976931348623157E308,
      10,
      21474836470L,
      null,
      "this is a simple string.")
    )

    val jsonDF2 = sqlContext.read.schema(schema).json(primitiveFieldAndType)

    assert(schema === jsonDF2.schema)

    jsonDF2.registerTempTable("jsonTable2")

    checkAnswer(
      sql("select * from jsonTable2"),
      Row(new java.math.BigDecimal("92233720368547758070"),
      true,
      1.7976931348623157E308,
      10,
      21474836470L,
      null,
      "this is a simple string.")
    )
  }

  test("Applying schemas with MapType") {
    val schemaWithSimpleMap = StructType(
      StructField("map", MapType(StringType, IntegerType, true), false) :: Nil)
    val jsonWithSimpleMap = sqlContext.read.schema(schemaWithSimpleMap).json(mapType1)

    jsonWithSimpleMap.registerTempTable("jsonWithSimpleMap")

    checkAnswer(
      sql("select `map` from jsonWithSimpleMap"),
      Row(Map("a" -> 1)) ::
      Row(Map("b" -> 2)) ::
      Row(Map("c" -> 3)) ::
      Row(Map("c" -> 1, "d" -> 4)) ::
      Row(Map("e" -> null)) :: Nil
    )

    checkAnswer(
      sql("select `map`['c'] from jsonWithSimpleMap"),
      Row(null) ::
      Row(null) ::
      Row(3) ::
      Row(1) ::
      Row(null) :: Nil
    )

    val innerStruct = StructType(
      StructField("field1", ArrayType(IntegerType, true), true) ::
      StructField("field2", IntegerType, true) :: Nil)
    val schemaWithComplexMap = StructType(
      StructField("map", MapType(StringType, innerStruct, true), false) :: Nil)

    val jsonWithComplexMap = sqlContext.read.schema(schemaWithComplexMap).json(mapType2)

    jsonWithComplexMap.registerTempTable("jsonWithComplexMap")

    checkAnswer(
      sql("select `map` from jsonWithComplexMap"),
      Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) ::
      Row(Map("b" -> Row(null, 2))) ::
      Row(Map("c" -> Row(Seq(), 4))) ::
      Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) ::
      Row(Map("e" -> null)) ::
      Row(Map("f" -> Row(null, null))) :: Nil
    )

    checkAnswer(
      sql("select `map`['a'].field1, `map`['c'].field2 from jsonWithComplexMap"),
      Row(Seq(1, 2, 3, null), null) ::
      Row(null, null) ::
      Row(null, 4) ::
      Row(null, 3) ::
      Row(null, null) ::
      Row(null, null) :: Nil
    )
  }

  test("SPARK-2096 Correctly parse dot notations") {
    val jsonDF = sqlContext.read.json(complexFieldAndType2)
    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"),
      Row(true, "str1")
    )
    checkAnswer(
      sql(
        """
          |select complexArrayOfStruct[0].field1[1].inner2[0], complexArrayOfStruct[1].field2[0][1]
          |from jsonTable
        """.stripMargin),
      Row("str2", 6)
    )
  }

  test("SPARK-3390 Complex arrays") {
    val jsonDF = sqlContext.read.json(complexFieldAndType2)
    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql(
        """
          |select arrayOfArray1[0][0][0], arrayOfArray1[1][0][1], arrayOfArray1[1][1][0]
          |from jsonTable
        """.stripMargin),
      Row(5, 7, 8)
    )
    checkAnswer(
      sql(
        """
          |select arrayOfArray2[0][0][0].inner1, arrayOfArray2[1][0],
          |arrayOfArray2[1][1][1].inner2[0], arrayOfArray2[2][0][0].inner3[0][0].inner4
          |from jsonTable
        """.stripMargin),
      Row("str1", Nil, "str4", 2)
    )
  }

  test("SPARK-3308 Read top level JSON arrays") {
    val jsonDF = sqlContext.read.json(jsonArray)
    jsonDF.registerTempTable("jsonTable")

    checkAnswer(
      sql(
        """
          |select a, b, c
          |from jsonTable
        """.stripMargin),
      Row("str_a_1", null, null) ::
        Row("str_a_2", null, null) ::
        Row(null, "str_b_3", null) ::
        Row("str_a_4", "str_b_4", "str_c_4") :: Nil
    )
  }

  test("Corrupt records: FAILFAST mode") {
    val schema = StructType(
        StructField("a", StringType, true) :: Nil)
    // `FAILFAST` mode should throw an exception for corrupt records.
    val exceptionOne = intercept[SparkException] {
      sqlContext.read
        .option("mode", "FAILFAST")
        .json(corruptRecords)
        .collect()
    }
    assert(exceptionOne.getMessage.contains("Malformed line in FAILFAST mode: {"))

    val exceptionTwo = intercept[SparkException] {
      sqlContext.read
        .option("mode", "FAILFAST")
        .schema(schema)
        .json(corruptRecords)
        .collect()
    }
    assert(exceptionTwo.getMessage.contains("Malformed line in FAILFAST mode: {"))
  }

  test("Corrupt records: DROPMALFORMED mode") {
    val schemaOne = StructType(
      StructField("a", StringType, true) ::
        StructField("b", StringType, true) ::
        StructField("c", StringType, true) :: Nil)
    val schemaTwo = StructType(
      StructField("a", StringType, true) :: Nil)
    // `DROPMALFORMED` mode should skip corrupt records
    val jsonDFOne = sqlContext.read
      .option("mode", "DROPMALFORMED")
      .json(corruptRecords)
    checkAnswer(
      jsonDFOne,
      Row("str_a_4", "str_b_4", "str_c_4") :: Nil
    )
    assert(jsonDFOne.schema === schemaOne)

    val jsonDFTwo = sqlContext.read
      .option("mode", "DROPMALFORMED")
      .schema(schemaTwo)
      .json(corruptRecords)
    checkAnswer(
      jsonDFTwo,
      Row("str_a_4") :: Nil)
    assert(jsonDFTwo.schema === schemaTwo)
  }

  test("Corrupt records: PERMISSIVE mode") {
    // Test if we can query corrupt records.
    withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") {
      withTempTable("jsonTable") {
        val jsonDF = sqlContext.read.json(corruptRecords)
        jsonDF.registerTempTable("jsonTable")
        val schema = StructType(
          StructField("_unparsed", StringType, true) ::
          StructField("a", StringType, true) ::
          StructField("b", StringType, true) ::
          StructField("c", StringType, true) :: Nil)

        assert(schema === jsonDF.schema)

        // In HiveContext, backticks should be used to access columns starting with a underscore.
        checkAnswer(
          sql(
            """
              |SELECT a, b, c, _unparsed
              |FROM jsonTable
            """.stripMargin),
          Row(null, null, null, "{") ::
            Row(null, null, null, """{"a":1, b:2}""") ::
            Row(null, null, null, """{"a":{, b:3}""") ::
            Row("str_a_4", "str_b_4", "str_c_4", null) ::
            Row(null, null, null, "]") :: Nil
        )

        checkAnswer(
          sql(
            """
              |SELECT a, b, c
              |FROM jsonTable
              |WHERE _unparsed IS NULL
            """.stripMargin),
          Row("str_a_4", "str_b_4", "str_c_4")
        )

        checkAnswer(
          sql(
            """
              |SELECT _unparsed
              |FROM jsonTable
              |WHERE _unparsed IS NOT NULL
            """.stripMargin),
          Row("{") ::
            Row("""{"a":1, b:2}""") ::
            Row("""{"a":{, b:3}""") ::
            Row("]") :: Nil
        )
      }
    }
  }

  test("SPARK-13953 Rename the corrupt record field via option") {
    val jsonDF = sqlContext.read
      .option("columnNameOfCorruptRecord", "_malformed")
      .json(corruptRecords)
    val schema = StructType(
      StructField("_malformed", StringType, true) ::
        StructField("a", StringType, true) ::
        StructField("b", StringType, true) ::
        StructField("c", StringType, true) :: Nil)

    assert(schema === jsonDF.schema)
    checkAnswer(
      jsonDF.selectExpr("a", "b", "c", "_malformed"),
      Row(null, null, null, "{") ::
        Row(null, null, null, """{"a":1, b:2}""") ::
        Row(null, null, null, """{"a":{, b:3}""") ::
        Row("str_a_4", "str_b_4", "str_c_4", null) ::
        Row(null, null, null, "]") :: Nil
    )
  }

  test("SPARK-4068: nulls in arrays") {
    val jsonDF = sqlContext.read.json(nullsInArrays)
    jsonDF.registerTempTable("jsonTable")

    val schema = StructType(
      StructField("field1",
        ArrayType(ArrayType(ArrayType(ArrayType(StringType, true), true), true), true), true) ::
      StructField("field2",
        ArrayType(ArrayType(
          StructType(StructField("Test", LongType, true) :: Nil), true), true), true) ::
      StructField("field3",
        ArrayType(ArrayType(
          StructType(StructField("Test", StringType, true) :: Nil), true), true), true) ::
      StructField("field4",
        ArrayType(ArrayType(ArrayType(LongType, true), true), true), true) :: Nil)

    assert(schema === jsonDF.schema)

    checkAnswer(
      sql(
        """
          |SELECT field1, field2, field3, field4
          |FROM jsonTable
        """.stripMargin),
      Row(Seq(Seq(null), Seq(Seq(Seq("Test")))), null, null, null) ::
        Row(null, Seq(null, Seq(Row(1))), null, null) ::
        Row(null, null, Seq(Seq(null), Seq(Row("2"))), null) ::
        Row(null, null, null, Seq(Seq(null, Seq(1, 2, 3)))) :: Nil
    )
  }

  test("SPARK-4228 DataFrame to JSON") {
    val schema1 = StructType(
      StructField("f1", IntegerType, false) ::
      StructField("f2", StringType, false) ::
      StructField("f3", BooleanType, false) ::
      StructField("f4", ArrayType(StringType), nullable = true) ::
      StructField("f5", IntegerType, true) :: Nil)

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

    val df1 = sqlContext.createDataFrame(rowRDD1, schema1)
    df1.registerTempTable("applySchema1")
    val df2 = df1.toDF
    val result = df2.toJSON.collect()
    // scalastyle:off
    assert(result(0) === "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}")
    assert(result(3) === "{\"f1\":4,\"f2\":\"D4\",\"f3\":true,\"f4\":[\"4\",\" D4\",\" true\",\" 2147483644\"],\"f5\":2147483644}")
    // scalastyle:on

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

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

    val df3 = sqlContext.createDataFrame(rowRDD2, schema2)
    df3.registerTempTable("applySchema2")
    val df4 = df3.toDF
    val result2 = df4.toJSON.collect()

    assert(result2(1) === "{\"f1\":{\"f11\":2,\"f12\":false},\"f2\":{\"B2\":null}}")
    assert(result2(3) === "{\"f1\":{\"f11\":4,\"f12\":true},\"f2\":{\"D4\":2147483644}}")

    val jsonDF = sqlContext.read.json(primitiveFieldAndType)
    val primTable = sqlContext.read.json(jsonDF.toJSON.rdd)
    primTable.registerTempTable("primitiveTable")
    checkAnswer(
        sql("select * from primitiveTable"),
      Row(new java.math.BigDecimal("92233720368547758070"),
        true,
        1.7976931348623157E308,
        10,
        21474836470L,
        "this is a simple string.")
      )

    val complexJsonDF = sqlContext.read.json(complexFieldAndType1)
    val compTable = sqlContext.read.json(complexJsonDF.toJSON.rdd)
    compTable.registerTempTable("complexTable")
    // Access elements of a primitive array.
    checkAnswer(
      sql("select arrayOfString[0], arrayOfString[1], arrayOfString[2] from complexTable"),
      Row("str1", "str2", null)
    )

    // Access an array of null values.
    checkAnswer(
      sql("select arrayOfNull from complexTable"),
      Row(Seq(null, null, null, null))
    )

    // Access elements of a BigInteger array (we use DecimalType internally).
    checkAnswer(
      sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] " +
        " from complexTable"),
      Row(new java.math.BigDecimal("922337203685477580700"),
        new java.math.BigDecimal("-922337203685477580800"), null)
    )

    // Access elements of an array of arrays.
    checkAnswer(
      sql("select arrayOfArray1[0], arrayOfArray1[1] from complexTable"),
      Row(Seq("1", "2", "3"), Seq("str1", "str2"))
    )

    // Access elements of an array of arrays.
    checkAnswer(
      sql("select arrayOfArray2[0], arrayOfArray2[1] from complexTable"),
      Row(Seq(1.0, 2.0, 3.0), Seq(1.1, 2.1, 3.1))
    )

    // Access elements of an array inside a filed with the type of ArrayType(ArrayType).
    checkAnswer(
      sql("select arrayOfArray1[1][1], arrayOfArray2[1][1] from complexTable"),
      Row("str2", 2.1)
    )

    // Access a struct and fields inside of it.
    checkAnswer(
      sql("select struct, struct.field1, struct.field2 from complexTable"),
      Row(
        Row(true, new java.math.BigDecimal("92233720368547758070")),
        true,
        new java.math.BigDecimal("92233720368547758070")) :: Nil
    )

    // Access an array field of a struct.
    checkAnswer(
      sql("select structWithArrayFields.field1, structWithArrayFields.field2 from complexTable"),
      Row(Seq(4, 5, 6), Seq("str1", "str2"))
    )

    // Access elements of an array field of a struct.
    checkAnswer(
      sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] " +
        "from complexTable"),
      Row(5, null)
    )
  }

  test("JSONRelation equality test") {
    withTempPath(dir => {
      val path = dir.getCanonicalFile.toURI.toString
      sparkContext.parallelize(1 to 100)
        .map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path)

      val d1 = DataSource(
        sqlContext,
        userSpecifiedSchema = None,
        partitionColumns = Array.empty[String],
        bucketSpec = None,
        className = classOf[DefaultSource].getCanonicalName,
        options = Map("path" -> path)).resolveRelation()

      val d2 = DataSource(
        sqlContext,
        userSpecifiedSchema = None,
        partitionColumns = Array.empty[String],
        bucketSpec = None,
        className = classOf[DefaultSource].getCanonicalName,
        options = Map("path" -> path)).resolveRelation()
      assert(d1 === d2)
    })
  }

  test("SPARK-6245 JsonRDD.inferSchema on empty RDD") {
    // This is really a test that it doesn't throw an exception
    val emptySchema = InferSchema.infer(empty, "", new JSONOptions(Map()))
    assert(StructType(Seq()) === emptySchema)
  }

  test("SPARK-7565 MapType in JsonRDD") {
    withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") {
      withTempDir { dir =>
        val schemaWithSimpleMap = StructType(
          StructField("map", MapType(StringType, IntegerType, true), false) :: Nil)
        val df = sqlContext.read.schema(schemaWithSimpleMap).json(mapType1)

        val path = dir.getAbsolutePath
        df.write.mode("overwrite").parquet(path)
        // order of MapType is not defined
        assert(sqlContext.read.parquet(path).count() == 5)

        val df2 = sqlContext.read.json(corruptRecords)
        df2.write.mode("overwrite").parquet(path)
        checkAnswer(sqlContext.read.parquet(path), df2.collect())
      }
    }
  }

  test("SPARK-8093 Erase empty structs") {
    val emptySchema = InferSchema.infer(emptyRecords, "", new JSONOptions(Map()))
    assert(StructType(Seq()) === emptySchema)
  }

  test("JSON with Partition") {
    def makePartition(rdd: RDD[String], parent: File, partName: String, partValue: Any): File = {
      val p = new File(parent, s"$partName=${partValue.toString}")
      rdd.saveAsTextFile(p.getCanonicalPath)
      p
    }

    withTempPath(root => {
      val d1 = new File(root, "d1=1")
      // root/dt=1/col1=abc
      val p1_col1 = makePartition(
        sparkContext.parallelize(2 to 5).map(i => s"""{"a": 1, "b": "str$i"}"""),
        d1,
        "col1",
        "abc")

      // root/dt=1/col1=abd
      val p2 = makePartition(
        sparkContext.parallelize(6 to 10).map(i => s"""{"a": 1, "b": "str$i"}"""),
        d1,
        "col1",
        "abd")

        sqlContext.read.json(root.getAbsolutePath).registerTempTable("test_myjson_with_part")
        checkAnswer(sql(
          "SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abc'"), Row(4))
        checkAnswer(sql(
          "SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abd'"), Row(5))
        checkAnswer(sql(
          "SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
    })
  }

  test("backward compatibility") {
    // This test we make sure our JSON support can read JSON data generated by previous version
    // of Spark generated through toJSON method and JSON data source.
    // The data is generated by the following program.
    // Here are a few notes:
    //  - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13)
    //      in the JSON object.
    //  - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to
    //      JSON objects generated by those Spark versions (col17).
    //  - If the type is NullType, we do not write data out.

    // Create the schema.
    val struct =
      StructType(
        StructField("f1", FloatType, true) ::
          StructField("f2", ArrayType(BooleanType), true) :: Nil)

    val dataTypes =
      Seq(
        StringType, BinaryType, NullType, BooleanType,
        ByteType, ShortType, IntegerType, LongType,
        FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5),
        DateType, TimestampType,
        ArrayType(IntegerType), MapType(StringType, LongType), struct,
        new MyDenseVectorUDT())
    val fields = dataTypes.zipWithIndex.map { case (dataType, index) =>
      StructField(s"col$index", dataType, nullable = true)
    }
    val schema = StructType(fields)

    val constantValues =
      Seq(
        "a string in binary".getBytes(StandardCharsets.UTF_8),
        null,
        true,
        1.toByte,
        2.toShort,
        3,
        Long.MaxValue,
        0.25.toFloat,
        0.75,
        new java.math.BigDecimal(s"1234.23456"),
        new java.math.BigDecimal(s"1.23456"),
        java.sql.Date.valueOf("2015-01-01"),
        java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"),
        Seq(2, 3, 4),
        Map("a string" -> 2000L),
        Row(4.75.toFloat, Seq(false, true)),
        new MyDenseVector(Array(0.25, 2.25, 4.25)))
    val data =
      Row.fromSeq(Seq("Spark " + sqlContext.sparkContext.version) ++ constantValues) :: Nil

    // Data generated by previous versions.
    // scalastyle:off
    val existingJSONData =
      """{"col0":"Spark 1.2.2","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
      """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
      """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
      """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
      """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
      """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
      """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"16436","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil
    // scalastyle:on

    // Generate data for the current version.
    val df = sqlContext.createDataFrame(sqlContext.sparkContext.parallelize(data, 1), schema)
    withTempPath { path =>
      df.write.format("json").mode("overwrite").save(path.getCanonicalPath)

      // df.toJSON will convert internal rows to external rows first and then generate
      // JSON objects. While, df.write.format("json") will write internal rows directly.
      val allJSON =
        existingJSONData ++
          df.toJSON.collect() ++
          sparkContext.textFile(path.getCanonicalPath).collect()

      Utils.deleteRecursively(path)
      sparkContext.parallelize(allJSON, 1).saveAsTextFile(path.getCanonicalPath)

      // Read data back with the schema specified.
      val col0Values =
        Seq(
          "Spark 1.2.2",
          "Spark 1.3.1",
          "Spark 1.3.1",
          "Spark 1.4.1",
          "Spark 1.4.1",
          "Spark 1.5.0",
          "Spark 1.5.0",
          "Spark " + sqlContext.sparkContext.version,
          "Spark " + sqlContext.sparkContext.version)
      val expectedResult = col0Values.map { v =>
        Row.fromSeq(Seq(v) ++ constantValues)
      }
      checkAnswer(
        sqlContext.read.format("json").schema(schema).load(path.getCanonicalPath),
        expectedResult
      )
    }
  }

  test("SPARK-11544 test pathfilter") {
    withTempPath { dir =>
      val path = dir.getCanonicalPath

      val df = sqlContext.range(2)
      df.write.json(path + "/p=1")
      df.write.json(path + "/p=2")
      assert(sqlContext.read.json(path).count() === 4)

      val clonedConf = new Configuration(hadoopConfiguration)
      try {
        // Setting it twice as the name of the propery has changed between hadoop versions.
        hadoopConfiguration.setClass(
          "mapred.input.pathFilter.class",
          classOf[TestFileFilter],
          classOf[PathFilter])
        hadoopConfiguration.setClass(
          "mapreduce.input.pathFilter.class",
          classOf[TestFileFilter],
          classOf[PathFilter])
        assert(sqlContext.read.json(path).count() === 2)
      } finally {
        // Hadoop 1 doesn't have `Configuration.unset`
        hadoopConfiguration.clear()
        clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue))
      }
    }
  }

  test("SPARK-12057 additional corrupt records do not throw exceptions") {
    // Test if we can query corrupt records.
    withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") {
      withTempTable("jsonTable") {
        val schema = StructType(
          StructField("_unparsed", StringType, true) ::
            StructField("dummy", StringType, true) :: Nil)

        {
          // We need to make sure we can infer the schema.
          val jsonDF = sqlContext.read.json(additionalCorruptRecords)
          assert(jsonDF.schema === schema)
        }

        {
          val jsonDF = sqlContext.read.schema(schema).json(additionalCorruptRecords)
          jsonDF.registerTempTable("jsonTable")

          // In HiveContext, backticks should be used to access columns starting with a underscore.
          checkAnswer(
            sql(
              """
                |SELECT dummy, _unparsed
                |FROM jsonTable
              """.stripMargin),
            Row("test", null) ::
              Row(null, """[1,2,3]""") ::
              Row(null, """":"test", "a":1}""") ::
              Row(null, """42""") ::
              Row(null, """     ","ian":"test"}""") :: Nil
          )
        }
      }
    }
  }

  test("Parse JSON rows having an array type and a struct type in the same field.") {
    withTempDir { dir =>
      val dir = Utils.createTempDir()
      dir.delete()
      val path = dir.getCanonicalPath
      arrayAndStructRecords.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)

      val schema =
        StructType(
          StructField("a", StructType(
            StructField("b", StringType) :: Nil
          )) :: Nil)
      val jsonDF = sqlContext.read.schema(schema).json(path)
      assert(jsonDF.count() == 2)
    }
  }

  test("SPARK-12872 Support to specify the option for compression codec") {
    withTempDir { dir =>
      val dir = Utils.createTempDir()
      dir.delete()
      val path = dir.getCanonicalPath
      primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)

      val jsonDF = sqlContext.read.json(path)
      val jsonDir = new File(dir, "json").getCanonicalPath
      jsonDF.coalesce(1).write
        .format("json")
        .option("compression", "gZiP")
        .save(jsonDir)

      val compressedFiles = new File(jsonDir).listFiles()
      assert(compressedFiles.exists(_.getName.endsWith(".json.gz")))

      val jsonCopy = sqlContext.read
        .format("json")
        .load(jsonDir)

      assert(jsonCopy.count == jsonDF.count)
      val jsonCopySome = jsonCopy.selectExpr("string", "long", "boolean")
      val jsonDFSome = jsonDF.selectExpr("string", "long", "boolean")
      checkAnswer(jsonCopySome, jsonDFSome)
    }
  }

  test("SPARK-13543 Write the output as uncompressed via option()") {
    val clonedConf = new Configuration(hadoopConfiguration)
    hadoopConfiguration.set("mapreduce.output.fileoutputformat.compress", "true")
    hadoopConfiguration
      .set("mapreduce.output.fileoutputformat.compress.type", CompressionType.BLOCK.toString)
    hadoopConfiguration
      .set("mapreduce.output.fileoutputformat.compress.codec", classOf[GzipCodec].getName)
    hadoopConfiguration.set("mapreduce.map.output.compress", "true")
    hadoopConfiguration.set("mapreduce.map.output.compress.codec", classOf[GzipCodec].getName)
    withTempDir { dir =>
      try {
        val dir = Utils.createTempDir()
        dir.delete()

        val path = dir.getCanonicalPath
        primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path)

        val jsonDF = sqlContext.read.json(path)
        val jsonDir = new File(dir, "json").getCanonicalPath
        jsonDF.coalesce(1).write
          .format("json")
          .option("compression", "none")
          .save(jsonDir)

        val compressedFiles = new File(jsonDir).listFiles()
        assert(compressedFiles.exists(!_.getName.endsWith(".json.gz")))

        val jsonCopy = sqlContext.read
          .format("json")
          .load(jsonDir)

        assert(jsonCopy.count == jsonDF.count)
        val jsonCopySome = jsonCopy.selectExpr("string", "long", "boolean")
        val jsonDFSome = jsonDF.selectExpr("string", "long", "boolean")
        checkAnswer(jsonCopySome, jsonDFSome)
      } finally {
        // Hadoop 1 doesn't have `Configuration.unset`
        hadoopConfiguration.clear()
        clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue))
      }
    }
  }

  test("Casting long as timestamp") {
    withTempTable("jsonTable") {
      val schema = (new StructType).add("ts", TimestampType)
      val jsonDF = sqlContext.read.schema(schema).json(timestampAsLong)

      jsonDF.registerTempTable("jsonTable")

      checkAnswer(
        sql("select ts from jsonTable"),
        Row(java.sql.Timestamp.valueOf("2016-01-02 03:04:05"))
      )
    }
  }

  test("wide nested json table") {
    val nested = (1 to 100).map { i =>
      s"""
         |"c$i": $i
       """.stripMargin
    }.mkString(", ")
    val json = s"""
       |{"a": [{$nested}], "b": [{$nested}]}
     """.stripMargin
    val rdd = sqlContext.sparkContext.makeRDD(Seq(json))
    val df = sqlContext.read.json(rdd)
    assert(df.schema.size === 2)
    df.collect()
  }
}