aboutsummaryrefslogtreecommitdiff
path: root/R/pkg/R/DataFrame.R
blob: 3b2fd733753eec52201145d9fabe25647369e710 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
#
# 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.
#

# DataFrame.R - SparkDataFrame class and methods implemented in S4 OO classes

#' @include generics.R jobj.R schema.R RDD.R pairRDD.R column.R group.R
NULL

setOldClass("jobj")

#' @title S4 class that represents a SparkDataFrame
#' @description DataFrames can be created using functions like \link{createDataFrame},
#'              \link{read.json}, \link{table} etc.
#' @family SparkDataFrame functions
#' @rdname SparkDataFrame
#' @docType class
#'
#' @slot env An R environment that stores bookkeeping states of the SparkDataFrame
#' @slot sdf A Java object reference to the backing Scala DataFrame
#' @seealso \link{createDataFrame}, \link{read.json}, \link{table}
#' @seealso \url{https://spark.apache.org/docs/latest/sparkr.html#sparkr-dataframes}
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' df <- createDataFrame(sqlContext, faithful)
#'}
setClass("SparkDataFrame",
         slots = list(env = "environment",
                      sdf = "jobj"))

setMethod("initialize", "SparkDataFrame", function(.Object, sdf, isCached) {
  .Object@env <- new.env()
  .Object@env$isCached <- isCached

  .Object@sdf <- sdf
  .Object
})

#' @rdname SparkDataFrame
#' @export
#' @param sdf A Java object reference to the backing Scala DataFrame
#' @param isCached TRUE if the SparkDataFrame is cached
dataFrame <- function(sdf, isCached = FALSE) {
  new("SparkDataFrame", sdf, isCached)
}

############################ SparkDataFrame Methods ##############################################

#' Print Schema of a SparkDataFrame
#'
#' Prints out the schema in tree format
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname printSchema
#' @name printSchema
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' printSchema(df)
#'}
setMethod("printSchema",
          signature(x = "SparkDataFrame"),
          function(x) {
            schemaString <- callJMethod(schema(x)$jobj, "treeString")
            cat(schemaString)
          })

#' Get schema object
#'
#' Returns the schema of this SparkDataFrame as a structType object.
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname schema
#' @name schema
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' dfSchema <- schema(df)
#'}
setMethod("schema",
          signature(x = "SparkDataFrame"),
          function(x) {
            structType(callJMethod(x@sdf, "schema"))
          })

#' Explain
#'
#' Print the logical and physical Catalyst plans to the console for debugging.
#'
#' @param x A SparkDataFrame
#' @param extended Logical. If extended is False, explain() only prints the physical plan.
#' @family SparkDataFrame functions
#' @rdname explain
#' @name explain
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' explain(df, TRUE)
#'}
setMethod("explain",
          signature(x = "SparkDataFrame"),
          function(x, extended = FALSE) {
            queryExec <- callJMethod(x@sdf, "queryExecution")
            if (extended) {
              cat(callJMethod(queryExec, "toString"))
            } else {
              execPlan <- callJMethod(queryExec, "executedPlan")
              cat(callJMethod(execPlan, "toString"))
            }
          })

#' isLocal
#'
#' Returns True if the `collect` and `take` methods can be run locally
#' (without any Spark executors).
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname isLocal
#' @name isLocal
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' isLocal(df)
#'}
setMethod("isLocal",
          signature(x = "SparkDataFrame"),
          function(x) {
            callJMethod(x@sdf, "isLocal")
          })

#' showDF
#'
#' Print the first numRows rows of a SparkDataFrame
#'
#' @param x A SparkDataFrame
#' @param numRows The number of rows to print. Defaults to 20.
#'
#' @family SparkDataFrame functions
#' @rdname showDF
#' @name showDF
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' showDF(df)
#'}
setMethod("showDF",
          signature(x = "SparkDataFrame"),
          function(x, numRows = 20, truncate = TRUE) {
            s <- callJMethod(x@sdf, "showString", numToInt(numRows), truncate)
            cat(s)
          })

#' show
#'
#' Print the SparkDataFrame column names and types
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname show
#' @name show
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' df
#'}
setMethod("show", "SparkDataFrame",
          function(object) {
            cols <- lapply(dtypes(object), function(l) {
              paste(l, collapse = ":")
            })
            s <- paste(cols, collapse = ", ")
            cat(paste(class(object), "[", s, "]\n", sep = ""))
          })

#' DataTypes
#'
#' Return all column names and their data types as a list
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname dtypes
#' @name dtypes
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' dtypes(df)
#'}
setMethod("dtypes",
          signature(x = "SparkDataFrame"),
          function(x) {
            lapply(schema(x)$fields(), function(f) {
              c(f$name(), f$dataType.simpleString())
            })
          })

#' Column names
#'
#' Return all column names as a list
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname columns
#' @name columns

#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' columns(df)
#' colnames(df)
#'}
setMethod("columns",
          signature(x = "SparkDataFrame"),
          function(x) {
            sapply(schema(x)$fields(), function(f) {
              f$name()
            })
          })

#' @rdname columns
#' @name names
setMethod("names",
          signature(x = "SparkDataFrame"),
          function(x) {
            columns(x)
          })

#' @rdname columns
#' @name names<-
setMethod("names<-",
          signature(x = "SparkDataFrame"),
          function(x, value) {
            if (!is.null(value)) {
              sdf <- callJMethod(x@sdf, "toDF", as.list(value))
              dataFrame(sdf)
            }
          })

#' @rdname columns
#' @name colnames
setMethod("colnames",
          signature(x = "SparkDataFrame"),
          function(x) {
            columns(x)
          })

#' @rdname columns
#' @name colnames<-
setMethod("colnames<-",
          signature(x = "SparkDataFrame"),
          function(x, value) {

            # Check parameter integrity
            if (class(value) != "character") {
              stop("Invalid column names.")
            }

            if (length(value) != ncol(x)) {
              stop(
                "Column names must have the same length as the number of columns in the dataset.")
            }

            if (any(is.na(value))) {
              stop("Column names cannot be NA.")
            }

            # Check if the column names have . in it
            if (any(regexec(".", value, fixed = TRUE)[[1]][1] != -1)) {
              stop("Colum names cannot contain the '.' symbol.")
            }

            sdf <- callJMethod(x@sdf, "toDF", as.list(value))
            dataFrame(sdf)
          })

#' coltypes
#'
#' Get column types of a SparkDataFrame
#'
#' @param x A SparkDataFrame
#' @return value A character vector with the column types of the given SparkDataFrame
#' @rdname coltypes
#' @name coltypes
#' @family SparkDataFrame functions
#' @export
#' @examples
#'\dontrun{
#' irisDF <- createDataFrame(sqlContext, iris)
#' coltypes(irisDF)
#'}
setMethod("coltypes",
          signature(x = "SparkDataFrame"),
          function(x) {
            # Get the data types of the SparkDataFrame by invoking dtypes() function
            types <- sapply(dtypes(x), function(x) {x[[2]]})

            # Map Spark data types into R's data types using DATA_TYPES environment
            rTypes <- sapply(types, USE.NAMES = F, FUN = function(x) {
              # Check for primitive types
              type <- PRIMITIVE_TYPES[[x]]

              if (is.null(type)) {
                # Check for complex types
                for (t in names(COMPLEX_TYPES)) {
                  if (substring(x, 1, nchar(t)) == t) {
                    type <- COMPLEX_TYPES[[t]]
                    break
                  }
                }

                if (is.null(type)) {
                  stop(paste("Unsupported data type: ", x))
                }
              }
              type
            })

            # Find which types don't have mapping to R
            naIndices <- which(is.na(rTypes))

            # Assign the original scala data types to the unmatched ones
            rTypes[naIndices] <- types[naIndices]

            rTypes
          })

#' coltypes
#'
#' Set the column types of a SparkDataFrame.
#'
#' @param x A SparkDataFrame
#' @param value A character vector with the target column types for the given
#'    SparkDataFrame. Column types can be one of integer, numeric/double, character, logical, or NA
#'    to keep that column as-is.
#' @rdname coltypes
#' @name coltypes<-
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' coltypes(df) <- c("character", "integer")
#' coltypes(df) <- c(NA, "numeric")
#'}
setMethod("coltypes<-",
          signature(x = "SparkDataFrame", value = "character"),
          function(x, value) {
            cols <- columns(x)
            ncols <- length(cols)
            if (length(value) == 0) {
              stop("Cannot set types of an empty SparkDataFrame with no Column")
            }
            if (length(value) != ncols) {
              stop("Length of type vector should match the number of columns for SparkDataFrame")
            }
            newCols <- lapply(seq_len(ncols), function(i) {
              col <- getColumn(x, cols[i])
              if (!is.na(value[i])) {
                stype <- rToSQLTypes[[value[i]]]
                if (is.null(stype)) {
                  stop("Only atomic type is supported for column types")
                }
                cast(col, stype)
              } else {
                col
              }
            })
            nx <- select(x, newCols)
            dataFrame(nx@sdf)
          })

#' Register Temporary Table
#'
#' Registers a SparkDataFrame as a Temporary Table in the SQLContext
#'
#' @param x A SparkDataFrame
#' @param tableName A character vector containing the name of the table
#'
#' @family SparkDataFrame functions
#' @rdname registerTempTable
#' @name registerTempTable
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' registerTempTable(df, "json_df")
#' new_df <- sql(sqlContext, "SELECT * FROM json_df")
#'}
setMethod("registerTempTable",
          signature(x = "SparkDataFrame", tableName = "character"),
          function(x, tableName) {
              invisible(callJMethod(x@sdf, "registerTempTable", tableName))
          })

#' insertInto
#'
#' Insert the contents of a SparkDataFrame into a table registered in the current SQL Context.
#'
#' @param x A SparkDataFrame
#' @param tableName A character vector containing the name of the table
#' @param overwrite A logical argument indicating whether or not to overwrite
#' the existing rows in the table.
#'
#' @family SparkDataFrame functions
#' @rdname insertInto
#' @name insertInto
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' df <- read.df(sqlContext, path, "parquet")
#' df2 <- read.df(sqlContext, path2, "parquet")
#' registerTempTable(df, "table1")
#' insertInto(df2, "table1", overwrite = TRUE)
#'}
setMethod("insertInto",
          signature(x = "SparkDataFrame", tableName = "character"),
          function(x, tableName, overwrite = FALSE) {
            jmode <- convertToJSaveMode(ifelse(overwrite, "overwrite", "append"))
            write <- callJMethod(x@sdf, "write")
            write <- callJMethod(write, "mode", jmode)
            callJMethod(write, "insertInto", tableName)
          })

#' Cache
#'
#' Persist with the default storage level (MEMORY_ONLY).
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname cache
#' @name cache
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' cache(df)
#'}
setMethod("cache",
          signature(x = "SparkDataFrame"),
          function(x) {
            cached <- callJMethod(x@sdf, "cache")
            x@env$isCached <- TRUE
            x
          })

#' Persist
#'
#' Persist this SparkDataFrame with the specified storage level. For details of the
#' supported storage levels, refer to
#' \url{http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence}.
#'
#' @param x The SparkDataFrame to persist
#'
#' @family SparkDataFrame functions
#' @rdname persist
#' @name persist
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' persist(df, "MEMORY_AND_DISK")
#'}
setMethod("persist",
          signature(x = "SparkDataFrame", newLevel = "character"),
          function(x, newLevel) {
            callJMethod(x@sdf, "persist", getStorageLevel(newLevel))
            x@env$isCached <- TRUE
            x
          })

#' Unpersist
#'
#' Mark this SparkDataFrame as non-persistent, and remove all blocks for it from memory and
#' disk.
#'
#' @param x The SparkDataFrame to unpersist
#' @param blocking Whether to block until all blocks are deleted
#'
#' @family SparkDataFrame functions
#' @rdname unpersist-methods
#' @name unpersist
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' persist(df, "MEMORY_AND_DISK")
#' unpersist(df)
#'}
setMethod("unpersist",
          signature(x = "SparkDataFrame"),
          function(x, blocking = TRUE) {
            callJMethod(x@sdf, "unpersist", blocking)
            x@env$isCached <- FALSE
            x
          })

#' Repartition
#'
#' Return a new SparkDataFrame that has exactly numPartitions partitions.
#'
#' @param x A SparkDataFrame
#' @param numPartitions The number of partitions to use.
#'
#' @family SparkDataFrame functions
#' @rdname repartition
#' @name repartition
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' newDF <- repartition(df, 2L)
#'}
setMethod("repartition",
          signature(x = "SparkDataFrame", numPartitions = "numeric"),
          function(x, numPartitions) {
            sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions))
            dataFrame(sdf)
          })

#' toJSON
#'
#' Convert the rows of a SparkDataFrame into JSON objects and return an RDD where
#' each element contains a JSON string.
#'
#' @param x A SparkDataFrame
#' @return A StringRRDD of JSON objects
#' @family SparkDataFrame functions
#' @rdname tojson
#' @noRd
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' newRDD <- toJSON(df)
#'}
setMethod("toJSON",
          signature(x = "SparkDataFrame"),
          function(x) {
            rdd <- callJMethod(x@sdf, "toJSON")
            jrdd <- callJMethod(rdd, "toJavaRDD")
            RDD(jrdd, serializedMode = "string")
          })

#' write.json
#'
#' Save the contents of a SparkDataFrame as a JSON file (one object per line). Files written out
#' with this method can be read back in as a SparkDataFrame using read.json().
#'
#' @param x A SparkDataFrame
#' @param path The directory where the file is saved
#'
#' @family SparkDataFrame functions
#' @rdname write.json
#' @name write.json
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' write.json(df, "/tmp/sparkr-tmp/")
#'}
setMethod("write.json",
          signature(x = "SparkDataFrame", path = "character"),
          function(x, path) {
            write <- callJMethod(x@sdf, "write")
            invisible(callJMethod(write, "json", path))
          })

#' write.parquet
#'
#' Save the contents of a SparkDataFrame as a Parquet file, preserving the schema. Files written out
#' with this method can be read back in as a SparkDataFrame using read.parquet().
#'
#' @param x A SparkDataFrame
#' @param path The directory where the file is saved
#'
#' @family SparkDataFrame functions
#' @rdname write.parquet
#' @name write.parquet
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' write.parquet(df, "/tmp/sparkr-tmp1/")
#' saveAsParquetFile(df, "/tmp/sparkr-tmp2/")
#'}
setMethod("write.parquet",
          signature(x = "SparkDataFrame", path = "character"),
          function(x, path) {
            write <- callJMethod(x@sdf, "write")
            invisible(callJMethod(write, "parquet", path))
          })

#' @rdname write.parquet
#' @name saveAsParquetFile
#' @export
setMethod("saveAsParquetFile",
          signature(x = "SparkDataFrame", path = "character"),
          function(x, path) {
            .Deprecated("write.parquet")
            write.parquet(x, path)
          })

#' write.text
#'
#' Saves the content of the SparkDataFrame in a text file at the specified path.
#' The SparkDataFrame must have only one column of string type with the name "value".
#' Each row becomes a new line in the output file.
#'
#' @param x A SparkDataFrame
#' @param path The directory where the file is saved
#'
#' @family SparkDataFrame functions
#' @rdname write.text
#' @name write.text
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.txt"
#' df <- read.text(sqlContext, path)
#' write.text(df, "/tmp/sparkr-tmp/")
#'}
setMethod("write.text",
          signature(x = "SparkDataFrame", path = "character"),
          function(x, path) {
            write <- callJMethod(x@sdf, "write")
            invisible(callJMethod(write, "text", path))
          })

#' Distinct
#'
#' Return a new SparkDataFrame containing the distinct rows in this SparkDataFrame.
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname distinct
#' @name distinct
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' distinctDF <- distinct(df)
#'}
setMethod("distinct",
          signature(x = "SparkDataFrame"),
          function(x) {
            sdf <- callJMethod(x@sdf, "distinct")
            dataFrame(sdf)
          })

#' @rdname distinct
#' @name unique
setMethod("unique",
          signature(x = "SparkDataFrame"),
          function(x) {
            distinct(x)
          })

#' Sample
#'
#' Return a sampled subset of this SparkDataFrame using a random seed.
#'
#' @param x A SparkDataFrame
#' @param withReplacement Sampling with replacement or not
#' @param fraction The (rough) sample target fraction
#' @param seed Randomness seed value
#'
#' @family SparkDataFrame functions
#' @rdname sample
#' @name sample
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' collect(sample(df, FALSE, 0.5))
#' collect(sample(df, TRUE, 0.5))
#'}
setMethod("sample",
          signature(x = "SparkDataFrame", withReplacement = "logical",
                    fraction = "numeric"),
          function(x, withReplacement, fraction, seed) {
            if (fraction < 0.0) stop(cat("Negative fraction value:", fraction))
            if (!missing(seed)) {
              # TODO : Figure out how to send integer as java.lang.Long to JVM so
              # we can send seed as an argument through callJMethod
              sdf <- callJMethod(x@sdf, "sample", withReplacement, fraction, as.integer(seed))
            } else {
              sdf <- callJMethod(x@sdf, "sample", withReplacement, fraction)
            }
            dataFrame(sdf)
          })

#' @rdname sample
#' @name sample_frac
setMethod("sample_frac",
          signature(x = "SparkDataFrame", withReplacement = "logical",
                    fraction = "numeric"),
          function(x, withReplacement, fraction, seed) {
            sample(x, withReplacement, fraction, seed)
          })

#' nrow
#'
#' Returns the number of rows in a SparkDataFrame
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname nrow
#' @name count
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' count(df)
#' }
setMethod("count",
          signature(x = "SparkDataFrame"),
          function(x) {
            callJMethod(x@sdf, "count")
          })

#' @name nrow
#' @rdname nrow
setMethod("nrow",
          signature(x = "SparkDataFrame"),
          function(x) {
            count(x)
          })

#' Returns the number of columns in a SparkDataFrame
#'
#' @param x a SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname ncol
#' @name ncol
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' ncol(df)
#' }
setMethod("ncol",
          signature(x = "SparkDataFrame"),
          function(x) {
            length(columns(x))
          })

#' Returns the dimentions (number of rows and columns) of a SparkDataFrame
#' @param x a SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname dim
#' @name dim
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' dim(df)
#' }
setMethod("dim",
          signature(x = "SparkDataFrame"),
          function(x) {
            c(count(x), ncol(x))
          })

#' Collects all the elements of a SparkDataFrame and coerces them into an R data.frame.
#'
#' @param x A SparkDataFrame
#' @param stringsAsFactors (Optional) A logical indicating whether or not string columns
#' should be converted to factors. FALSE by default.
#'
#' @family SparkDataFrame functions
#' @rdname collect
#' @name collect
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' collected <- collect(df)
#' firstName <- collected[[1]]$name
#' }
setMethod("collect",
          signature(x = "SparkDataFrame"),
          function(x, stringsAsFactors = FALSE) {
            dtypes <- dtypes(x)
            ncol <- length(dtypes)
            if (ncol <= 0) {
              # empty data.frame with 0 columns and 0 rows
              data.frame()
            } else {
              # listCols is a list of columns
              listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf)
              stopifnot(length(listCols) == ncol)

              # An empty data.frame with 0 columns and number of rows as collected
              nrow <- length(listCols[[1]])
              if (nrow <= 0) {
                df <- data.frame()
              } else {
                df <- data.frame(row.names = 1 : nrow)
              }

              # Append columns one by one
              for (colIndex in 1 : ncol) {
                # Note: appending a column of list type into a data.frame so that
                # data of complex type can be held. But getting a cell from a column
                # of list type returns a list instead of a vector. So for columns of
                # non-complex type, append them as vector.
                #
                # For columns of complex type, be careful to access them.
                # Get a column of complex type returns a list.
                # Get a cell from a column of complex type returns a list instead of a vector.
                col <- listCols[[colIndex]]
                if (length(col) <= 0) {
                  df[[colIndex]] <- col
                } else {
                  colType <- dtypes[[colIndex]][[2]]
                  # Note that "binary" columns behave like complex types.
                  if (!is.null(PRIMITIVE_TYPES[[colType]]) && colType != "binary") {
                    vec <- do.call(c, col)
                    stopifnot(class(vec) != "list")
                    df[[colIndex]] <- vec
                  } else {
                    df[[colIndex]] <- col
                  }
                }
              }
              names(df) <- names(x)
              df
            }
          })

#' Limit
#'
#' Limit the resulting SparkDataFrame to the number of rows specified.
#'
#' @param x A SparkDataFrame
#' @param num The number of rows to return
#' @return A new SparkDataFrame containing the number of rows specified.
#'
#' @family SparkDataFrame functions
#' @rdname limit
#' @name limit
#' @export
#' @examples
#' \dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' limitedDF <- limit(df, 10)
#' }
setMethod("limit",
          signature(x = "SparkDataFrame", num = "numeric"),
          function(x, num) {
            res <- callJMethod(x@sdf, "limit", as.integer(num))
            dataFrame(res)
          })

#' Take the first NUM rows of a SparkDataFrame and return a the results as a data.frame
#'
#' @family SparkDataFrame functions
#' @rdname take
#' @name take
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' take(df, 2)
#' }
setMethod("take",
          signature(x = "SparkDataFrame", num = "numeric"),
          function(x, num) {
            limited <- limit(x, num)
            collect(limited)
          })

#' Head
#'
#' Return the first NUM rows of a SparkDataFrame as a data.frame. If NUM is NULL,
#' then head() returns the first 6 rows in keeping with the current data.frame
#' convention in R.
#'
#' @param x A SparkDataFrame
#' @param num The number of rows to return. Default is 6.
#' @return A data.frame
#'
#' @family SparkDataFrame functions
#' @rdname head
#' @name head
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' head(df)
#' }
setMethod("head",
          signature(x = "SparkDataFrame"),
          function(x, num = 6L) {
          # Default num is 6L in keeping with R's data.frame convention
            take(x, num)
          })

#' Return the first row of a SparkDataFrame
#'
#' @param x A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname first
#' @name first
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' first(df)
#' }
setMethod("first",
          signature(x = "SparkDataFrame"),
          function(x) {
            take(x, 1)
          })

#' toRDD
#'
#' Converts a SparkDataFrame to an RDD while preserving column names.
#'
#' @param x A SparkDataFrame
#'
#' @noRd
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' rdd <- toRDD(df)
#'}
setMethod("toRDD",
          signature(x = "SparkDataFrame"),
          function(x) {
            jrdd <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToRowRDD", x@sdf)
            colNames <- callJMethod(x@sdf, "columns")
            rdd <- RDD(jrdd, serializedMode = "row")
            lapply(rdd, function(row) {
              names(row) <- colNames
              row
            })
          })

#' GroupBy
#'
#' Groups the SparkDataFrame using the specified columns, so we can run aggregation on them.
#'
#' @param x a SparkDataFrame
#' @return a GroupedData
#' @seealso GroupedData
#' @family SparkDataFrame functions
#' @rdname groupBy
#' @name groupBy
#' @export
#' @examples
#' \dontrun{
#'   # Compute the average for all numeric columns grouped by department.
#'   avg(groupBy(df, "department"))
#'
#'   # Compute the max age and average salary, grouped by department and gender.
#'   agg(groupBy(df, "department", "gender"), salary="avg", "age" -> "max")
#' }
setMethod("groupBy",
           signature(x = "SparkDataFrame"),
           function(x, ...) {
             cols <- list(...)
             if (length(cols) >= 1 && class(cols[[1]]) == "character") {
               sgd <- callJMethod(x@sdf, "groupBy", cols[[1]], cols[-1])
             } else {
               jcol <- lapply(cols, function(c) { c@jc })
               sgd <- callJMethod(x@sdf, "groupBy", jcol)
             }
             groupedData(sgd)
           })

#' @rdname groupBy
#' @name group_by
setMethod("group_by",
          signature(x = "SparkDataFrame"),
          function(x, ...) {
            groupBy(x, ...)
          })

#' Summarize data across columns
#'
#' Compute aggregates by specifying a list of columns
#'
#' @param x a SparkDataFrame
#' @family SparkDataFrame functions
#' @rdname agg
#' @name agg
#' @export
setMethod("agg",
          signature(x = "SparkDataFrame"),
          function(x, ...) {
            agg(groupBy(x), ...)
          })

#' @rdname agg
#' @name summarize
setMethod("summarize",
          signature(x = "SparkDataFrame"),
          function(x, ...) {
            agg(x, ...)
          })


############################## RDD Map Functions ##################################
# All of the following functions mirror the existing RDD map functions,           #
# but allow for use with DataFrames by first converting to an RRDD before calling #
# the requested map function.                                                     #
###################################################################################

#' @rdname lapply
#' @noRd
setMethod("lapply",
          signature(X = "SparkDataFrame", FUN = "function"),
          function(X, FUN) {
            rdd <- toRDD(X)
            lapply(rdd, FUN)
          })

#' @rdname lapply
#' @noRd
setMethod("map",
          signature(X = "SparkDataFrame", FUN = "function"),
          function(X, FUN) {
            lapply(X, FUN)
          })

#' @rdname flatMap
#' @noRd
setMethod("flatMap",
          signature(X = "SparkDataFrame", FUN = "function"),
          function(X, FUN) {
            rdd <- toRDD(X)
            flatMap(rdd, FUN)
          })

#' @rdname lapplyPartition
#' @noRd
setMethod("lapplyPartition",
          signature(X = "SparkDataFrame", FUN = "function"),
          function(X, FUN) {
            rdd <- toRDD(X)
            lapplyPartition(rdd, FUN)
          })

#' @rdname lapplyPartition
#' @noRd
setMethod("mapPartitions",
          signature(X = "SparkDataFrame", FUN = "function"),
          function(X, FUN) {
            lapplyPartition(X, FUN)
          })

#' @rdname foreach
#' @noRd
setMethod("foreach",
          signature(x = "SparkDataFrame", func = "function"),
          function(x, func) {
            rdd <- toRDD(x)
            foreach(rdd, func)
          })

#' @rdname foreach
#' @noRd
setMethod("foreachPartition",
          signature(x = "SparkDataFrame", func = "function"),
          function(x, func) {
            rdd <- toRDD(x)
            foreachPartition(rdd, func)
          })


############################## SELECT ##################################

getColumn <- function(x, c) {
  column(callJMethod(x@sdf, "col", c))
}

#' @rdname select
#' @name $
setMethod("$", signature(x = "SparkDataFrame"),
          function(x, name) {
            getColumn(x, name)
          })

#' @rdname select
#' @name $<-
setMethod("$<-", signature(x = "SparkDataFrame"),
          function(x, name, value) {
            stopifnot(class(value) == "Column" || is.null(value))

            if (is.null(value)) {
              nx <- drop(x, name)
            } else {
              nx <- withColumn(x, name, value)
            }
            x@sdf <- nx@sdf
            x
          })

setClassUnion("numericOrcharacter", c("numeric", "character"))

#' @rdname subset
#' @name [[
setMethod("[[", signature(x = "SparkDataFrame", i = "numericOrcharacter"),
          function(x, i) {
            if (is.numeric(i)) {
              cols <- columns(x)
              i <- cols[[i]]
            }
            getColumn(x, i)
          })

#' @rdname subset
#' @name [
setMethod("[", signature(x = "SparkDataFrame", i = "missing"),
          function(x, i, j, ...) {
            if (is.numeric(j)) {
              cols <- columns(x)
              j <- cols[j]
            }
            if (length(j) > 1) {
              j <- as.list(j)
            }
            select(x, j)
          })

#' @rdname subset
#' @name [
setMethod("[", signature(x = "SparkDataFrame", i = "Column"),
          function(x, i, j, ...) {
            # It could handle i as "character" but it seems confusing and not required
            # https://stat.ethz.ch/R-manual/R-devel/library/base/html/Extract.data.frame.html
            filtered <- filter(x, i)
            if (!missing(j)) {
              filtered[, j, ...]
            } else {
              filtered
            }
          })

#' Subset
#'
#' Return subsets of SparkDataFrame according to given conditions
#' @param x A SparkDataFrame
#' @param subset (Optional) A logical expression to filter on rows
#' @param select expression for the single Column or a list of columns to select from the
#' SparkDataFrame
#' @return A new SparkDataFrame containing only the rows that meet the condition with selected
#' columns
#' @export
#' @family SparkDataFrame functions
#' @rdname subset
#' @name subset
#' @family subsetting functions
#' @examples
#' \dontrun{
#'   # Columns can be selected using `[[` and `[`
#'   df[[2]] == df[["age"]]
#'   df[,2] == df[,"age"]
#'   df[,c("name", "age")]
#'   # Or to filter rows
#'   df[df$age > 20,]
#'   # SparkDataFrame can be subset on both rows and Columns
#'   df[df$name == "Smith", c(1,2)]
#'   df[df$age %in% c(19, 30), 1:2]
#'   subset(df, df$age %in% c(19, 30), 1:2)
#'   subset(df, df$age %in% c(19), select = c(1,2))
#'   subset(df, select = c(1,2))
#' }
setMethod("subset", signature(x = "SparkDataFrame"),
          function(x, subset, select, ...) {
            if (missing(subset)) {
              x[, select, ...]
            } else {
              x[subset, select, ...]
            }
          })

#' Select
#'
#' Selects a set of columns with names or Column expressions.
#' @param x A SparkDataFrame
#' @param col A list of columns or single Column or name
#' @return A new SparkDataFrame with selected columns
#' @export
#' @family SparkDataFrame functions
#' @rdname select
#' @name select
#' @family subsetting functions
#' @examples
#' \dontrun{
#'   select(df, "*")
#'   select(df, "col1", "col2")
#'   select(df, df$name, df$age + 1)
#'   select(df, c("col1", "col2"))
#'   select(df, list(df$name, df$age + 1))
#'   # Similar to R data frames columns can also be selected using `$`
#'   df[,df$age]
#' }
setMethod("select", signature(x = "SparkDataFrame", col = "character"),
          function(x, col, ...) {
            if (length(col) > 1) {
              if (length(list(...)) > 0) {
                stop("To select multiple columns, use a character vector or list for col")
              }

              select(x, as.list(col))
            } else {
              sdf <- callJMethod(x@sdf, "select", col, list(...))
              dataFrame(sdf)
            }
          })

#' @family SparkDataFrame functions
#' @rdname select
#' @export
setMethod("select", signature(x = "SparkDataFrame", col = "Column"),
          function(x, col, ...) {
            jcols <- lapply(list(col, ...), function(c) {
              c@jc
            })
            sdf <- callJMethod(x@sdf, "select", jcols)
            dataFrame(sdf)
          })

#' @family SparkDataFrame functions
#' @rdname select
#' @export
setMethod("select",
          signature(x = "SparkDataFrame", col = "list"),
          function(x, col) {
            cols <- lapply(col, function(c) {
              if (class(c) == "Column") {
                c@jc
              } else {
                col(c)@jc
              }
            })
            sdf <- callJMethod(x@sdf, "select", cols)
            dataFrame(sdf)
          })

#' SelectExpr
#'
#' Select from a SparkDataFrame using a set of SQL expressions.
#'
#' @param x A SparkDataFrame to be selected from.
#' @param expr A string containing a SQL expression
#' @param ... Additional expressions
#' @return A SparkDataFrame
#' @family SparkDataFrame functions
#' @rdname selectExpr
#' @name selectExpr
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' selectExpr(df, "col1", "(col2 * 5) as newCol")
#' }
setMethod("selectExpr",
          signature(x = "SparkDataFrame", expr = "character"),
          function(x, expr, ...) {
            exprList <- list(expr, ...)
            sdf <- callJMethod(x@sdf, "selectExpr", exprList)
            dataFrame(sdf)
          })

#' WithColumn
#'
#' Return a new SparkDataFrame by adding a column or replacing the existing column
#' that has the same name.
#'
#' @param x A SparkDataFrame
#' @param colName A column name.
#' @param col A Column expression.
#' @return A SparkDataFrame with the new column added or the existing column replaced.
#' @family SparkDataFrame functions
#' @rdname withColumn
#' @name withColumn
#' @seealso \link{rename} \link{mutate}
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' newDF <- withColumn(df, "newCol", df$col1 * 5)
#' # Replace an existing column
#' newDF2 <- withColumn(newDF, "newCol", newDF$col1)
#' }
setMethod("withColumn",
          signature(x = "SparkDataFrame", colName = "character", col = "Column"),
          function(x, colName, col) {
            sdf <- callJMethod(x@sdf, "withColumn", colName, col@jc)
            dataFrame(sdf)
          })

#' Mutate
#'
#' Return a new SparkDataFrame with the specified columns added.
#'
#' @param .data A SparkDataFrame
#' @param col a named argument of the form name = col
#' @return A new SparkDataFrame with the new columns added.
#' @family SparkDataFrame functions
#' @rdname mutate
#' @name mutate
#' @seealso \link{rename} \link{withColumn}
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' newDF <- mutate(df, newCol = df$col1 * 5, newCol2 = df$col1 * 2)
#' names(newDF) # Will contain newCol, newCol2
#' newDF2 <- transform(df, newCol = df$col1 / 5, newCol2 = df$col1 * 2)
#' }
setMethod("mutate",
          signature(.data = "SparkDataFrame"),
          function(.data, ...) {
            x <- .data
            cols <- list(...)
            stopifnot(length(cols) > 0)
            stopifnot(class(cols[[1]]) == "Column")
            ns <- names(cols)
            if (!is.null(ns)) {
              for (n in ns) {
                if (n != "") {
                  cols[[n]] <- alias(cols[[n]], n)
                }
              }
            }
            do.call(select, c(x, x$"*", cols))
          })

#' @export
#' @rdname mutate
#' @name transform
setMethod("transform",
          signature(`_data` = "SparkDataFrame"),
          function(`_data`, ...) {
            mutate(`_data`, ...)
          })

#' rename
#'
#' Rename an existing column in a SparkDataFrame.
#'
#' @param x A SparkDataFrame
#' @param existingCol The name of the column you want to change.
#' @param newCol The new column name.
#' @return A SparkDataFrame with the column name changed.
#' @family SparkDataFrame functions
#' @rdname rename
#' @name withColumnRenamed
#' @seealso \link{mutate}
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' newDF <- withColumnRenamed(df, "col1", "newCol1")
#' }
setMethod("withColumnRenamed",
          signature(x = "SparkDataFrame", existingCol = "character", newCol = "character"),
          function(x, existingCol, newCol) {
            cols <- lapply(columns(x), function(c) {
              if (c == existingCol) {
                alias(col(c), newCol)
              } else {
                col(c)
              }
            })
            select(x, cols)
          })

#' @param newColPair A named pair of the form new_column_name = existing_column
#' @rdname rename
#' @name rename
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' newDF <- rename(df, col1 = df$newCol1)
#' }
setMethod("rename",
          signature(x = "SparkDataFrame"),
          function(x, ...) {
            renameCols <- list(...)
            stopifnot(length(renameCols) > 0)
            stopifnot(class(renameCols[[1]]) == "Column")
            newNames <- names(renameCols)
            oldNames <- lapply(renameCols, function(col) {
              callJMethod(col@jc, "toString")
            })
            cols <- lapply(columns(x), function(c) {
              if (c %in% oldNames) {
                alias(col(c), newNames[[match(c, oldNames)]])
              } else {
                col(c)
              }
            })
            select(x, cols)
          })

setClassUnion("characterOrColumn", c("character", "Column"))

#' Arrange
#'
#' Sort a SparkDataFrame by the specified column(s).
#'
#' @param x A SparkDataFrame to be sorted.
#' @param col A character or Column object vector indicating the fields to sort on
#' @param ... Additional sorting fields
#' @param decreasing A logical argument indicating sorting order for columns when
#'                   a character vector is specified for col
#' @return A SparkDataFrame where all elements are sorted.
#' @family SparkDataFrame functions
#' @rdname arrange
#' @name arrange
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' arrange(df, df$col1)
#' arrange(df, asc(df$col1), desc(abs(df$col2)))
#' arrange(df, "col1", decreasing = TRUE)
#' arrange(df, "col1", "col2", decreasing = c(TRUE, FALSE))
#' }
setMethod("arrange",
          signature(x = "SparkDataFrame", col = "Column"),
          function(x, col, ...) {
              jcols <- lapply(list(col, ...), function(c) {
                c@jc
              })

            sdf <- callJMethod(x@sdf, "sort", jcols)
            dataFrame(sdf)
          })

#' @rdname arrange
#' @name arrange
#' @export
setMethod("arrange",
          signature(x = "SparkDataFrame", col = "character"),
          function(x, col, ..., decreasing = FALSE) {

            # all sorting columns
            by <- list(col, ...)

            if (length(decreasing) == 1) {
              # in case only 1 boolean argument - decreasing value is specified,
              # it will be used for all columns
              decreasing <- rep(decreasing, length(by))
            } else if (length(decreasing) != length(by)) {
              stop("Arguments 'col' and 'decreasing' must have the same length")
            }

            # builds a list of columns of type Column
            # example: [[1]] Column Species ASC
            #          [[2]] Column Petal_Length DESC
            jcols <- lapply(seq_len(length(decreasing)), function(i){
              if (decreasing[[i]]) {
                desc(getColumn(x, by[[i]]))
              } else {
                asc(getColumn(x, by[[i]]))
              }
            })

            do.call("arrange", c(x, jcols))
          })

#' @rdname arrange
#' @name orderBy
#' @export
setMethod("orderBy",
          signature(x = "SparkDataFrame", col = "characterOrColumn"),
          function(x, col) {
            arrange(x, col)
          })

#' Filter
#'
#' Filter the rows of a SparkDataFrame according to a given condition.
#'
#' @param x A SparkDataFrame to be sorted.
#' @param condition The condition to filter on. This may either be a Column expression
#' or a string containing a SQL statement
#' @return A SparkDataFrame containing only the rows that meet the condition.
#' @family SparkDataFrame functions
#' @rdname filter
#' @name filter
#' @family subsetting functions
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' filter(df, "col1 > 0")
#' filter(df, df$col2 != "abcdefg")
#' }
setMethod("filter",
          signature(x = "SparkDataFrame", condition = "characterOrColumn"),
          function(x, condition) {
            if (class(condition) == "Column") {
              condition <- condition@jc
            }
            sdf <- callJMethod(x@sdf, "filter", condition)
            dataFrame(sdf)
          })

#' @family SparkDataFrame functions
#' @rdname filter
#' @name where
setMethod("where",
          signature(x = "SparkDataFrame", condition = "characterOrColumn"),
          function(x, condition) {
            filter(x, condition)
          })

#' dropDuplicates
#'
#' Returns a new SparkDataFrame with duplicate rows removed, considering only
#' the subset of columns.
#'
#' @param x A SparkDataFrame.
#' @param colnames A character vector of column names.
#' @return A SparkDataFrame with duplicate rows removed.
#' @family SparkDataFrame functions
#' @rdname dropduplicates
#' @name dropDuplicates
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' dropDuplicates(df)
#' dropDuplicates(df, c("col1", "col2"))
#' }
setMethod("dropDuplicates",
          signature(x = "SparkDataFrame"),
          function(x, colNames = columns(x)) {
            stopifnot(class(colNames) == "character")

            sdf <- callJMethod(x@sdf, "dropDuplicates", as.list(colNames))
            dataFrame(sdf)
          })

#' Join
#'
#' Join two SparkDataFrames based on the given join expression.
#'
#' @param x A SparkDataFrame
#' @param y A SparkDataFrame
#' @param joinExpr (Optional) The expression used to perform the join. joinExpr must be a
#' Column expression. If joinExpr is omitted, join() will perform a Cartesian join
#' @param joinType The type of join to perform. The following join types are available:
#' 'inner', 'outer', 'full', 'fullouter', leftouter', 'left_outer', 'left',
#' 'right_outer', 'rightouter', 'right', and 'leftsemi'. The default joinType is "inner".
#' @return A SparkDataFrame containing the result of the join operation.
#' @family SparkDataFrame functions
#' @rdname join
#' @name join
#' @seealso \link{merge}
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' df1 <- read.json(sqlContext, path)
#' df2 <- read.json(sqlContext, path2)
#' join(df1, df2) # Performs a Cartesian
#' join(df1, df2, df1$col1 == df2$col2) # Performs an inner join based on expression
#' join(df1, df2, df1$col1 == df2$col2, "right_outer")
#' }
setMethod("join",
          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
          function(x, y, joinExpr = NULL, joinType = NULL) {
            if (is.null(joinExpr)) {
              sdf <- callJMethod(x@sdf, "join", y@sdf)
            } else {
              if (class(joinExpr) != "Column") stop("joinExpr must be a Column")
              if (is.null(joinType)) {
                sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc)
              } else {
                if (joinType %in% c("inner", "outer", "full", "fullouter",
                    "leftouter", "left_outer", "left",
                    "rightouter", "right_outer", "right", "leftsemi")) {
                  joinType <- gsub("_", "", joinType)
                  sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType)
                } else {
                  stop("joinType must be one of the following types: ",
                      "'inner', 'outer', 'full', 'fullouter', 'leftouter', 'left_outer', 'left',
                      'rightouter', 'right_outer', 'right', 'leftsemi'")
                }
              }
            }
            dataFrame(sdf)
          })

#' @name merge
#' @title Merges two data frames
#' @param x the first data frame to be joined
#' @param y the second data frame to be joined
#' @param by a character vector specifying the join columns. If by is not
#'   specified, the common column names in \code{x} and \code{y} will be used.
#' @param by.x a character vector specifying the joining columns for x.
#' @param by.y a character vector specifying the joining columns for y.
#' @param all.x a boolean value indicating whether all the rows in x should
#'              be including in the join
#' @param all.y a boolean value indicating whether all the rows in y should
#'              be including in the join
#' @param sort a logical argument indicating whether the resulting columns should be sorted
#' @details  If all.x and all.y are set to FALSE, a natural join will be returned. If
#'   all.x is set to TRUE and all.y is set to FALSE, a left outer join will
#'   be returned. If all.x is set to FALSE and all.y is set to TRUE, a right
#'   outer join will be returned. If all.x and all.y are set to TRUE, a full
#'   outer join will be returned.
#' @family SparkDataFrame functions
#' @rdname merge
#' @seealso \link{join}
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' df1 <- read.json(sqlContext, path)
#' df2 <- read.json(sqlContext, path2)
#' merge(df1, df2) # Performs a Cartesian
#' merge(df1, df2, by = "col1") # Performs an inner join based on expression
#' merge(df1, df2, by.x = "col1", by.y = "col2", all.y = TRUE)
#' merge(df1, df2, by.x = "col1", by.y = "col2", all.x = TRUE)
#' merge(df1, df2, by.x = "col1", by.y = "col2", all.x = TRUE, all.y = TRUE)
#' merge(df1, df2, by.x = "col1", by.y = "col2", all = TRUE, sort = FALSE)
#' merge(df1, df2, by = "col1", all = TRUE, suffixes = c("-X", "-Y"))
#' }
setMethod("merge",
          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
          function(x, y, by = intersect(names(x), names(y)), by.x = by, by.y = by,
                   all = FALSE, all.x = all, all.y = all,
                   sort = TRUE, suffixes = c("_x", "_y"), ... ) {

            if (length(suffixes) != 2) {
              stop("suffixes must have length 2")
            }

            # join type is identified based on the values of all, all.x and all.y
            # default join type is inner, according to R it should be natural but since it
            # is not supported in spark inner join is used
            joinType <- "inner"
            if (all || (all.x && all.y)) {
              joinType <- "outer"
            } else if (all.x) {
              joinType <- "left_outer"
            } else if (all.y) {
              joinType <- "right_outer"
            }

            # join expression is based on by.x, by.y if both by.x and by.y are not missing
            # or on by, if by.x or by.y are missing or have different lengths
            if (length(by.x) > 0 && length(by.x) == length(by.y)) {
              joinX <- by.x
              joinY <- by.y
            } else if (length(by) > 0) {
              # if join columns have the same name for both dataframes,
              # they are used in join expression
              joinX <- by
              joinY <- by
            } else {
              # if by or both by.x and by.y have length 0, use Cartesian Product
              joinRes <- join(x, y)
              return (joinRes)
            }

            # sets alias for making colnames unique in dataframes 'x' and 'y'
            colsX <- generateAliasesForIntersectedCols(x, by, suffixes[1])
            colsY <- generateAliasesForIntersectedCols(y, by, suffixes[2])

            # selects columns with their aliases from dataframes
            # in case same column names are present in both data frames
            xsel <- select(x, colsX)
            ysel <- select(y, colsY)

            # generates join conditions and adds them into a list
            # it also considers alias names of the columns while generating join conditions
            joinColumns <- lapply(seq_len(length(joinX)), function(i) {
              colX <- joinX[[i]]
              colY <- joinY[[i]]

              if (colX %in% by) {
                colX <- paste(colX, suffixes[1], sep = "")
              }
              if (colY %in% by) {
                colY <- paste(colY, suffixes[2], sep = "")
              }

              colX <- getColumn(xsel, colX)
              colY <- getColumn(ysel, colY)

              colX == colY
            })

            # concatenates join columns with '&' and executes join
            joinExpr <- Reduce("&", joinColumns)
            joinRes <- join(xsel, ysel, joinExpr, joinType)

            # sorts the result by 'by' columns if sort = TRUE
            if (sort && length(by) > 0) {
              colNameWithSuffix <- paste(by, suffixes[2], sep = "")
              joinRes <- do.call("arrange", c(joinRes, colNameWithSuffix, decreasing = FALSE))
            }

            joinRes
          })

#'
#' Creates a list of columns by replacing the intersected ones with aliases.
#' The name of the alias column is formed by concatanating the original column name and a suffix.
#'
#' @param x a SparkDataFrame on which the
#' @param intersectedColNames a list of intersected column names
#' @param suffix a suffix for the column name
#' @return list of columns
#'
generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) {
  allColNames <- names(x)
  # sets alias for making colnames unique in dataframe 'x'
  cols <- lapply(allColNames, function(colName) {
    col <- getColumn(x, colName)
    if (colName %in% intersectedColNames) {
      newJoin <- paste(colName, suffix, sep = "")
      if (newJoin %in% allColNames){
        stop ("The following column name: ", newJoin, " occurs more than once in the 'DataFrame'.",
          "Please use different suffixes for the intersected columns.")
      }
      col <- alias(col, newJoin)
    }
    col
  })
  cols
}

#' rbind
#'
#' Return a new SparkDataFrame containing the union of rows in this SparkDataFrame
#' and another SparkDataFrame. This is equivalent to `UNION ALL` in SQL.
#' Note that this does not remove duplicate rows across the two SparkDataFrames.
#'
#' @param x A SparkDataFrame
#' @param y A SparkDataFrame
#' @return A SparkDataFrame containing the result of the union.
#' @family SparkDataFrame functions
#' @rdname rbind
#' @name unionAll
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' df1 <- read.json(sqlContext, path)
#' df2 <- read.json(sqlContext, path2)
#' unioned <- unionAll(df, df2)
#' }
setMethod("unionAll",
          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
          function(x, y) {
            unioned <- callJMethod(x@sdf, "unionAll", y@sdf)
            dataFrame(unioned)
          })

#' @title Union two or more SparkDataFrames
#' @description Returns a new SparkDataFrame containing rows of all parameters.
#'
#' @rdname rbind
#' @name rbind
#' @export
setMethod("rbind",
          signature(... = "SparkDataFrame"),
          function(x, ..., deparse.level = 1) {
            if (nargs() == 3) {
              unionAll(x, ...)
            } else {
              unionAll(x, Recall(..., deparse.level = 1))
            }
          })

#' Intersect
#'
#' Return a new SparkDataFrame containing rows only in both this SparkDataFrame
#' and another SparkDataFrame. This is equivalent to `INTERSECT` in SQL.
#'
#' @param x A SparkDataFrame
#' @param y A SparkDataFrame
#' @return A SparkDataFrame containing the result of the intersect.
#' @family SparkDataFrame functions
#' @rdname intersect
#' @name intersect
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' df1 <- read.json(sqlContext, path)
#' df2 <- read.json(sqlContext, path2)
#' intersectDF <- intersect(df, df2)
#' }
setMethod("intersect",
          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
          function(x, y) {
            intersected <- callJMethod(x@sdf, "intersect", y@sdf)
            dataFrame(intersected)
          })

#' except
#'
#' Return a new SparkDataFrame containing rows in this SparkDataFrame
#' but not in another SparkDataFrame. This is equivalent to `EXCEPT` in SQL.
#'
#' @param x A SparkDataFrame
#' @param y A SparkDataFrame
#' @return A SparkDataFrame containing the result of the except operation.
#' @family SparkDataFrame functions
#' @rdname except
#' @name except
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' df1 <- read.json(sqlContext, path)
#' df2 <- read.json(sqlContext, path2)
#' exceptDF <- except(df, df2)
#' }
#' @rdname except
#' @export
setMethod("except",
          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
          function(x, y) {
            excepted <- callJMethod(x@sdf, "except", y@sdf)
            dataFrame(excepted)
          })

#' Save the contents of the SparkDataFrame to a data source
#'
#' The data source is specified by the `source` and a set of options (...).
#' If `source` is not specified, the default data source configured by
#' spark.sql.sources.default will be used.
#'
#' Additionally, mode is used to specify the behavior of the save operation when
#' data already exists in the data source. There are four modes: \cr
#'  append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr
#'  overwrite: Existing data is expected to be overwritten by the contents of this
#'     SparkDataFrame. \cr
#'  error: An exception is expected to be thrown. \cr
#'  ignore: The save operation is expected to not save the contents of the SparkDataFrame
#'     and to not change the existing data. \cr
#'
#' @param df A SparkDataFrame
#' @param path A name for the table
#' @param source A name for external data source
#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default)
#'
#' @family SparkDataFrame functions
#' @rdname write.df
#' @name write.df
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' write.df(df, "myfile", "parquet", "overwrite")
#' saveDF(df, parquetPath2, "parquet", mode = saveMode, mergeSchema = mergeSchema)
#' }
setMethod("write.df",
          signature(df = "SparkDataFrame", path = "character"),
          function(df, path, source = NULL, mode = "error", ...){
            if (is.null(source)) {
              if (exists(".sparkRSQLsc", envir = .sparkREnv)) {
                sqlContext <- get(".sparkRSQLsc", envir = .sparkREnv)
              } else if (exists(".sparkRHivesc", envir = .sparkREnv)) {
                sqlContext <- get(".sparkRHivesc", envir = .sparkREnv)
              } else {
                stop("sparkRHive or sparkRSQL context has to be specified")
              }
              source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default",
                                    "org.apache.spark.sql.parquet")
            }
            jmode <- convertToJSaveMode(mode)
            options <- varargsToEnv(...)
            if (!is.null(path)) {
                options[["path"]] <- path
            }
            write <- callJMethod(df@sdf, "write")
            write <- callJMethod(write, "format", source)
            write <- callJMethod(write, "mode", jmode)
            write <- callJMethod(write, "save", path)
          })

#' @rdname write.df
#' @name saveDF
#' @export
setMethod("saveDF",
          signature(df = "SparkDataFrame", path = "character"),
          function(df, path, source = NULL, mode = "error", ...){
            write.df(df, path, source, mode, ...)
          })

#' saveAsTable
#'
#' Save the contents of the SparkDataFrame to a data source as a table
#'
#' The data source is specified by the `source` and a set of options (...).
#' If `source` is not specified, the default data source configured by
#' spark.sql.sources.default will be used.
#'
#' Additionally, mode is used to specify the behavior of the save operation when
#' data already exists in the data source. There are four modes: \cr
#'  append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr
#'  overwrite: Existing data is expected to be overwritten by the contents of this
#'     SparkDataFrame. \cr
#'  error: An exception is expected to be thrown. \cr
#'  ignore: The save operation is expected to not save the contents of the SparkDataFrame
#'     and to not change the existing data. \cr
#'
#' @param df A SparkDataFrame
#' @param tableName A name for the table
#' @param source A name for external data source
#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default)
#'
#' @family SparkDataFrame functions
#' @rdname saveAsTable
#' @name saveAsTable
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' saveAsTable(df, "myfile")
#' }
setMethod("saveAsTable",
          signature(df = "SparkDataFrame", tableName = "character"),
          function(df, tableName, source = NULL, mode="error", ...){
            if (is.null(source)) {
              if (exists(".sparkRSQLsc", envir = .sparkREnv)) {
                sqlContext <- get(".sparkRSQLsc", envir = .sparkREnv)
              } else if (exists(".sparkRHivesc", envir = .sparkREnv)) {
                sqlContext <- get(".sparkRHivesc", envir = .sparkREnv)
              } else {
                stop("sparkRHive or sparkRSQL context has to be specified")
              }
               source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default",
                                     "org.apache.spark.sql.parquet")
            }
            jmode <- convertToJSaveMode(mode)
            options <- varargsToEnv(...)

            write <- callJMethod(df@sdf, "write")
            write <- callJMethod(write, "format", source)
            write <- callJMethod(write, "mode", jmode)
            write <- callJMethod(write, "options", options)
            callJMethod(write, "saveAsTable", tableName)
          })

#' summary
#'
#' Computes statistics for numeric columns.
#' If no columns are given, this function computes statistics for all numerical columns.
#'
#' @param x A SparkDataFrame to be computed.
#' @param col A string of name
#' @param ... Additional expressions
#' @return A SparkDataFrame
#' @family SparkDataFrame functions
#' @rdname summary
#' @name describe
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlContext, path)
#' describe(df)
#' describe(df, "col1")
#' describe(df, "col1", "col2")
#' }
setMethod("describe",
          signature(x = "SparkDataFrame", col = "character"),
          function(x, col, ...) {
            colList <- list(col, ...)
            sdf <- callJMethod(x@sdf, "describe", colList)
            dataFrame(sdf)
          })

#' @rdname summary
#' @name describe
setMethod("describe",
          signature(x = "SparkDataFrame"),
          function(x) {
            colList <- as.list(c(columns(x)))
            sdf <- callJMethod(x@sdf, "describe", colList)
            dataFrame(sdf)
          })

#' @rdname summary
#' @name summary
setMethod("summary",
          signature(object = "SparkDataFrame"),
          function(object, ...) {
            describe(object)
          })


#' dropna
#'
#' Returns a new SparkDataFrame omitting rows with null values.
#'
#' @param x A SparkDataFrame.
#' @param how "any" or "all".
#'            if "any", drop a row if it contains any nulls.
#'            if "all", drop a row only if all its values are null.
#'            if minNonNulls is specified, how is ignored.
#' @param minNonNulls If specified, drop rows that have less than
#'                    minNonNulls non-null values.
#'                    This overwrites the how parameter.
#' @param cols Optional list of column names to consider.
#' @return A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname nafunctions
#' @name dropna
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlCtx, path)
#' dropna(df)
#' }
setMethod("dropna",
          signature(x = "SparkDataFrame"),
          function(x, how = c("any", "all"), minNonNulls = NULL, cols = NULL) {
            how <- match.arg(how)
            if (is.null(cols)) {
              cols <- columns(x)
            }
            if (is.null(minNonNulls)) {
              minNonNulls <- if (how == "any") { length(cols) } else { 1 }
            }

            naFunctions <- callJMethod(x@sdf, "na")
            sdf <- callJMethod(naFunctions, "drop",
                               as.integer(minNonNulls), as.list(cols))
            dataFrame(sdf)
          })

#' @rdname nafunctions
#' @name na.omit
#' @export
setMethod("na.omit",
          signature(object = "SparkDataFrame"),
          function(object, how = c("any", "all"), minNonNulls = NULL, cols = NULL) {
            dropna(object, how, minNonNulls, cols)
          })

#' fillna
#'
#' Replace null values.
#'
#' @param x A SparkDataFrame.
#' @param value Value to replace null values with.
#'              Should be an integer, numeric, character or named list.
#'              If the value is a named list, then cols is ignored and
#'              value must be a mapping from column name (character) to
#'              replacement value. The replacement value must be an
#'              integer, numeric or character.
#' @param cols optional list of column names to consider.
#'             Columns specified in cols that do not have matching data
#'             type are ignored. For example, if value is a character, and
#'             subset contains a non-character column, then the non-character
#'             column is simply ignored.
#'
#' @rdname nafunctions
#' @name fillna
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlCtx, path)
#' fillna(df, 1)
#' fillna(df, list("age" = 20, "name" = "unknown"))
#' }
setMethod("fillna",
          signature(x = "SparkDataFrame"),
          function(x, value, cols = NULL) {
            if (!(class(value) %in% c("integer", "numeric", "character", "list"))) {
              stop("value should be an integer, numeric, charactor or named list.")
            }

            if (class(value) == "list") {
              # Check column names in the named list
              colNames <- names(value)
              if (length(colNames) == 0 || !all(colNames != "")) {
                stop("value should be an a named list with each name being a column name.")
              }
              # Check each item in the named list is of valid type
              lapply(value, function(v) {
                if (!(class(v) %in% c("integer", "numeric", "character"))) {
                  stop("Each item in value should be an integer, numeric or charactor.")
                }
              })

              # Convert to the named list to an environment to be passed to JVM
              valueMap <- convertNamedListToEnv(value)

              # When value is a named list, caller is expected not to pass in cols
              if (!is.null(cols)) {
                warning("When value is a named list, cols is ignored!")
                cols <- NULL
              }

              value <- valueMap
            } else if (is.integer(value)) {
              # Cast an integer to a numeric
              value <- as.numeric(value)
            }

            naFunctions <- callJMethod(x@sdf, "na")
            sdf <- if (length(cols) == 0) {
              callJMethod(naFunctions, "fill", value)
            } else {
              callJMethod(naFunctions, "fill", value, as.list(cols))
            }
            dataFrame(sdf)
          })

#' This function downloads the contents of a SparkDataFrame into an R's data.frame.
#' Since data.frames are held in memory, ensure that you have enough memory
#' in your system to accommodate the contents.
#'
#' @title Download data from a SparkDataFrame into a data.frame
#' @param x a SparkDataFrame
#' @return a data.frame
#' @family SparkDataFrame functions
#' @rdname as.data.frame
#' @examples \dontrun{
#'
#' irisDF <- createDataFrame(sqlContext, iris)
#' df <- as.data.frame(irisDF[irisDF$Species == "setosa", ])
#' }
setMethod("as.data.frame",
          signature(x = "SparkDataFrame"),
          function(x, row.names = NULL, optional = FALSE, ...) {
            as.data.frame(collect(x), row.names, optional, ...)
          })

#' The specified SparkDataFrame is attached to the R search path. This means that
#' the SparkDataFrame is searched by R when evaluating a variable, so columns in
#' the SparkDataFrame can be accessed by simply giving their names.
#'
#' @family SparkDataFrame functions
#' @rdname attach
#' @title Attach SparkDataFrame to R search path
#' @param what (SparkDataFrame) The SparkDataFrame to attach
#' @param pos (integer) Specify position in search() where to attach.
#' @param name (character) Name to use for the attached SparkDataFrame. Names
#'   starting with package: are reserved for library.
#' @param warn.conflicts (logical) If TRUE, warnings are printed about conflicts
#' from attaching the database, unless that SparkDataFrame contains an object
#' @examples
#' \dontrun{
#' attach(irisDf)
#' summary(Sepal_Width)
#' }
#' @seealso \link{detach}
setMethod("attach",
          signature(what = "SparkDataFrame"),
          function(what, pos = 2, name = deparse(substitute(what)), warn.conflicts = TRUE) {
            newEnv <- assignNewEnv(what)
            attach(newEnv, pos = pos, name = name, warn.conflicts = warn.conflicts)
          })

#' Evaluate a R expression in an environment constructed from a SparkDataFrame
#' with() allows access to columns of a SparkDataFrame by simply referring to
#' their name. It appends every column of a SparkDataFrame into a new
#' environment. Then, the given expression is evaluated in this new
#' environment.
#'
#' @rdname with
#' @title Evaluate a R expression in an environment constructed from a SparkDataFrame
#' @param data (SparkDataFrame) SparkDataFrame to use for constructing an environment.
#' @param expr (expression) Expression to evaluate.
#' @param ... arguments to be passed to future methods.
#' @examples
#' \dontrun{
#' with(irisDf, nrow(Sepal_Width))
#' }
#' @seealso \link{attach}
setMethod("with",
          signature(data = "SparkDataFrame"),
          function(data, expr, ...) {
            newEnv <- assignNewEnv(data)
            eval(substitute(expr), envir = newEnv, enclos = newEnv)
          })

#' Display the structure of a SparkDataFrame, including column names, column types, as well as a
#' a small sample of rows.
#' @name str
#' @title Compactly display the structure of a dataset
#' @rdname str
#' @family SparkDataFrame functions
#' @param object a SparkDataFrame
#' @examples \dontrun{
#' # Create a SparkDataFrame from the Iris dataset
#' irisDF <- createDataFrame(sqlContext, iris)
#'
#' # Show the structure of the SparkDataFrame
#' str(irisDF)
#' }
setMethod("str",
          signature(object = "SparkDataFrame"),
          function(object) {

            # TODO: These could be made global parameters, though in R it's not the case
            MAX_CHAR_PER_ROW <- 120
            MAX_COLS <- 100

            # Get the column names and types of the DataFrame
            names <- names(object)
            types <- coltypes(object)

            # Get the first elements of the dataset. Limit number of columns accordingly
            localDF <- if (ncol(object) > MAX_COLS) {
              head(object[, c(1:MAX_COLS)])
            } else {
              head(object)
            }

            # The number of observations will not be displayed as computing the
            # number of rows is a very expensive operation
            cat(paste0("'", class(object), "': ", length(names), " variables:\n"))

            if (nrow(localDF) > 0) {
              for (i in 1 : ncol(localDF)) {
                # Get the first elements for each column

                firstElements <- if (types[i] == "character") {
                  paste(paste0("\"", localDF[, i], "\""), collapse = " ")
                } else {
                  paste(localDF[, i], collapse = " ")
                }

                # Add the corresponding number of spaces for alignment
                spaces <- paste(rep(" ", max(nchar(names) - nchar(names[i]))), collapse = "")

                # Get the short type. For 'character', it would be 'chr';
                # 'for numeric', it's 'num', etc.
                dataType <- SHORT_TYPES[[types[i]]]
                if (is.null(dataType)) {
                  dataType <- substring(types[i], 1, 3)
                }

                # Concatenate the colnames, coltypes, and first
                # elements of each column
                line <- paste0(" $ ", names[i], spaces, ": ",
                               dataType, " ", firstElements)

                # Chop off extra characters if this is too long
                cat(substr(line, 1, MAX_CHAR_PER_ROW))
                cat("\n")
              }

              if (ncol(localDF) < ncol(object)) {
                cat(paste0("\nDisplaying first ", ncol(localDF), " columns only."))
              }
            }
          })

#' drop
#'
#' Returns a new SparkDataFrame with columns dropped.
#' This is a no-op if schema doesn't contain column name(s).
#'
#' @param x A SparkDataFrame.
#' @param cols A character vector of column names or a Column.
#' @return A SparkDataFrame
#'
#' @family SparkDataFrame functions
#' @rdname drop
#' @name drop
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- read.json(sqlCtx, path)
#' drop(df, "col1")
#' drop(df, c("col1", "col2"))
#' drop(df, df$col1)
#' }
setMethod("drop",
          signature(x = "SparkDataFrame"),
          function(x, col) {
            stopifnot(class(col) == "character" || class(col) == "Column")

            if (class(col) == "Column") {
              sdf <- callJMethod(x@sdf, "drop", col@jc)
            } else {
              sdf <- callJMethod(x@sdf, "drop", as.list(col))
            }
            dataFrame(sdf)
          })

# Expose base::drop
setMethod("drop",
          signature(x = "ANY"),
          function(x) {
            base::drop(x)
          })

#' Saves the content of the SparkDataFrame to an external database table via JDBC
#'
#' Additional JDBC database connection properties can be set (...)
#'
#' Also, mode is used to specify the behavior of the save operation when
#' data already exists in the data source. There are four modes: \cr
#'  append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr
#'  overwrite: Existing data is expected to be overwritten by the contents of this
#'     SparkDataFrame. \cr
#'  error: An exception is expected to be thrown. \cr
#'  ignore: The save operation is expected to not save the contents of the SparkDataFrame
#'     and to not change the existing data. \cr
#'
#' @param x A SparkDataFrame
#' @param url JDBC database url of the form `jdbc:subprotocol:subname`
#' @param tableName The name of the table in the external database
#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default)
#' @family SparkDataFrame functions
#' @rdname write.jdbc
#' @name write.jdbc
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlContext <- sparkRSQL.init(sc)
#' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename"
#' write.jdbc(df, jdbcUrl, "table", user = "username", password = "password")
#' }
setMethod("write.jdbc",
          signature(x = "SparkDataFrame", url = "character", tableName = "character"),
          function(x, url, tableName, mode = "error", ...){
            jmode <- convertToJSaveMode(mode)
            jprops <- varargsToJProperties(...)
            write <- callJMethod(x@sdf, "write")
            write <- callJMethod(write, "mode", jmode)
            invisible(callJMethod(write, "jdbc", url, tableName, jprops))
          })