[SPARK-5654] Integrate SparkR

This pull requests integrates SparkR, an R frontend for Spark. The SparkR package contains both RDD and DataFrame APIs in R and is integrated with Spark's submission scripts to work on different cluster managers.

Some integration points that would be great to get feedback on:

1. Build procedure: SparkR requires R to be installed on the machine to be built. Right now we have a new Maven profile `-PsparkR` that can be used to enable SparkR builds

2. YARN cluster mode: The R package that is built needs to be present on the driver and all the worker nodes during execution. The R package location is currently set using SPARK_HOME, but this might not work on YARN cluster mode.

The SparkR package represents the work of many contributors and attached below is a list of people along with areas they worked on

edwardt (edwart) - Documentation improvements
Felix Cheung (felixcheung) - Documentation improvements
Hossein Falaki (falaki)  - Documentation improvements
Chris Freeman (cafreeman) - DataFrame API, Programming Guide
Todd Gao (7c00) - R worker Internals
Ryan Hafen (hafen) - SparkR Internals
Qian Huang (hqzizania) - RDD API
Hao Lin (hlin09) - RDD API, Closure cleaner
Evert Lammerts (evertlammerts) - DataFrame API
Davies Liu (davies) - DataFrame API, R worker internals, Merging with Spark
Yi Lu (lythesia) - RDD API, Worker internals
Matt Massie (massie) - Jenkins build
Harihar Nahak (hnahak87) - SparkR examples
Oscar Olmedo (oscaroboto) - Spark configuration
Antonio Piccolboni (piccolbo) - SparkR examples, Namespace bug fixes
Dan Putler (dputler) - Dataframe API, SparkR Install Guide
Ashutosh Raina (ashutoshraina) - Build improvements
Josh Rosen (joshrosen) - Travis CI build
Sun Rui (sun-rui)- RDD API, JVM Backend, Shuffle improvements
Shivaram Venkataraman (shivaram) - RDD API, JVM Backend, Worker Internals
Zongheng Yang (concretevitamin) - RDD API, Pipelined RDDs, Examples and EC2 guide

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>
Author: Shivaram Venkataraman <shivaram.venkataraman@gmail.com>
Author: Zongheng Yang <zongheng.y@gmail.com>
Author: cafreeman <cfreeman@alteryx.com>
Author: Shivaram Venkataraman <shivaram@eecs.berkeley.edu>
Author: Davies Liu <davies@databricks.com>
Author: Davies Liu <davies.liu@gmail.com>
Author: hlin09 <hlin09pu@gmail.com>
Author: Sun Rui <rui.sun@intel.com>
Author: lythesia <iranaikimi@gmail.com>
Author: oscaroboto <oscarjr@gmail.com>
Author: Antonio Piccolboni <antonio@piccolboni.info>
Author: root <edward>
Author: edwardt <edwardt.tril@gmail.com>
Author: hqzizania <qian.huang@intel.com>
Author: dputler <dan.putler@gmail.com>
Author: Todd Gao <todd.gao.2013@gmail.com>
Author: Chris Freeman <cfreeman@alteryx.com>
Author: Felix Cheung <fcheung@AVVOMAC-119.local>
Author: Hossein <hossein@databricks.com>
Author: Evert Lammerts <evert@apache.org>
Author: Felix Cheung <fcheung@avvomac-119.t-mobile.com>
Author: felixcheung <felixcheung_m@hotmail.com>
Author: Ryan Hafen <rhafen@gmail.com>
Author: Ashutosh Raina <ashutoshraina@users.noreply.github.com>
Author: Oscar Olmedo <oscarjr@gmail.com>
Author: Josh Rosen <rosenville@gmail.com>
Author: Yi Lu <iranaikimi@gmail.com>
Author: Harihar Nahak <hnahak87@users.noreply.github.com>

Closes #5096 from shivaram/R and squashes the following commits:

da64742 [Davies Liu] fix Date serialization
59266d1 [Davies Liu] check exclusive of primary-py-file and primary-r-file
55808e4 [Davies Liu] fix tests
5581c75 [Davies Liu] update author of SparkR
f731b48 [Shivaram Venkataraman] Only run SparkR tests if R is installed
64eda24 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R
d7c3f22 [Shivaram Venkataraman] Address code review comments Changes include 1. Adding SparkR docs to API docs generated 2. Style fixes in SparkR scala files 3. Clean up of shell scripts and explanation of install-dev.sh
377151f [Shivaram Venkataraman] Merge remote-tracking branch 'apache/master' into R
eb5da53 [Shivaram Venkataraman] Merge pull request #3 from davies/R2
a18ff5c [Davies Liu] Update sparkR.R
5133f3a [Shivaram Venkataraman] Merge pull request #7 from hqzizania/R3
940b631 [hqzizania] [SPARKR-92] Phase 2: implement sum(rdd)
0e788c0 [Shivaram Venkataraman] Merge pull request #5 from hlin09/doc-fix
3487461 [hlin09] Add tests log in .gitignore.
1d1802e [Shivaram Venkataraman] Merge pull request #4 from felixcheung/r-require
11981b7 [felixcheung] Update R to fail early if SparkR package is missing
c300e08 [Davies Liu] remove duplicated file
b045701 [Davies Liu] Merge branch 'remote_r' into R
19c9368 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into remote_r
f8fa8af [Davies Liu] mute logging when start/stop context
e7104b6 [Davies Liu] remove ::: in SparkR
a1777eb [Davies Liu] move rules into R/.gitignore
e88b649 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
6e20e71 [Davies Liu] address comments
b433817 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R
a1cedad [Shivaram Venkataraman] Merge pull request #228 from felixcheung/doc
e089151 [Davies Liu] Merge pull request #225 from sun-rui/SPARKR-154_2
463e28c [Davies Liu] Merge pull request #2 from shivaram/doc-fixes
bc2d6d8 [Shivaram Venkataraman] Remove arg from sparkR.stop and update docs
d425363 [Shivaram Venkataraman] Some doc fixes for column, generics, group
1f1a7e0 [Shivaram Venkataraman] Some fixes to DataFrame, RDD, SQLContext docs
104ad4e [Shivaram Venkataraman] Check the right env in exists
cf5cd99 [Shivaram Venkataraman] Remove unused numCols argument
85a50ec [Shivaram Venkataraman] Merge pull request #226 from RevolutionAnalytics/master
3eacfc0 [Davies Liu] fix flaky test
733380d [Davies Liu] update R examples (remove master from args)
b21a0da [Davies Liu] Merge pull request #1 from shivaram/log4j-tests
a1493d7 [Shivaram Venkataraman] Address comments
e1f83ab [Shivaram Venkataraman] Send Spark INFO logs to a file in SparkR tests
58276f5 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R
52cc92d [Shivaram Venkataraman] Add license to create-docs.sh
6ff5ea2 [Shivaram Venkataraman] Add instructions to generate docs
1f478c5 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R
02b4833 [Shivaram Venkataraman] Add a script to generate R docs (Rd, html) Also fix some issues with our documentation
d6d3729 [Davies Liu] enable spark and pyspark tests
0e5a83f [Davies Liu] fix code style
afd8a77 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
d87a181 [Davies Liu] fix flaky tests
7100fb9 [Shivaram Venkataraman] Fix libPaths in README
bdf3a14 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
05e7375 [Davies Liu] sort generics
b44e371 [Shivaram Venkataraman] Include RStudio instructions in README
855537f [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
9fb6af3 [Davies Liu] mark R classes/objects are private
423ea3c [Shivaram Venkataraman] Ignore unknown jobj in cleanup
974e4ea [Davies Liu] fix flaky test
410ec18 [Davies Liu] fix zipRDD() tests
d8b24fc [Davies Liu] disable spark and python tests temporary
ce3ca62 [Davies Liu] fix license check
7da0049 [Davies Liu] fix build
2892e29 [Davies Liu] support R in YARN cluster
ebd4d07 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
38cbf59 [Davies Liu] fix test of zipRDD()
756ece0 [Shivaram Venkataraman] Update README remove outdated TODO
d436f26 [Davies Liu] add missing files
40d193a [Shivaram Venkataraman] Merge pull request #224 from sun-rui/SPARKR-224-new
1a16cd6 [Davies Liu] rm PROJECT_HOME
56670ef [Davies Liu] rm man page
ba4b80b [Davies Liu] Merge branch 'remote_r' into R
f04080c [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into remote_r
028cbfb [Davies Liu] fix exit code of sparkr unit test
42d8b4c [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
ef26015 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
a1870e8 [Shivaram Venkataraman] Merge pull request #214 from sun-rui/SPARKR-156_3
cb6e5e3 [Shivaram Venkataraman] Add scripts to start SparkR on windows
8030847 [Shivaram Venkataraman] Set windows file separators, install dirs
05afef0 [Shivaram Venkataraman] Only stop backend JVM if R launched it
95d2de3 [Davies Liu] fix spark-submit with R scripot
baefd9e [Shivaram Venkataraman] Make bin/sparkR use spark-submit As a part of this move the R initialization functions into first.R and first-submit.R
d6f2bdd [Shivaram Venkataraman] Fix run-tests path
ea90fab [Davies Liu] fix spark-submit with R path and sparkR -h
0e2412c [Davies Liu] fix bin/sparkR
9f6aa1f [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
479e3fe [Davies Liu] change println() to logging
52ca6e5 [Shivaram Venkataraman] Add missing comma
716b16f [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R
2d235d4 [Shivaram Venkataraman] Build SparkR with Maven profile
aae881b [Davies Liu] fix rat
ff776aa [Shivaram Venkataraman] Fix style
e4f1937 [Shivaram Venkataraman] Remove DFC example
f7b6936 [Davies Liu] remove Spark prefix for class
043959e [Davies Liu] cleanup
ba53b09 [Davies Liu] support R in spark-submit
f403b4a [Davies Liu] rm .travis.yml
c4a5bdf [Davies Liu] run sparkr tests in Spark
e8fc7ca [Davies Liu] fix .gitignore
35e5755 [Davies Liu] reduce size of example data
50bff63 [Davies Liu] add LICENSE header for R sources
facb6e0 [Davies Liu] add .gitignore for .o, .so, .Rd
18e5eed [Davies Liu] update docs
0a0e632 [Davies Liu] move sparkR into bin/
a76472f [Davies Liu] fix path of assembly jar
df3eeea [Davies Liu] move R/examples into examples/src/main/r
3415cc7 [Davies Liu] move Scala source into core/ and sql/
180fc9c [Davies Liu] move scala
014d253 [Davies Liu] delete man pages
49a8133 [Davies Liu] Merge branch 'remote_r' into R
44994c2 [Davies Liu] Moved files to R/
2fc553f [Shivaram Venkataraman] Merge pull request #222 from davies/column2
b043876 [Davies Liu] fix test
5e610cb [Davies Liu] add more API for Column
6f95d49 [Shivaram Venkataraman] Merge pull request #221 from shivaram/sparkr-stop-start
3214c6d [Shivaram Venkataraman] Merge pull request #217 from hlin09/cleanClosureFix
f5d3355 [Shivaram Venkataraman] Merge pull request #218 from davies/merge
70f620c [Davies Liu] address comments
4b1628d [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into merge
3139325 [Shivaram Venkataraman] Merge pull request #212 from davies/toDF
6122e0e [Davies Liu] handle NULL
bc2ff38 [Davies Liu] handle NULL
7f5e70c [Davies Liu] Update SerDe.scala
46454e4 [Davies Liu] address comments
dd52cbc [Shivaram Venkataraman] Merge pull request #220 from shivaram/sparkr-utils-include
662938a [Shivaram Venkataraman] Include utils before SparkR for `head` to work Before this change calling `head` on a DataFrame would not work from the sparkR script as utils would be loaded after SparkR and placed ahead in the search list. This change requires utils to be loaded before SparkR
1bc2998 [Shivaram Venkataraman] Merge pull request #179 from evertlammerts/sparkr-sql
7695d36 [Evert Lammerts] added tests
8190127 [Evert Lammerts] fixed parquetFile signature
d8c8fcc [Shivaram Venkataraman] Merge pull request #219 from shivaram/sparkr-build-final
963c7ee [Davies Liu] Merge branch 'master' into merge
8bff523 [Shivaram Venkataraman] Remove staging repo now that 1.3 is released
e52258f [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into toDF
05b9126 [Shivaram Venkataraman] Merge pull request #215 from davies/agg
8e1497d [Davies Liu] Update DataFrame.R
72adb14 [Davies Liu] Update SQLContext.R
66cc92a [Davies Liu] address commets
55c38bc [Shivaram Venkataraman] Merge pull request #216 from davies/select2
3e0555d [Shivaram Venkataraman] Merge pull request #193 from davies/daemon
0467474 [Davies Liu] add more selecter for DataFrame
9a6be74 [Davies Liu] include grouping columns in agg()
e87bb98 [Davies Liu] improve comment and logging
a6dc435 [Davies Liu] remove dependency of jsonlite
26a3621 [Davies Liu] support date.frame and Date/Time
4e4908a [Davies Liu] createDataFrame from rdd
5757b95 [Shivaram Venkataraman] Merge pull request #196 from davies/die
90f2692 [Shivaram Venkataraman] Merge pull request #211 from hlin09/generics
8583968 [Davies Liu] readFully()
46cea3d [Davies Liu] retry
01aa5ee [Davies Liu] add config for using daemon, refactor
ff948db [hlin09] Remove missingOrInteger.
ecdfda1 [hlin09] Remove duplication.
411b751 [Davies Liu] make RStudio happy
8f8813f [Davies Liu] switch back to use parallel
6bccbbf [hlin09] Move roxygen doc back to implementation.
ffd6e8e [Shivaram Venkataraman] Merge pull request #210 from hlin09/hlin09
471c794 [hlin09] Move getJRDD and broadcast's value to 00-generic.R.
89b886d [hlin09] Move setGeneric() to 00-generics.R.
97dde1a [hlin09] Add a test for access operators.
09ff163 [Shivaram Venkataraman] Merge pull request #204 from cafreeman/sparkr-sql
15a713f [cafreeman] Fix example for `dropTempTable`
dc1291b [hlin09] Add checks for namespace access operators in cleanClosure.
b4c0b2e [Davies Liu] use fork package
3db5649 [cafreeman] Merge branch 'sparkr-sql' of https://github.com/amplab-extras/SparkR-pkg into sparkr-sql
789be97 [Shivaram Venkataraman] Merge pull request #207 from shivaram/err-remove
e60578a [cafreeman] update tests to guarantee row order
5eec6fc [Shivaram Venkataraman] Merge pull request #206 from sun-rui/SPARKR-156_2
3f7aed6 [Sun Rui] Fix minor typos in the function description.
a8cebf0 [Shivaram Venkataraman] Remove print statement in SparkRBackendHandler This print statement is noisy for SQL methods which have multiple APIs (like loadDF). We already have a better error message when no valid methods are found
5e3a576 [Sun Rui] Fix indentation.
f3d99a6 [Sun Rui] [SPARKR-156] phase 2: implement zipWithIndex() of the RDD class.
a582810 [cafreeman] Merge branch 'dfMethods' into sparkr-sql
7a5d6fd [cafreeman] `withColumn` and `withColumnRenamed`
c5fa3b9 [cafreeman] New `select` method
bcb0bf5 [Shivaram Venkataraman] Merge pull request #180 from davies/group
9dd6a5a [Davies Liu] Update SparkRBackendHandler.scala
e6fb8d8 [Davies Liu] improve logging
428a99a [Davies Liu] remove test, catch exception
fef99de [cafreeman] `intersect`, `subtract`, `unionAll`
befbd32 [cafreeman] `insertInto`
9d01bcd [cafreeman] `dropTempTable`
d8c1c09 [Davies Liu] add test to start and stop context multiple times
18c6004 [Shivaram Venkataraman] Merge pull request #201 from sun-rui/SPARKR-156_1
dfb399a [Davies Liu] address comments
f06ccec [Sun Rui] Use mapply() instead of for statement.
3c7674f [Davies Liu] Merge branch 'die' of github.com:davies/SparkR-pkg into die
ac8a852 [Davies Liu] close monitor connection in sparkR.stop()
4d0fb56 [Shivaram Venkataraman] Merge pull request #203 from shivaram/sparkr-hive-fix
62b0760 [Shivaram Venkataraman] Fix test hive context package name
47a613f [Shivaram Venkataraman] Fix HiveContext package name
fb3b139 [Davies Liu] fix tests
d0d4626 [Shivaram Venkataraman] Merge pull request #199 from davies/load
8b7fb67 [Davies Liu] fix HiveContext
bb46832 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load
e9e2a03 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into group
b875b4f [Davies Liu] fix style
de2abfa [Shivaram Venkataraman] Merge pull request #202 from cafreeman/sparkr-sql
3675fcf [cafreeman] Update `explain` and fixed doc for `toJSON`
5fd9575 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load
6fac596 [Davies Liu] support Column expression in agg()
f10a24e [Davies Liu] address comments
ff8b005 [cafreeman] 'saveAsParquetFile`
a5c2887 [cafreeman] fix test
3fab0f8 [cafreeman] `showDF`
779c102 [cafreeman] `isLocal`
68b11cf [cafreeman] `toJSON`
0ac4abc [cafreeman] 'explain`
20242c4 [cafreeman] clean up docs
6a1fe64 [Shivaram Venkataraman] Merge pull request #198 from cafreeman/sparkr-sql
198c130 [Shivaram Venkataraman] Merge pull request #200 from shivaram/sparkr-sql-build
870acd4 [Shivaram Venkataraman] Use rc2 explicitly
8b9a963 [cafreeman] Merge branch 'sparkr-sql' of https://github.com/amplab-extras/SparkR-pkg into sparkr-sql
bc90115 [cafreeman] Fixed docs
3865f39 [Sun Rui] [SPARKR-156] phase 1: implement zipWithUniqueId() of the RDD class.
a37fd80 [Davies Liu] Update sparkR.R
d18f9d3 [Shivaram Venkataraman] Remove SparkR snapshot build We now have 1.3.0 RC2 on Apache Staging
8de958d [Davies Liu] Update SparkRBackend.scala
4e0becc [Shivaram Venkataraman] Merge pull request #194 from davies/api
197a79b [Davies Liu] add HiveContext (commented)
32aa01d [Shivaram Venkataraman] Merge pull request #191 from felixcheung/doc
5073e07 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load
7918634 [cafreeman] Fix test
acea146 [cafreeman] remove extra line
74269f3 [cafreeman] Merge branch 'dfMethods' into sparkr-sql
cd7ac8a [Shivaram Venkataraman] Merge pull request #197 from cafreeman/sparkr-sql
494a4dd [cafreeman] update export
e14c328 [cafreeman] `selectExpr`
32b37d1 [cafreeman] Fixed indent in `join` test.
2e7b190 [Felix Cheung] small update on yarn deploy mode.
8ff29d6 [Davies Liu] fix tests
12a6db2 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into api
294ca4a [cafreeman] `join`, `sort`, and `filter`
4fa6343 [cafreeman] Refactor `join` generic for use with `DataFrame`
3f22c8d [Shivaram Venkataraman] Merge pull request #195 from cafreeman/sparkr-sql
2b6f980 [Davies Liu] shutdown the JVM after R process die
e8639c3 [cafreeman] New 1.3 repo and updates to `column.R`
ed9a89f [Davies Liu] address comments
03bcf20 [Davies Liu] Merge branch 'group' of github.com:davies/SparkR-pkg into group
39c253d [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into group
98cc97a [Davies Liu] fix test and docs
e2d144a [Felix Cheung] Fixed small typos
3beadcf [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into api
06cbc2d [Davies Liu] launch R worker by a daemon
8a676b1 [Shivaram Venkataraman] Merge pull request #188 from davies/column
524c122 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into column
f798402 [Davies Liu] Update column.R
1d0f2ae [Davies Liu] Update DataFrame.R
03402eb [Felix Cheung] Updates as per feedback on sparkR-submit
76cf2e0 [Shivaram Venkataraman] Merge pull request #192 from cafreeman/sparkr-sql
1955a09 [cafreeman] return object instead of a list of one object
f585929 [cafreeman] Fix brackets
e998356 [cafreeman] define generic for 'first' in RDD API
71d66a1 [Davies Liu] fix first(0
8ec21af [Davies Liu] fix signature
acae527 [Davies Liu] refactor
d7b17a4 [Davies Liu] fix approxCountDistinct
7dfe27d [Davies Liu] fix cyclic namespace dependency
8caf5bb [Davies Liu] use S4 methods
5c0bb24 [Felix Cheung] Doc updates: build and running on YARN
773baf0 [Zongheng Yang] Merge pull request #178 from davies/random
862f07c [Shivaram Venkataraman] Merge pull request #190 from shivaram/SPARKR-79
b457833 [Shivaram Venkataraman] Merge pull request #189 from shivaram/stdErrFix
f7caeb8 [Davies Liu] Update SparkRBackend.scala
8c4deae [Shivaram Venkataraman] Remove unused function
6e51c7f [Shivaram Venkataraman] Fix stderr redirection on executors
7afa4c9 [Shivaram Venkataraman] Merge pull request #186 from hlin09/funcDep3
4d36ab1 [hlin09] Add tests for broadcast variables.
3f57e56 [hlin09] Fix comments.
7b72487 [hlin09] Fix comments.
ae05bf1 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into column
abb4bb9 [Davies Liu] add Column and expression
eb8ac11 [Shivaram Venkataraman] Set Spark version 1.3.0 in Windows build
5c72e73 [Davies Liu] wait atmost 100 seconds
e425437 [Shivaram Venkataraman] Merge pull request #177 from lythesia/master
a00f502 [lythesia] fix indents
0346e5f [Davies Liu] address comment
6134649 [Shivaram Venkataraman] Merge pull request #187 from cafreeman/sparkr-sql
ad0935e [lythesia] minor fixes
b0e7f73 [cafreeman] Update `sampleDF` test
7b0d070 [lythesia] keep partitions check
889c265 [cafreeman] numToInt utility function
27dd3a0 [lythesia] modify tests for repartition
cad0f0c [cafreeman] Fix docs and indents
2808dcf [cafreeman] Three more DataFrame methods
5ef66fb [Davies Liu] send back the port via temporary file
3b46429 [Davies Liu] Merge branch 'master' of github.com:amplab-extras/SparkR-pkg into random
798f453 [cafreeman] Merge branch 'sparkr-sql' into dev
9aa4acf [Shivaram Venkataraman] Merge pull request #184 from davies/socket
020bce8 [Shivaram Venkataraman] Merge pull request #183 from cafreeman/sparkr-sql
222e06b [cafreeman] Lazy evaluation and formatting changes
e776324 [Davies Liu] fix import
211cc15 [cafreeman] Merge branch 'sparkr-sql' into dev
3351afd [hlin09] Replaces getDependencies with cleanClosure, to serialize UDFs to workers.
e7c56d6 [lythesia] fix random partition key
50c74b1 [Davies Liu] address comments
083c89f [cafreeman] Remove commented lines an unused import
dfa119b [hlin09] Improve the coverage of processClosure.
a41c9b9 [cafreeman] Merge branch 'wrapper' into sparkr-sql
1cd714f [cafreeman] Wrapper function docs.
db0cd9e [cafreeman] Clean up for wrapper functions
818c19f [cafreeman] Update schema-related functions
a57884e [cafreeman] Remove unused import
d72e830 [cafreeman] Add wrapper for `StructField` and `StructType`
2ea2ecf [lythesia] use generic arg
09b9512 [hlin09] add docs
f4f077c [hlin09] Add recursive cleanClosure for function access.
f84ad27 [hlin09] Merge remote-tracking branch 'upstream/master' into funcDep2
5300766 [Shivaram Venkataraman] Merge pull request #185 from hlin09/hlin09
07aa7c0 [hlin09] Unifies the implementation of lapply with lapplyParitionsWithIndex.
f4dbb0b [Davies Liu] use socket in worker
8282c59 [Davies Liu] Update DataFrame.R
ba495a8 [Davies Liu] Update NAMESPACE
36dffb3 [cafreeman] Add 'head` and `first`
534a95f [cafreeman] Schema-related methods
64f488d [cafreeman] Cache and Persist Methods
30d71fd [cafreeman] Standardize method arguments for DataFrame methods
785898b [Shivaram Venkataraman] Merge pull request #182 from cafreeman/sparkr-sql
2619003 [Shivaram Venkataraman] Merge pull request #181 from cafreeman/master
a9bbe0b [cafreeman] Update existing SparkSQL functions
8c241a3 [cafreeman] Merge with master, include changes to method args
68d6de4 [cafreeman] Fix typos
8d2ec6e [Davies Liu] add sum/max/min/avg/mean
774e687 [Davies Liu] add missing API in SQLContext
1e72b4b [Davies Liu] missing API in SQLContext
3294949 [Chris Freeman] Restore `rdd` argument to `getJRDD`
3a58ebc [Davies Liu] rm unrelated file
8bd93b5 [Davies Liu] fix signature
c652b4c [cafreeman] Update method signatures to use generic arg
48c8827 [Davies Liu] update NAMESPACE
84e2d8c [Davies Liu] groupBy and agg()
7c3ddbd [Davies Liu] create jmode in JVM
9465426 [Davies Liu] load and save
982f342 [lythesia] fix numeric issue
7651d84 [lythesia] fix coalesce
4e712e1 [Davies Liu] use random port in backend
041d22b [Shivaram Venkataraman] Merge pull request #172 from cafreeman/sparkr-sql
0d07770 [cafreeman] Added `limit` and updated `take`
301d8e5 [cafreeman] Remove extraneous map functions
0387db2 [cafreeman] Remove colNames
04c4b65 [lythesia] add repartition/coalesce
231deab [cafreeman] Change reserialize to serializeToBytes
acf7e1a [cafreeman] Rework the Scala to R DataFrame Conversion
481ae37 [cafreeman] Updated stale comments and standardized arg names
21d4a97 [hlin09] Adds cleanClosure to capture the function closures.
d24ffb4 [hlin09] Merge remote-tracking branch 'upstream/master' into funcDep2
8be02de [hlin09] Revert "loop 1-12 test pass."
fddb9cc [hlin09] Revert "add docs"
f8ef0ab [hlin09] Revert "More docs"
8e4b3da [hlin09] Revert "More docs"
57e005b [hlin09] Revert "fix tests."
c10148e [Shivaram Venkataraman] Merge pull request #174 from shivaram/sparkr-runner
910e3be [Shivaram Venkataraman] Add a timeout for initialization Also move sparkRBackend.stop into a finally block
bf52b17 [Shivaram Venkataraman] Merge remote-tracking branch 'amplab-sparkr/master' into sparkr-runner
08102b0 [Shivaram Venkataraman] Merge pull request #176 from lythesia/master
9c77b20 [Chris Freeman] Merge pull request #2 from shivaram/sparkr-sql
179ab38 [lythesia] add try counts and increase time interval
71a73b2 [Shivaram Venkataraman] Use a getter for serialization mode This change encapsulates the semantics of serialization mode for RDDs inside a getter function. For PipelinedRDDs if a backing JavaRDD is available we use that else we fall back to a default serialization mode
06bf250 [Shivaram Venkataraman] Merge pull request #173 from shivaram/windows-space-fix
88bf97f [Shivaram Venkataraman] Create SparkContext for R shell launch
f9268d9 [Shivaram Venkataraman] Fix code review comments
e6ad12d [Shivaram Venkataraman] Update comment describing sparkR-submit
17eda4c [Shivaram Venkataraman] Merge pull request #175 from falaki/docfix
ba2b72b [Hossein] Spark 1.1.0 is default
4cd7d3f [lythesia] retry backend connection
749e2d0 [Hossein] Updated README
bc04cf4 [Shivaram Venkataraman] Use SPARKR_BACKEND_PORT in sparkR.R as default Change SparkRRunner to use EXISTING_SPARKR_BACKEND_PORT to differentiate between the two
22a19ac [Shivaram Venkataraman] Use a semaphore to wait for backend to initalize Also pick a random port to avoid collisions
7f1f0f8 [cafreeman] Move comments to fit 100 char line length
8b84e4e [cafreeman] Make if statements more explicit
ce5d5ab [cafreeman] New tests for Union and Object File
b063320 [cafreeman] Changed 'serialized' to 'serializedMode'
0981dff [Zongheng Yang] Merge pull request #168 from sun-rui/SPARKR-153_2
86fc639 [Shivaram Venkataraman] Move sparkR-submit into pkg/inst
fd8f8a9 [Shivaram Venkataraman] Merge branch 'hqzizania-master'
a33dbea [Shivaram Venkataraman] Merge branch 'master' of https://github.com/hqzizania/SparkR-pkg into hqzizania-master
384e6e2 [Shivaram Venkataraman] Merge pull request #171 from hlin09/hlin09
1f5a6ac [hlin09] fixed comments
7f7596a [cafreeman] Additional handling for "row" serialization
8c3b8c5 [cafreeman] Add test for UnionRDD on "row" serialization
b1141f8 [cafreeman] Fixed formatting issues.
5db30bf [cafreeman] Changed serialized from bool to string
2f0c0b8 [cafreeman] Add check for serialized type
d243dfb [cafreeman] Clean up code
5ff63a2 [cafreeman] Change test from boolean to string
77fec1a [cafreeman] Updated .Rd files
9224989 [cafreeman] Various updates for DataFrame to RRDD
26af62b [cafreeman] DataFrame to RRDD
e004481 [cafreeman] Update UnionRDD test
5292be7 [hlin09] Adds support of pipeRDD().
e2a7560 [Shivaram Venkataraman] Merge pull request #170 from cafreeman/sparkr-sql
5d537f4 [cafreeman] Add pairRDD to Description
b6fa88e [cafreeman] Updating to current master
0cda231 [Sun Rui] [SPARKR-153] phase 2: implement aggregateByKey() and foldByKey().
95ee6b4 [Shivaram Venkataraman] Merge remote-tracking branch 'amplab-sparkr/master' into sparkr-runner
67fbc60 [Shivaram Venkataraman] Add support for SparkR shell to use spark-submit This ensures that SparkConf options are read in both in batch and interactive modes
2271030 [Shivaram Venkataraman] Merge pull request #167 from sun-rui/removePartionByInRDD
7fcb46a [Sun Rui] Remove partitionBy() in RDD.
52f94c4 [Shivaram Venkataraman] Merge pull request #160 from lythesia/master
59e2d54 [lythesia] merge with upstream
5836650 [Zongheng Yang] Merge pull request #163 from sun-rui/SPARKR-153_1
141723e [Sun Rui] fix comments.
f73a07e [Shivaram Venkataraman] Merge pull request #165 from shivaram/sparkr-sql-build
10ffc6d [Shivaram Venkataraman] Set Spark version to 1.3 using staging dependency Also fix the maven build
c91ede2 [Shivaram Venkataraman] Merge pull request #164 from hlin09/hlin09
9d335a9 [hlin09] Makes git to ignore Eclipse meta files.
94066bf [Sun Rui] [SPARKR-153] phase 1: implement fold() and aggregate().
9c391c7 [hqzizania] Merge remote-tracking branch 'upstream/master'
5f29551 [hqzizania] 	modified:   pkg/R/RDD.R 	modified:   pkg/R/context.R
d968664 [lythesia] fix comment
7972858 [Shivaram Venkataraman] Merge pull request #159 from sun-rui/SPARKR-150_2
7690878 [lythesia] separate out pair RDD functions
f4573c1 [Sun Rui] Use reduce() instead of sortBy().take() to get the ordered elements.
63e62ed [Sun Rui] [SPARKR-150] phase 2: implement takeOrdered() and top().
050390b [Shivaram Venkataraman] Fix bugs in inferring R file
8398f2e [Shivaram Venkataraman] Add sparkR-submit helper script Also adjust R file path for YARN cluster mode
bd6705b [Zongheng Yang] Merge pull request #154 from sun-rui/SPARKR-150
c7964c9 [Sun Rui] Merge with upstream master.
7feac38 [Sun Rui] Use default arguments for sortBy() and sortKeyBy().
de2bfb3 [Sun Rui] Fix minor comments and add more test cases.
0c6e071 [Zongheng Yang] Merge pull request #157 from lythesia/master
f5038c0 [lythesia] pull out anonymous functions in groupByKey
ba6f044 [lythesia] fixes for reduceByKeyLocally
343b6ab [Oscar Olmedo] Export sparkR.stop Closes #156 from oscaroboto/master
25639cf [Shivaram Venkataraman] Replace tabs with spaces
bb25920 [Shivaram Venkataraman] Merge branch 'dputler-master'
fd836db [hlin09] fix tests.
24a7f13 [hlin09] More docs
a465165 [hlin09] More docs
6ad4fc3 [hlin09] add docs
b082a35 [lythesia] add reduceByKeyLocally
7ca6512 [Shivaram Venkataraman] First cut of SparkRRunner
193f5fe [hlin09] loop 1-12 test pass.
345f1b8 [dputler] [SPARKR-195] Implemented project style guidelines for if-else statements
8043559 [Sun Rui] Add a TODO to use binary search in the range partitioner.
91b2fd6 [Sun Rui] Add more test cases.
e8ebbe4 [Shivaram Venkataraman] Merge pull request #152 from cafreeman/sparkr-sql
0c53d6c [dputler] Data frames now coerced to lists, and messages issued for a data frame or matrix on how they are parallelized
6d57ec0 [cafreeman] Remove json test file since we're using a temp
ac1ef09 [cafreeman] Update registerTempTable test
d9da451 [Sun Rui] [SPARKR-150] phase 1: implement sortBy() and sortByKey().
08ff30b [Shivaram Venkataraman] Merge pull request #153 from hqzizania/master
9767e8e [hqzizania] 	modified:   pkg/man/collect-methods.Rd
5d69f0a [hqzizania] 	modified:   pkg/R/RDD.R
4914091 [hqzizania] 	modified:   pkg/inst/tests/test_rdd.R
742a68b [cafreeman] Update test_sparkRSQL.R
a95823e [hqzizania] 	modified:   pkg/R/RDD.R
2d04526 [cafreeman] Formatting
fae9bdd [cafreeman] Renamed to SQLUtils.scala
39888ea [Chris Freeman] Update test_sparkSQL.R
fce2453 [cafreeman] Updated documentation for SQLContext
13fbf12 [cafreeman] Regenerated .Rd files
51ecf41 [cafreeman] Updated Scala object
30d7337 [cafreeman] Added SparkSQL test
74b3ed6 [cafreeman] Incorporate code feedback
554bda0 [Zongheng Yang] Merge pull request #147 from shivaram/sparkr-ec2-fixes
a5f4f8f [cafreeman] Squashed commit of the following:
f34bb88 [Shivaram Venkataraman] Remove profiling information from this PR
c662f29 [Zongheng Yang] Merge pull request #146 from shivaram/spark-1.2-build
21e9b74 [Zongheng Yang] Merge pull request #145 from lythesia/master
76f6b9e [Shivaram Venkataraman] Merge pull request #149 from hqzizania/master
1c2dbec [lythesia] minor fix for refactoring join code
5b380d3 [hqzizania] 	modified:   pkg/man/combineByKey.Rd 	modified:   pkg/man/groupByKey.Rd 	modified:   pkg/man/partitionBy.Rd 	modified:   pkg/man/reduceByKey.Rd
98794fe [hqzizania] 	modified:   pkg/R/RDD.R
b66534d [Zongheng Yang] Merge pull request #144 from shivaram/fix-rd-files
60da1df [Shivaram Venkataraman] Initialize timing variables
179aa75 [Shivaram Venkataraman] Bunch of fixes for longer running jobs 1. Increase the timeout for socket connection to wait for long jobs 2. Add some profiling information in worker.R 3. Put temp file writes before stdin writes in RRDD.scala
06d99f0 [Shivaram Venkataraman] Fix URI to have right number of slashes
add97f5 [Shivaram Venkataraman] Use URL encode to create valid URIs for jars
4eec962 [lythesia] refactor join functions
73430c6 [Shivaram Venkataraman] Make SparkR work on paths with spaces on Windows
aaf8f47 [Shivaram Venkataraman] Exclude hadoop client from Spark dependency
227ee42 [Zongheng Yang] Merge pull request #141 from shivaram/SPARKR-140
ac5ceb1 [Shivaram Venkataraman] Fix code review comments
32394de [Shivaram Venkataraman] Regenerate Rd files for SparkR This fixes a number of issues in SparkR man pages. The main changes are 1. Don't export or generate docs for PipelineRDD 2. Fix variable names for Filter, count to match base methods 3. Document missing arguments for sparkR.init, print.jobj etc.
e157bf6 [Shivaram Venkataraman] Use prev_serialized to track if JRDD is serialized This changes introduces a new variable in PipelineRDD environment to track if the prev_jrdd is serialized or not.
7428a7e [Zongheng Yang] Merge pull request #143 from shivaram/SPARKR-181
7dd1797 [Shivaram Venkataraman] Address code review comments
8f81c45 [Shivaram Venkataraman] Remove roxygen export for PipelinedRDD
0cb90f1 [Zongheng Yang] Merge pull request #142 from shivaram/SPARKR-169
d1c6e6c [Shivaram Venkataraman] Buffer stderr from R and return it on Exception This change buffers the last 100 lines from R process and passes these lines back to the driver if we have an exception. This will help users debug why their tasks failed on the cluster
d6c1393 [Shivaram Venkataraman] Suppress warnings from normalizePath
a382835 [Shivaram Venkataraman] Fix serialization tracking in pipelined RDDs When creating a pipeline RDD, we need to check if the JavaRDD belonging to the parent is serialized.
da39529 [Zongheng Yang] Merge pull request #140 from sun-rui/SPARKR-183
2814caa [Sun Rui] Merge with upstream master.
cd2a5b3 [Sun Rui] Add reference to Nagle's algorithm and clean code.
52356b6 [Shivaram Venkataraman] Merge pull request #139 from shivaram/fix-backend-exit
97e5a1f [Sun Rui] [SPARKR-183] Fix the issue that parallelize collect tests are slow.
a9f8e8e [Shivaram Venkataraman] Merge pull request #138 from concretevitamin/fix-collect-test
125ae43 [Shivaram Venkataraman] Fix SparkR backend to exit in more cases This change has two fixes 1. When the workspace is saved (from R or RStudio) the backend connection seems to be closed before the finalizer is run. In such cases we reopen the connection and stop the backend 2. With RStudio when R is restarted, there are port-conflicts which appear due to a race condition between the JVM and rsession restart. This change adds a 1 sec sleep to avoid this race.
12c102a [Zongheng Yang] Simplify a unit test.
9c0637a [Zongheng Yang] Merge pull request #137 from shivaram/fix-docs
0df0e18 [Shivaram Venkataraman] Fix documentation for includePackage
7549f88 [Zongheng Yang] Merge pull request #136 from shivaram/man-updates
7edbe46 [Shivaram Venkataraman] Add missing man pages
9cb9567 [Shivaram Venkataraman] Merge pull request #131 from shivaram/rJavaExpt
1fa722e [Shivaram Venkataraman] Rename to SerDe now
2fcb051 [Shivaram Venkataraman] Rename to SerDeJVMR
d112cf0 [Shivaram Venkataraman] Style fixes
9fd01cc [Shivaram Venkataraman] Remove unnecessary braces
0881931 [Shivaram Venkataraman] Some more style fixes
f00b531 [Shivaram Venkataraman] Address code review comments. Big changes include style fixes throughout for named arguments
c09ba05 [Shivaram Venkataraman] Change jobj id to be just an integer Add a new print.jobj that gets the class name and prints it Also add a utility function isInstanceOf
be05b16 [Shivaram Venkataraman] Check if context, connection exist before stopping
d596a23 [Shivaram Venkataraman] Address code review comments
396e7ac [Shivaram Venkataraman] Changes to make new backend work on Windows This change uses file.path to construct the Java binary path in a OS agnostic way and uses system2 to handle quoting binary paths correctly. Tests pass on Mac OSX and a Windows EC2 instance.
e7a4e03 [Shivaram Venkataraman] Remove unused file BACKEND.md
62f380b [Shivaram Venkataraman] Update worker.R to use new deserialization call
8b9c4e6 [Shivaram Venkataraman] Change RDD name, setName to use new backend
6dcd5c5 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into rJavaExpt
0873397 [Shivaram Venkataraman] Refactor java object tracking into a new singleton. Also add comments describing each class
95db964 [Shivaram Venkataraman] Add comments, cleanup new R code
bcd4258 [Zongheng Yang] Merge pull request #130 from lythesia/master
74dbc5e [Sun Rui] Match method using parameter types.
7ad4a4d [Sun Rui] Use 1 char to represent types on the backend->client direction.
bace887 [Sun Rui] Use an integer count for the backend java object ID because Uniqueness isn't guaranteed by System.identityHashCode().
b38d04f [Sun Rui] Use 1 char to represent types on the client -> backend direction.
f88bc68 [lythesia] Merge branch 'master' of github.com:lythesia/SparkR-pkg
71d41f5 [lythesia] add test case for fullOuterJoin
eb4f423 [lythesia] --amend
cffecc5 [lythesia] add test case for fullOuterJoin
a547dd2 [Shivaram Venkataraman] Move classTag, rddRef into newJObject call This avoids them getting eagerly garbage collected
1255391 [Shivaram Venkataraman] Add a finalizer for jobj objects This enables Java objects to be garbage collected on the backend when they are no longer referenced in R. Also rename newJava to newJObject to be more consistent with callJMethod
70fa409 [Sun Rui] Add YARN Conf Dir to the class path when launching the backend.
a1108ca [lythesia] add fullOuterJoin in RDD.R
2152727 [Shivaram Venkataraman] Remove empty file
cd08bee [Shivaram Venkataraman] Update all functions to use new backend All unit tests pass.
9de49b7 [Shivaram Venkataraman] Add high level calls for methods, constructors Also update BACKEND.md
5a97ea4 [Shivaram Venkataraman] Add jobj S3 class that holds backend refs
e071d3e [Shivaram Venkataraman] Change SparkRBackend to use general method calls This change uses a custom protocl + JNI to invoke any method on a given object type. Also update serializers, deserializers to make code more concise
49f0404 [Shivaram Venkataraman] Merge pull request #129 from lythesia/master
7f8cd82 [lythesia] update man
4715ed2 [Yi Lu] Update RDD.R
5a53801 [lythesia] fix name,setName
4f3870b [lythesia] add name,setName in RDD.R
1c25700 [Shivaram Venkataraman] Merge pull request #128 from sun-rui/SPARKR-165
c8507d8 [Sun Rui] [SPARKR-165] IS_SCALAR is not present in R before 3.1
2cff2bd [Sun Rui] Add function to invoke Java method.
7a31da1 [Shivaram Venkataraman] Merge branch 'dputler-master'. Closes #119
0ceba82 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/dputler/SparkR-pkg into dputler-master
735f70c [Shivaram Venkataraman] Merge pull request #125 from 7c00/rawcon
fccfe6c [Shivaram Venkataraman] Merge pull request #127 from sun-rui/SPARKR-164
387bd57 [Sun Rui] [SPARKR-164] Temporary files used by SparkR accumulat as time goes on.
5f2268f [Shivaram Venkataraman] Add support to stop backend
5f745c0 [Shivaram Venkataraman] Update notes in backend
22015c1 [Shivaram Venkataraman] Add first cut of SparkR Backend
52821da [Todd Gao] switch the order of packages and function deps
d7b0007 [Todd Gao] remove memCompress
cb6873e [Shivaram Venkataraman] Merge pull request #126 from sun-rui/SPARKR-147
c5962eb [Todd Gao] further optimize using rawConnection
f04c6e0 [Sun Rui] [SPARKR-147] Support multiple directories as input to textFile.
b7de604 [Todd Gao] optimize execFunctionDeps loading in worker.R
4d4fc30 [Shivaram Venkataraman] Merge pull request #122 from cafreeman/master
b508877 [cafreeman] Update SparkR_IDE_Setup.sh
21ed9d7 [cafreeman] Update build.sbt
f73ec16 [cafreeman] Delete SparkR_IDE_Setup_Guide.md
d63b026 [cafreeman] Delete SparkR_Quick_Start_Guide.md
6e6cb62 [cafreeman] Update SparkR_IDE_Setup.sh
bc6042b [cafreeman] Update build.sbt
a8197d5 [cafreeman] Merge remote-tracking branch 'upstream/master'
d671564 [Zongheng Yang] Merge pull request #123 from shivaram/jcheck-void
76b8d00 [Zongheng Yang] Merge pull request #124 from shivaram/master
b690d58 [Shivaram Venkataraman] Specify how to change Spark versions in README
0fb003d [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into jcheck-void
1c227b4 [Shivaram Venkataraman] Also add a check in context.R
96812b6 [Shivaram Venkataraman] Check for exceptions after void method calls
f5c216d [cafreeman] Merge remote-tracking branch 'upstream/master'
90c8933 [Zongheng Yang] Merge pull request #121 from shivaram/fix-sort-order
bd0e3b4 [Shivaram Venkataraman] Fix saveAsTextFile test case
2e55f67 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into fix-sort-order
f10c607 [Shivaram Venkataraman] Merge pull request #118 from sun-rui/saveAsTextFile
6c9bfc0 [Sun Rui] Merge remote-tracking branch 'SparkR_upstream/master' into saveAsTextFile
6faedbe [cafreeman] Update SparkR_IDE_Setup_Guide.md
57008bc [cafreeman] Update SparkR_IDE_Setup.sh
bb1c17d [cafreeman] Update SparkR_IDE_Setup.sh
538bfdb [cafreeman] Update SparkR_Quick_Start_Guide.md
31322c6 [cafreeman] Update SparkR_IDE_Setup.sh
ca3f593 [Sun Rui] Refactor RRDD code.
df58d95 [cafreeman] Update SparkR_Quick_Start_Guide.md
b488c88 [cafreeman] Rename Spark_IDE_Setup.sh to SparkR_IDE_Setup.sh
b2545a4 [cafreeman] Added IDE Setup Guide
0ffb5de [cafreeman] Merge branch 'master' of https://github.com/cafreeman/SparkR-pkg
bd8fbfb [cafreeman] Merge remote-tracking branch 'upstream/master'
98efa5b [cafreeman] Added Quick Start Guide
3cf88f2 [Shivaram Venkataraman] Sort lists before comparing in unit tests Since Spark doesn't guarantee that shuffle results will always be in the same order, we need to sort the results before comparing for deterministic behavior
d621dbc [Shivaram Venkataraman] Merge pull request #120 from sun-rui/objectFile
c4a44d7 [Sun Rui] Add @seealso in comments and extract some common code into a function.
724e3a4 [cafreeman] Update Spark_IDE_Setup.sh
8153e5a [Sun Rui] [SPARKR-146] Support read/save object files in SparkR.
17f9909 [cafreeman] Update Spark_IDE_Setup.sh
a9eb080 [cafreeman] IDE Shell Script
64d800c [dputler] Merge remote branch 'upstream/master'
1fbdb2e [dputler] Added the ability for the user to specify a text file location throught the use of tilde expansion or just the file name if it is in the working directory.
d83c017 [Shivaram Venkataraman] Merge pull request #113 from sun-rui/stringHashCodeInC
a7d9cdb [Sun Rui] Fix build on Windows.
7d81b05 [Shivaram Venkataraman] Merge pull request #114 from hlin09/hlin09
47c4bb7 [hlin09] fix reviews
a457f7f [Shivaram Venkataraman] Merge pull request #116 from dputler/master
0fa48d1 [Shivaram Venkataraman] Merge pull request #117 from sun-rui/keyBy
85cfeb4 [Sun Rui] [SPARKR-144] Implement saveAsTextFile() in the RDD class.
09083d9 [Sun Rui] Add keyBy() to the RDD class.
caad5d7 [dputler] Adding the script to install software on the Cloudera Quick Start VM.
dca3d05 [hlin09] Minor fix.
ece5f7d [hlin09] Merge remote-tracking branch 'upstream/master' into hlin09
a40874b [hlin09] Use extendible accumulators aggregate the cogroup values.
d0347ce [Zongheng Yang] Merge pull request #112 from sun-rui/outer_join
492f76e [Sun Rui] Refine code and add description.
ba01358 [Shivaram Venkataraman] Merge pull request #115 from sun-rui/SPARKR-130
5c8e46e [Sun Rui] Fix per the review comments.
7190a2c [Sun Rui] Update comment to add a reference to storage levels.
1da705e [hlin09] Fix the review comments.
c4b77be [Sun Rui] [SPARKR-130] Add persist(storageLevel) API to RDD.
b424a1a [hlin09] Add function cogroup().
9770312 [Shivaram Venkataraman] Merge pull request #111 from hlin09/hlin09
cead7df [hlin09] fix review comments.
54f712e [Sun Rui] Implement string hash code in C.
425f0c6 [Sun Rui] Add leftOuterJoin() and rightOuterJoin() to the RDD class.
39509c7 [hlin09] add Rd file for foreach and foreachPartition.
63d6ac7 [hlin09] Adds function foreach() and foreachPartition().
9c954df [Zongheng Yang] Merge pull request #105 from sun-rui/join
c71228d [Sun Rui] Pre-allocate list with fixed length. Add test case for join() using string key.
bc3e9f6 [Shivaram Venkataraman] Merge pull request #108 from concretevitamin/take-optimize
c06fc90 [Zongheng Yang] Fix: only optimize for unserialized dataset case.
d399aeb [Zongheng Yang] Apply size-capping on logical representation instead of physical.
e4217dd [Zongheng Yang] Merge pull request #107 from shivaram/master
7952180 [Shivaram Venkataraman] Copy, use getLocalDirs from Spark Utils.scala
08e24c3 [Zongheng Yang] Merge pull request #109 from hlin09/hlin09
97d4e02 [Zongheng Yang] Min() upper-bound size with actual size.
bb779bf [hlin09] Rename the filter function to filterRDD to follow the API consistency. Filter() is also kept.
ce1661f [Zongheng Yang] Fix slow take(): deserialize only up to necessary # of elements.
4dca9b1 [Shivaram Venkataraman] Merge pull request #106 from hlin09/hlin09
1220d92 [hlin09] Adds function numPartitions().
2326a65 [Shivaram Venkataraman] Use SPARK_LOCAL_DIRS to create tmp files
e119757 [hlin09] Minor fix.
9c24c8b [hlin09] Adds function countByKey().
48fce67 [hlin09] Adds countByValue().
6679eef [Sun Rui] Update documentation for join().
70586b4 [Sun Rui] Add join() to the RDD class.
e6fb999 [Zongheng Yang] Merge pull request #103 from shivaram/rlibdir-fix
a21f146 [Shivaram Venkataraman] Merge pull request #102 from hlin09/hlin09
32eb619 [Shivaram Venkataraman] Merge pull request #104 from sun-rui/add_keys_values
d8692e9 [Sun Rui] Add keys() and values() for the RDD class.
18b9be1 [Shivaram Venkataraman] Allow users to set where SparkR is installed This also adds a warning if somebody tries to call sparkR.init multiple times.
a17f135 [hlin09] Adds tests for flatMap and flatMapValues.
4bcf59b [hlin09] Adds function flatMapValues.
4a193ef [Zongheng Yang] Merge pull request #101 from ashutoshraina/master
60d22f2 [Ashutosh Raina] changed sbt version
5400793 [Zongheng Yang] Merge pull request #98 from shivaram/windows-fixes-build
36d61a7 [Shivaram Venkataraman] Merge pull request #97 from hlin09/hlin09
f7d7d89 [hlin09] Remove redundant code in test.
6bbe823 [hlin09] minor style fix.
9b47f3a [Shivaram Venkataraman] Merge pull request #100 from hnahak87/patch-1
7f6e4ea [Harihar Nahak] Update logistic_regression.R
a605047 [Shivaram Venkataraman] Merge pull request #99 from hlin09/makefile
323151d [hlin09] Fix yar flag in Makefile to remove build error in Maven.
8911897 [hlin09] Make reserialize() private function in package.
79aee73 [Shivaram Venkataraman] Add notes on how to build SparkR on windows
49a99e7 [Shivaram Venkataraman] Clean up some commented code
ddc271b [Shivaram Venkataraman] Only append file:/// to non empty jar paths
a53952e [Shivaram Venkataraman] Add windows build scripts
325b179 [hlin09] Merge remote-tracking branch 'upstream/master' into hlin09
daf5040 [hlin09] Add reserialize() before union if two RDDs are not both serialized.
536afb1 [hlin09] Add new function of union().
7044677 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into windows-fixes
d22a02d [Zongheng Yang] Merge pull request #94 from shivaram/windows-fixes-stdin
51924f7 [Shivaram Venkataraman] Merge pull request #90 from oscaroboto/master
eb97d85 [Shivaram Venkataraman] Merge pull request #96 from sun-rui/add_clarification_readme
5a128f4 [Sun Rui] Add clarification on setting Spark master when launching the SparkR shell.
187526a [oscaroboto] Update sparkR.R
32c567b [Shivaram Venkataraman] Merge pull request #95 from concretevitamin/master
4cd2d5e [Zongheng Yang] Notes about spark-ec2.
1c28e3b [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into windows-fixes
8e8a029 [Zongheng Yang] Merge pull request #92 from shivaram/sparkr-yarn
721043b [Zongheng Yang] Update README.md with YARN instructions.
1681f58 [Shivaram Venkataraman] Use temporary files for input instead of stdin This fixes a bug for Windows where stdin would get truncated
b084314 [oscaroboto] removed ... from example
44c93d4 [oscaroboto] Added example to SparkR.R
be82dcc [Shivaram Venkataraman] Merge pull request #93 from hlin09/hlin09
868554d [oscaroboto] Update sparkR.R
488ac47 [hlin09] Add generated Rd file of previous added functions, distinct() and mapValues().
b2740ad [hlin09] Add test for filter all elements. Add filter() as alias.
08d3631 [hlin09] Minor style fixes.
2c0e34f [hlin09] Adds function Filter(), which extracts the elements that satisfy a predicate.
5951d3b [Shivaram Venkataraman] Remove SBT plugin
4e70ced [oscaroboto] changed ExecutorEnv to sparkExecutorEnvMap, to make it consistent with sparkEnvirMap
903d18a [oscaroboto] changed executorEnv to sparkExecutorEnvMap,  will do the same in R
f97346e [oscaroboto] executorEnv to lower-case e
88a524e [oscaroboto] Added LD_LIBRARY_PATH to the ExecutorEnv. This is need so that the nodes can find libjvm.so, or if the master has a different LD_LIBRARY_PATH then the nodes. Make sure to export LD_LIBRARY_PATH  that includes the path to libjvm.so in the nodes.
1d208ae [oscaroboto] added the YARN_CONF_DIR to the classpath
8a9b75c [oscaroboto] forgot to change hm and ee inside the for loops
579db58 [Shivaram Venkataraman] Merge pull request #91 from sun-rui/add_max_min
4381efa [Sun Rui] use reduce() to implemement max() and min().
a5459c5 [Shivaram Venkataraman] Consolidate yarn flags
86b04eb [Shivaram Venkataraman] Don't use quotes around yarn
bf0797f [Shivaram Venkataraman] Add dependency on spark yarn module
af5fe77 [Shivaram Venkataraman] Fix SBT build, add dependency tree plugin
4917607 [Sun Rui] Add maximum() and minimum() API to RDD.
51bbbe4 [Shivaram Venkataraman] Changes to make SparkR work with YARN
9d5e3ab [oscaroboto] a few stylistic changes. Also change vars to sparkEnvirMap and eevars to ExecutorEnv, to match sparkR.R
578f545 [oscaroboto] a few stylistic changes
39eea2f [oscaroboto] Modification to dynamically create a sparkContext with YARN. Added .setExecutorEnv to the sparkConf in createSparkContext within the RRDD object. This modification was made together with sparkR.R
17ec42e [oscaroboto] A modification to dynamically create a sparkContext with YARN. sparkR.R modified to pass custom Jar file names and EnvironmentEnv to the sparkConf. RRDD.scala was also modified to accept the new inputs to creatSparkContext.
624ac9d [Shivaram Venkataraman] Merge pull request #87 from sun-rui/SPARKR-125
4f213db [Shivaram Venkataraman] Merge pull request #89 from sun-rui/SPARKR-108
eb833c5 [Shivaram Venkataraman] Merge pull request #88 from hlin09/hlin09
07bf971 [Sun Rui] [SPARKR-108] Implement map-side reduction for reduceByKey().
4accba1 [hlin09] Fixes style and adds an optional param 'numPartition' in distinct().
80d303a [hlin09] typo fixed.
e37a9b5 [hlin09] Adds function distinct() and mapValues().
08dac06 [Sun Rui] [SPARKR-125] Get the iterator of the parent RDD before launching a R worker process in compute() of RRDD/PairwiseRRDD
c4ba53c [Shivaram Venkataraman] Merge pull request #85 from edwardt/master
72a9d27 [root] reorder to keep relative ordering the same
f3fcb10 [root] fix up build.sbt also to match pom.xml
5ecbe3e [root] Make spark verison configurable in build script per ISSUE122
a44e63d [Shivaram Venkataraman] Merge pull request #84 from sun-rui/SPARKR-94
fbb5663 [Sun Rui] Add {} to one-line functions and add a test case for lookup where no match is found.
95beb4e [Shivaram Venkataraman] Merge pull request #82 from edwardt/master
36776c5 [edwardt] missed one 0.9.0 revert
b26deec [Sun Rui] [SPARKR-94] Add a  method to get an element of a pair RDD object by key.
1ba256e [edwardt] Keep 0.9.0 and says uses 1.1.0 by default
5380c43 [root] missed one version
21f74da [root] upgrade to spark version 1.1.0 to match lastest merge list
ddfcde9 [root] merge
67d067a [Shivaram Venkataraman] Merge pull request #81 from sun-rui/SparkR-117
993868f [Sun Rui] [SPARKR-117] Update Spark dependency to 1.1.0
d20661a [Zongheng Yang] Merge pull request #80 from sun-rui/master
0b2da9f [Sun Rui] Update Rd file and add a test case for mapPartitions.
5879648 [Sun Rui] Add mapPartitions() method to RDD for API consistency.
c033461 [Shivaram Venkataraman] Merge pull request #79 from sun-rui/fix-kmeans
f62b77e [Sun Rui] Adjust coding style.
b40911d [Sun Rui] Fix syntax error in examples/kmeans.R.
5304451 [Shivaram Venkataraman] Merge pull request #78 from sun-rui/master
70ffbfb [Sun Rui] Fix a bug that modifications to build.sbt won't trigger rebuilding.
a25696c [Shivaram Venkataraman] Merge pull request #76 from edwardt/addjira
b8bbd93 [edwardt] Update README.md
615d930 [edwardt] Update README.md
e522e69 [edwardt] Update README.md
03e6ced [edwardt] Update README.md
3007015 [root] don't check in gedit buffer file'
c35c9a6 [root] Add where to enter bugs ad feeback
469eae3 [edwardt] Update README.md
61b4a43 [edwardt] Update Makefile (style uniformity)
ce3337d [edwardt] Update README.md
7ff68fc [root] Merge branch 'master' of https://github.com/edwardt/SparkR-pkg
16353f5 [root] add links to devtools and install_github
513b9e5 [Shivaram Venkataraman] Merge pull request #72 from edwardt/master
31608a4 [edwardt] Update Makefile (style uniformity)
4ffe146 [root] Makefile: factor out SPARKR_VERSION to reduce potential copy&paste error; cp & rm called with -f in build/clean phase; .gitignore includes checkpoints and unit test log generated by run-tests.sh
715275f [Zongheng Yang] Merge pull request #68 from shivaram/master
90e2083 [Shivaram Venkataraman] Add return type to hasNext
8eb983d [Shivaram Venkataraman] Fix up comment
2206164 [Shivaram Venkataraman] Delete temporary files after they are read This change deletes temporary files used for communication between Rscript and the JVM once they have been completely read.
5881da7 [Zongheng Yang] Merge pull request #67 from shivaram/improve-shuffle
81251e2 [Shivaram Venkataraman] Address code review comments
a5f573f [Shivaram Venkataraman] Use a better list append in shuffles This is helpful in scenarios where we have a large number of values in a bucket
388e64d [Shivaram Venkataraman] Merge pull request #55 from RevolutionAnalytics/master
e1f95b6 [Zongheng Yang] Merge pull request #65 from concretevitamin/parallelize-fix
fc1a71a [Zongheng Yang] Fix that collect(parallelize(sc,1:72,15)) drops elements.
b8204c5 [Zongheng Yang] Minor: update a URL in README.
86f30c3 [Antonio Piccolboni] better fix for amplab-extras/SparkR-pkg#53
b3c318d [Antonio Piccolboni] delayed loading to have all namespaces available.
f323e97 [Antonio Piccolboni] tentative fix for amplab-extras/SparkR-pkg#53
6f82269 [Zongheng Yang] Merge pull request #48 from shivaram/master
8f433e5 [Shivaram Venkataraman] Move up Hadoop in pom.xml and add back protobufs As Hadoop 1.0.4 doesn't use protobufs, we can't exclude protobufs from Spark always. This change tries to order the dependencies so that the shader first picks up Hadoop's protobufs over Mesos.
bfe7e26 [Shivaram Venkataraman] Merge pull request #36 from RevolutionAnalytics/vectorize-examples
059ae41 [Antonio Piccolboni] and more formatting
9dbd531 [Antonio Piccolboni] more formatting per committer request
948738a [Antonio Piccolboni] converted tabs to spaces per project request
49f5f5a [Shivaram Venkataraman] Merge pull request #35 from shivaram/master
3eb5ad3 [Shivaram Venkataraman] on_failure -> after_failure in travis.yml
139bdee [Shivaram Venkataraman] Cache sbt, maven, ivy dependencies
4ebced2 [Shivaram Venkataraman] Merge pull request #34 from shivaram/master
8437061 [Shivaram Venkataraman] Exclude protobuf from Spark dependency in Maven This avoids pulling in multiple versions of protobuf from Mesos and Hadoop.
91aa527 [Antonio Piccolboni] vectorized version, 36s 10 slices 10^6 per slice. The older version takes 30 sec on 1/10th of data.
f137a57 [Antonio Piccolboni] for rstudio users
1f7ffb0 [Antonio Piccolboni] implemented using matrices and vectorized calls wherever possible
46b23df [Antonio Piccolboni] replace require with library
b15d7db [Antonio Piccolboni] faster parsing
8b7aeb3 [Antonio Piccolboni] 22x speed improvement, 3X mem impovement
c5bce07 [Zongheng Yang] Merge pull request #30 from shivaram/string-tests
21fa2d8 [Shivaram Venkataraman] Fix bug where serialized was not changed for RRRD Reason: When an RRDD is created in getJRDD we have converted any possibly unserialized RDD to a serialized RDD.
9d1ea20 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg into string-tests
7b9348c [Shivaram Venkataraman] Add tests for partition with string keys Add two tests one with a string array and one from a textFile to test both codepaths
aacd726 [Shivaram Venkataraman] Update README with maven proxy instructions
803e62c [Shivaram Venkataraman] Merge pull request #28 from concretevitamin/master
7c093e6 [Zongheng Yang] Use inherits() to test an object's class.
061c591 [Shivaram Venkataraman] Merge pull request #26 from hafen/master
90f9fda [Ryan Hafen] Fix isRdd() to properly check for class
5b10cc7 [Zongheng Yang] Merge pull request #24 from shivaram/master
7014f83 [Shivaram Venkataraman] Remove unused transformers in maven's pom.xml
b00cea5 [Shivaram Venkataraman] Add support for a Maven build
11ec9b2 [Shivaram Venkataraman] Merge pull request #12 from concretevitamin/pipelined
6b18a90 [Zongheng Yang] Merge branch 'master' into pipelined
57127b8 [Zongheng Yang] Merge pull request #23 from shivaram/master
1ac3940 [Zongheng Yang] Review feedback.
a06fb34 [Zongheng Yang] Remove outdated comment.
0a1fc13 [Shivaram Venkataraman] Fixes for using SparkR with Hadoop2. 1. Exclude ASM, Netty from Hadoop similar to Spark. 2. Concat services files to ensure HDFS filesystems work. 3. Update README with an example
9a1db44 [Zongheng Yang] Merge pull request #22 from shivaram/master
e462448 [Shivaram Venkataraman] Use `$` for calling `put` instead of .jrcall
ed4559a [Shivaram Venkataraman] Add support for passing Spark environment vars This change creates a new `createSparkContext` method in RRDD as we can't pass Map<String, String> through rJava. Also use SPARK_MEM in local mode to increase heap size and update the README with some examples.
10228fb [Shivaram Venkataraman] Merge pull request #20 from concretevitamin/digit-ex
1398d9f [Zongheng Yang] Add linear_solver_mnist to examples/.
d484c2a [Zongheng Yang] Add tests for actions on PipelinedRDD.
d9cb95c [Zongheng Yang] Add setCheckpointDir() to context.R; comment fix.
f8bc8a9 [Zongheng Yang] Minor edits per Shivaram's comments.
8cd67f7 [Shivaram Venkataraman] Merge pull request #15 from shivaram/master
d4468a9 [Shivaram Venkataraman] Remove trailing comma
e2714b8 [Shivaram Venkataraman] Remove Apache Staging repo and update README
334eace [Zongheng Yang] Add a multi-transformation test to benchmark on pipelining.
5650ad7 [Zongheng Yang] Put serialized field inside env for both RDD and PipelinedRDD.
0b9e8bb [Zongheng Yang] First cut at PipelinedRDD.
a4c431e [Zongheng Yang] Add `isCheckpointed` field and checkpoint().
dac0795 [Zongheng Yang] Minor inline comment style fix.
bfb8e26 [Zongheng Yang] Add isCached field (inside an env) and unpersist().
295bff6 [Zongheng Yang] Merge pull request #11 from shivaram/master
4cb209c [Shivaram Venkataraman] Search rLibDir in worker before libPaths This ensures we pick up the SparkR intended and not an older version installed on the same machine
ef198ff [Zongheng Yang] Merge pull request #10 from shivaram/unit-tests
e0557a8 [Shivaram Venkataraman] Update travis to install plyr
8b18bc1 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg into unit-tests
4a9ca31 [Shivaram Venkataraman] Use smaller broadcast and plyr instead of Matrix Matrix package takes around 2s to load and slows down unit tests.
21c6a61 [Zongheng Yang] Merge pull request #8 from shivaram/master
08c2947 [Shivaram Venkataraman] Move dev install directory to front of libPaths
bda42ee [Shivaram Venkataraman] Merge pull request #7 from JoshRosen/travis
cc5f5c0 [Josh Rosen] Add Travis CI integration (using craigcitro/r-travis)
b6c864b [Shivaram Venkataraman] Merge pull request #6 from concretevitamin/env-style-fix
4fcef22 [Zongheng Yang] Use one style ($) for accessing names in environments.
8a948c6 [Shivaram Venkataraman] Merge pull request #4 from shivaram/master
24978eb [Shivaram Venkataraman] Update README to use install_github
8899db4 [Shivaram Venkataraman] Update TODO.md
91792de [Shivaram Venkataraman] Update Spark requirements
f34f4bf [Shivaram Venkataraman] Check tests for failures and output error msg
cd750d3 [Shivaram Venkataraman] Update run-tests to use new path
1877b7c [Shivaram Venkataraman] Unset R_TESTS to make tests work with R CMD check Also silence Akka remoting logs and update Makefile to build on log4j changes
e60e18a [Shivaram Venkataraman] Update README to remove Spark installation notes
4450189 [Shivaram Venkataraman] Add Spark 0.9 dependency from Apache Staging Also clean up assembly jar from inst on make clean
5eb2131 [Shivaram Venkataraman] Update repo path in README
ec8210e [Shivaram Venkataraman] Remove broadcastId hack as it is public in Spark
9f0e080 [Shivaram Venkataraman] Merge branch 'install-github'
5c88fbd [Shivaram Venkataraman] Add helper script to run tests
77450a1 [Shivaram Venkataraman] Remove dependency on Spark Logging
6cb00d1 [Shivaram Venkataraman] Update README and add helper script install-dev.sh
28346ca [Shivaram Venkataraman] Only normalize if SPARK_HOME is not empty
0fd6571 [Shivaram Venkataraman] Normalize SPARK_HOME before passing it
ff96d5c [Shivaram Venkataraman] Pass in SPARK_HOME and jar file path
34c4dce [Shivaram Venkataraman] Move src into pkg and update Makefile This enables the package to be installed using install_github using devtools and automates the build procedure.
b25afed [Shivaram Venkataraman] Change package name to edu.berkeley.cs.amplab
c691464 [Shivaram Venkataraman] Add Apache 2.0 License file
27a4a4b [Shivaram Venkataraman] Add notes on how to compile roxygen2 docs
ca63844 [Shivaram Venkataraman] Add broadcast documentation Also generate documentation for sample, takeSample etc.
e4dd976 [Shivaram Venkataraman] Update TODO.md
e42d435 [Shivaram Venkataraman] Add support for broadcast variables
6b638e7 [Shivaram Venkataraman] Add the assembly jar to SparkContext
bf24e32 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg
43c05ce [Zongheng Yang] Fix a flaky/incorrect test for sampleRDD().
c6a9dfc [Zongheng Yang] Initial port of the kmeans example.
6885581 [Zongheng Yang] Implement element-level sampleRDD() and takeSample() with tests.
d3a4987 [Zongheng Yang] Add a test for lapplyPartitionsWithIndex on pairwise RDD.
c7899c1 [Zongheng Yang] Add lapplyPartitionsWithIndex, with a test and an alias function.
a9a7436 [Shivaram Venkataraman] Add DFC example from Tselil, Benjamin and Jonah
fbc5a95 [Zongheng Yang] Implement take() and takeSample().
c4a3409 [Shivaram Venkataraman] Use RDD instead of RRDD
dfad3f5 [Zongheng Yang] Add test_utils.R: a unit test for convertJListToRList().
a45227d [Zongheng Yang] Update .gitignore.
238fe6e [Zongheng Yang] Add a unit test for textFile().
a88898b [Zongheng Yang] Rename test_rrd to test_rrdd
10c8baa [Shivaram Venkataraman] Make SparkR work as a standalone package. Changes include: 1. Adding a new `sbt` project that builds RRDD.scala 2. Change the onLoad functions to load the assembly jar for SparkR 3. Set rLibDir in RRDD.scala and worker.R to load things correctly
78adcd8 [Shivaram Venkataraman] Add a gitignore
ca6108f [Shivaram Venkataraman] Merge branch 'SparkR-scalacode' of ../SparkR
999bd61 [Shivaram Venkataraman] Update collectPartition in R and use ClassTag
c58f63e [Shivaram Venkataraman] Update collectPartition in R and use ClassTag
48265fd [Shivaram Venkataraman] Use new version of collectPartitions in take
d4fe086 [Shivaram Venkataraman] Move collectPartitions to JavaRDDLike Also remove numPartitions in JavaRDD and update R code
bfecd7b [Shivaram Venkataraman] Scala 2.10 changes 1. Update sparkR script 2. Use classTag instead of classManifest
092a4b3 [Shivaram Venkataraman] Add combineByKey, update TODO
ac0d81d [Shivaram Venkataraman] Add more documentation
d1dc3fa [Shivaram Venkataraman] Add more documentation
c515e3a [Shivaram Venkataraman] Update TODO
db56a34 [Shivaram Venkataraman] Add a test case for include package
41cea51 [Shivaram Venkataraman] Ensure all parent environments are serialized. Also add a test case with an inline function
a978e84 [Shivaram Venkataraman] Add support to include packages in the worker
12bf8ce [Shivaram Venkataraman] Add support to include packages in the worker
fb7e72c [Shivaram Venkataraman] Cleanup TODO
16ac314 [Shivaram Venkataraman] Add documentation for functions in context, sparkR
85b1d25 [Shivaram Venkataraman] Set license to Apache
88f1101 [Shivaram Venkataraman] Add unit test running instructions
c40768e [Shivaram Venkataraman] Update TODO
0c7efbf [Shivaram Venkataraman] Refactor RRDD.scala and add comments to functions
5880d42 [Shivaram Venkataraman] Refactor RRDD.scala and add comments to functions
2dee36c [Shivaram Venkataraman] Remove empty test file
a82219b [Shivaram Venkataraman] Update TODOs
5db00dc [Shivaram Venkataraman] Add reduceByKey, groupByKey and refactor shuffle Other changes include 1. Adding unit tests for basic RDD functions and shuffle 2. Add a word count example 3. Change the dependency serialization to handle double loading of SparkR    package 4. Allow partitionBy to operate on any RDDs to create pair-wise RDD.
f196479 [Shivaram Venkataraman] Add reduceByKey, groupByKey and refactor shuffle Other changes include 1. Adding unit tests for basic RDD functions and shuffle 2. Add a word count example 3. Change the dependency serialization to handle double loading of SparkR    package 4. Allow partitionBy to operate on any RDDs to create pair-wise RDD.
987e36f [Shivaram Venkataraman] Add perf todo
0b03265 [Shivaram Venkataraman] Update TODO with testing, docs todo
685aaad [Zongheng Yang] First cut at refactoring worker.R. Remove pairwiseWorker.R.
95b9ddc [Zongheng Yang] First cut at refactoring worker.R. Remove pairwiseWorker.R.
4f00895 [Zongheng Yang] Remove the unnecessary `pairwise' flag in RRDD class. Reasons:
75d36d9 [Zongheng Yang] Working versions: partitionBy() and collectPartition() for RRDD.
e3fbd9d [Zongheng Yang] Working versions: partitionBy() and collectPartition() for RRDD.
67a4335 [Zongheng Yang] Add unit test for parallelize() and collect() pairwise data.
100ae65 [Zongheng Yang] Properly parallelize() and collect() pairwise data.
cd0a5e2 [Zongheng Yang] Properly parallelize() and collect() pairwise data.
aea16c3 [Zongheng Yang] WIP: second cut at partitionBy. Running into R/Scala communication issues.
45eb943 [Zongheng Yang] WIP: second cut at partitionBy. Running into R/Scala communication issues.
11c893b [Zongheng Yang] WIP: need to figure out the logic of (whether or not) shipping a hash func
82c201a [Zongheng Yang] WIP: need to figure out the logic of (whether or not) shipping a hash func
b3bfad2 [Zongheng Yang] Update TODO: take() done.
0e45293 [Zongheng Yang] Add ability to parallelize key-val collections in R.
f60406a [Zongheng Yang] Add ability to parallelize key-val collections in R.
7d7fe3b [Zongheng Yang] Re-implement take(): take a partition at a time and append.
a054e55 [Zongheng Yang] Fix take() tests(): mode difference.
9de0935 [Zongheng Yang] Implement take() for RRDD.
1e4427e [Zongheng Yang] Implement take() for RRDD.
ec3cd67 [Shivaram Venkataraman] Use temp file in Spark to pipe output
417aaed [Shivaram Venkataraman] Use temp file in Spark to pipe output
bb0a3c3 [Shivaram Venkataraman] Add conf directory to classpath
9594d8a [Shivaram Venkataraman] Clean up LR example
3b26b58 [Shivaram Venkataraman] Add a list of things to do.
cabce68 [Shivaram Venkataraman] Fix warnings from package check
fde3f9c [Shivaram Venkataraman] Flatten by default and disable recursive unlist
ab2e061 [Shivaram Venkataraman] Create LIB_DIR before installing SparkR package
555220a [Shivaram Venkataraman] Add readme and update Makefile
1319cda [Shivaram Venkataraman] Make standalone programs run with sparkR
ae19fa8 [Shivaram Venkataraman] Add support for cache and use `tempfile`
4e89ca4 [Shivaram Venkataraman] Add support for apply, reduce, count Also serialize closures using `save` and add two examples
25a0bea [Shivaram Venkataraman] Add support for apply, reduce, count Also serialize closures using `save` and add two examples
f50223f [Zongheng Yang] Make parallelize() and collect() use lists. Add a few more tests for them.
fc7693f [Zongheng Yang] Refactor and enhance the previously added unit test a little bit.
6de9b81 [Zongheng Yang] Add a simple unit test for parallelize().
8b95155 [Zongheng Yang] Add testthat skeleton infrastructure
ef305bf [Zongheng Yang] parallelize() followed by collect() now work for vectors/lists of strings and numerics (should work for other primitives as well).
dc16af4 [Zongheng Yang] Comment: toArray() allocates memory for a copy
f50121e [Zongheng Yang] Make parallelize() return JavaRDD[Array[Byte]]. Add RRDD.scala with a helper function in the singleton object.
46eb063 [Zongheng Yang] Make parallelize() return JavaRDD[Array[Byte]]. Add RRDD.scala with a helper function in the singleton object.
6b4938a [Zongheng Yang] parallelize(): a raw can be parallelized by JavaSparkContext and get back JavaRDD
978aa0f [Zongheng Yang] Add parallelize() skeleton: only return serialized slices now
84c1fd2 [Zongheng Yang] Use .jsimplify() to get around generic List's get() type erasure problem
f16b891 [Zongheng Yang] Convert a few reflectionc alls to .jcall
1284c13 [Zongheng Yang] WIP on collect(): JavaListToRList() failed with errors.
4c2e516 [Zongheng Yang] Add simple prototype of S4 class RRDD. Make TextFile() returns an RRDD.
82aa17a [Zongheng Yang] Add textFile()
83ce63f [Zongheng Yang] Create a JavaSparkContext and save it in .sparkEnv using sparkR.init()
01cdf0e [Zongheng Yang] Add Makefile for SparkR
fc9cae2 [Shivaram Venkataraman] Add skeleton R package
This commit is contained in:
Shivaram Venkataraman 2015-04-08 22:45:40 -07:00
parent 1b2aab8d5b
commit 2fe0a1aaee
83 changed files with 12044 additions and 56 deletions

2
.gitignore vendored
View file

@ -63,6 +63,8 @@ ec2/lib/
rat-results.txt
scalastyle.txt
scalastyle-output.xml
R-unit-tests.log
R/unit-tests.out
# For Hive
metastore_db/

View file

@ -67,3 +67,5 @@ logs
.*scalastyle-output.xml
.*dependency-reduced-pom.xml
known_translations
DESCRIPTION
NAMESPACE

6
R/.gitignore vendored Normal file
View file

@ -0,0 +1,6 @@
*.o
*.so
*.Rd
lib
pkg/man
pkg/html

12
R/DOCUMENTATION.md Normal file
View file

@ -0,0 +1,12 @@
# SparkR Documentation
SparkR documentation is generated using in-source comments annotated using using
`roxygen2`. After making changes to the documentation, to generate man pages,
you can run the following from an R console in the SparkR home directory
library(devtools)
devtools::document(pkg="./pkg", roclets=c("rd"))
You can verify if your changes are good by running
R CMD check pkg/

67
R/README.md Normal file
View file

@ -0,0 +1,67 @@
# R on Spark
SparkR is an R package that provides a light-weight frontend to use Spark from R.
### SparkR development
#### Build Spark
Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-PsparkR` profile to build the R package. For example to use the default Hadoop versions you can run
```
build/mvn -DskipTests -Psparkr package
```
#### Running sparkR
You can start using SparkR by launching the SparkR shell with
./bin/sparkR
The `sparkR` script automatically creates a SparkContext with Spark by default in
local mode. To specify the Spark master of a cluster for the automatically created
SparkContext, you can run
./bin/sparkR --master "local[2]"
To set other options like driver memory, executor memory etc. you can pass in the [spark-submit](http://spark.apache.org/docs/latest/submitting-applications.html) arguments to `./bin/sparkR`
#### Using SparkR from RStudio
If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example
```
# Set this to where Spark is installed
Sys.setenv(SPARK_HOME="/Users/shivaram/spark")
# This line loads SparkR from the installed directory
.libPaths(c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"), .libPaths()))
library(SparkR)
sc <- sparkR.init(master="local")
```
#### Making changes to SparkR
The [instructions](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) for making contributions to Spark also apply to SparkR.
If you only make R file changes (i.e. no Scala changes) then you can just re-install the R package using `R/install-dev.sh` and test your changes.
Once you have made your changes, please include unit tests for them and run existing unit tests using the `run-tests.sh` script as described below.
#### Generating documentation
The SparkR documentation (Rd files and HTML files) are not a part of the source repository. To generate them you can run the script `R/create-docs.sh`. This script uses `devtools` and `knitr` to generate the docs and these packages need to be installed on the machine before using the script.
### Examples, Unit tests
SparkR comes with several sample programs in the `examples/src/main/r` directory.
To run one of them, use `./bin/sparkR <filename> <args>`. For example:
./bin/sparkR examples/src/main/r/pi.R local[2]
You can also run the unit-tests for SparkR by running (you need to install the [testthat](http://cran.r-project.org/web/packages/testthat/index.html) package first):
R -e 'install.packages("testthat", repos="http://cran.us.r-project.org")'
./R/run-tests.sh
### Running on YARN
The `./bin/spark-submit` and `./bin/sparkR` can also be used to submit jobs to YARN clusters. You will need to set YARN conf dir before doing so. For example on CDH you can run
```
export YARN_CONF_DIR=/etc/hadoop/conf
./bin/spark-submit --master yarn examples/src/main/r/pi.R 4
```

13
R/WINDOWS.md Normal file
View file

@ -0,0 +1,13 @@
## Building SparkR on Windows
To build SparkR on Windows, the following steps are required
1. Install R (>= 3.1) and [Rtools](http://cran.r-project.org/bin/windows/Rtools/). Make sure to
include Rtools and R in `PATH`.
2. Install
[JDK7](http://www.oracle.com/technetwork/java/javase/downloads/jdk7-downloads-1880260.html) and set
`JAVA_HOME` in the system environment variables.
3. Download and install [Maven](http://maven.apache.org/download.html). Also include the `bin`
directory in Maven in `PATH`.
4. Set `MAVEN_OPTS` as described in [Building Spark](http://spark.apache.org/docs/latest/building-spark.html).
5. Open a command shell (`cmd`) in the Spark directory and run `mvn -DskipTests -Psparkr package`

46
R/create-docs.sh Executable file
View file

@ -0,0 +1,46 @@
#!/bin/bash
#
# 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.
#
# Script to create API docs for SparkR
# This requires `devtools` and `knitr` to be installed on the machine.
# After running this script the html docs can be found in
# $SPARK_HOME/R/pkg/html
# Figure out where the script is
export FWDIR="$(cd "`dirname "$0"`"; pwd)"
pushd $FWDIR
# Generate Rd file
Rscript -e 'library(devtools); devtools::document(pkg="./pkg", roclets=c("rd"))'
# Install the package
./install-dev.sh
# Now create HTML files
# knit_rd puts html in current working directory
mkdir -p pkg/html
pushd pkg/html
Rscript -e 'library(SparkR, lib.loc="../../lib"); library(knitr); knit_rd("SparkR")'
popd
popd

27
R/install-dev.bat Normal file
View file

@ -0,0 +1,27 @@
@echo off
rem
rem Licensed to the Apache Software Foundation (ASF) under one or more
rem contributor license agreements. See the NOTICE file distributed with
rem this work for additional information regarding copyright ownership.
rem The ASF licenses this file to You under the Apache License, Version 2.0
rem (the "License"); you may not use this file except in compliance with
rem the License. You may obtain a copy of the License at
rem
rem http://www.apache.org/licenses/LICENSE-2.0
rem
rem Unless required by applicable law or agreed to in writing, software
rem distributed under the License is distributed on an "AS IS" BASIS,
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
rem Install development version of SparkR
rem
set SPARK_HOME=%~dp0..
MKDIR %SPARK_HOME%\R\lib
R.exe CMD INSTALL --library="%SPARK_HOME%\R\lib" %SPARK_HOME%\R\pkg\

36
R/install-dev.sh Executable file
View file

@ -0,0 +1,36 @@
#!/bin/bash
#
# 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.
#
# This scripts packages the SparkR source files (R and C files) and
# creates a package that can be loaded in R. The package is by default installed to
# $FWDIR/lib and the package can be loaded by using the following command in R:
#
# library(SparkR, lib.loc="$FWDIR/lib")
#
# NOTE(shivaram): Right now we use $SPARK_HOME/R/lib to be the installation directory
# to load the SparkR package on the worker nodes.
FWDIR="$(cd `dirname $0`; pwd)"
LIB_DIR="$FWDIR/lib"
mkdir -p $LIB_DIR
# Install R
R CMD INSTALL --library=$LIB_DIR $FWDIR/pkg/

28
R/log4j.properties Normal file
View file

@ -0,0 +1,28 @@
#
# 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.
#
# Set everything to be logged to the file target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=true
log4j.appender.file.file=R-unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
# Ignore messages below warning level from Jetty, because it's a bit verbose
log4j.logger.org.eclipse.jetty=WARN
org.eclipse.jetty.LEVEL=WARN

35
R/pkg/DESCRIPTION Normal file
View file

@ -0,0 +1,35 @@
Package: SparkR
Type: Package
Title: R frontend for Spark
Version: 1.4.0
Date: 2013-09-09
Author: The Apache Software Foundation
Maintainer: Shivaram Venkataraman <shivaram@cs.berkeley.edu>
Imports:
methods
Depends:
R (>= 3.0),
methods,
Suggests:
testthat
Description: R frontend for Spark
License: Apache License (== 2.0)
Collate:
'generics.R'
'jobj.R'
'SQLTypes.R'
'RDD.R'
'pairRDD.R'
'column.R'
'group.R'
'DataFrame.R'
'SQLContext.R'
'broadcast.R'
'context.R'
'deserialize.R'
'serialize.R'
'sparkR.R'
'backend.R'
'client.R'
'utils.R'
'zzz.R'

182
R/pkg/NAMESPACE Normal file
View file

@ -0,0 +1,182 @@
#exportPattern("^[[:alpha:]]+")
exportClasses("RDD")
exportClasses("Broadcast")
exportMethods(
"aggregateByKey",
"aggregateRDD",
"cache",
"checkpoint",
"coalesce",
"cogroup",
"collect",
"collectAsMap",
"collectPartition",
"combineByKey",
"count",
"countByKey",
"countByValue",
"distinct",
"Filter",
"filterRDD",
"first",
"flatMap",
"flatMapValues",
"fold",
"foldByKey",
"foreach",
"foreachPartition",
"fullOuterJoin",
"glom",
"groupByKey",
"join",
"keyBy",
"keys",
"length",
"lapply",
"lapplyPartition",
"lapplyPartitionsWithIndex",
"leftOuterJoin",
"lookup",
"map",
"mapPartitions",
"mapPartitionsWithIndex",
"mapValues",
"maximum",
"minimum",
"numPartitions",
"partitionBy",
"persist",
"pipeRDD",
"reduce",
"reduceByKey",
"reduceByKeyLocally",
"repartition",
"rightOuterJoin",
"sampleRDD",
"saveAsTextFile",
"saveAsObjectFile",
"sortBy",
"sortByKey",
"sumRDD",
"take",
"takeOrdered",
"takeSample",
"top",
"unionRDD",
"unpersist",
"value",
"values",
"zipRDD",
"zipWithIndex",
"zipWithUniqueId"
)
# S3 methods exported
export(
"textFile",
"objectFile",
"parallelize",
"hashCode",
"includePackage",
"broadcast",
"setBroadcastValue",
"setCheckpointDir"
)
export("sparkR.init")
export("sparkR.stop")
export("print.jobj")
useDynLib(SparkR, stringHashCode)
importFrom(methods, setGeneric, setMethod, setOldClass)
# SparkRSQL
exportClasses("DataFrame")
exportMethods("columns",
"distinct",
"dtypes",
"explain",
"filter",
"groupBy",
"head",
"insertInto",
"intersect",
"isLocal",
"limit",
"orderBy",
"names",
"printSchema",
"registerTempTable",
"repartition",
"sampleDF",
"saveAsParquetFile",
"saveAsTable",
"saveDF",
"schema",
"select",
"selectExpr",
"show",
"showDF",
"sortDF",
"subtract",
"toJSON",
"toRDD",
"unionAll",
"where",
"withColumn",
"withColumnRenamed")
exportClasses("Column")
exportMethods("abs",
"alias",
"approxCountDistinct",
"asc",
"avg",
"cast",
"contains",
"countDistinct",
"desc",
"endsWith",
"getField",
"getItem",
"isNotNull",
"isNull",
"last",
"like",
"lower",
"max",
"mean",
"min",
"rlike",
"sqrt",
"startsWith",
"substr",
"sum",
"sumDistinct",
"upper")
exportClasses("GroupedData")
exportMethods("agg")
export("sparkRSQL.init",
"sparkRHive.init")
export("cacheTable",
"clearCache",
"createDataFrame",
"createExternalTable",
"dropTempTable",
"jsonFile",
"jsonRDD",
"loadDF",
"parquetFile",
"sql",
"table",
"tableNames",
"tables",
"toDF",
"uncacheTable")
export("print.structType",
"print.structField")

1270
R/pkg/R/DataFrame.R Normal file

File diff suppressed because it is too large Load diff

1539
R/pkg/R/RDD.R Normal file

File diff suppressed because it is too large Load diff

520
R/pkg/R/SQLContext.R Normal file
View file

@ -0,0 +1,520 @@
#
# 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.
#
# SQLcontext.R: SQLContext-driven functions
#' infer the SQL type
infer_type <- function(x) {
if (is.null(x)) {
stop("can not infer type from NULL")
}
# class of POSIXlt is c("POSIXlt" "POSIXt")
type <- switch(class(x)[[1]],
integer = "integer",
character = "string",
logical = "boolean",
double = "double",
numeric = "double",
raw = "binary",
list = "array",
environment = "map",
Date = "date",
POSIXlt = "timestamp",
POSIXct = "timestamp",
stop(paste("Unsupported type for DataFrame:", class(x))))
if (type == "map") {
stopifnot(length(x) > 0)
key <- ls(x)[[1]]
list(type = "map",
keyType = "string",
valueType = infer_type(get(key, x)),
valueContainsNull = TRUE)
} else if (type == "array") {
stopifnot(length(x) > 0)
names <- names(x)
if (is.null(names)) {
list(type = "array", elementType = infer_type(x[[1]]), containsNull = TRUE)
} else {
# StructType
types <- lapply(x, infer_type)
fields <- lapply(1:length(x), function(i) {
list(name = names[[i]], type = types[[i]], nullable = TRUE)
})
list(type = "struct", fields = fields)
}
} else if (length(x) > 1) {
list(type = "array", elementType = type, containsNull = TRUE)
} else {
type
}
}
#' dump the schema into JSON string
tojson <- function(x) {
if (is.list(x)) {
names <- names(x)
if (!is.null(names)) {
items <- lapply(names, function(n) {
safe_n <- gsub('"', '\\"', n)
paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '')
})
d <- paste(items, collapse = ', ')
paste('{', d, '}', sep = '')
} else {
l <- paste(lapply(x, tojson), collapse = ', ')
paste('[', l, ']', sep = '')
}
} else if (is.character(x)) {
paste('"', x, '"', sep = '')
} else if (is.logical(x)) {
if (x) "true" else "false"
} else {
stop(paste("unexpected type:", class(x)))
}
}
#' Create a DataFrame from an RDD
#'
#' Converts an RDD to a DataFrame by infer the types.
#'
#' @param sqlCtx A SQLContext
#' @param data An RDD or list or data.frame
#' @param schema a list of column names or named list (StructType), optional
#' @return an DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x)))
#' df <- createDataFrame(sqlCtx, rdd)
#' }
# TODO(davies): support sampling and infer type from NA
createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) {
if (is.data.frame(data)) {
# get the names of columns, they will be put into RDD
schema <- names(data)
n <- nrow(data)
m <- ncol(data)
# get rid of factor type
dropFactor <- function(x) {
if (is.factor(x)) {
as.character(x)
} else {
x
}
}
data <- lapply(1:n, function(i) {
lapply(1:m, function(j) { dropFactor(data[i,j]) })
})
}
if (is.list(data)) {
sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sqlCtx)
rdd <- parallelize(sc, data)
} else if (inherits(data, "RDD")) {
rdd <- data
} else {
stop(paste("unexpected type:", class(data)))
}
if (is.null(schema) || is.null(names(schema))) {
row <- first(rdd)
names <- if (is.null(schema)) {
names(row)
} else {
as.list(schema)
}
if (is.null(names)) {
names <- lapply(1:length(row), function(x) {
paste("_", as.character(x), sep = "")
})
}
# SPAKR-SQL does not support '.' in column name, so replace it with '_'
# TODO(davies): remove this once SPARK-2775 is fixed
names <- lapply(names, function(n) {
nn <- gsub("[.]", "_", n)
if (nn != n) {
warning(paste("Use", nn, "instead of", n, " as column name"))
}
nn
})
types <- lapply(row, infer_type)
fields <- lapply(1:length(row), function(i) {
list(name = names[[i]], type = types[[i]], nullable = TRUE)
})
schema <- list(type = "struct", fields = fields)
}
stopifnot(class(schema) == "list")
stopifnot(schema$type == "struct")
stopifnot(class(schema$fields) == "list")
schemaString <- tojson(schema)
jrdd <- getJRDD(lapply(rdd, function(x) x), "row")
srdd <- callJMethod(jrdd, "rdd")
sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF",
srdd, schemaString, sqlCtx)
dataFrame(sdf)
}
#' toDF
#'
#' Converts an RDD to a DataFrame by infer the types.
#'
#' @param x An RDD
#'
#' @rdname DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x)))
#' df <- toDF(rdd)
#' }
setGeneric("toDF", function(x, ...) { standardGeneric("toDF") })
setMethod("toDF", signature(x = "RDD"),
function(x, ...) {
sqlCtx <- if (exists(".sparkRHivesc", envir = .sparkREnv)) {
get(".sparkRHivesc", envir = .sparkREnv)
} else if (exists(".sparkRSQLsc", envir = .sparkREnv)) {
get(".sparkRSQLsc", envir = .sparkREnv)
} else {
stop("no SQL context available")
}
createDataFrame(sqlCtx, x, ...)
})
#' Create a DataFrame from a JSON file.
#'
#' Loads a JSON file (one object per line), returning the result as a DataFrame
#' It goes through the entire dataset once to determine the schema.
#'
#' @param sqlCtx SQLContext to use
#' @param path Path of file to read. A vector of multiple paths is allowed.
#' @return DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- jsonFile(sqlCtx, path)
#' }
jsonFile <- function(sqlCtx, path) {
# Allow the user to have a more flexible definiton of the text file path
path <- normalizePath(path)
# Convert a string vector of paths to a string containing comma separated paths
path <- paste(path, collapse = ",")
sdf <- callJMethod(sqlCtx, "jsonFile", path)
dataFrame(sdf)
}
#' JSON RDD
#'
#' Loads an RDD storing one JSON object per string as a DataFrame.
#'
#' @param sqlCtx SQLContext to use
#' @param rdd An RDD of JSON string
#' @param schema A StructType object to use as schema
#' @param samplingRatio The ratio of simpling used to infer the schema
#' @return A DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' rdd <- texFile(sc, "path/to/json")
#' df <- jsonRDD(sqlCtx, rdd)
#' }
# TODO: support schema
jsonRDD <- function(sqlCtx, rdd, schema = NULL, samplingRatio = 1.0) {
rdd <- serializeToString(rdd)
if (is.null(schema)) {
sdf <- callJMethod(sqlCtx, "jsonRDD", callJMethod(getJRDD(rdd), "rdd"), samplingRatio)
dataFrame(sdf)
} else {
stop("not implemented")
}
}
#' Create a DataFrame from a Parquet file.
#'
#' Loads a Parquet file, returning the result as a DataFrame.
#'
#' @param sqlCtx SQLContext to use
#' @param ... Path(s) of parquet file(s) to read.
#' @return DataFrame
#' @export
# TODO: Implement saveasParquetFile and write examples for both
parquetFile <- function(sqlCtx, ...) {
# Allow the user to have a more flexible definiton of the text file path
paths <- lapply(list(...), normalizePath)
sdf <- callJMethod(sqlCtx, "parquetFile", paths)
dataFrame(sdf)
}
#' SQL Query
#'
#' Executes a SQL query using Spark, returning the result as a DataFrame.
#'
#' @param sqlCtx SQLContext to use
#' @param sqlQuery A character vector containing the SQL query
#' @return DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- jsonFile(sqlCtx, path)
#' registerTempTable(df, "table")
#' new_df <- sql(sqlCtx, "SELECT * FROM table")
#' }
sql <- function(sqlCtx, sqlQuery) {
sdf <- callJMethod(sqlCtx, "sql", sqlQuery)
dataFrame(sdf)
}
#' Create a DataFrame from a SparkSQL Table
#'
#' Returns the specified Table as a DataFrame. The Table must have already been registered
#' in the SQLContext.
#'
#' @param sqlCtx SQLContext to use
#' @param tableName The SparkSQL Table to convert to a DataFrame.
#' @return DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- jsonFile(sqlCtx, path)
#' registerTempTable(df, "table")
#' new_df <- table(sqlCtx, "table")
#' }
table <- function(sqlCtx, tableName) {
sdf <- callJMethod(sqlCtx, "table", tableName)
dataFrame(sdf)
}
#' Tables
#'
#' Returns a DataFrame containing names of tables in the given database.
#'
#' @param sqlCtx SQLContext to use
#' @param databaseName name of the database
#' @return a DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' tables(sqlCtx, "hive")
#' }
tables <- function(sqlCtx, databaseName = NULL) {
jdf <- if (is.null(databaseName)) {
callJMethod(sqlCtx, "tables")
} else {
callJMethod(sqlCtx, "tables", databaseName)
}
dataFrame(jdf)
}
#' Table Names
#'
#' Returns the names of tables in the given database as an array.
#'
#' @param sqlCtx SQLContext to use
#' @param databaseName name of the database
#' @return a list of table names
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' tableNames(sqlCtx, "hive")
#' }
tableNames <- function(sqlCtx, databaseName = NULL) {
if (is.null(databaseName)) {
callJMethod(sqlCtx, "tableNames")
} else {
callJMethod(sqlCtx, "tableNames", databaseName)
}
}
#' Cache Table
#'
#' Caches the specified table in-memory.
#'
#' @param sqlCtx SQLContext to use
#' @param tableName The name of the table being cached
#' @return DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- jsonFile(sqlCtx, path)
#' registerTempTable(df, "table")
#' cacheTable(sqlCtx, "table")
#' }
cacheTable <- function(sqlCtx, tableName) {
callJMethod(sqlCtx, "cacheTable", tableName)
}
#' Uncache Table
#'
#' Removes the specified table from the in-memory cache.
#'
#' @param sqlCtx SQLContext to use
#' @param tableName The name of the table being uncached
#' @return DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' path <- "path/to/file.json"
#' df <- jsonFile(sqlCtx, path)
#' registerTempTable(df, "table")
#' uncacheTable(sqlCtx, "table")
#' }
uncacheTable <- function(sqlCtx, tableName) {
callJMethod(sqlCtx, "uncacheTable", tableName)
}
#' Clear Cache
#'
#' Removes all cached tables from the in-memory cache.
#'
#' @param sqlCtx SQLContext to use
#' @examples
#' \dontrun{
#' clearCache(sqlCtx)
#' }
clearCache <- function(sqlCtx) {
callJMethod(sqlCtx, "clearCache")
}
#' Drop Temporary Table
#'
#' Drops the temporary table with the given table name in the catalog.
#' If the table has been cached/persisted before, it's also unpersisted.
#'
#' @param sqlCtx SQLContext to use
#' @param tableName The name of the SparkSQL table to be dropped.
#' @examples
#' \dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' df <- loadDF(sqlCtx, path, "parquet")
#' registerTempTable(df, "table")
#' dropTempTable(sqlCtx, "table")
#' }
dropTempTable <- function(sqlCtx, tableName) {
if (class(tableName) != "character") {
stop("tableName must be a string.")
}
callJMethod(sqlCtx, "dropTempTable", tableName)
}
#' Load an DataFrame
#'
#' Returns the dataset in a data source as a DataFrame
#'
#' 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.
#'
#' @param sqlCtx SQLContext to use
#' @param path The path of files to load
#' @param source the name of external data source
#' @return DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' df <- load(sqlCtx, "path/to/file.json", source = "json")
#' }
loadDF <- function(sqlCtx, path = NULL, source = NULL, ...) {
options <- varargsToEnv(...)
if (!is.null(path)) {
options[['path']] <- path
}
sdf <- callJMethod(sqlCtx, "load", source, options)
dataFrame(sdf)
}
#' Create an external table
#'
#' Creates an external table based on the dataset in a data source,
#' Returns the DataFrame associated with the external 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.
#'
#' @param sqlCtx SQLContext to use
#' @param tableName A name of the table
#' @param path The path of files to load
#' @param source the name of external data source
#' @return DataFrame
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#' df <- sparkRSQL.createExternalTable(sqlCtx, "myjson", path="path/to/json", source="json")
#' }
createExternalTable <- function(sqlCtx, tableName, path = NULL, source = NULL, ...) {
options <- varargsToEnv(...)
if (!is.null(path)) {
options[['path']] <- path
}
sdf <- callJMethod(sqlCtx, "createExternalTable", tableName, source, options)
dataFrame(sdf)
}

64
R/pkg/R/SQLTypes.R Normal file
View file

@ -0,0 +1,64 @@
#
# 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.
#
# Utility functions for handling SparkSQL DataTypes.
# Handler for StructType
structType <- function(st) {
obj <- structure(new.env(parent = emptyenv()), class = "structType")
obj$jobj <- st
obj$fields <- function() { lapply(callJMethod(st, "fields"), structField) }
obj
}
#' Print a Spark StructType.
#'
#' This function prints the contents of a StructType returned from the
#' SparkR JVM backend.
#'
#' @param x A StructType object
#' @param ... further arguments passed to or from other methods
print.structType <- function(x, ...) {
fieldsList <- lapply(x$fields(), function(i) { i$print() })
print(fieldsList)
}
# Handler for StructField
structField <- function(sf) {
obj <- structure(new.env(parent = emptyenv()), class = "structField")
obj$jobj <- sf
obj$name <- function() { callJMethod(sf, "name") }
obj$dataType <- function() { callJMethod(sf, "dataType") }
obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") }
obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") }
obj$nullable <- function() { callJMethod(sf, "nullable") }
obj$print <- function() { paste("StructField(",
paste(obj$name(), obj$dataType.toString(), obj$nullable(), sep = ", "),
")", sep = "") }
obj
}
#' Print a Spark StructField.
#'
#' This function prints the contents of a StructField returned from the
#' SparkR JVM backend.
#'
#' @param x A StructField object
#' @param ... further arguments passed to or from other methods
print.structField <- function(x, ...) {
cat(x$print())
}

115
R/pkg/R/backend.R Normal file
View file

@ -0,0 +1,115 @@
#
# 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.
#
# Methods to call into SparkRBackend.
# Returns TRUE if object is an instance of given class
isInstanceOf <- function(jobj, className) {
stopifnot(class(jobj) == "jobj")
cls <- callJStatic("java.lang.Class", "forName", className)
callJMethod(cls, "isInstance", jobj)
}
# Call a Java method named methodName on the object
# specified by objId. objId should be a "jobj" returned
# from the SparkRBackend.
callJMethod <- function(objId, methodName, ...) {
stopifnot(class(objId) == "jobj")
if (!isValidJobj(objId)) {
stop("Invalid jobj ", objId$id,
". If SparkR was restarted, Spark operations need to be re-executed.")
}
invokeJava(isStatic = FALSE, objId$id, methodName, ...)
}
# Call a static method on a specified className
callJStatic <- function(className, methodName, ...) {
invokeJava(isStatic = TRUE, className, methodName, ...)
}
# Create a new object of the specified class name
newJObject <- function(className, ...) {
invokeJava(isStatic = TRUE, className, methodName = "<init>", ...)
}
# Remove an object from the SparkR backend. This is done
# automatically when a jobj is garbage collected.
removeJObject <- function(objId) {
invokeJava(isStatic = TRUE, "SparkRHandler", "rm", objId)
}
isRemoveMethod <- function(isStatic, objId, methodName) {
isStatic == TRUE && objId == "SparkRHandler" && methodName == "rm"
}
# Invoke a Java method on the SparkR backend. Users
# should typically use one of the higher level methods like
# callJMethod, callJStatic etc. instead of using this.
#
# isStatic - TRUE if the method to be called is static
# objId - String that refers to the object on which method is invoked
# Should be a jobj id for non-static methods and the classname
# for static methods
# methodName - name of method to be invoked
invokeJava <- function(isStatic, objId, methodName, ...) {
if (!exists(".sparkRCon", .sparkREnv)) {
stop("No connection to backend found. Please re-run sparkR.init")
}
# If this isn't a removeJObject call
if (!isRemoveMethod(isStatic, objId, methodName)) {
objsToRemove <- ls(.toRemoveJobjs)
if (length(objsToRemove) > 0) {
sapply(objsToRemove,
function(e) {
removeJObject(e)
})
rm(list = objsToRemove, envir = .toRemoveJobjs)
}
}
rc <- rawConnection(raw(0), "r+")
writeBoolean(rc, isStatic)
writeString(rc, objId)
writeString(rc, methodName)
args <- list(...)
writeInt(rc, length(args))
writeArgs(rc, args)
# Construct the whole request message to send it once,
# avoiding write-write-read pattern in case of Nagle's algorithm.
# Refer to http://en.wikipedia.org/wiki/Nagle%27s_algorithm for the details.
bytesToSend <- rawConnectionValue(rc)
close(rc)
rc <- rawConnection(raw(0), "r+")
writeInt(rc, length(bytesToSend))
writeBin(bytesToSend, rc)
requestMessage <- rawConnectionValue(rc)
close(rc)
conn <- get(".sparkRCon", .sparkREnv)
writeBin(requestMessage, conn)
# TODO: check the status code to output error information
returnStatus <- readInt(conn)
stopifnot(returnStatus == 0)
readObject(conn)
}

86
R/pkg/R/broadcast.R Normal file
View file

@ -0,0 +1,86 @@
#
# 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.
#
# S4 class representing Broadcast variables
# Hidden environment that holds values for broadcast variables
# This will not be serialized / shipped by default
.broadcastNames <- new.env()
.broadcastValues <- new.env()
.broadcastIdToName <- new.env()
#' @title S4 class that represents a Broadcast variable
#' @description Broadcast variables can be created using the broadcast
#' function from a \code{SparkContext}.
#' @rdname broadcast-class
#' @seealso broadcast
#'
#' @param id Id of the backing Spark broadcast variable
#' @export
setClass("Broadcast", slots = list(id = "character"))
#' @rdname broadcast-class
#' @param value Value of the broadcast variable
#' @param jBroadcastRef reference to the backing Java broadcast object
#' @param objName name of broadcasted object
#' @export
Broadcast <- function(id, value, jBroadcastRef, objName) {
.broadcastValues[[id]] <- value
.broadcastNames[[as.character(objName)]] <- jBroadcastRef
.broadcastIdToName[[id]] <- as.character(objName)
new("Broadcast", id = id)
}
#' @description
#' \code{value} can be used to get the value of a broadcast variable inside
#' a distributed function.
#'
#' @param bcast The broadcast variable to get
#' @rdname broadcast
#' @aliases value,Broadcast-method
setMethod("value",
signature(bcast = "Broadcast"),
function(bcast) {
if (exists(bcast@id, envir = .broadcastValues)) {
get(bcast@id, envir = .broadcastValues)
} else {
NULL
}
})
#' Internal function to set values of a broadcast variable.
#'
#' This function is used internally by Spark to set the value of a broadcast
#' variable on workers. Not intended for use outside the package.
#'
#' @rdname broadcast-internal
#' @seealso broadcast, value
#' @param bcastId The id of broadcast variable to set
#' @param value The value to be set
#' @export
setBroadcastValue <- function(bcastId, value) {
bcastIdStr <- as.character(bcastId)
.broadcastValues[[bcastIdStr]] <- value
}
#' Helper function to clear the list of broadcast variables we know about
#' Should be called when the SparkR JVM backend is shutdown
clearBroadcastVariables <- function() {
bcasts <- ls(.broadcastNames)
rm(list = bcasts, envir = .broadcastNames)
}

57
R/pkg/R/client.R Normal file
View file

@ -0,0 +1,57 @@
#
# 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.
#
# Client code to connect to SparkRBackend
# Creates a SparkR client connection object
# if one doesn't already exist
connectBackend <- function(hostname, port, timeout = 6000) {
if (exists(".sparkRcon", envir = .sparkREnv)) {
if (isOpen(.sparkREnv[[".sparkRCon"]])) {
cat("SparkRBackend client connection already exists\n")
return(get(".sparkRcon", envir = .sparkREnv))
}
}
con <- socketConnection(host = hostname, port = port, server = FALSE,
blocking = TRUE, open = "wb", timeout = timeout)
assign(".sparkRCon", con, envir = .sparkREnv)
con
}
launchBackend <- function(args, sparkHome, jars, sparkSubmitOpts) {
if (.Platform$OS.type == "unix") {
sparkSubmitBinName = "spark-submit"
} else {
sparkSubmitBinName = "spark-submit.cmd"
}
if (sparkHome != "") {
sparkSubmitBin <- file.path(sparkHome, "bin", sparkSubmitBinName)
} else {
sparkSubmitBin <- sparkSubmitBinName
}
if (jars != "") {
jars <- paste("--jars", jars)
}
combinedArgs <- paste(jars, sparkSubmitOpts, args, sep = " ")
cat("Launching java with spark-submit command", sparkSubmitBin, combinedArgs, "\n")
invisible(system2(sparkSubmitBin, combinedArgs, wait = F))
}

199
R/pkg/R/column.R Normal file
View file

@ -0,0 +1,199 @@
#
# 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.
#
# Column Class
#' @include generics.R jobj.R
NULL
setOldClass("jobj")
#' @title S4 class that represents a DataFrame column
#' @description The column class supports unary, binary operations on DataFrame columns
#' @rdname column
#'
#' @param jc reference to JVM DataFrame column
#' @export
setClass("Column",
slots = list(jc = "jobj"))
setMethod("initialize", "Column", function(.Object, jc) {
.Object@jc <- jc
.Object
})
column <- function(jc) {
new("Column", jc)
}
col <- function(x) {
column(callJStatic("org.apache.spark.sql.functions", "col", x))
}
#' @rdname show
setMethod("show", "Column",
function(object) {
cat("Column", callJMethod(object@jc, "toString"), "\n")
})
operators <- list(
"+" = "plus", "-" = "minus", "*" = "multiply", "/" = "divide", "%%" = "mod",
"==" = "equalTo", ">" = "gt", "<" = "lt", "!=" = "notEqual", "<=" = "leq", ">=" = "geq",
# we can not override `&&` and `||`, so use `&` and `|` instead
"&" = "and", "|" = "or" #, "!" = "unary_$bang"
)
column_functions1 <- c("asc", "desc", "isNull", "isNotNull")
column_functions2 <- c("like", "rlike", "startsWith", "endsWith", "getField", "getItem", "contains")
functions <- c("min", "max", "sum", "avg", "mean", "count", "abs", "sqrt",
"first", "last", "lower", "upper", "sumDistinct")
createOperator <- function(op) {
setMethod(op,
signature(e1 = "Column"),
function(e1, e2) {
jc <- if (missing(e2)) {
if (op == "-") {
callJMethod(e1@jc, "unary_$minus")
} else {
callJMethod(e1@jc, operators[[op]])
}
} else {
if (class(e2) == "Column") {
e2 <- e2@jc
}
callJMethod(e1@jc, operators[[op]], e2)
}
column(jc)
})
}
createColumnFunction1 <- function(name) {
setMethod(name,
signature(x = "Column"),
function(x) {
column(callJMethod(x@jc, name))
})
}
createColumnFunction2 <- function(name) {
setMethod(name,
signature(x = "Column"),
function(x, data) {
if (class(data) == "Column") {
data <- data@jc
}
jc <- callJMethod(x@jc, name, data)
column(jc)
})
}
createStaticFunction <- function(name) {
setMethod(name,
signature(x = "Column"),
function(x) {
jc <- callJStatic("org.apache.spark.sql.functions", name, x@jc)
column(jc)
})
}
createMethods <- function() {
for (op in names(operators)) {
createOperator(op)
}
for (name in column_functions1) {
createColumnFunction1(name)
}
for (name in column_functions2) {
createColumnFunction2(name)
}
for (x in functions) {
createStaticFunction(x)
}
}
createMethods()
#' alias
#'
#' Set a new name for a column
setMethod("alias",
signature(object = "Column"),
function(object, data) {
if (is.character(data)) {
column(callJMethod(object@jc, "as", data))
} else {
stop("data should be character")
}
})
#' An expression that returns a substring.
#'
#' @param start starting position
#' @param stop ending position
setMethod("substr", signature(x = "Column"),
function(x, start, stop) {
jc <- callJMethod(x@jc, "substr", as.integer(start - 1), as.integer(stop - start + 1))
column(jc)
})
#' Casts the column to a different data type.
#' @examples
#' \dontrun{
#' cast(df$age, "string")
#' cast(df$name, list(type="array", elementType="byte", containsNull = TRUE))
#' }
setMethod("cast",
signature(x = "Column"),
function(x, dataType) {
if (is.character(dataType)) {
column(callJMethod(x@jc, "cast", dataType))
} else if (is.list(dataType)) {
json <- tojson(dataType)
jdataType <- callJStatic("org.apache.spark.sql.types.DataType", "fromJson", json)
column(callJMethod(x@jc, "cast", jdataType))
} else {
stop("dataType should be character or list")
}
})
#' Approx Count Distinct
#'
#' Returns the approximate number of distinct items in a group.
#'
setMethod("approxCountDistinct",
signature(x = "Column"),
function(x, rsd = 0.95) {
jc <- callJStatic("org.apache.spark.sql.functions", "approxCountDistinct", x@jc, rsd)
column(jc)
})
#' Count Distinct
#'
#' returns the number of distinct items in a group.
#'
setMethod("countDistinct",
signature(x = "Column"),
function(x, ...) {
jcol <- lapply(list(...), function (x) {
x@jc
})
jc <- callJStatic("org.apache.spark.sql.functions", "countDistinct", x@jc,
listToSeq(jcol))
column(jc)
})

225
R/pkg/R/context.R Normal file
View file

@ -0,0 +1,225 @@
#
# 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.
#
# context.R: SparkContext driven functions
getMinSplits <- function(sc, minSplits) {
if (is.null(minSplits)) {
defaultParallelism <- callJMethod(sc, "defaultParallelism")
minSplits <- min(defaultParallelism, 2)
}
as.integer(minSplits)
}
#' Create an RDD from a text file.
#'
#' This function reads a text file from HDFS, a local file system (available on all
#' nodes), or any Hadoop-supported file system URI, and creates an
#' RDD of strings from it.
#'
#' @param sc SparkContext to use
#' @param path Path of file to read. A vector of multiple paths is allowed.
#' @param minSplits Minimum number of splits to be created. If NULL, the default
#' value is chosen based on available parallelism.
#' @return RDD where each item is of type \code{character}
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' lines <- textFile(sc, "myfile.txt")
#'}
textFile <- function(sc, path, minSplits = NULL) {
# Allow the user to have a more flexible definiton of the text file path
path <- suppressWarnings(normalizePath(path))
#' Convert a string vector of paths to a string containing comma separated paths
path <- paste(path, collapse = ",")
jrdd <- callJMethod(sc, "textFile", path, getMinSplits(sc, minSplits))
# jrdd is of type JavaRDD[String]
RDD(jrdd, "string")
}
#' Load an RDD saved as a SequenceFile containing serialized objects.
#'
#' The file to be loaded should be one that was previously generated by calling
#' saveAsObjectFile() of the RDD class.
#'
#' @param sc SparkContext to use
#' @param path Path of file to read. A vector of multiple paths is allowed.
#' @param minSplits Minimum number of splits to be created. If NULL, the default
#' value is chosen based on available parallelism.
#' @return RDD containing serialized R objects.
#' @seealso saveAsObjectFile
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- objectFile(sc, "myfile")
#'}
objectFile <- function(sc, path, minSplits = NULL) {
# Allow the user to have a more flexible definiton of the text file path
path <- suppressWarnings(normalizePath(path))
#' Convert a string vector of paths to a string containing comma separated paths
path <- paste(path, collapse = ",")
jrdd <- callJMethod(sc, "objectFile", path, getMinSplits(sc, minSplits))
# Assume the RDD contains serialized R objects.
RDD(jrdd, "byte")
}
#' Create an RDD from a homogeneous list or vector.
#'
#' This function creates an RDD from a local homogeneous list in R. The elements
#' in the list are split into \code{numSlices} slices and distributed to nodes
#' in the cluster.
#'
#' @param sc SparkContext to use
#' @param coll collection to parallelize
#' @param numSlices number of partitions to create in the RDD
#' @return an RDD created from this collection
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, 1:10, 2)
#' # The RDD should contain 10 elements
#' length(rdd)
#'}
parallelize <- function(sc, coll, numSlices = 1) {
# TODO: bound/safeguard numSlices
# TODO: unit tests for if the split works for all primitives
# TODO: support matrix, data frame, etc
if ((!is.list(coll) && !is.vector(coll)) || is.data.frame(coll)) {
if (is.data.frame(coll)) {
message(paste("context.R: A data frame is parallelized by columns."))
} else {
if (is.matrix(coll)) {
message(paste("context.R: A matrix is parallelized by elements."))
} else {
message(paste("context.R: parallelize() currently only supports lists and vectors.",
"Calling as.list() to coerce coll into a list."))
}
}
coll <- as.list(coll)
}
if (numSlices > length(coll))
numSlices <- length(coll)
sliceLen <- ceiling(length(coll) / numSlices)
slices <- split(coll, rep(1:(numSlices + 1), each = sliceLen)[1:length(coll)])
# Serialize each slice: obtain a list of raws, or a list of lists (slices) of
# 2-tuples of raws
serializedSlices <- lapply(slices, serialize, connection = NULL)
jrdd <- callJStatic("org.apache.spark.api.r.RRDD",
"createRDDFromArray", sc, serializedSlices)
RDD(jrdd, "byte")
}
#' Include this specified package on all workers
#'
#' This function can be used to include a package on all workers before the
#' user's code is executed. This is useful in scenarios where other R package
#' functions are used in a function passed to functions like \code{lapply}.
#' NOTE: The package is assumed to be installed on every node in the Spark
#' cluster.
#'
#' @param sc SparkContext to use
#' @param pkg Package name
#'
#' @export
#' @examples
#'\dontrun{
#' library(Matrix)
#'
#' sc <- sparkR.init()
#' # Include the matrix library we will be using
#' includePackage(sc, Matrix)
#'
#' generateSparse <- function(x) {
#' sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3))
#' }
#'
#' rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse)
#' collect(rdd)
#'}
includePackage <- function(sc, pkg) {
pkg <- as.character(substitute(pkg))
if (exists(".packages", .sparkREnv)) {
packages <- .sparkREnv$.packages
} else {
packages <- list()
}
packages <- c(packages, pkg)
.sparkREnv$.packages <- packages
}
#' @title Broadcast a variable to all workers
#'
#' @description
#' Broadcast a read-only variable to the cluster, returning a \code{Broadcast}
#' object for reading it in distributed functions.
#'
#' @param sc Spark Context to use
#' @param object Object to be broadcast
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, 1:2, 2L)
#'
#' # Large Matrix object that we want to broadcast
#' randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000))
#' randomMatBr <- broadcast(sc, randomMat)
#'
#' # Use the broadcast variable inside the function
#' useBroadcast <- function(x) {
#' sum(value(randomMatBr) * x)
#' }
#' sumRDD <- lapply(rdd, useBroadcast)
#'}
broadcast <- function(sc, object) {
objName <- as.character(substitute(object))
serializedObj <- serialize(object, connection = NULL)
jBroadcast <- callJMethod(sc, "broadcast", serializedObj)
id <- as.character(callJMethod(jBroadcast, "id"))
Broadcast(id, object, jBroadcast, objName)
}
#' @title Set the checkpoint directory
#'
#' Set the directory under which RDDs are going to be checkpointed. The
#' directory must be a HDFS path if running on a cluster.
#'
#' @param sc Spark Context to use
#' @param dirName Directory path
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' setCheckpointDir(sc, "~/checkpoints")
#' rdd <- parallelize(sc, 1:2, 2L)
#' checkpoint(rdd)
#'}
setCheckpointDir <- function(sc, dirName) {
invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName))))
}

184
R/pkg/R/deserialize.R Normal file
View file

@ -0,0 +1,184 @@
#
# 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.
#
# Utility functions to deserialize objects from Java.
# Type mapping from Java to R
#
# void -> NULL
# Int -> integer
# String -> character
# Boolean -> logical
# Double -> double
# Long -> double
# Array[Byte] -> raw
# Date -> Date
# Time -> POSIXct
#
# Array[T] -> list()
# Object -> jobj
readObject <- function(con) {
# Read type first
type <- readType(con)
readTypedObject(con, type)
}
readTypedObject <- function(con, type) {
switch (type,
"i" = readInt(con),
"c" = readString(con),
"b" = readBoolean(con),
"d" = readDouble(con),
"r" = readRaw(con),
"D" = readDate(con),
"t" = readTime(con),
"l" = readList(con),
"n" = NULL,
"j" = getJobj(readString(con)),
stop(paste("Unsupported type for deserialization", type)))
}
readString <- function(con) {
stringLen <- readInt(con)
string <- readBin(con, raw(), stringLen, endian = "big")
rawToChar(string)
}
readInt <- function(con) {
readBin(con, integer(), n = 1, endian = "big")
}
readDouble <- function(con) {
readBin(con, double(), n = 1, endian = "big")
}
readBoolean <- function(con) {
as.logical(readInt(con))
}
readType <- function(con) {
rawToChar(readBin(con, "raw", n = 1L))
}
readDate <- function(con) {
as.Date(readString(con))
}
readTime <- function(con) {
t <- readDouble(con)
as.POSIXct(t, origin = "1970-01-01")
}
# We only support lists where all elements are of same type
readList <- function(con) {
type <- readType(con)
len <- readInt(con)
if (len > 0) {
l <- vector("list", len)
for (i in 1:len) {
l[[i]] <- readTypedObject(con, type)
}
l
} else {
list()
}
}
readRaw <- function(con) {
dataLen <- readInt(con)
data <- readBin(con, raw(), as.integer(dataLen), endian = "big")
}
readRawLen <- function(con, dataLen) {
data <- readBin(con, raw(), as.integer(dataLen), endian = "big")
}
readDeserialize <- function(con) {
# We have two cases that are possible - In one, the entire partition is
# encoded as a byte array, so we have only one value to read. If so just
# return firstData
dataLen <- readInt(con)
firstData <- unserialize(
readBin(con, raw(), as.integer(dataLen), endian = "big"))
# Else, read things into a list
dataLen <- readInt(con)
if (length(dataLen) > 0 && dataLen > 0) {
data <- list(firstData)
while (length(dataLen) > 0 && dataLen > 0) {
data[[length(data) + 1L]] <- unserialize(
readBin(con, raw(), as.integer(dataLen), endian = "big"))
dataLen <- readInt(con)
}
unlist(data, recursive = FALSE)
} else {
firstData
}
}
readDeserializeRows <- function(inputCon) {
# readDeserializeRows will deserialize a DataOutputStream composed of
# a list of lists. Since the DOS is one continuous stream and
# the number of rows varies, we put the readRow function in a while loop
# that termintates when the next row is empty.
data <- list()
while(TRUE) {
row <- readRow(inputCon)
if (length(row) == 0) {
break
}
data[[length(data) + 1L]] <- row
}
data # this is a list of named lists now
}
readRowList <- function(obj) {
# readRowList is meant for use inside an lapply. As a result, it is
# necessary to open a standalone connection for the row and consume
# the numCols bytes inside the read function in order to correctly
# deserialize the row.
rawObj <- rawConnection(obj, "r+")
on.exit(close(rawObj))
readRow(rawObj)
}
readRow <- function(inputCon) {
numCols <- readInt(inputCon)
if (length(numCols) > 0 && numCols > 0) {
lapply(1:numCols, function(x) {
obj <- readObject(inputCon)
if (is.null(obj)) {
NA
} else {
obj
}
}) # each row is a list now
} else {
list()
}
}
# Take a single column as Array[Byte] and deserialize it into an atomic vector
readCol <- function(inputCon, numRows) {
# sapply can not work with POSIXlt
do.call(c, lapply(1:numRows, function(x) {
value <- readObject(inputCon)
# Replace NULL with NA so we can coerce to vectors
if (is.null(value)) NA else value
}))
}

543
R/pkg/R/generics.R Normal file
View file

@ -0,0 +1,543 @@
#
# 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.
#
############ RDD Actions and Transformations ############
#' @rdname aggregateRDD
#' @seealso reduce
#' @export
setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") })
#' @rdname cache-methods
#' @export
setGeneric("cache", function(x) { standardGeneric("cache") })
#' @rdname coalesce
#' @seealso repartition
#' @export
setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") })
#' @rdname checkpoint-methods
#' @export
setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") })
#' @rdname collect-methods
#' @export
setGeneric("collect", function(x, ...) { standardGeneric("collect") })
#' @rdname collect-methods
#' @export
setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") })
#' @rdname collect-methods
#' @export
setGeneric("collectPartition",
function(x, partitionId) {
standardGeneric("collectPartition")
})
#' @rdname count
#' @export
setGeneric("count", function(x) { standardGeneric("count") })
#' @rdname countByValue
#' @export
setGeneric("countByValue", function(x) { standardGeneric("countByValue") })
#' @rdname distinct
#' @export
setGeneric("distinct", function(x, numPartitions = 1L) { standardGeneric("distinct") })
#' @rdname filterRDD
#' @export
setGeneric("filterRDD", function(x, f) { standardGeneric("filterRDD") })
#' @rdname first
#' @export
setGeneric("first", function(x) { standardGeneric("first") })
#' @rdname flatMap
#' @export
setGeneric("flatMap", function(X, FUN) { standardGeneric("flatMap") })
#' @rdname fold
#' @seealso reduce
#' @export
setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") })
#' @rdname foreach
#' @export
setGeneric("foreach", function(x, func) { standardGeneric("foreach") })
#' @rdname foreach
#' @export
setGeneric("foreachPartition", function(x, func) { standardGeneric("foreachPartition") })
# The jrdd accessor function.
setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") })
#' @rdname glom
#' @export
setGeneric("glom", function(x) { standardGeneric("glom") })
#' @rdname keyBy
#' @export
setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") })
#' @rdname lapplyPartition
#' @export
setGeneric("lapplyPartition", function(X, FUN) { standardGeneric("lapplyPartition") })
#' @rdname lapplyPartitionsWithIndex
#' @export
setGeneric("lapplyPartitionsWithIndex",
function(X, FUN) {
standardGeneric("lapplyPartitionsWithIndex")
})
#' @rdname lapply
#' @export
setGeneric("map", function(X, FUN) { standardGeneric("map") })
#' @rdname lapplyPartition
#' @export
setGeneric("mapPartitions", function(X, FUN) { standardGeneric("mapPartitions") })
#' @rdname lapplyPartitionsWithIndex
#' @export
setGeneric("mapPartitionsWithIndex",
function(X, FUN) { standardGeneric("mapPartitionsWithIndex") })
#' @rdname maximum
#' @export
setGeneric("maximum", function(x) { standardGeneric("maximum") })
#' @rdname minimum
#' @export
setGeneric("minimum", function(x) { standardGeneric("minimum") })
#' @rdname sumRDD
#' @export
setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") })
#' @rdname name
#' @export
setGeneric("name", function(x) { standardGeneric("name") })
#' @rdname numPartitions
#' @export
setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") })
#' @rdname persist
#' @export
setGeneric("persist", function(x, newLevel) { standardGeneric("persist") })
#' @rdname pipeRDD
#' @export
setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD")})
#' @rdname reduce
#' @export
setGeneric("reduce", function(x, func) { standardGeneric("reduce") })
#' @rdname repartition
#' @seealso coalesce
#' @export
setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") })
#' @rdname sampleRDD
#' @export
setGeneric("sampleRDD",
function(x, withReplacement, fraction, seed) {
standardGeneric("sampleRDD")
})
#' @rdname saveAsObjectFile
#' @seealso objectFile
#' @export
setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") })
#' @rdname saveAsTextFile
#' @export
setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") })
#' @rdname setName
#' @export
setGeneric("setName", function(x, name) { standardGeneric("setName") })
#' @rdname sortBy
#' @export
setGeneric("sortBy",
function(x, func, ascending = TRUE, numPartitions = 1L) {
standardGeneric("sortBy")
})
#' @rdname take
#' @export
setGeneric("take", function(x, num) { standardGeneric("take") })
#' @rdname takeOrdered
#' @export
setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") })
#' @rdname takeSample
#' @export
setGeneric("takeSample",
function(x, withReplacement, num, seed) {
standardGeneric("takeSample")
})
#' @rdname top
#' @export
setGeneric("top", function(x, num) { standardGeneric("top") })
#' @rdname unionRDD
#' @export
setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") })
#' @rdname unpersist-methods
#' @export
setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") })
#' @rdname zipRDD
#' @export
setGeneric("zipRDD", function(x, other) { standardGeneric("zipRDD") })
#' @rdname zipWithIndex
#' @seealso zipWithUniqueId
#' @export
setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") })
#' @rdname zipWithUniqueId
#' @seealso zipWithIndex
#' @export
setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") })
############ Binary Functions #############
#' @rdname countByKey
#' @export
setGeneric("countByKey", function(x) { standardGeneric("countByKey") })
#' @rdname flatMapValues
#' @export
setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") })
#' @rdname keys
#' @export
setGeneric("keys", function(x) { standardGeneric("keys") })
#' @rdname lookup
#' @export
setGeneric("lookup", function(x, key) { standardGeneric("lookup") })
#' @rdname mapValues
#' @export
setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") })
#' @rdname values
#' @export
setGeneric("values", function(x) { standardGeneric("values") })
############ Shuffle Functions ############
#' @rdname aggregateByKey
#' @seealso foldByKey, combineByKey
#' @export
setGeneric("aggregateByKey",
function(x, zeroValue, seqOp, combOp, numPartitions) {
standardGeneric("aggregateByKey")
})
#' @rdname cogroup
#' @export
setGeneric("cogroup",
function(..., numPartitions) {
standardGeneric("cogroup")
},
signature = "...")
#' @rdname combineByKey
#' @seealso groupByKey, reduceByKey
#' @export
setGeneric("combineByKey",
function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) {
standardGeneric("combineByKey")
})
#' @rdname foldByKey
#' @seealso aggregateByKey, combineByKey
#' @export
setGeneric("foldByKey",
function(x, zeroValue, func, numPartitions) {
standardGeneric("foldByKey")
})
#' @rdname join-methods
#' @export
setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") })
#' @rdname groupByKey
#' @seealso reduceByKey
#' @export
setGeneric("groupByKey", function(x, numPartitions) { standardGeneric("groupByKey") })
#' @rdname join-methods
#' @export
setGeneric("join", function(x, y, ...) { standardGeneric("join") })
#' @rdname join-methods
#' @export
setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") })
#' @rdname partitionBy
#' @export
setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") })
#' @rdname reduceByKey
#' @seealso groupByKey
#' @export
setGeneric("reduceByKey", function(x, combineFunc, numPartitions) { standardGeneric("reduceByKey")})
#' @rdname reduceByKeyLocally
#' @seealso reduceByKey
#' @export
setGeneric("reduceByKeyLocally",
function(x, combineFunc) {
standardGeneric("reduceByKeyLocally")
})
#' @rdname join-methods
#' @export
setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") })
#' @rdname sortByKey
#' @export
setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1L) {
standardGeneric("sortByKey")
})
################### Broadcast Variable Methods #################
#' @rdname broadcast
#' @export
setGeneric("value", function(bcast) { standardGeneric("value") })
#################### DataFrame Methods ########################
#' @rdname schema
#' @export
setGeneric("columns", function(x) {standardGeneric("columns") })
#' @rdname schema
#' @export
setGeneric("dtypes", function(x) { standardGeneric("dtypes") })
#' @rdname explain
#' @export
setGeneric("explain", function(x, ...) { standardGeneric("explain") })
#' @rdname filter
#' @export
setGeneric("filter", function(x, condition) { standardGeneric("filter") })
#' @rdname DataFrame
#' @export
setGeneric("groupBy", function(x, ...) { standardGeneric("groupBy") })
#' @rdname insertInto
#' @export
setGeneric("insertInto", function(x, tableName, ...) { standardGeneric("insertInto") })
#' @rdname intersect
#' @export
setGeneric("intersect", function(x, y) { standardGeneric("intersect") })
#' @rdname isLocal
#' @export
setGeneric("isLocal", function(x) { standardGeneric("isLocal") })
#' @rdname limit
#' @export
setGeneric("limit", function(x, num) {standardGeneric("limit") })
#' @rdname sortDF
#' @export
setGeneric("orderBy", function(x, col) { standardGeneric("orderBy") })
#' @rdname schema
#' @export
setGeneric("printSchema", function(x) { standardGeneric("printSchema") })
#' @rdname registerTempTable
#' @export
setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") })
#' @rdname sampleDF
#' @export
setGeneric("sampleDF",
function(x, withReplacement, fraction, seed) {
standardGeneric("sampleDF")
})
#' @rdname saveAsParquetFile
#' @export
setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") })
#' @rdname saveAsTable
#' @export
setGeneric("saveAsTable", function(df, tableName, source, mode, ...) {
standardGeneric("saveAsTable")
})
#' @rdname saveAsTable
#' @export
setGeneric("saveDF", function(df, path, source, mode, ...) { standardGeneric("saveDF") })
#' @rdname schema
#' @export
setGeneric("schema", function(x) { standardGeneric("schema") })
#' @rdname select
#' @export
setGeneric("select", function(x, col, ...) { standardGeneric("select") } )
#' @rdname select
#' @export
setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") })
#' @rdname showDF
#' @export
setGeneric("showDF", function(x,...) { standardGeneric("showDF") })
#' @rdname sortDF
#' @export
setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") })
#' @rdname subtract
#' @export
setGeneric("subtract", function(x, y) { standardGeneric("subtract") })
#' @rdname tojson
#' @export
setGeneric("toJSON", function(x) { standardGeneric("toJSON") })
#' @rdname DataFrame
#' @export
setGeneric("toRDD", function(x) { standardGeneric("toRDD") })
#' @rdname unionAll
#' @export
setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") })
#' @rdname filter
#' @export
setGeneric("where", function(x, condition) { standardGeneric("where") })
#' @rdname withColumn
#' @export
setGeneric("withColumn", function(x, colName, col) { standardGeneric("withColumn") })
#' @rdname withColumnRenamed
#' @export
setGeneric("withColumnRenamed", function(x, existingCol, newCol) {
standardGeneric("withColumnRenamed") })
###################### Column Methods ##########################
#' @rdname column
#' @export
setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") })
#' @rdname column
#' @export
setGeneric("asc", function(x) { standardGeneric("asc") })
#' @rdname column
#' @export
setGeneric("avg", function(x, ...) { standardGeneric("avg") })
#' @rdname column
#' @export
setGeneric("cast", function(x, dataType) { standardGeneric("cast") })
#' @rdname column
#' @export
setGeneric("contains", function(x, ...) { standardGeneric("contains") })
#' @rdname column
#' @export
setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") })
#' @rdname column
#' @export
setGeneric("desc", function(x) { standardGeneric("desc") })
#' @rdname column
#' @export
setGeneric("endsWith", function(x, ...) { standardGeneric("endsWith") })
#' @rdname column
#' @export
setGeneric("getField", function(x, ...) { standardGeneric("getField") })
#' @rdname column
#' @export
setGeneric("getItem", function(x, ...) { standardGeneric("getItem") })
#' @rdname column
#' @export
setGeneric("isNull", function(x) { standardGeneric("isNull") })
#' @rdname column
#' @export
setGeneric("isNotNull", function(x) { standardGeneric("isNotNull") })
#' @rdname column
#' @export
setGeneric("last", function(x) { standardGeneric("last") })
#' @rdname column
#' @export
setGeneric("like", function(x, ...) { standardGeneric("like") })
#' @rdname column
#' @export
setGeneric("lower", function(x) { standardGeneric("lower") })
#' @rdname column
#' @export
setGeneric("rlike", function(x, ...) { standardGeneric("rlike") })
#' @rdname column
#' @export
setGeneric("startsWith", function(x, ...) { standardGeneric("startsWith") })
#' @rdname column
#' @export
setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") })
#' @rdname column
#' @export
setGeneric("upper", function(x) { standardGeneric("upper") })

132
R/pkg/R/group.R Normal file
View file

@ -0,0 +1,132 @@
#
# 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.
#
# group.R - GroupedData class and methods implemented in S4 OO classes
setOldClass("jobj")
#' @title S4 class that represents a GroupedData
#' @description GroupedDatas can be created using groupBy() on a DataFrame
#' @rdname GroupedData
#' @seealso groupBy
#'
#' @param sgd A Java object reference to the backing Scala GroupedData
#' @export
setClass("GroupedData",
slots = list(sgd = "jobj"))
setMethod("initialize", "GroupedData", function(.Object, sgd) {
.Object@sgd <- sgd
.Object
})
#' @rdname DataFrame
groupedData <- function(sgd) {
new("GroupedData", sgd)
}
#' @rdname show
setMethod("show", "GroupedData",
function(object) {
cat("GroupedData\n")
})
#' Count
#'
#' Count the number of rows for each group.
#' The resulting DataFrame will also contain the grouping columns.
#'
#' @param x a GroupedData
#' @return a DataFrame
#' @export
#' @examples
#' \dontrun{
#' count(groupBy(df, "name"))
#' }
setMethod("count",
signature(x = "GroupedData"),
function(x) {
dataFrame(callJMethod(x@sgd, "count"))
})
#' Agg
#'
#' Aggregates on the entire DataFrame without groups.
#' The resulting DataFrame will also contain the grouping columns.
#'
#' df2 <- agg(df, <column> = <aggFunction>)
#' df2 <- agg(df, newColName = aggFunction(column))
#'
#' @param x a GroupedData
#' @return a DataFrame
#' @rdname agg
#' @examples
#' \dontrun{
#' df2 <- agg(df, age = "sum") # new column name will be created as 'SUM(age#0)'
#' df2 <- agg(df, ageSum = sum(df$age)) # Creates a new column named ageSum
#' }
setGeneric("agg", function (x, ...) { standardGeneric("agg") })
setMethod("agg",
signature(x = "GroupedData"),
function(x, ...) {
cols = list(...)
stopifnot(length(cols) > 0)
if (is.character(cols[[1]])) {
cols <- varargsToEnv(...)
sdf <- callJMethod(x@sgd, "agg", cols)
} else if (class(cols[[1]]) == "Column") {
ns <- names(cols)
if (!is.null(ns)) {
for (n in ns) {
if (n != "") {
cols[[n]] = alias(cols[[n]], n)
}
}
}
jcols <- lapply(cols, function(c) { c@jc })
# the GroupedData.agg(col, cols*) API does not contain grouping Column
sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "aggWithGrouping",
x@sgd, listToSeq(jcols))
} else {
stop("agg can only support Column or character")
}
dataFrame(sdf)
})
# sum/mean/avg/min/max
methods <- c("sum", "mean", "avg", "min", "max")
createMethod <- function(name) {
setMethod(name,
signature(x = "GroupedData"),
function(x, ...) {
sdf <- callJMethod(x@sgd, name, toSeq(...))
dataFrame(sdf)
})
}
createMethods <- function() {
for (name in methods) {
createMethod(name)
}
}
createMethods()

101
R/pkg/R/jobj.R Normal file
View file

@ -0,0 +1,101 @@
#
# 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.
#
# References to objects that exist on the JVM backend
# are maintained using the jobj.
# Maintain a reference count of Java object references
# This allows us to GC the java object when it is safe
.validJobjs <- new.env(parent = emptyenv())
# List of object ids to be removed
.toRemoveJobjs <- new.env(parent = emptyenv())
# Check if jobj was created with the current SparkContext
isValidJobj <- function(jobj) {
if (exists(".scStartTime", envir = .sparkREnv)) {
jobj$appId == get(".scStartTime", envir = .sparkREnv)
} else {
FALSE
}
}
getJobj <- function(objId) {
newObj <- jobj(objId)
if (exists(objId, .validJobjs)) {
.validJobjs[[objId]] <- .validJobjs[[objId]] + 1
} else {
.validJobjs[[objId]] <- 1
}
newObj
}
# Handler for a java object that exists on the backend.
jobj <- function(objId) {
if (!is.character(objId)) {
stop("object id must be a character")
}
# NOTE: We need a new env for a jobj as we can only register
# finalizers for environments or external references pointers.
obj <- structure(new.env(parent = emptyenv()), class = "jobj")
obj$id <- objId
obj$appId <- get(".scStartTime", envir = .sparkREnv)
# Register a finalizer to remove the Java object when this reference
# is garbage collected in R
reg.finalizer(obj, cleanup.jobj)
obj
}
#' Print a JVM object reference.
#'
#' This function prints the type and id for an object stored
#' in the SparkR JVM backend.
#'
#' @param x The JVM object reference
#' @param ... further arguments passed to or from other methods
print.jobj <- function(x, ...) {
cls <- callJMethod(x, "getClass")
name <- callJMethod(cls, "getName")
cat("Java ref type", name, "id", x$id, "\n", sep = " ")
}
cleanup.jobj <- function(jobj) {
if (isValidJobj(jobj)) {
objId <- jobj$id
# If we don't know anything about this jobj, ignore it
if (exists(objId, envir = .validJobjs)) {
.validJobjs[[objId]] <- .validJobjs[[objId]] - 1
if (.validJobjs[[objId]] == 0) {
rm(list = objId, envir = .validJobjs)
# NOTE: We cannot call removeJObject here as the finalizer may be run
# in the middle of another RPC. Thus we queue up this object Id to be removed
# and then run all the removeJObject when the next RPC is called.
.toRemoveJobjs[[objId]] <- 1
}
}
}
}
clearJobjs <- function() {
valid <- ls(.validJobjs)
rm(list = valid, envir = .validJobjs)
removeList <- ls(.toRemoveJobjs)
rm(list = removeList, envir = .toRemoveJobjs)
}

789
R/pkg/R/pairRDD.R Normal file
View file

@ -0,0 +1,789 @@
#
# 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.
#
# Operations supported on RDDs contains pairs (i.e key, value)
############ Actions and Transformations ############
#' Look up elements of a key in an RDD
#'
#' @description
#' \code{lookup} returns a list of values in this RDD for key key.
#'
#' @param x The RDD to collect
#' @param key The key to look up for
#' @return a list of values in this RDD for key key
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' pairs <- list(c(1, 1), c(2, 2), c(1, 3))
#' rdd <- parallelize(sc, pairs)
#' lookup(rdd, 1) # list(1, 3)
#'}
#' @rdname lookup
#' @aliases lookup,RDD-method
setMethod("lookup",
signature(x = "RDD", key = "ANY"),
function(x, key) {
partitionFunc <- function(part) {
filtered <- part[unlist(lapply(part, function(i) { identical(key, i[[1]]) }))]
lapply(filtered, function(i) { i[[2]] })
}
valsRDD <- lapplyPartition(x, partitionFunc)
collect(valsRDD)
})
#' Count the number of elements for each key, and return the result to the
#' master as lists of (key, count) pairs.
#'
#' Same as countByKey in Spark.
#'
#' @param x The RDD to count keys.
#' @return list of (key, count) pairs, where count is number of each key in rdd.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1)))
#' countByKey(rdd) # ("a", 2L), ("b", 1L)
#'}
#' @rdname countByKey
#' @aliases countByKey,RDD-method
setMethod("countByKey",
signature(x = "RDD"),
function(x) {
keys <- lapply(x, function(item) { item[[1]] })
countByValue(keys)
})
#' Return an RDD with the keys of each tuple.
#'
#' @param x The RDD from which the keys of each tuple is returned.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)))
#' collect(keys(rdd)) # list(1, 3)
#'}
#' @rdname keys
#' @aliases keys,RDD
setMethod("keys",
signature(x = "RDD"),
function(x) {
func <- function(k) {
k[[1]]
}
lapply(x, func)
})
#' Return an RDD with the values of each tuple.
#'
#' @param x The RDD from which the values of each tuple is returned.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)))
#' collect(values(rdd)) # list(2, 4)
#'}
#' @rdname values
#' @aliases values,RDD
setMethod("values",
signature(x = "RDD"),
function(x) {
func <- function(v) {
v[[2]]
}
lapply(x, func)
})
#' Applies a function to all values of the elements, without modifying the keys.
#'
#' The same as `mapValues()' in Spark.
#'
#' @param X The RDD to apply the transformation.
#' @param FUN the transformation to apply on the value of each element.
#' @return a new RDD created by the transformation.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, 1:10)
#' makePairs <- lapply(rdd, function(x) { list(x, x) })
#' collect(mapValues(makePairs, function(x) { x * 2) })
#' Output: list(list(1,2), list(2,4), list(3,6), ...)
#'}
#' @rdname mapValues
#' @aliases mapValues,RDD,function-method
setMethod("mapValues",
signature(X = "RDD", FUN = "function"),
function(X, FUN) {
func <- function(x) {
list(x[[1]], FUN(x[[2]]))
}
lapply(X, func)
})
#' Pass each value in the key-value pair RDD through a flatMap function without
#' changing the keys; this also retains the original RDD's partitioning.
#'
#' The same as 'flatMapValues()' in Spark.
#'
#' @param X The RDD to apply the transformation.
#' @param FUN the transformation to apply on the value of each element.
#' @return a new RDD created by the transformation.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4))))
#' collect(flatMapValues(rdd, function(x) { x }))
#' Output: list(list(1,1), list(1,2), list(2,3), list(2,4))
#'}
#' @rdname flatMapValues
#' @aliases flatMapValues,RDD,function-method
setMethod("flatMapValues",
signature(X = "RDD", FUN = "function"),
function(X, FUN) {
flatMapFunc <- function(x) {
lapply(FUN(x[[2]]), function(v) { list(x[[1]], v) })
}
flatMap(X, flatMapFunc)
})
############ Shuffle Functions ############
#' Partition an RDD by key
#'
#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V).
#' For each element of this RDD, the partitioner is used to compute a hash
#' function and the RDD is partitioned using this hash value.
#'
#' @param x The RDD to partition. Should be an RDD where each element is
#' list(K, V) or c(K, V).
#' @param numPartitions Number of partitions to create.
#' @param ... Other optional arguments to partitionBy.
#'
#' @param partitionFunc The partition function to use. Uses a default hashCode
#' function if not provided
#' @return An RDD partitioned using the specified partitioner.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4))
#' rdd <- parallelize(sc, pairs)
#' parts <- partitionBy(rdd, 2L)
#' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4)
#'}
#' @rdname partitionBy
#' @aliases partitionBy,RDD,integer-method
setMethod("partitionBy",
signature(x = "RDD", numPartitions = "integer"),
function(x, numPartitions, partitionFunc = hashCode) {
#if (missing(partitionFunc)) {
# partitionFunc <- hashCode
#}
partitionFunc <- cleanClosure(partitionFunc)
serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL)
packageNamesArr <- serialize(.sparkREnv$.packages,
connection = NULL)
broadcastArr <- lapply(ls(.broadcastNames), function(name) {
get(name, .broadcastNames) })
jrdd <- getJRDD(x)
# We create a PairwiseRRDD that extends RDD[(Array[Byte],
# Array[Byte])], where the key is the hashed split, the value is
# the content (key-val pairs).
pairwiseRRDD <- newJObject("org.apache.spark.api.r.PairwiseRRDD",
callJMethod(jrdd, "rdd"),
as.integer(numPartitions),
serializedHashFuncBytes,
getSerializedMode(x),
packageNamesArr,
as.character(.sparkREnv$libname),
broadcastArr,
callJMethod(jrdd, "classTag"))
# Create a corresponding partitioner.
rPartitioner <- newJObject("org.apache.spark.HashPartitioner",
as.integer(numPartitions))
# Call partitionBy on the obtained PairwiseRDD.
javaPairRDD <- callJMethod(pairwiseRRDD, "asJavaPairRDD")
javaPairRDD <- callJMethod(javaPairRDD, "partitionBy", rPartitioner)
# Call .values() on the result to get back the final result, the
# shuffled acutal content key-val pairs.
r <- callJMethod(javaPairRDD, "values")
RDD(r, serializedMode = "byte")
})
#' Group values by key
#'
#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V).
#' and group values for each key in the RDD into a single sequence.
#'
#' @param x The RDD to group. Should be an RDD where each element is
#' list(K, V) or c(K, V).
#' @param numPartitions Number of partitions to create.
#' @return An RDD where each element is list(K, list(V))
#' @seealso reduceByKey
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4))
#' rdd <- parallelize(sc, pairs)
#' parts <- groupByKey(rdd, 2L)
#' grouped <- collect(parts)
#' grouped[[1]] # Should be a list(1, list(2, 4))
#'}
#' @rdname groupByKey
#' @aliases groupByKey,RDD,integer-method
setMethod("groupByKey",
signature(x = "RDD", numPartitions = "integer"),
function(x, numPartitions) {
shuffled <- partitionBy(x, numPartitions)
groupVals <- function(part) {
vals <- new.env()
keys <- new.env()
pred <- function(item) exists(item$hash, keys)
appendList <- function(acc, i) {
addItemToAccumulator(acc, i)
acc
}
makeList <- function(i) {
acc <- initAccumulator()
addItemToAccumulator(acc, i)
acc
}
# Each item in the partition is list of (K, V)
lapply(part,
function(item) {
item$hash <- as.character(hashCode(item[[1]]))
updateOrCreatePair(item, keys, vals, pred,
appendList, makeList)
})
# extract out data field
vals <- eapply(vals,
function(i) {
length(i$data) <- i$counter
i$data
})
# Every key in the environment contains a list
# Convert that to list(K, Seq[V])
convertEnvsToList(keys, vals)
}
lapplyPartition(shuffled, groupVals)
})
#' Merge values by key
#'
#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V).
#' and merges the values for each key using an associative reduce function.
#'
#' @param x The RDD to reduce by key. Should be an RDD where each element is
#' list(K, V) or c(K, V).
#' @param combineFunc The associative reduce function to use.
#' @param numPartitions Number of partitions to create.
#' @return An RDD where each element is list(K, V') where V' is the merged
#' value
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4))
#' rdd <- parallelize(sc, pairs)
#' parts <- reduceByKey(rdd, "+", 2L)
#' reduced <- collect(parts)
#' reduced[[1]] # Should be a list(1, 6)
#'}
#' @rdname reduceByKey
#' @aliases reduceByKey,RDD,integer-method
setMethod("reduceByKey",
signature(x = "RDD", combineFunc = "ANY", numPartitions = "integer"),
function(x, combineFunc, numPartitions) {
reduceVals <- function(part) {
vals <- new.env()
keys <- new.env()
pred <- function(item) exists(item$hash, keys)
lapply(part,
function(item) {
item$hash <- as.character(hashCode(item[[1]]))
updateOrCreatePair(item, keys, vals, pred, combineFunc, identity)
})
convertEnvsToList(keys, vals)
}
locallyReduced <- lapplyPartition(x, reduceVals)
shuffled <- partitionBy(locallyReduced, numPartitions)
lapplyPartition(shuffled, reduceVals)
})
#' Merge values by key locally
#'
#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V).
#' and merges the values for each key using an associative reduce function, but return the
#' results immediately to the driver as an R list.
#'
#' @param x The RDD to reduce by key. Should be an RDD where each element is
#' list(K, V) or c(K, V).
#' @param combineFunc The associative reduce function to use.
#' @return A list of elements of type list(K, V') where V' is the merged value for each key
#' @seealso reduceByKey
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4))
#' rdd <- parallelize(sc, pairs)
#' reduced <- reduceByKeyLocally(rdd, "+")
#' reduced # list(list(1, 6), list(1.1, 3))
#'}
#' @rdname reduceByKeyLocally
#' @aliases reduceByKeyLocally,RDD,integer-method
setMethod("reduceByKeyLocally",
signature(x = "RDD", combineFunc = "ANY"),
function(x, combineFunc) {
reducePart <- function(part) {
vals <- new.env()
keys <- new.env()
pred <- function(item) exists(item$hash, keys)
lapply(part,
function(item) {
item$hash <- as.character(hashCode(item[[1]]))
updateOrCreatePair(item, keys, vals, pred, combineFunc, identity)
})
list(list(keys, vals)) # return hash to avoid re-compute in merge
}
mergeParts <- function(accum, x) {
pred <- function(item) {
exists(item$hash, accum[[1]])
}
lapply(ls(x[[1]]),
function(name) {
item <- list(x[[1]][[name]], x[[2]][[name]])
item$hash <- name
updateOrCreatePair(item, accum[[1]], accum[[2]], pred, combineFunc, identity)
})
accum
}
reduced <- mapPartitions(x, reducePart)
merged <- reduce(reduced, mergeParts)
convertEnvsToList(merged[[1]], merged[[2]])
})
#' Combine values by key
#'
#' Generic function to combine the elements for each key using a custom set of
#' aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)],
#' for a "combined type" C. Note that V and C can be different -- for example, one
#' might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]).
#' Users provide three functions:
#' \itemize{
#' \item createCombiner, which turns a V into a C (e.g., creates a one-element list)
#' \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) -
#' \item mergeCombiners, to combine two C's into a single one (e.g., concatentates
#' two lists).
#' }
#'
#' @param x The RDD to combine. Should be an RDD where each element is
#' list(K, V) or c(K, V).
#' @param createCombiner Create a combiner (C) given a value (V)
#' @param mergeValue Merge the given value (V) with an existing combiner (C)
#' @param mergeCombiners Merge two combiners and return a new combiner
#' @param numPartitions Number of partitions to create.
#' @return An RDD where each element is list(K, C) where C is the combined type
#'
#' @seealso groupByKey, reduceByKey
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4))
#' rdd <- parallelize(sc, pairs)
#' parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L)
#' combined <- collect(parts)
#' combined[[1]] # Should be a list(1, 6)
#'}
#' @rdname combineByKey
#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method
setMethod("combineByKey",
signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY",
mergeCombiners = "ANY", numPartitions = "integer"),
function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) {
combineLocally <- function(part) {
combiners <- new.env()
keys <- new.env()
pred <- function(item) exists(item$hash, keys)
lapply(part,
function(item) {
item$hash <- as.character(item[[1]])
updateOrCreatePair(item, keys, combiners, pred, mergeValue, createCombiner)
})
convertEnvsToList(keys, combiners)
}
locallyCombined <- lapplyPartition(x, combineLocally)
shuffled <- partitionBy(locallyCombined, numPartitions)
mergeAfterShuffle <- function(part) {
combiners <- new.env()
keys <- new.env()
pred <- function(item) exists(item$hash, keys)
lapply(part,
function(item) {
item$hash <- as.character(item[[1]])
updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, identity)
})
convertEnvsToList(keys, combiners)
}
lapplyPartition(shuffled, mergeAfterShuffle)
})
#' Aggregate a pair RDD by each key.
#'
#' Aggregate the values of each key in an RDD, using given combine functions
#' and a neutral "zero value". This function can return a different result type,
#' U, than the type of the values in this RDD, V. Thus, we need one operation
#' for merging a V into a U and one operation for merging two U's, The former
#' operation is used for merging values within a partition, and the latter is
#' used for merging values between partitions. To avoid memory allocation, both
#' of these functions are allowed to modify and return their first argument
#' instead of creating a new U.
#'
#' @param x An RDD.
#' @param zeroValue A neutral "zero value".
#' @param seqOp A function to aggregate the values of each key. It may return
#' a different result type from the type of the values.
#' @param combOp A function to aggregate results of seqOp.
#' @return An RDD containing the aggregation result.
#' @seealso foldByKey, combineByKey
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4)))
#' zeroValue <- list(0, 0)
#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) }
#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) }
#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L)
#' # list(list(1, list(3, 2)), list(2, list(7, 2)))
#'}
#' @rdname aggregateByKey
#' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method
setMethod("aggregateByKey",
signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY",
combOp = "ANY", numPartitions = "integer"),
function(x, zeroValue, seqOp, combOp, numPartitions) {
createCombiner <- function(v) {
do.call(seqOp, list(zeroValue, v))
}
combineByKey(x, createCombiner, seqOp, combOp, numPartitions)
})
#' Fold a pair RDD by each key.
#'
#' Aggregate the values of each key in an RDD, using an associative function "func"
#' and a neutral "zero value" which may be added to the result an arbitrary
#' number of times, and must not change the result (e.g., 0 for addition, or
#' 1 for multiplication.).
#'
#' @param x An RDD.
#' @param zeroValue A neutral "zero value".
#' @param func An associative function for folding values of each key.
#' @return An RDD containing the aggregation result.
#' @seealso aggregateByKey, combineByKey
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4)))
#' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7))
#'}
#' @rdname foldByKey
#' @aliases foldByKey,RDD,ANY,ANY,integer-method
setMethod("foldByKey",
signature(x = "RDD", zeroValue = "ANY",
func = "ANY", numPartitions = "integer"),
function(x, zeroValue, func, numPartitions) {
aggregateByKey(x, zeroValue, func, func, numPartitions)
})
############ Binary Functions #############
#' Join two RDDs
#'
#' @description
#' \code{join} This function joins two RDDs where every element is of the form list(K, V).
#' The key types of the two RDDs should be the same.
#'
#' @param x An RDD to be joined. Should be an RDD where each element is
#' list(K, V).
#' @param y An RDD to be joined. Should be an RDD where each element is
#' list(K, V).
#' @param numPartitions Number of partitions to create.
#' @return a new RDD containing all pairs of elements with matching keys in
#' two input RDDs.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4)))
#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3)))
#' join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3))
#'}
#' @rdname join-methods
#' @aliases join,RDD,RDD-method
setMethod("join",
signature(x = "RDD", y = "RDD"),
function(x, y, numPartitions) {
xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) })
yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) })
doJoin <- function(v) {
joinTaggedList(v, list(FALSE, FALSE))
}
joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)),
doJoin)
})
#' Left outer join two RDDs
#'
#' @description
#' \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V).
#' The key types of the two RDDs should be the same.
#'
#' @param x An RDD to be joined. Should be an RDD where each element is
#' list(K, V).
#' @param y An RDD to be joined. Should be an RDD where each element is
#' list(K, V).
#' @param numPartitions Number of partitions to create.
#' @return For each element (k, v) in x, the resulting RDD will either contain
#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL))
#' if no elements in rdd2 have key k.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4)))
#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3)))
#' leftOuterJoin(rdd1, rdd2, 2L)
#' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL)))
#'}
#' @rdname join-methods
#' @aliases leftOuterJoin,RDD,RDD-method
setMethod("leftOuterJoin",
signature(x = "RDD", y = "RDD", numPartitions = "integer"),
function(x, y, numPartitions) {
xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) })
yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) })
doJoin <- function(v) {
joinTaggedList(v, list(FALSE, TRUE))
}
joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin)
})
#' Right outer join two RDDs
#'
#' @description
#' \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V).
#' The key types of the two RDDs should be the same.
#'
#' @param x An RDD to be joined. Should be an RDD where each element is
#' list(K, V).
#' @param y An RDD to be joined. Should be an RDD where each element is
#' list(K, V).
#' @param numPartitions Number of partitions to create.
#' @return For each element (k, w) in y, the resulting RDD will either contain
#' all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w))
#' if no elements in x have key k.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3)))
#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4)))
#' rightOuterJoin(rdd1, rdd2, 2L)
#' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)))
#'}
#' @rdname join-methods
#' @aliases rightOuterJoin,RDD,RDD-method
setMethod("rightOuterJoin",
signature(x = "RDD", y = "RDD", numPartitions = "integer"),
function(x, y, numPartitions) {
xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) })
yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) })
doJoin <- function(v) {
joinTaggedList(v, list(TRUE, FALSE))
}
joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin)
})
#' Full outer join two RDDs
#'
#' @description
#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V).
#' The key types of the two RDDs should be the same.
#'
#' @param x An RDD to be joined. Should be an RDD where each element is
#' list(K, V).
#' @param y An RDD to be joined. Should be an RDD where each element is
#' list(K, V).
#' @param numPartitions Number of partitions to create.
#' @return For each element (k, v) in x and (k, w) in y, the resulting RDD
#' will contain all pairs (k, (v, w)) for both (k, v) in x and
#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements
#' in x/y have key k.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3)))
#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4)))
#' fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)),
#' # list(1, list(3, 1)),
#' # list(2, list(NULL, 4)))
#' # list(3, list(3, NULL)),
#'}
#' @rdname join-methods
#' @aliases fullOuterJoin,RDD,RDD-method
setMethod("fullOuterJoin",
signature(x = "RDD", y = "RDD", numPartitions = "integer"),
function(x, y, numPartitions) {
xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) })
yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) })
doJoin <- function(v) {
joinTaggedList(v, list(TRUE, TRUE))
}
joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin)
})
#' For each key k in several RDDs, return a resulting RDD that
#' whose values are a list of values for the key in all RDDs.
#'
#' @param ... Several RDDs.
#' @param numPartitions Number of partitions to create.
#' @return a new RDD containing all pairs of elements with values in a list
#' in all RDDs.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4)))
#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3)))
#' cogroup(rdd1, rdd2, numPartitions = 2L)
#' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list()))
#'}
#' @rdname cogroup
#' @aliases cogroup,RDD-method
setMethod("cogroup",
"RDD",
function(..., numPartitions) {
rdds <- list(...)
rddsLen <- length(rdds)
for (i in 1:rddsLen) {
rdds[[i]] <- lapply(rdds[[i]],
function(x) { list(x[[1]], list(i, x[[2]])) })
# TODO(hao): As issue [SparkR-142] mentions, the right value of i
# will not be captured into UDF if getJRDD is not invoked.
# It should be resolved together with that issue.
getJRDD(rdds[[i]]) # Capture the closure.
}
union.rdd <- Reduce(unionRDD, rdds)
group.func <- function(vlist) {
res <- list()
length(res) <- rddsLen
for (x in vlist) {
i <- x[[1]]
acc <- res[[i]]
# Create an accumulator.
if (is.null(acc)) {
acc <- initAccumulator()
}
addItemToAccumulator(acc, x[[2]])
res[[i]] <- acc
}
lapply(res, function(acc) {
if (is.null(acc)) {
list()
} else {
acc$data
}
})
}
cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions),
group.func)
})
#' Sort a (k, v) pair RDD by k.
#'
#' @param x A (k, v) pair RDD to be sorted.
#' @param ascending A flag to indicate whether the sorting is ascending or descending.
#' @param numPartitions Number of partitions to create.
#' @return An RDD where all (k, v) pair elements are sorted.
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3)))
#' collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1))
#'}
#' @rdname sortByKey
#' @aliases sortByKey,RDD,RDD-method
setMethod("sortByKey",
signature(x = "RDD"),
function(x, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) {
rangeBounds <- list()
if (numPartitions > 1) {
rddSize <- count(x)
# constant from Spark's RangePartitioner
maxSampleSize <- numPartitions * 20
fraction <- min(maxSampleSize / max(rddSize, 1), 1.0)
samples <- collect(keys(sampleRDD(x, FALSE, fraction, 1L)))
# Note: the built-in R sort() function only works on atomic vectors
samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending)
if (length(samples) > 0) {
rangeBounds <- lapply(seq_len(numPartitions - 1),
function(i) {
j <- ceiling(length(samples) * i / numPartitions)
samples[j]
})
}
}
rangePartitionFunc <- function(key) {
partition <- 0
# TODO: Use binary search instead of linear search, similar with Spark
while (partition < length(rangeBounds) && key > rangeBounds[[partition + 1]]) {
partition <- partition + 1
}
if (ascending) {
partition
} else {
numPartitions - partition - 1
}
}
partitionFunc <- function(part) {
sortKeyValueList(part, decreasing = !ascending)
}
newRDD <- partitionBy(x, numPartitions, rangePartitionFunc)
lapplyPartition(newRDD, partitionFunc)
})

195
R/pkg/R/serialize.R Normal file
View file

@ -0,0 +1,195 @@
#
# 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.
#
# Utility functions to serialize R objects so they can be read in Java.
# Type mapping from R to Java
#
# NULL -> Void
# integer -> Int
# character -> String
# logical -> Boolean
# double, numeric -> Double
# raw -> Array[Byte]
# Date -> Date
# POSIXct,POSIXlt -> Time
#
# list[T] -> Array[T], where T is one of above mentioned types
# environment -> Map[String, T], where T is a native type
# jobj -> Object, where jobj is an object created in the backend
writeObject <- function(con, object, writeType = TRUE) {
# NOTE: In R vectors have same type as objects. So we don't support
# passing in vectors as arrays and instead require arrays to be passed
# as lists.
type <- class(object)[[1]] # class of POSIXlt is c("POSIXlt", "POSIXt")
if (writeType) {
writeType(con, type)
}
switch(type,
NULL = writeVoid(con),
integer = writeInt(con, object),
character = writeString(con, object),
logical = writeBoolean(con, object),
double = writeDouble(con, object),
numeric = writeDouble(con, object),
raw = writeRaw(con, object),
list = writeList(con, object),
jobj = writeJobj(con, object),
environment = writeEnv(con, object),
Date = writeDate(con, object),
POSIXlt = writeTime(con, object),
POSIXct = writeTime(con, object),
stop(paste("Unsupported type for serialization", type)))
}
writeVoid <- function(con) {
# no value for NULL
}
writeJobj <- function(con, value) {
if (!isValidJobj(value)) {
stop("invalid jobj ", value$id)
}
writeString(con, value$id)
}
writeString <- function(con, value) {
writeInt(con, as.integer(nchar(value) + 1))
writeBin(value, con, endian = "big")
}
writeInt <- function(con, value) {
writeBin(as.integer(value), con, endian = "big")
}
writeDouble <- function(con, value) {
writeBin(value, con, endian = "big")
}
writeBoolean <- function(con, value) {
# TRUE becomes 1, FALSE becomes 0
writeInt(con, as.integer(value))
}
writeRawSerialize <- function(outputCon, batch) {
outputSer <- serialize(batch, ascii = FALSE, connection = NULL)
writeRaw(outputCon, outputSer)
}
writeRowSerialize <- function(outputCon, rows) {
invisible(lapply(rows, function(r) {
bytes <- serializeRow(r)
writeRaw(outputCon, bytes)
}))
}
serializeRow <- function(row) {
rawObj <- rawConnection(raw(0), "wb")
on.exit(close(rawObj))
writeRow(rawObj, row)
rawConnectionValue(rawObj)
}
writeRow <- function(con, row) {
numCols <- length(row)
writeInt(con, numCols)
for (i in 1:numCols) {
writeObject(con, row[[i]])
}
}
writeRaw <- function(con, batch) {
writeInt(con, length(batch))
writeBin(batch, con, endian = "big")
}
writeType <- function(con, class) {
type <- switch(class,
NULL = "n",
integer = "i",
character = "c",
logical = "b",
double = "d",
numeric = "d",
raw = "r",
list = "l",
jobj = "j",
environment = "e",
Date = "D",
POSIXlt = 't',
POSIXct = 't',
stop(paste("Unsupported type for serialization", class)))
writeBin(charToRaw(type), con)
}
# Used to pass arrays where all the elements are of the same type
writeList <- function(con, arr) {
# All elements should be of same type
elemType <- unique(sapply(arr, function(elem) { class(elem) }))
stopifnot(length(elemType) <= 1)
# TODO: Empty lists are given type "character" right now.
# This may not work if the Java side expects array of any other type.
if (length(elemType) == 0) {
elemType <- class("somestring")
}
writeType(con, elemType)
writeInt(con, length(arr))
if (length(arr) > 0) {
for (a in arr) {
writeObject(con, a, FALSE)
}
}
}
# Used to pass in hash maps required on Java side.
writeEnv <- function(con, env) {
len <- length(env)
writeInt(con, len)
if (len > 0) {
writeList(con, as.list(ls(env)))
vals <- lapply(ls(env), function(x) { env[[x]] })
writeList(con, as.list(vals))
}
}
writeDate <- function(con, date) {
writeString(con, as.character(date))
}
writeTime <- function(con, time) {
writeDouble(con, as.double(time))
}
# Used to serialize in a list of objects where each
# object can be of a different type. Serialization format is
# <object type> <object> for each object
writeArgs <- function(con, args) {
if (length(args) > 0) {
for (a in args) {
writeObject(con, a)
}
}
}
writeStrings <- function(con, stringList) {
writeLines(unlist(stringList), con)
}

266
R/pkg/R/sparkR.R Normal file
View file

@ -0,0 +1,266 @@
#
# 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.
#
.sparkREnv <- new.env()
sparkR.onLoad <- function(libname, pkgname) {
.sparkREnv$libname <- libname
}
# Utility function that returns TRUE if we have an active connection to the
# backend and FALSE otherwise
connExists <- function(env) {
tryCatch({
exists(".sparkRCon", envir = env) && isOpen(env[[".sparkRCon"]])
}, error = function(err) {
return(FALSE)
})
}
#' Stop the Spark context.
#'
#' Also terminates the backend this R session is connected to
sparkR.stop <- function() {
env <- .sparkREnv
if (exists(".sparkRCon", envir = env)) {
# cat("Stopping SparkR\n")
if (exists(".sparkRjsc", envir = env)) {
sc <- get(".sparkRjsc", envir = env)
callJMethod(sc, "stop")
rm(".sparkRjsc", envir = env)
}
if (exists(".backendLaunched", envir = env)) {
callJStatic("SparkRHandler", "stopBackend")
}
# Also close the connection and remove it from our env
conn <- get(".sparkRCon", envir = env)
close(conn)
rm(".sparkRCon", envir = env)
rm(".scStartTime", envir = env)
}
if (exists(".monitorConn", envir = env)) {
conn <- get(".monitorConn", envir = env)
close(conn)
rm(".monitorConn", envir = env)
}
# Clear all broadcast variables we have
# as the jobj will not be valid if we restart the JVM
clearBroadcastVariables()
# Clear jobj maps
clearJobjs()
}
#' Initialize a new Spark Context.
#'
#' This function initializes a new SparkContext.
#'
#' @param master The Spark master URL.
#' @param appName Application name to register with cluster manager
#' @param sparkHome Spark Home directory
#' @param sparkEnvir Named list of environment variables to set on worker nodes.
#' @param sparkExecutorEnv Named list of environment variables to be used when launching executors.
#' @param sparkJars Character string vector of jar files to pass to the worker nodes.
#' @param sparkRLibDir The path where R is installed on the worker nodes.
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark")
#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark",
#' list(spark.executor.memory="1g"))
#' sc <- sparkR.init("yarn-client", "SparkR", "/home/spark",
#' list(spark.executor.memory="1g"),
#' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"),
#' c("jarfile1.jar","jarfile2.jar"))
#'}
sparkR.init <- function(
master = "",
appName = "SparkR",
sparkHome = Sys.getenv("SPARK_HOME"),
sparkEnvir = list(),
sparkExecutorEnv = list(),
sparkJars = "",
sparkRLibDir = "") {
if (exists(".sparkRjsc", envir = .sparkREnv)) {
cat("Re-using existing Spark Context. Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n")
return(get(".sparkRjsc", envir = .sparkREnv))
}
sparkMem <- Sys.getenv("SPARK_MEM", "512m")
jars <- suppressWarnings(normalizePath(as.character(sparkJars)))
# Classpath separator is ";" on Windows
# URI needs four /// as from http://stackoverflow.com/a/18522792
if (.Platform$OS.type == "unix") {
collapseChar <- ":"
uriSep <- "//"
} else {
collapseChar <- ";"
uriSep <- "////"
}
existingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "")
if (existingPort != "") {
backendPort <- existingPort
} else {
path <- tempfile(pattern = "backend_port")
launchBackend(
args = path,
sparkHome = sparkHome,
jars = jars,
sparkSubmitOpts = Sys.getenv("SPARKR_SUBMIT_ARGS", "sparkr-shell"))
# wait atmost 100 seconds for JVM to launch
wait <- 0.1
for (i in 1:25) {
Sys.sleep(wait)
if (file.exists(path)) {
break
}
wait <- wait * 1.25
}
if (!file.exists(path)) {
stop("JVM is not ready after 10 seconds")
}
f <- file(path, open='rb')
backendPort <- readInt(f)
monitorPort <- readInt(f)
close(f)
file.remove(path)
if (length(backendPort) == 0 || backendPort == 0 ||
length(monitorPort) == 0 || monitorPort == 0) {
stop("JVM failed to launch")
}
assign(".monitorConn", socketConnection(port = monitorPort), envir = .sparkREnv)
assign(".backendLaunched", 1, envir = .sparkREnv)
}
.sparkREnv$backendPort <- backendPort
tryCatch({
connectBackend("localhost", backendPort)
}, error = function(err) {
stop("Failed to connect JVM\n")
})
if (nchar(sparkHome) != 0) {
sparkHome <- normalizePath(sparkHome)
}
if (nchar(sparkRLibDir) != 0) {
.sparkREnv$libname <- sparkRLibDir
}
sparkEnvirMap <- new.env()
for (varname in names(sparkEnvir)) {
sparkEnvirMap[[varname]] <- sparkEnvir[[varname]]
}
sparkExecutorEnvMap <- new.env()
if (!any(names(sparkExecutorEnv) == "LD_LIBRARY_PATH")) {
sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH"))
}
for (varname in names(sparkExecutorEnv)) {
sparkExecutorEnvMap[[varname]] <- sparkExecutorEnv[[varname]]
}
nonEmptyJars <- Filter(function(x) { x != "" }, jars)
localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) })
# Set the start time to identify jobjs
# Seconds resolution is good enough for this purpose, so use ints
assign(".scStartTime", as.integer(Sys.time()), envir = .sparkREnv)
assign(
".sparkRjsc",
callJStatic(
"org.apache.spark.api.r.RRDD",
"createSparkContext",
master,
appName,
as.character(sparkHome),
as.list(localJarPaths),
sparkEnvirMap,
sparkExecutorEnvMap),
envir = .sparkREnv
)
sc <- get(".sparkRjsc", envir = .sparkREnv)
# Register a finalizer to sleep 1 seconds on R exit to make RStudio happy
reg.finalizer(.sparkREnv, function(x) { Sys.sleep(1) }, onexit = TRUE)
sc
}
#' Initialize a new SQLContext.
#'
#' This function creates a SparkContext from an existing JavaSparkContext and
#' then uses it to initialize a new SQLContext
#'
#' @param jsc The existing JavaSparkContext created with SparkR.init()
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRSQL.init(sc)
#'}
sparkRSQL.init <- function(jsc) {
if (exists(".sparkRSQLsc", envir = .sparkREnv)) {
return(get(".sparkRSQLsc", envir = .sparkREnv))
}
sqlCtx <- callJStatic("org.apache.spark.sql.api.r.SQLUtils",
"createSQLContext",
jsc)
assign(".sparkRSQLsc", sqlCtx, envir = .sparkREnv)
sqlCtx
}
#' Initialize a new HiveContext.
#'
#' This function creates a HiveContext from an existing JavaSparkContext
#'
#' @param jsc The existing JavaSparkContext created with SparkR.init()
#' @export
#' @examples
#'\dontrun{
#' sc <- sparkR.init()
#' sqlCtx <- sparkRHive.init(sc)
#'}
sparkRHive.init <- function(jsc) {
if (exists(".sparkRHivesc", envir = .sparkREnv)) {
return(get(".sparkRHivesc", envir = .sparkREnv))
}
ssc <- callJMethod(jsc, "sc")
hiveCtx <- tryCatch({
newJObject("org.apache.spark.sql.hive.HiveContext", ssc)
}, error = function(err) {
stop("Spark SQL is not built with Hive support")
})
assign(".sparkRHivesc", hiveCtx, envir = .sparkREnv)
hiveCtx
}

467
R/pkg/R/utils.R Normal file
View file

@ -0,0 +1,467 @@
#
# 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.
#
# Utilities and Helpers
# Given a JList<T>, returns an R list containing the same elements, the number
# of which is optionally upper bounded by `logicalUpperBound` (by default,
# return all elements). Takes care of deserializations and type conversions.
convertJListToRList <- function(jList, flatten, logicalUpperBound = NULL,
serializedMode = "byte") {
arrSize <- callJMethod(jList, "size")
# Datasets with serializedMode == "string" (such as an RDD directly generated by textFile()):
# each partition is not dense-packed into one Array[Byte], and `arrSize`
# here corresponds to number of logical elements. Thus we can prune here.
if (serializedMode == "string" && !is.null(logicalUpperBound)) {
arrSize <- min(arrSize, logicalUpperBound)
}
results <- if (arrSize > 0) {
lapply(0:(arrSize - 1),
function(index) {
obj <- callJMethod(jList, "get", as.integer(index))
# Assume it is either an R object or a Java obj ref.
if (inherits(obj, "jobj")) {
if (isInstanceOf(obj, "scala.Tuple2")) {
# JavaPairRDD[Array[Byte], Array[Byte]].
keyBytes = callJMethod(obj, "_1")
valBytes = callJMethod(obj, "_2")
res <- list(unserialize(keyBytes),
unserialize(valBytes))
} else {
stop(paste("utils.R: convertJListToRList only supports",
"RDD[Array[Byte]] and",
"JavaPairRDD[Array[Byte], Array[Byte]] for now"))
}
} else {
if (inherits(obj, "raw")) {
if (serializedMode == "byte") {
# RDD[Array[Byte]]. `obj` is a whole partition.
res <- unserialize(obj)
# For serialized datasets, `obj` (and `rRaw`) here corresponds to
# one whole partition dense-packed together. We deserialize the
# whole partition first, then cap the number of elements to be returned.
} else if (serializedMode == "row") {
res <- readRowList(obj)
# For DataFrames that have been converted to RRDDs, we call readRowList
# which will read in each row of the RRDD as a list and deserialize
# each element.
flatten <<- FALSE
# Use global assignment to change the flatten flag. This means
# we don't have to worry about the default argument in other functions
# e.g. collect
}
# TODO: is it possible to distinguish element boundary so that we can
# unserialize only what we need?
if (!is.null(logicalUpperBound)) {
res <- head(res, n = logicalUpperBound)
}
} else {
# obj is of a primitive Java type, is simplified to R's
# corresponding type.
res <- list(obj)
}
}
res
})
} else {
list()
}
if (flatten) {
as.list(unlist(results, recursive = FALSE))
} else {
as.list(results)
}
}
# Returns TRUE if `name` refers to an RDD in the given environment `env`
isRDD <- function(name, env) {
obj <- get(name, envir = env)
inherits(obj, "RDD")
}
#' Compute the hashCode of an object
#'
#' Java-style function to compute the hashCode for the given object. Returns
#' an integer value.
#'
#' @details
#' This only works for integer, numeric and character types right now.
#'
#' @param key the object to be hashed
#' @return the hash code as an integer
#' @export
#' @examples
#' hashCode(1L) # 1
#' hashCode(1.0) # 1072693248
#' hashCode("1") # 49
hashCode <- function(key) {
if (class(key) == "integer") {
as.integer(key[[1]])
} else if (class(key) == "numeric") {
# Convert the double to long and then calculate the hash code
rawVec <- writeBin(key[[1]], con = raw())
intBits <- packBits(rawToBits(rawVec), "integer")
as.integer(bitwXor(intBits[2], intBits[1]))
} else if (class(key) == "character") {
.Call("stringHashCode", key)
} else {
warning(paste("Could not hash object, returning 0", sep = ""))
as.integer(0)
}
}
# Create a new RDD with serializedMode == "byte".
# Return itself if already in "byte" format.
serializeToBytes <- function(rdd) {
if (!inherits(rdd, "RDD")) {
stop("Argument 'rdd' is not an RDD type.")
}
if (getSerializedMode(rdd) != "byte") {
ser.rdd <- lapply(rdd, function(x) { x })
return(ser.rdd)
} else {
return(rdd)
}
}
# Create a new RDD with serializedMode == "string".
# Return itself if already in "string" format.
serializeToString <- function(rdd) {
if (!inherits(rdd, "RDD")) {
stop("Argument 'rdd' is not an RDD type.")
}
if (getSerializedMode(rdd) != "string") {
ser.rdd <- lapply(rdd, function(x) { toString(x) })
# force it to create jrdd using "string"
getJRDD(ser.rdd, serializedMode = "string")
return(ser.rdd)
} else {
return(rdd)
}
}
# Fast append to list by using an accumulator.
# http://stackoverflow.com/questions/17046336/here-we-go-again-append-an-element-to-a-list-in-r
#
# The accumulator should has three fields size, counter and data.
# This function amortizes the allocation cost by doubling
# the size of the list every time it fills up.
addItemToAccumulator <- function(acc, item) {
if(acc$counter == acc$size) {
acc$size <- acc$size * 2
length(acc$data) <- acc$size
}
acc$counter <- acc$counter + 1
acc$data[[acc$counter]] <- item
}
initAccumulator <- function() {
acc <- new.env()
acc$counter <- 0
acc$data <- list(NULL)
acc$size <- 1
acc
}
# Utility function to sort a list of key value pairs
# Used in unit tests
sortKeyValueList <- function(kv_list, decreasing = FALSE) {
keys <- sapply(kv_list, function(x) x[[1]])
kv_list[order(keys, decreasing = decreasing)]
}
# Utility function to generate compact R lists from grouped rdd
# Used in Join-family functions
# param:
# tagged_list R list generated via groupByKey with tags(1L, 2L, ...)
# cnull Boolean list where each element determines whether the corresponding list should
# be converted to list(NULL)
genCompactLists <- function(tagged_list, cnull) {
len <- length(tagged_list)
lists <- list(vector("list", len), vector("list", len))
index <- list(1, 1)
for (x in tagged_list) {
tag <- x[[1]]
idx <- index[[tag]]
lists[[tag]][[idx]] <- x[[2]]
index[[tag]] <- idx + 1
}
len <- lapply(index, function(x) x - 1)
for (i in (1:2)) {
if (cnull[[i]] && len[[i]] == 0) {
lists[[i]] <- list(NULL)
} else {
length(lists[[i]]) <- len[[i]]
}
}
lists
}
# Utility function to merge compact R lists
# Used in Join-family functions
# param:
# left/right Two compact lists ready for Cartesian product
mergeCompactLists <- function(left, right) {
result <- list()
length(result) <- length(left) * length(right)
index <- 1
for (i in left) {
for (j in right) {
result[[index]] <- list(i, j)
index <- index + 1
}
}
result
}
# Utility function to wrapper above two operations
# Used in Join-family functions
# param (same as genCompactLists):
# tagged_list R list generated via groupByKey with tags(1L, 2L, ...)
# cnull Boolean list where each element determines whether the corresponding list should
# be converted to list(NULL)
joinTaggedList <- function(tagged_list, cnull) {
lists <- genCompactLists(tagged_list, cnull)
mergeCompactLists(lists[[1]], lists[[2]])
}
# Utility function to reduce a key-value list with predicate
# Used in *ByKey functions
# param
# pair key-value pair
# keys/vals env of key/value with hashes
# updateOrCreatePred predicate function
# updateFn update or merge function for existing pair, similar with `mergeVal` @combineByKey
# createFn create function for new pair, similar with `createCombiner` @combinebykey
updateOrCreatePair <- function(pair, keys, vals, updateOrCreatePred, updateFn, createFn) {
# assume hashVal bind to `$hash`, key/val with index 1/2
hashVal <- pair$hash
key <- pair[[1]]
val <- pair[[2]]
if (updateOrCreatePred(pair)) {
assign(hashVal, do.call(updateFn, list(get(hashVal, envir = vals), val)), envir = vals)
} else {
assign(hashVal, do.call(createFn, list(val)), envir = vals)
assign(hashVal, key, envir = keys)
}
}
# Utility function to convert key&values envs into key-val list
convertEnvsToList <- function(keys, vals) {
lapply(ls(keys),
function(name) {
list(keys[[name]], vals[[name]])
})
}
# Utility function to capture the varargs into environment object
varargsToEnv <- function(...) {
pairs <- as.list(substitute(list(...)))[-1L]
env <- new.env()
for (name in names(pairs)) {
env[[name]] <- pairs[[name]]
}
env
}
getStorageLevel <- function(newLevel = c("DISK_ONLY",
"DISK_ONLY_2",
"MEMORY_AND_DISK",
"MEMORY_AND_DISK_2",
"MEMORY_AND_DISK_SER",
"MEMORY_AND_DISK_SER_2",
"MEMORY_ONLY",
"MEMORY_ONLY_2",
"MEMORY_ONLY_SER",
"MEMORY_ONLY_SER_2",
"OFF_HEAP")) {
match.arg(newLevel)
storageLevel <- switch(newLevel,
"DISK_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY"),
"DISK_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY_2"),
"MEMORY_AND_DISK" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK"),
"MEMORY_AND_DISK_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_2"),
"MEMORY_AND_DISK_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER"),
"MEMORY_AND_DISK_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER_2"),
"MEMORY_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY"),
"MEMORY_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_2"),
"MEMORY_ONLY_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER"),
"MEMORY_ONLY_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER_2"),
"OFF_HEAP" = callJStatic("org.apache.spark.storage.StorageLevel", "OFF_HEAP"))
}
# Utility function for functions where an argument needs to be integer but we want to allow
# the user to type (for example) `5` instead of `5L` to avoid a confusing error message.
numToInt <- function(num) {
if (as.integer(num) != num) {
warning(paste("Coercing", as.list(sys.call())[[2]], "to integer."))
}
as.integer(num)
}
# create a Seq in JVM
toSeq <- function(...) {
callJStatic("org.apache.spark.sql.api.r.SQLUtils", "toSeq", list(...))
}
# create a Seq in JVM from a list
listToSeq <- function(l) {
callJStatic("org.apache.spark.sql.api.r.SQLUtils", "toSeq", l)
}
# Utility function to recursively traverse the Abstract Syntax Tree (AST) of a
# user defined function (UDF), and to examine variables in the UDF to decide
# if their values should be included in the new function environment.
# param
# node The current AST node in the traversal.
# oldEnv The original function environment.
# defVars An Accumulator of variables names defined in the function's calling environment,
# including function argument and local variable names.
# checkedFunc An environment of function objects examined during cleanClosure. It can
# be considered as a "name"-to-"list of functions" mapping.
# newEnv A new function environment to store necessary function dependencies, an output argument.
processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) {
nodeLen <- length(node)
if (nodeLen > 1 && typeof(node) == "language") {
# Recursive case: current AST node is an internal node, check for its children.
if (length(node[[1]]) > 1) {
for (i in 1:nodeLen) {
processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv)
}
} else { # if node[[1]] is length of 1, check for some R special functions.
nodeChar <- as.character(node[[1]])
if (nodeChar == "{" || nodeChar == "(") { # Skip start symbol.
for (i in 2:nodeLen) {
processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv)
}
} else if (nodeChar == "<-" || nodeChar == "=" ||
nodeChar == "<<-") { # Assignment Ops.
defVar <- node[[2]]
if (length(defVar) == 1 && typeof(defVar) == "symbol") {
# Add the defined variable name into defVars.
addItemToAccumulator(defVars, as.character(defVar))
} else {
processClosure(node[[2]], oldEnv, defVars, checkedFuncs, newEnv)
}
for (i in 3:nodeLen) {
processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv)
}
} else if (nodeChar == "function") { # Function definition.
# Add parameter names.
newArgs <- names(node[[2]])
lapply(newArgs, function(arg) { addItemToAccumulator(defVars, arg) })
for (i in 3:nodeLen) {
processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv)
}
} else if (nodeChar == "$") { # Skip the field.
processClosure(node[[2]], oldEnv, defVars, checkedFuncs, newEnv)
} else if (nodeChar == "::" || nodeChar == ":::") {
processClosure(node[[3]], oldEnv, defVars, checkedFuncs, newEnv)
} else {
for (i in 1:nodeLen) {
processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv)
}
}
}
} else if (nodeLen == 1 &&
(typeof(node) == "symbol" || typeof(node) == "language")) {
# Base case: current AST node is a leaf node and a symbol or a function call.
nodeChar <- as.character(node)
if (!nodeChar %in% defVars$data) { # Not a function parameter or local variable.
func.env <- oldEnv
topEnv <- parent.env(.GlobalEnv)
# Search in function environment, and function's enclosing environments
# up to global environment. There is no need to look into package environments
# above the global or namespace environment that is not SparkR below the global,
# as they are assumed to be loaded on workers.
while (!identical(func.env, topEnv)) {
# Namespaces other than "SparkR" will not be searched.
if (!isNamespace(func.env) ||
(getNamespaceName(func.env) == "SparkR" &&
!(nodeChar %in% getNamespaceExports("SparkR")))) { # Only include SparkR internals.
# Set parameter 'inherits' to FALSE since we do not need to search in
# attached package environments.
if (tryCatch(exists(nodeChar, envir = func.env, inherits = FALSE),
error = function(e) { FALSE })) {
obj <- get(nodeChar, envir = func.env, inherits = FALSE)
if (is.function(obj)) { # If the node is a function call.
funcList <- mget(nodeChar, envir = checkedFuncs, inherits = F,
ifnotfound = list(list(NULL)))[[1]]
found <- sapply(funcList, function(func) {
ifelse(identical(func, obj), TRUE, FALSE)
})
if (sum(found) > 0) { # If function has been examined, ignore.
break
}
# Function has not been examined, record it and recursively clean its closure.
assign(nodeChar,
if (is.null(funcList[[1]])) {
list(obj)
} else {
append(funcList, obj)
},
envir = checkedFuncs)
obj <- cleanClosure(obj, checkedFuncs)
}
assign(nodeChar, obj, envir = newEnv)
break
}
}
# Continue to search in enclosure.
func.env <- parent.env(func.env)
}
}
}
}
# Utility function to get user defined function (UDF) dependencies (closure).
# More specifically, this function captures the values of free variables defined
# outside a UDF, and stores them in the function's environment.
# param
# func A function whose closure needs to be captured.
# checkedFunc An environment of function objects examined during cleanClosure. It can be
# considered as a "name"-to-"list of functions" mapping.
# return value
# a new version of func that has an correct environment (closure).
cleanClosure <- function(func, checkedFuncs = new.env()) {
if (is.function(func)) {
newEnv <- new.env(parent = .GlobalEnv)
func.body <- body(func)
oldEnv <- environment(func)
# defVars is an Accumulator of variables names defined in the function's calling
# environment. First, function's arguments are added to defVars.
defVars <- initAccumulator()
argNames <- names(as.list(args(func)))
for (i in 1:(length(argNames) - 1)) { # Remove the ending NULL in pairlist.
addItemToAccumulator(defVars, argNames[i])
}
# Recursively examine variables in the function body.
processClosure(func.body, oldEnv, defVars, checkedFuncs, newEnv)
environment(func) <- newEnv
}
func
}

21
R/pkg/R/zzz.R Normal file
View file

@ -0,0 +1,21 @@
#
# 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.
#
.onLoad <- function(libname, pkgname) {
sparkR.onLoad(libname, pkgname)
}

View file

@ -0,0 +1,22 @@
#
# 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.
#
.First <- function() {
home <- Sys.getenv("SPARK_HOME")
.libPaths(c(file.path(home, "R", "lib"), .libPaths()))
Sys.setenv(NOAWT=1)
}

View file

@ -0,0 +1,31 @@
#
# 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.
#
.First <- function() {
home <- Sys.getenv("SPARK_HOME")
.libPaths(c(file.path(home, "R", "lib"), .libPaths()))
Sys.setenv(NOAWT=1)
library(utils)
library(SparkR)
sc <- sparkR.init(Sys.getenv("MASTER", unset = ""))
assign("sc", sc, envir=.GlobalEnv)
sqlCtx <- sparkRSQL.init(sc)
assign("sqlCtx", sqlCtx, envir=.GlobalEnv)
cat("\n Welcome to SparkR!")
cat("\n Spark context is available as sc, SQL context is available as sqlCtx\n")
}

View file

@ -0,0 +1,90 @@
#
# 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.
#
context("functions on binary files")
# JavaSparkContext handle
sc <- sparkR.init()
mockFile = c("Spark is pretty.", "Spark is awesome.")
test_that("saveAsObjectFile()/objectFile() following textFile() works", {
fileName1 <- tempfile(pattern="spark-test", fileext=".tmp")
fileName2 <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName1)
rdd <- textFile(sc, fileName1)
saveAsObjectFile(rdd, fileName2)
rdd <- objectFile(sc, fileName2)
expect_equal(collect(rdd), as.list(mockFile))
unlink(fileName1)
unlink(fileName2, recursive = TRUE)
})
test_that("saveAsObjectFile()/objectFile() works on a parallelized list", {
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
l <- list(1, 2, 3)
rdd <- parallelize(sc, l)
saveAsObjectFile(rdd, fileName)
rdd <- objectFile(sc, fileName)
expect_equal(collect(rdd), l)
unlink(fileName, recursive = TRUE)
})
test_that("saveAsObjectFile()/objectFile() following RDD transformations works", {
fileName1 <- tempfile(pattern="spark-test", fileext=".tmp")
fileName2 <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName1)
rdd <- textFile(sc, fileName1)
words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] })
wordCount <- lapply(words, function(word) { list(word, 1L) })
counts <- reduceByKey(wordCount, "+", 2L)
saveAsObjectFile(counts, fileName2)
counts <- objectFile(sc, fileName2)
output <- collect(counts)
expected <- list(list("awesome.", 1), list("Spark", 2), list("pretty.", 1),
list("is", 2))
expect_equal(sortKeyValueList(output), sortKeyValueList(expected))
unlink(fileName1)
unlink(fileName2, recursive = TRUE)
})
test_that("saveAsObjectFile()/objectFile() works with multiple paths", {
fileName1 <- tempfile(pattern="spark-test", fileext=".tmp")
fileName2 <- tempfile(pattern="spark-test", fileext=".tmp")
rdd1 <- parallelize(sc, "Spark is pretty.")
saveAsObjectFile(rdd1, fileName1)
rdd2 <- parallelize(sc, "Spark is awesome.")
saveAsObjectFile(rdd2, fileName2)
rdd <- objectFile(sc, c(fileName1, fileName2))
expect_true(count(rdd) == 2)
unlink(fileName1, recursive = TRUE)
unlink(fileName2, recursive = TRUE)
})

View file

@ -0,0 +1,68 @@
#
# 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.
#
context("binary functions")
# JavaSparkContext handle
sc <- sparkR.init()
# Data
nums <- 1:10
rdd <- parallelize(sc, nums, 2L)
# File content
mockFile <- c("Spark is pretty.", "Spark is awesome.")
test_that("union on two RDDs", {
actual <- collect(unionRDD(rdd, rdd))
expect_equal(actual, as.list(rep(nums, 2)))
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName)
text.rdd <- textFile(sc, fileName)
union.rdd <- unionRDD(rdd, text.rdd)
actual <- collect(union.rdd)
expect_equal(actual, c(as.list(nums), mockFile))
expect_true(getSerializedMode(union.rdd) == "byte")
rdd<- map(text.rdd, function(x) {x})
union.rdd <- unionRDD(rdd, text.rdd)
actual <- collect(union.rdd)
expect_equal(actual, as.list(c(mockFile, mockFile)))
expect_true(getSerializedMode(union.rdd) == "byte")
unlink(fileName)
})
test_that("cogroup on two RDDs", {
rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4)))
rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3)))
cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L)
actual <- collect(cogroup.rdd)
expect_equal(actual,
list(list(1, list(list(1), list(2, 3))), list(2, list(list(4), list()))))
rdd1 <- parallelize(sc, list(list("a", 1), list("a", 4)))
rdd2 <- parallelize(sc, list(list("b", 2), list("a", 3)))
cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L)
actual <- collect(cogroup.rdd)
expected <- list(list("b", list(list(), list(2))), list("a", list(list(1, 4), list(3))))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(expected))
})

View file

@ -0,0 +1,48 @@
#
# 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.
#
context("broadcast variables")
# JavaSparkContext handle
sc <- sparkR.init()
# Partitioned data
nums <- 1:2
rrdd <- parallelize(sc, nums, 2L)
test_that("using broadcast variable", {
randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100))
randomMatBr <- broadcast(sc, randomMat)
useBroadcast <- function(x) {
sum(value(randomMatBr) * x)
}
actual <- collect(lapply(rrdd, useBroadcast))
expected <- list(sum(randomMat) * 1, sum(randomMat) * 2)
expect_equal(actual, expected)
})
test_that("without using broadcast variable", {
randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100))
useBroadcast <- function(x) {
sum(randomMat * x)
}
actual <- collect(lapply(rrdd, useBroadcast))
expected <- list(sum(randomMat) * 1, sum(randomMat) * 2)
expect_equal(actual, expected)
})

View file

@ -0,0 +1,50 @@
#
# 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.
#
context("test functions in sparkR.R")
test_that("repeatedly starting and stopping SparkR", {
for (i in 1:4) {
sc <- sparkR.init()
rdd <- parallelize(sc, 1:20, 2L)
expect_equal(count(rdd), 20)
sparkR.stop()
}
})
test_that("rdd GC across sparkR.stop", {
sparkR.stop()
sc <- sparkR.init() # sc should get id 0
rdd1 <- parallelize(sc, 1:20, 2L) # rdd1 should get id 1
rdd2 <- parallelize(sc, 1:10, 2L) # rdd2 should get id 2
sparkR.stop()
sc <- sparkR.init() # sc should get id 0 again
# GC rdd1 before creating rdd3 and rdd2 after
rm(rdd1)
gc()
rdd3 <- parallelize(sc, 1:20, 2L) # rdd3 should get id 1 now
rdd4 <- parallelize(sc, 1:10, 2L) # rdd4 should get id 2 now
rm(rdd2)
gc()
count(rdd3)
count(rdd4)
})

View file

@ -0,0 +1,57 @@
#
# 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.
#
context("include R packages")
# JavaSparkContext handle
sc <- sparkR.init()
# Partitioned data
nums <- 1:2
rdd <- parallelize(sc, nums, 2L)
test_that("include inside function", {
# Only run the test if plyr is installed.
if ("plyr" %in% rownames(installed.packages())) {
suppressPackageStartupMessages(library(plyr))
generateData <- function(x) {
suppressPackageStartupMessages(library(plyr))
attach(airquality)
result <- transform(Ozone, logOzone = log(Ozone))
result
}
data <- lapplyPartition(rdd, generateData)
actual <- collect(data)
}
})
test_that("use include package", {
# Only run the test if plyr is installed.
if ("plyr" %in% rownames(installed.packages())) {
suppressPackageStartupMessages(library(plyr))
generateData <- function(x) {
attach(airquality)
result <- transform(Ozone, logOzone = log(Ozone))
result
}
includePackage(sc, plyr)
data <- lapplyPartition(rdd, generateData)
actual <- collect(data)
}
})

View file

@ -0,0 +1,109 @@
#
# 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.
#
context("parallelize() and collect()")
# Mock data
numVector <- c(-10:97)
numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10)
strVector <- c("Dexter Morgan: I suppose I should be upset, even feel",
"violated, but I'm not. No, in fact, I think this is a friendly",
"message, like \"Hey, wanna play?\" and yes, I want to play. ",
"I really, really do.")
strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ",
"other times it helps me control the chaos.",
"Dexter Morgan: Harry and Dorris Morgan did a wonderful job ",
"raising me. But they're both dead now. I didn't kill them. Honest.")
numPairs <- list(list(1, 1), list(1, 2), list(2, 2), list(2, 3))
strPairs <- list(list(strList, strList), list(strList, strList))
# JavaSparkContext handle
jsc <- sparkR.init()
# Tests
test_that("parallelize() on simple vectors and lists returns an RDD", {
numVectorRDD <- parallelize(jsc, numVector, 1)
numVectorRDD2 <- parallelize(jsc, numVector, 10)
numListRDD <- parallelize(jsc, numList, 1)
numListRDD2 <- parallelize(jsc, numList, 4)
strVectorRDD <- parallelize(jsc, strVector, 2)
strVectorRDD2 <- parallelize(jsc, strVector, 3)
strListRDD <- parallelize(jsc, strList, 4)
strListRDD2 <- parallelize(jsc, strList, 1)
rdds <- c(numVectorRDD,
numVectorRDD2,
numListRDD,
numListRDD2,
strVectorRDD,
strVectorRDD2,
strListRDD,
strListRDD2)
for (rdd in rdds) {
expect_true(inherits(rdd, "RDD"))
expect_true(.hasSlot(rdd, "jrdd")
&& inherits(rdd@jrdd, "jobj")
&& isInstanceOf(rdd@jrdd, "org.apache.spark.api.java.JavaRDD"))
}
})
test_that("collect(), following a parallelize(), gives back the original collections", {
numVectorRDD <- parallelize(jsc, numVector, 10)
expect_equal(collect(numVectorRDD), as.list(numVector))
numListRDD <- parallelize(jsc, numList, 1)
numListRDD2 <- parallelize(jsc, numList, 4)
expect_equal(collect(numListRDD), as.list(numList))
expect_equal(collect(numListRDD2), as.list(numList))
strVectorRDD <- parallelize(jsc, strVector, 2)
strVectorRDD2 <- parallelize(jsc, strVector, 3)
expect_equal(collect(strVectorRDD), as.list(strVector))
expect_equal(collect(strVectorRDD2), as.list(strVector))
strListRDD <- parallelize(jsc, strList, 4)
strListRDD2 <- parallelize(jsc, strList, 1)
expect_equal(collect(strListRDD), as.list(strList))
expect_equal(collect(strListRDD2), as.list(strList))
})
test_that("regression: collect() following a parallelize() does not drop elements", {
# 10 %/% 6 = 1, ceiling(10 / 6) = 2
collLen <- 10
numPart <- 6
expected <- runif(collLen)
actual <- collect(parallelize(jsc, expected, numPart))
expect_equal(actual, as.list(expected))
})
test_that("parallelize() and collect() work for lists of pairs (pairwise data)", {
# use the pairwise logical to indicate pairwise data
numPairsRDDD1 <- parallelize(jsc, numPairs, 1)
numPairsRDDD2 <- parallelize(jsc, numPairs, 2)
numPairsRDDD3 <- parallelize(jsc, numPairs, 3)
expect_equal(collect(numPairsRDDD1), numPairs)
expect_equal(collect(numPairsRDDD2), numPairs)
expect_equal(collect(numPairsRDDD3), numPairs)
# can also leave out the parameter name, if the params are supplied in order
strPairsRDDD1 <- parallelize(jsc, strPairs, 1)
strPairsRDDD2 <- parallelize(jsc, strPairs, 2)
expect_equal(collect(strPairsRDDD1), strPairs)
expect_equal(collect(strPairsRDDD2), strPairs)
})

644
R/pkg/inst/tests/test_rdd.R Normal file
View file

@ -0,0 +1,644 @@
#
# 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.
#
context("basic RDD functions")
# JavaSparkContext handle
sc <- sparkR.init()
# Data
nums <- 1:10
rdd <- parallelize(sc, nums, 2L)
intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200))
intRdd <- parallelize(sc, intPairs, 2L)
test_that("get number of partitions in RDD", {
expect_equal(numPartitions(rdd), 2)
expect_equal(numPartitions(intRdd), 2)
})
test_that("first on RDD", {
expect_true(first(rdd) == 1)
newrdd <- lapply(rdd, function(x) x + 1)
expect_true(first(newrdd) == 2)
})
test_that("count and length on RDD", {
expect_equal(count(rdd), 10)
expect_equal(length(rdd), 10)
})
test_that("count by values and keys", {
mods <- lapply(rdd, function(x) { x %% 3 })
actual <- countByValue(mods)
expected <- list(list(0, 3L), list(1, 4L), list(2, 3L))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
actual <- countByKey(intRdd)
expected <- list(list(2L, 2L), list(1L, 2L))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("lapply on RDD", {
multiples <- lapply(rdd, function(x) { 2 * x })
actual <- collect(multiples)
expect_equal(actual, as.list(nums * 2))
})
test_that("lapplyPartition on RDD", {
sums <- lapplyPartition(rdd, function(part) { sum(unlist(part)) })
actual <- collect(sums)
expect_equal(actual, list(15, 40))
})
test_that("mapPartitions on RDD", {
sums <- mapPartitions(rdd, function(part) { sum(unlist(part)) })
actual <- collect(sums)
expect_equal(actual, list(15, 40))
})
test_that("flatMap() on RDDs", {
flat <- flatMap(intRdd, function(x) { list(x, x) })
actual <- collect(flat)
expect_equal(actual, rep(intPairs, each=2))
})
test_that("filterRDD on RDD", {
filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 })
actual <- collect(filtered.rdd)
expect_equal(actual, list(2, 4, 6, 8, 10))
filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd)
actual <- collect(filtered.rdd)
expect_equal(actual, list(list(1L, -1)))
# Filter out all elements.
filtered.rdd <- filterRDD(rdd, function(x) { x > 10 })
actual <- collect(filtered.rdd)
expect_equal(actual, list())
})
test_that("lookup on RDD", {
vals <- lookup(intRdd, 1L)
expect_equal(vals, list(-1, 200))
vals <- lookup(intRdd, 3L)
expect_equal(vals, list())
})
test_that("several transformations on RDD (a benchmark on PipelinedRDD)", {
rdd2 <- rdd
for (i in 1:12)
rdd2 <- lapplyPartitionsWithIndex(
rdd2, function(split, part) {
part <- as.list(unlist(part) * split + i)
})
rdd2 <- lapply(rdd2, function(x) x + x)
actual <- collect(rdd2)
expected <- list(24, 24, 24, 24, 24,
168, 170, 172, 174, 176)
expect_equal(actual, expected)
})
test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", {
# RDD
rdd2 <- rdd
# PipelinedRDD
rdd2 <- lapplyPartitionsWithIndex(
rdd2,
function(split, part) {
part <- as.list(unlist(part) * split)
})
cache(rdd2)
expect_true(rdd2@env$isCached)
rdd2 <- lapply(rdd2, function(x) x)
expect_false(rdd2@env$isCached)
unpersist(rdd2)
expect_false(rdd2@env$isCached)
persist(rdd2, "MEMORY_AND_DISK")
expect_true(rdd2@env$isCached)
rdd2 <- lapply(rdd2, function(x) x)
expect_false(rdd2@env$isCached)
unpersist(rdd2)
expect_false(rdd2@env$isCached)
setCheckpointDir(sc, "checkpoints")
checkpoint(rdd2)
expect_true(rdd2@env$isCheckpointed)
rdd2 <- lapply(rdd2, function(x) x)
expect_false(rdd2@env$isCached)
expect_false(rdd2@env$isCheckpointed)
# make sure the data is collectable
collect(rdd2)
unlink("checkpoints")
})
test_that("reduce on RDD", {
sum <- reduce(rdd, "+")
expect_equal(sum, 55)
# Also test with an inline function
sumInline <- reduce(rdd, function(x, y) { x + y })
expect_equal(sumInline, 55)
})
test_that("lapply with dependency", {
fa <- 5
multiples <- lapply(rdd, function(x) { fa * x })
actual <- collect(multiples)
expect_equal(actual, as.list(nums * 5))
})
test_that("lapplyPartitionsWithIndex on RDDs", {
func <- function(splitIndex, part) { list(splitIndex, Reduce("+", part)) }
actual <- collect(lapplyPartitionsWithIndex(rdd, func), flatten = FALSE)
expect_equal(actual, list(list(0, 15), list(1, 40)))
pairsRDD <- parallelize(sc, list(list(1, 2), list(3, 4), list(4, 8)), 1L)
partitionByParity <- function(key) { if (key %% 2 == 1) 0 else 1 }
mkTup <- function(splitIndex, part) { list(splitIndex, part) }
actual <- collect(lapplyPartitionsWithIndex(
partitionBy(pairsRDD, 2L, partitionByParity),
mkTup),
FALSE)
expect_equal(actual, list(list(0, list(list(1, 2), list(3, 4))),
list(1, list(list(4, 8)))))
})
test_that("sampleRDD() on RDDs", {
expect_equal(unlist(collect(sampleRDD(rdd, FALSE, 1.0, 2014L))), nums)
})
test_that("takeSample() on RDDs", {
# ported from RDDSuite.scala, modified seeds
data <- parallelize(sc, 1:100, 2L)
for (seed in 4:5) {
s <- takeSample(data, FALSE, 20L, seed)
expect_equal(length(s), 20L)
expect_equal(length(unique(s)), 20L)
for (elem in s) {
expect_true(elem >= 1 && elem <= 100)
}
}
for (seed in 4:5) {
s <- takeSample(data, FALSE, 200L, seed)
expect_equal(length(s), 100L)
expect_equal(length(unique(s)), 100L)
for (elem in s) {
expect_true(elem >= 1 && elem <= 100)
}
}
for (seed in 4:5) {
s <- takeSample(data, TRUE, 20L, seed)
expect_equal(length(s), 20L)
for (elem in s) {
expect_true(elem >= 1 && elem <= 100)
}
}
for (seed in 4:5) {
s <- takeSample(data, TRUE, 100L, seed)
expect_equal(length(s), 100L)
# Chance of getting all distinct elements is astronomically low, so test we
# got < 100
expect_true(length(unique(s)) < 100L)
}
for (seed in 4:5) {
s <- takeSample(data, TRUE, 200L, seed)
expect_equal(length(s), 200L)
# Chance of getting all distinct elements is still quite low, so test we
# got < 100
expect_true(length(unique(s)) < 100L)
}
})
test_that("mapValues() on pairwise RDDs", {
multiples <- mapValues(intRdd, function(x) { x * 2 })
actual <- collect(multiples)
expected <- lapply(intPairs, function(x) {
list(x[[1]], x[[2]] * 2)
})
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("flatMapValues() on pairwise RDDs", {
l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4))))
actual <- collect(flatMapValues(l, function(x) { x }))
expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4)))
# Generate x to x+1 for every value
actual <- collect(flatMapValues(intRdd, function(x) { x:(x + 1) }))
expect_equal(actual,
list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101),
list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201)))
})
test_that("reduceByKeyLocally() on PairwiseRDDs", {
pairs <- parallelize(sc, list(list(1, 2), list(1.1, 3), list(1, 4)), 2L)
actual <- reduceByKeyLocally(pairs, "+")
expect_equal(sortKeyValueList(actual),
sortKeyValueList(list(list(1, 6), list(1.1, 3))))
pairs <- parallelize(sc, list(list("abc", 1.2), list(1.1, 0), list("abc", 1.3),
list("bb", 5)), 4L)
actual <- reduceByKeyLocally(pairs, "+")
expect_equal(sortKeyValueList(actual),
sortKeyValueList(list(list("abc", 2.5), list(1.1, 0), list("bb", 5))))
})
test_that("distinct() on RDDs", {
nums.rep2 <- rep(1:10, 2)
rdd.rep2 <- parallelize(sc, nums.rep2, 2L)
uniques <- distinct(rdd.rep2)
actual <- sort(unlist(collect(uniques)))
expect_equal(actual, nums)
})
test_that("maximum() on RDDs", {
max <- maximum(rdd)
expect_equal(max, 10)
})
test_that("minimum() on RDDs", {
min <- minimum(rdd)
expect_equal(min, 1)
})
test_that("sumRDD() on RDDs", {
sum <- sumRDD(rdd)
expect_equal(sum, 55)
})
test_that("keyBy on RDDs", {
func <- function(x) { x*x }
keys <- keyBy(rdd, func)
actual <- collect(keys)
expect_equal(actual, lapply(nums, function(x) { list(func(x), x) }))
})
test_that("repartition/coalesce on RDDs", {
rdd <- parallelize(sc, 1:20, 4L) # each partition contains 5 elements
# repartition
r1 <- repartition(rdd, 2)
expect_equal(numPartitions(r1), 2L)
count <- length(collectPartition(r1, 0L))
expect_true(count >= 8 && count <= 12)
r2 <- repartition(rdd, 6)
expect_equal(numPartitions(r2), 6L)
count <- length(collectPartition(r2, 0L))
expect_true(count >=0 && count <= 4)
# coalesce
r3 <- coalesce(rdd, 1)
expect_equal(numPartitions(r3), 1L)
count <- length(collectPartition(r3, 0L))
expect_equal(count, 20)
})
test_that("sortBy() on RDDs", {
sortedRdd <- sortBy(rdd, function(x) { x * x }, ascending = FALSE)
actual <- collect(sortedRdd)
expect_equal(actual, as.list(sort(nums, decreasing = TRUE)))
rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L)
sortedRdd2 <- sortBy(rdd2, function(x) { x * x })
actual <- collect(sortedRdd2)
expect_equal(actual, as.list(nums))
})
test_that("takeOrdered() on RDDs", {
l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7)
rdd <- parallelize(sc, l)
actual <- takeOrdered(rdd, 6L)
expect_equal(actual, as.list(sort(unlist(l)))[1:6])
l <- list("e", "d", "c", "d", "a")
rdd <- parallelize(sc, l)
actual <- takeOrdered(rdd, 3L)
expect_equal(actual, as.list(sort(unlist(l)))[1:3])
})
test_that("top() on RDDs", {
l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7)
rdd <- parallelize(sc, l)
actual <- top(rdd, 6L)
expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:6])
l <- list("e", "d", "c", "d", "a")
rdd <- parallelize(sc, l)
actual <- top(rdd, 3L)
expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:3])
})
test_that("fold() on RDDs", {
actual <- fold(rdd, 0, "+")
expect_equal(actual, Reduce("+", nums, 0))
rdd <- parallelize(sc, list())
actual <- fold(rdd, 0, "+")
expect_equal(actual, 0)
})
test_that("aggregateRDD() on RDDs", {
rdd <- parallelize(sc, list(1, 2, 3, 4))
zeroValue <- list(0, 0)
seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) }
combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) }
actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp)
expect_equal(actual, list(10, 4))
rdd <- parallelize(sc, list())
actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp)
expect_equal(actual, list(0, 0))
})
test_that("zipWithUniqueId() on RDDs", {
rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L)
actual <- collect(zipWithUniqueId(rdd))
expected <- list(list("a", 0), list("b", 3), list("c", 1),
list("d", 4), list("e", 2))
expect_equal(actual, expected)
rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L)
actual <- collect(zipWithUniqueId(rdd))
expected <- list(list("a", 0), list("b", 1), list("c", 2),
list("d", 3), list("e", 4))
expect_equal(actual, expected)
})
test_that("zipWithIndex() on RDDs", {
rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L)
actual <- collect(zipWithIndex(rdd))
expected <- list(list("a", 0), list("b", 1), list("c", 2),
list("d", 3), list("e", 4))
expect_equal(actual, expected)
rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L)
actual <- collect(zipWithIndex(rdd))
expected <- list(list("a", 0), list("b", 1), list("c", 2),
list("d", 3), list("e", 4))
expect_equal(actual, expected)
})
test_that("glom() on RDD", {
rdd <- parallelize(sc, as.list(1:4), 2L)
actual <- collect(glom(rdd))
expect_equal(actual, list(list(1, 2), list(3, 4)))
})
test_that("keys() on RDDs", {
keys <- keys(intRdd)
actual <- collect(keys)
expect_equal(actual, lapply(intPairs, function(x) { x[[1]] }))
})
test_that("values() on RDDs", {
values <- values(intRdd)
actual <- collect(values)
expect_equal(actual, lapply(intPairs, function(x) { x[[2]] }))
})
test_that("pipeRDD() on RDDs", {
actual <- collect(pipeRDD(rdd, "more"))
expected <- as.list(as.character(1:10))
expect_equal(actual, expected)
trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n"))
actual <- collect(pipeRDD(trailed.rdd, "sort"))
expected <- list("", "1", "2", "3")
expect_equal(actual, expected)
rev.nums <- 9:0
rev.rdd <- parallelize(sc, rev.nums, 2L)
actual <- collect(pipeRDD(rev.rdd, "sort"))
expected <- as.list(as.character(c(5:9, 0:4)))
expect_equal(actual, expected)
})
test_that("zipRDD() on RDDs", {
rdd1 <- parallelize(sc, 0:4, 2)
rdd2 <- parallelize(sc, 1000:1004, 2)
actual <- collect(zipRDD(rdd1, rdd2))
expect_equal(actual,
list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)))
mockFile = c("Spark is pretty.", "Spark is awesome.")
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName)
rdd <- textFile(sc, fileName, 1)
actual <- collect(zipRDD(rdd, rdd))
expected <- lapply(mockFile, function(x) { list(x ,x) })
expect_equal(actual, expected)
rdd1 <- parallelize(sc, 0:1, 1)
actual <- collect(zipRDD(rdd1, rdd))
expected <- lapply(0:1, function(x) { list(x, mockFile[x + 1]) })
expect_equal(actual, expected)
rdd1 <- map(rdd, function(x) { x })
actual <- collect(zipRDD(rdd, rdd1))
expected <- lapply(mockFile, function(x) { list(x, x) })
expect_equal(actual, expected)
unlink(fileName)
})
test_that("join() on pairwise RDDs", {
rdd1 <- parallelize(sc, list(list(1,1), list(2,4)))
rdd2 <- parallelize(sc, list(list(1,2), list(1,3)))
actual <- collect(join(rdd1, rdd2, 2L))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(list(list(1, list(1, 2)), list(1, list(1, 3)))))
rdd1 <- parallelize(sc, list(list("a",1), list("b",4)))
rdd2 <- parallelize(sc, list(list("a",2), list("a",3)))
actual <- collect(join(rdd1, rdd2, 2L))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(list(list("a", list(1, 2)), list("a", list(1, 3)))))
rdd1 <- parallelize(sc, list(list(1,1), list(2,2)))
rdd2 <- parallelize(sc, list(list(3,3), list(4,4)))
actual <- collect(join(rdd1, rdd2, 2L))
expect_equal(actual, list())
rdd1 <- parallelize(sc, list(list("a",1), list("b",2)))
rdd2 <- parallelize(sc, list(list("c",3), list("d",4)))
actual <- collect(join(rdd1, rdd2, 2L))
expect_equal(actual, list())
})
test_that("leftOuterJoin() on pairwise RDDs", {
rdd1 <- parallelize(sc, list(list(1,1), list(2,4)))
rdd2 <- parallelize(sc, list(list(1,2), list(1,3)))
actual <- collect(leftOuterJoin(rdd1, rdd2, 2L))
expected <- list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL)))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(expected))
rdd1 <- parallelize(sc, list(list("a",1), list("b",4)))
rdd2 <- parallelize(sc, list(list("a",2), list("a",3)))
actual <- collect(leftOuterJoin(rdd1, rdd2, 2L))
expected <- list(list("b", list(4, NULL)), list("a", list(1, 2)), list("a", list(1, 3)))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(expected))
rdd1 <- parallelize(sc, list(list(1,1), list(2,2)))
rdd2 <- parallelize(sc, list(list(3,3), list(4,4)))
actual <- collect(leftOuterJoin(rdd1, rdd2, 2L))
expected <- list(list(1, list(1, NULL)), list(2, list(2, NULL)))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(expected))
rdd1 <- parallelize(sc, list(list("a",1), list("b",2)))
rdd2 <- parallelize(sc, list(list("c",3), list("d",4)))
actual <- collect(leftOuterJoin(rdd1, rdd2, 2L))
expected <- list(list("b", list(2, NULL)), list("a", list(1, NULL)))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(expected))
})
test_that("rightOuterJoin() on pairwise RDDs", {
rdd1 <- parallelize(sc, list(list(1,2), list(1,3)))
rdd2 <- parallelize(sc, list(list(1,1), list(2,4)))
actual <- collect(rightOuterJoin(rdd1, rdd2, 2L))
expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
rdd1 <- parallelize(sc, list(list("a",2), list("a",3)))
rdd2 <- parallelize(sc, list(list("a",1), list("b",4)))
actual <- collect(rightOuterJoin(rdd1, rdd2, 2L))
expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1)))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(expected))
rdd1 <- parallelize(sc, list(list(1,1), list(2,2)))
rdd2 <- parallelize(sc, list(list(3,3), list(4,4)))
actual <- collect(rightOuterJoin(rdd1, rdd2, 2L))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(list(list(3, list(NULL, 3)), list(4, list(NULL, 4)))))
rdd1 <- parallelize(sc, list(list("a",1), list("b",2)))
rdd2 <- parallelize(sc, list(list("c",3), list("d",4)))
actual <- collect(rightOuterJoin(rdd1, rdd2, 2L))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(list(list("d", list(NULL, 4)), list("c", list(NULL, 3)))))
})
test_that("fullOuterJoin() on pairwise RDDs", {
rdd1 <- parallelize(sc, list(list(1,2), list(1,3), list(3,3)))
rdd2 <- parallelize(sc, list(list(1,1), list(2,4)))
actual <- collect(fullOuterJoin(rdd1, rdd2, 2L))
expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)), list(3, list(3, NULL)))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
rdd1 <- parallelize(sc, list(list("a",2), list("a",3), list("c", 1)))
rdd2 <- parallelize(sc, list(list("a",1), list("b",4)))
actual <- collect(fullOuterJoin(rdd1, rdd2, 2L))
expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1)), list("c", list(1, NULL)))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(expected))
rdd1 <- parallelize(sc, list(list(1,1), list(2,2)))
rdd2 <- parallelize(sc, list(list(3,3), list(4,4)))
actual <- collect(fullOuterJoin(rdd1, rdd2, 2L))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), list(3, list(NULL, 3)), list(4, list(NULL, 4)))))
rdd1 <- parallelize(sc, list(list("a",1), list("b",2)))
rdd2 <- parallelize(sc, list(list("c",3), list("d",4)))
actual <- collect(fullOuterJoin(rdd1, rdd2, 2L))
expect_equal(sortKeyValueList(actual),
sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), list("d", list(NULL, 4)), list("c", list(NULL, 3)))))
})
test_that("sortByKey() on pairwise RDDs", {
numPairsRdd <- map(rdd, function(x) { list (x, x) })
sortedRdd <- sortByKey(numPairsRdd, ascending = FALSE)
actual <- collect(sortedRdd)
numPairs <- lapply(nums, function(x) { list (x, x) })
expect_equal(actual, sortKeyValueList(numPairs, decreasing = TRUE))
rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L)
numPairsRdd2 <- map(rdd2, function(x) { list (x, x) })
sortedRdd2 <- sortByKey(numPairsRdd2)
actual <- collect(sortedRdd2)
expect_equal(actual, numPairs)
# sort by string keys
l <- list(list("a", 1), list("b", 2), list("1", 3), list("d", 4), list("2", 5))
rdd3 <- parallelize(sc, l, 2L)
sortedRdd3 <- sortByKey(rdd3)
actual <- collect(sortedRdd3)
expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4)))
# test on the boundary cases
# boundary case 1: the RDD to be sorted has only 1 partition
rdd4 <- parallelize(sc, l, 1L)
sortedRdd4 <- sortByKey(rdd4)
actual <- collect(sortedRdd4)
expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4)))
# boundary case 2: the sorted RDD has only 1 partition
rdd5 <- parallelize(sc, l, 2L)
sortedRdd5 <- sortByKey(rdd5, numPartitions = 1L)
actual <- collect(sortedRdd5)
expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4)))
# boundary case 3: the RDD to be sorted has only 1 element
l2 <- list(list("a", 1))
rdd6 <- parallelize(sc, l2, 2L)
sortedRdd6 <- sortByKey(rdd6)
actual <- collect(sortedRdd6)
expect_equal(actual, l2)
# boundary case 4: the RDD to be sorted has 0 element
l3 <- list()
rdd7 <- parallelize(sc, l3, 2L)
sortedRdd7 <- sortByKey(rdd7)
actual <- collect(sortedRdd7)
expect_equal(actual, l3)
})
test_that("collectAsMap() on a pairwise RDD", {
rdd <- parallelize(sc, list(list(1, 2), list(3, 4)))
vals <- collectAsMap(rdd)
expect_equal(vals, list(`1` = 2, `3` = 4))
rdd <- parallelize(sc, list(list("a", 1), list("b", 2)))
vals <- collectAsMap(rdd)
expect_equal(vals, list(a = 1, b = 2))
rdd <- parallelize(sc, list(list(1.1, 2.2), list(1.2, 2.4)))
vals <- collectAsMap(rdd)
expect_equal(vals, list(`1.1` = 2.2, `1.2` = 2.4))
rdd <- parallelize(sc, list(list(1, "a"), list(2, "b")))
vals <- collectAsMap(rdd)
expect_equal(vals, list(`1` = "a", `2` = "b"))
})

View file

@ -0,0 +1,209 @@
#
# 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.
#
context("partitionBy, groupByKey, reduceByKey etc.")
# JavaSparkContext handle
sc <- sparkR.init()
# Data
intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200))
intRdd <- parallelize(sc, intPairs, 2L)
doublePairs <- list(list(1.5, -1), list(2.5, 100), list(2.5, 1), list(1.5, 200))
doubleRdd <- parallelize(sc, doublePairs, 2L)
numPairs <- list(list(1L, 100), list(2L, 200), list(4L, -1), list(3L, 1),
list(3L, 0))
numPairsRdd <- parallelize(sc, numPairs, length(numPairs))
strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge and ",
"Dexter Morgan: Harry and Dorris Morgan did a wonderful job ")
strListRDD <- parallelize(sc, strList, 4)
test_that("groupByKey for integers", {
grouped <- groupByKey(intRdd, 2L)
actual <- collect(grouped)
expected <- list(list(2L, list(100, 1)), list(1L, list(-1, 200)))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("groupByKey for doubles", {
grouped <- groupByKey(doubleRdd, 2L)
actual <- collect(grouped)
expected <- list(list(1.5, list(-1, 200)), list(2.5, list(100, 1)))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("reduceByKey for ints", {
reduced <- reduceByKey(intRdd, "+", 2L)
actual <- collect(reduced)
expected <- list(list(2L, 101), list(1L, 199))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("reduceByKey for doubles", {
reduced <- reduceByKey(doubleRdd, "+", 2L)
actual <- collect(reduced)
expected <- list(list(1.5, 199), list(2.5, 101))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("combineByKey for ints", {
reduced <- combineByKey(intRdd, function(x) { x }, "+", "+", 2L)
actual <- collect(reduced)
expected <- list(list(2L, 101), list(1L, 199))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("combineByKey for doubles", {
reduced <- combineByKey(doubleRdd, function(x) { x }, "+", "+", 2L)
actual <- collect(reduced)
expected <- list(list(1.5, 199), list(2.5, 101))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("aggregateByKey", {
# test aggregateByKey for int keys
rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4)))
zeroValue <- list(0, 0)
seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) }
combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) }
aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L)
actual <- collect(aggregatedRDD)
expected <- list(list(1, list(3, 2)), list(2, list(7, 2)))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
# test aggregateByKey for string keys
rdd <- parallelize(sc, list(list("a", 1), list("a", 2), list("b", 3), list("b", 4)))
zeroValue <- list(0, 0)
seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) }
combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) }
aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L)
actual <- collect(aggregatedRDD)
expected <- list(list("a", list(3, 2)), list("b", list(7, 2)))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
})
test_that("foldByKey", {
# test foldByKey for int keys
folded <- foldByKey(intRdd, 0, "+", 2L)
actual <- collect(folded)
expected <- list(list(2L, 101), list(1L, 199))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
# test foldByKey for double keys
folded <- foldByKey(doubleRdd, 0, "+", 2L)
actual <- collect(folded)
expected <- list(list(1.5, 199), list(2.5, 101))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
# test foldByKey for string keys
stringKeyPairs <- list(list("a", -1), list("b", 100), list("b", 1), list("a", 200))
stringKeyRDD <- parallelize(sc, stringKeyPairs)
folded <- foldByKey(stringKeyRDD, 0, "+", 2L)
actual <- collect(folded)
expected <- list(list("b", 101), list("a", 199))
expect_equal(sortKeyValueList(actual), sortKeyValueList(expected))
# test foldByKey for empty pair RDD
rdd <- parallelize(sc, list())
folded <- foldByKey(rdd, 0, "+", 2L)
actual <- collect(folded)
expected <- list()
expect_equal(actual, expected)
# test foldByKey for RDD with only 1 pair
rdd <- parallelize(sc, list(list(1, 1)))
folded <- foldByKey(rdd, 0, "+", 2L)
actual <- collect(folded)
expected <- list(list(1, 1))
expect_equal(actual, expected)
})
test_that("partitionBy() partitions data correctly", {
# Partition by magnitude
partitionByMagnitude <- function(key) { if (key >= 3) 1 else 0 }
resultRDD <- partitionBy(numPairsRdd, 2L, partitionByMagnitude)
expected_first <- list(list(1, 100), list(2, 200)) # key < 3
expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key >= 3
actual_first <- collectPartition(resultRDD, 0L)
actual_second <- collectPartition(resultRDD, 1L)
expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first))
expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second))
})
test_that("partitionBy works with dependencies", {
kOne <- 1
partitionByParity <- function(key) { if (key %% 2 == kOne) 7 else 4 }
# Partition by parity
resultRDD <- partitionBy(numPairsRdd, numPartitions = 2L, partitionByParity)
# keys even; 100 %% 2 == 0
expected_first <- list(list(2, 200), list(4, -1))
# keys odd; 3 %% 2 == 1
expected_second <- list(list(1, 100), list(3, 1), list(3, 0))
actual_first <- collectPartition(resultRDD, 0L)
actual_second <- collectPartition(resultRDD, 1L)
expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first))
expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second))
})
test_that("test partitionBy with string keys", {
words <- flatMap(strListRDD, function(line) { strsplit(line, " ")[[1]] })
wordCount <- lapply(words, function(word) { list(word, 1L) })
resultRDD <- partitionBy(wordCount, 2L)
expected_first <- list(list("Dexter", 1), list("Dexter", 1))
expected_second <- list(list("and", 1), list("and", 1))
actual_first <- Filter(function(item) { item[[1]] == "Dexter" },
collectPartition(resultRDD, 0L))
actual_second <- Filter(function(item) { item[[1]] == "and" },
collectPartition(resultRDD, 1L))
expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first))
expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second))
})

View file

@ -0,0 +1,695 @@
#
# 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.
#
library(testthat)
context("SparkSQL functions")
# Tests for SparkSQL functions in SparkR
sc <- sparkR.init()
sqlCtx <- sparkRSQL.init(sc)
mockLines <- c("{\"name\":\"Michael\"}",
"{\"name\":\"Andy\", \"age\":30}",
"{\"name\":\"Justin\", \"age\":19}")
jsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp")
parquetPath <- tempfile(pattern="sparkr-test", fileext=".parquet")
writeLines(mockLines, jsonPath)
test_that("infer types", {
expect_equal(infer_type(1L), "integer")
expect_equal(infer_type(1.0), "double")
expect_equal(infer_type("abc"), "string")
expect_equal(infer_type(TRUE), "boolean")
expect_equal(infer_type(as.Date("2015-03-11")), "date")
expect_equal(infer_type(as.POSIXlt("2015-03-11 12:13:04.043")), "timestamp")
expect_equal(infer_type(c(1L, 2L)),
list(type = 'array', elementType = "integer", containsNull = TRUE))
expect_equal(infer_type(list(1L, 2L)),
list(type = 'array', elementType = "integer", containsNull = TRUE))
expect_equal(infer_type(list(a = 1L, b = "2")),
list(type = "struct",
fields = list(list(name = "a", type = "integer", nullable = TRUE),
list(name = "b", type = "string", nullable = TRUE))))
e <- new.env()
assign("a", 1L, envir = e)
expect_equal(infer_type(e),
list(type = "map", keyType = "string", valueType = "integer",
valueContainsNull = TRUE))
})
test_that("create DataFrame from RDD", {
rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) })
df <- createDataFrame(sqlCtx, rdd, list("a", "b"))
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 10)
expect_equal(columns(df), c("a", "b"))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
df <- createDataFrame(sqlCtx, rdd)
expect_true(inherits(df, "DataFrame"))
expect_equal(columns(df), c("_1", "_2"))
fields <- list(list(name = "a", type = "integer", nullable = TRUE),
list(name = "b", type = "string", nullable = TRUE))
schema <- list(type = "struct", fields = fields)
df <- createDataFrame(sqlCtx, rdd, schema)
expect_true(inherits(df, "DataFrame"))
expect_equal(columns(df), c("a", "b"))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) })
df <- createDataFrame(sqlCtx, rdd)
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 10)
expect_equal(columns(df), c("a", "b"))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
})
test_that("toDF", {
rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) })
df <- toDF(rdd, list("a", "b"))
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 10)
expect_equal(columns(df), c("a", "b"))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
df <- toDF(rdd)
expect_true(inherits(df, "DataFrame"))
expect_equal(columns(df), c("_1", "_2"))
fields <- list(list(name = "a", type = "integer", nullable = TRUE),
list(name = "b", type = "string", nullable = TRUE))
schema <- list(type = "struct", fields = fields)
df <- toDF(rdd, schema)
expect_true(inherits(df, "DataFrame"))
expect_equal(columns(df), c("a", "b"))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) })
df <- toDF(rdd)
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 10)
expect_equal(columns(df), c("a", "b"))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
})
test_that("create DataFrame from list or data.frame", {
l <- list(list(1, 2), list(3, 4))
df <- createDataFrame(sqlCtx, l, c("a", "b"))
expect_equal(columns(df), c("a", "b"))
l <- list(list(a=1, b=2), list(a=3, b=4))
df <- createDataFrame(sqlCtx, l)
expect_equal(columns(df), c("a", "b"))
a <- 1:3
b <- c("a", "b", "c")
ldf <- data.frame(a, b)
df <- createDataFrame(sqlCtx, ldf)
expect_equal(columns(df), c("a", "b"))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
expect_equal(count(df), 3)
ldf2 <- collect(df)
expect_equal(ldf$a, ldf2$a)
})
test_that("create DataFrame with different data types", {
l <- list(a = 1L, b = 2, c = TRUE, d = "ss", e = as.Date("2012-12-13"),
f = as.POSIXct("2015-03-15 12:13:14.056"))
df <- createDataFrame(sqlCtx, list(l))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "double"), c("c", "boolean"),
c("d", "string"), c("e", "date"), c("f", "timestamp")))
expect_equal(count(df), 1)
expect_equal(collect(df), data.frame(l, stringsAsFactors = FALSE))
})
# TODO: enable this test after fix serialization for nested object
#test_that("create DataFrame with nested array and struct", {
# e <- new.env()
# assign("n", 3L, envir = e)
# l <- list(1:10, list("a", "b"), e, list(a="aa", b=3L))
# df <- createDataFrame(sqlCtx, list(l), c("a", "b", "c", "d"))
# expect_equal(dtypes(df), list(c("a", "array<int>"), c("b", "array<string>"),
# c("c", "map<string,int>"), c("d", "struct<a:string,b:int>")))
# expect_equal(count(df), 1)
# ldf <- collect(df)
# expect_equal(ldf[1,], l[[1]])
#})
test_that("jsonFile() on a local file returns a DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 3)
})
test_that("jsonRDD() on a RDD with json string", {
rdd <- parallelize(sc, mockLines)
expect_true(count(rdd) == 3)
df <- jsonRDD(sqlCtx, rdd)
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 3)
rdd2 <- flatMap(rdd, function(x) c(x, x))
df <- jsonRDD(sqlCtx, rdd2)
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 6)
})
test_that("test cache, uncache and clearCache", {
df <- jsonFile(sqlCtx, jsonPath)
registerTempTable(df, "table1")
cacheTable(sqlCtx, "table1")
uncacheTable(sqlCtx, "table1")
clearCache(sqlCtx)
dropTempTable(sqlCtx, "table1")
})
test_that("test tableNames and tables", {
df <- jsonFile(sqlCtx, jsonPath)
registerTempTable(df, "table1")
expect_true(length(tableNames(sqlCtx)) == 1)
df <- tables(sqlCtx)
expect_true(count(df) == 1)
dropTempTable(sqlCtx, "table1")
})
test_that("registerTempTable() results in a queryable table and sql() results in a new DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
registerTempTable(df, "table1")
newdf <- sql(sqlCtx, "SELECT * FROM table1 where name = 'Michael'")
expect_true(inherits(newdf, "DataFrame"))
expect_true(count(newdf) == 1)
dropTempTable(sqlCtx, "table1")
})
test_that("insertInto() on a registered table", {
df <- loadDF(sqlCtx, jsonPath, "json")
saveDF(df, parquetPath, "parquet", "overwrite")
dfParquet <- loadDF(sqlCtx, parquetPath, "parquet")
lines <- c("{\"name\":\"Bob\", \"age\":24}",
"{\"name\":\"James\", \"age\":35}")
jsonPath2 <- tempfile(pattern="jsonPath2", fileext=".tmp")
parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet")
writeLines(lines, jsonPath2)
df2 <- loadDF(sqlCtx, jsonPath2, "json")
saveDF(df2, parquetPath2, "parquet", "overwrite")
dfParquet2 <- loadDF(sqlCtx, parquetPath2, "parquet")
registerTempTable(dfParquet, "table1")
insertInto(dfParquet2, "table1")
expect_true(count(sql(sqlCtx, "select * from table1")) == 5)
expect_true(first(sql(sqlCtx, "select * from table1 order by age"))$name == "Michael")
dropTempTable(sqlCtx, "table1")
registerTempTable(dfParquet, "table1")
insertInto(dfParquet2, "table1", overwrite = TRUE)
expect_true(count(sql(sqlCtx, "select * from table1")) == 2)
expect_true(first(sql(sqlCtx, "select * from table1 order by age"))$name == "Bob")
dropTempTable(sqlCtx, "table1")
})
test_that("table() returns a new DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
registerTempTable(df, "table1")
tabledf <- table(sqlCtx, "table1")
expect_true(inherits(tabledf, "DataFrame"))
expect_true(count(tabledf) == 3)
dropTempTable(sqlCtx, "table1")
})
test_that("toRDD() returns an RRDD", {
df <- jsonFile(sqlCtx, jsonPath)
testRDD <- toRDD(df)
expect_true(inherits(testRDD, "RDD"))
expect_true(count(testRDD) == 3)
})
test_that("union on two RDDs created from DataFrames returns an RRDD", {
df <- jsonFile(sqlCtx, jsonPath)
RDD1 <- toRDD(df)
RDD2 <- toRDD(df)
unioned <- unionRDD(RDD1, RDD2)
expect_true(inherits(unioned, "RDD"))
expect_true(SparkR:::getSerializedMode(unioned) == "byte")
expect_true(collect(unioned)[[2]]$name == "Andy")
})
test_that("union on mixed serialization types correctly returns a byte RRDD", {
# Byte RDD
nums <- 1:10
rdd <- parallelize(sc, nums, 2L)
# String RDD
textLines <- c("Michael",
"Andy, 30",
"Justin, 19")
textPath <- tempfile(pattern="sparkr-textLines", fileext=".tmp")
writeLines(textLines, textPath)
textRDD <- textFile(sc, textPath)
df <- jsonFile(sqlCtx, jsonPath)
dfRDD <- toRDD(df)
unionByte <- unionRDD(rdd, dfRDD)
expect_true(inherits(unionByte, "RDD"))
expect_true(SparkR:::getSerializedMode(unionByte) == "byte")
expect_true(collect(unionByte)[[1]] == 1)
expect_true(collect(unionByte)[[12]]$name == "Andy")
unionString <- unionRDD(textRDD, dfRDD)
expect_true(inherits(unionString, "RDD"))
expect_true(SparkR:::getSerializedMode(unionString) == "byte")
expect_true(collect(unionString)[[1]] == "Michael")
expect_true(collect(unionString)[[5]]$name == "Andy")
})
test_that("objectFile() works with row serialization", {
objectPath <- tempfile(pattern="spark-test", fileext=".tmp")
df <- jsonFile(sqlCtx, jsonPath)
dfRDD <- toRDD(df)
saveAsObjectFile(coalesce(dfRDD, 1L), objectPath)
objectIn <- objectFile(sc, objectPath)
expect_true(inherits(objectIn, "RDD"))
expect_equal(SparkR:::getSerializedMode(objectIn), "byte")
expect_equal(collect(objectIn)[[2]]$age, 30)
})
test_that("lapply() on a DataFrame returns an RDD with the correct columns", {
df <- jsonFile(sqlCtx, jsonPath)
testRDD <- lapply(df, function(row) {
row$newCol <- row$age + 5
row
})
expect_true(inherits(testRDD, "RDD"))
collected <- collect(testRDD)
expect_true(collected[[1]]$name == "Michael")
expect_true(collected[[2]]$newCol == "35")
})
test_that("collect() returns a data.frame", {
df <- jsonFile(sqlCtx, jsonPath)
rdf <- collect(df)
expect_true(is.data.frame(rdf))
expect_true(names(rdf)[1] == "age")
expect_true(nrow(rdf) == 3)
expect_true(ncol(rdf) == 2)
})
test_that("limit() returns DataFrame with the correct number of rows", {
df <- jsonFile(sqlCtx, jsonPath)
dfLimited <- limit(df, 2)
expect_true(inherits(dfLimited, "DataFrame"))
expect_true(count(dfLimited) == 2)
})
test_that("collect() and take() on a DataFrame return the same number of rows and columns", {
df <- jsonFile(sqlCtx, jsonPath)
expect_true(nrow(collect(df)) == nrow(take(df, 10)))
expect_true(ncol(collect(df)) == ncol(take(df, 10)))
})
test_that("multiple pipeline transformations starting with a DataFrame result in an RDD with the correct values", {
df <- jsonFile(sqlCtx, jsonPath)
first <- lapply(df, function(row) {
row$age <- row$age + 5
row
})
second <- lapply(first, function(row) {
row$testCol <- if (row$age == 35 && !is.na(row$age)) TRUE else FALSE
row
})
expect_true(inherits(second, "RDD"))
expect_true(count(second) == 3)
expect_true(collect(second)[[2]]$age == 35)
expect_true(collect(second)[[2]]$testCol)
expect_false(collect(second)[[3]]$testCol)
})
test_that("cache(), persist(), and unpersist() on a DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
expect_false(df@env$isCached)
cache(df)
expect_true(df@env$isCached)
unpersist(df)
expect_false(df@env$isCached)
persist(df, "MEMORY_AND_DISK")
expect_true(df@env$isCached)
unpersist(df)
expect_false(df@env$isCached)
# make sure the data is collectable
expect_true(is.data.frame(collect(df)))
})
test_that("schema(), dtypes(), columns(), names() return the correct values/format", {
df <- jsonFile(sqlCtx, jsonPath)
testSchema <- schema(df)
expect_true(length(testSchema$fields()) == 2)
expect_true(testSchema$fields()[[1]]$dataType.toString() == "LongType")
expect_true(testSchema$fields()[[2]]$dataType.simpleString() == "string")
expect_true(testSchema$fields()[[1]]$name() == "age")
testTypes <- dtypes(df)
expect_true(length(testTypes[[1]]) == 2)
expect_true(testTypes[[1]][1] == "age")
testCols <- columns(df)
expect_true(length(testCols) == 2)
expect_true(testCols[2] == "name")
testNames <- names(df)
expect_true(length(testNames) == 2)
expect_true(testNames[2] == "name")
})
test_that("head() and first() return the correct data", {
df <- jsonFile(sqlCtx, jsonPath)
testHead <- head(df)
expect_true(nrow(testHead) == 3)
expect_true(ncol(testHead) == 2)
testHead2 <- head(df, 2)
expect_true(nrow(testHead2) == 2)
expect_true(ncol(testHead2) == 2)
testFirst <- first(df)
expect_true(nrow(testFirst) == 1)
})
test_that("distinct() on DataFrames", {
lines <- c("{\"name\":\"Michael\"}",
"{\"name\":\"Andy\", \"age\":30}",
"{\"name\":\"Justin\", \"age\":19}",
"{\"name\":\"Justin\", \"age\":19}")
jsonPathWithDup <- tempfile(pattern="sparkr-test", fileext=".tmp")
writeLines(lines, jsonPathWithDup)
df <- jsonFile(sqlCtx, jsonPathWithDup)
uniques <- distinct(df)
expect_true(inherits(uniques, "DataFrame"))
expect_true(count(uniques) == 3)
})
test_that("sampleDF on a DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
sampled <- sampleDF(df, FALSE, 1.0)
expect_equal(nrow(collect(sampled)), count(df))
expect_true(inherits(sampled, "DataFrame"))
sampled2 <- sampleDF(df, FALSE, 0.1)
expect_true(count(sampled2) < 3)
})
test_that("select operators", {
df <- select(jsonFile(sqlCtx, jsonPath), "name", "age")
expect_true(inherits(df$name, "Column"))
expect_true(inherits(df[[2]], "Column"))
expect_true(inherits(df[["age"]], "Column"))
expect_true(inherits(df[,1], "DataFrame"))
expect_equal(columns(df[,1]), c("name"))
expect_equal(columns(df[,"age"]), c("age"))
df2 <- df[,c("age", "name")]
expect_true(inherits(df2, "DataFrame"))
expect_equal(columns(df2), c("age", "name"))
df$age2 <- df$age
expect_equal(columns(df), c("name", "age", "age2"))
expect_equal(count(where(df, df$age2 == df$age)), 2)
df$age2 <- df$age * 2
expect_equal(columns(df), c("name", "age", "age2"))
expect_equal(count(where(df, df$age2 == df$age * 2)), 2)
})
test_that("select with column", {
df <- jsonFile(sqlCtx, jsonPath)
df1 <- select(df, "name")
expect_true(columns(df1) == c("name"))
expect_true(count(df1) == 3)
df2 <- select(df, df$age)
expect_true(columns(df2) == c("age"))
expect_true(count(df2) == 3)
})
test_that("selectExpr() on a DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
selected <- selectExpr(df, "age * 2")
expect_true(names(selected) == "(age * 2)")
expect_equal(collect(selected), collect(select(df, df$age * 2L)))
selected2 <- selectExpr(df, "name as newName", "abs(age) as age")
expect_equal(names(selected2), c("newName", "age"))
expect_true(count(selected2) == 3)
})
test_that("column calculation", {
df <- jsonFile(sqlCtx, jsonPath)
d <- collect(select(df, alias(df$age + 1, "age2")))
expect_true(names(d) == c("age2"))
df2 <- select(df, lower(df$name), abs(df$age))
expect_true(inherits(df2, "DataFrame"))
expect_true(count(df2) == 3)
})
test_that("load() from json file", {
df <- loadDF(sqlCtx, jsonPath, "json")
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 3)
})
test_that("save() as parquet file", {
df <- loadDF(sqlCtx, jsonPath, "json")
saveDF(df, parquetPath, "parquet", mode="overwrite")
df2 <- loadDF(sqlCtx, parquetPath, "parquet")
expect_true(inherits(df2, "DataFrame"))
expect_true(count(df2) == 3)
})
test_that("test HiveContext", {
hiveCtx <- tryCatch({
newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc)
}, error = function(err) {
skip("Hive is not build with SparkSQL, skipped")
})
df <- createExternalTable(hiveCtx, "json", jsonPath, "json")
expect_true(inherits(df, "DataFrame"))
expect_true(count(df) == 3)
df2 <- sql(hiveCtx, "select * from json")
expect_true(inherits(df2, "DataFrame"))
expect_true(count(df2) == 3)
jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp")
saveAsTable(df, "json", "json", "append", path = jsonPath2)
df3 <- sql(hiveCtx, "select * from json")
expect_true(inherits(df3, "DataFrame"))
expect_true(count(df3) == 6)
})
test_that("column operators", {
c <- SparkR:::col("a")
c2 <- (- c + 1 - 2) * 3 / 4.0
c3 <- (c + c2 - c2) * c2 %% c2
c4 <- (c > c2) & (c2 <= c3) | (c == c2) & (c2 != c3)
})
test_that("column functions", {
c <- SparkR:::col("a")
c2 <- min(c) + max(c) + sum(c) + avg(c) + count(c) + abs(c) + sqrt(c)
c3 <- lower(c) + upper(c) + first(c) + last(c)
c4 <- approxCountDistinct(c) + countDistinct(c) + cast(c, "string")
})
test_that("string operators", {
df <- jsonFile(sqlCtx, jsonPath)
expect_equal(count(where(df, like(df$name, "A%"))), 1)
expect_equal(count(where(df, startsWith(df$name, "A"))), 1)
expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi")
expect_equal(collect(select(df, cast(df$age, "string")))[[2, 1]], "30")
})
test_that("group by", {
df <- jsonFile(sqlCtx, jsonPath)
df1 <- agg(df, name = "max", age = "sum")
expect_true(1 == count(df1))
df1 <- agg(df, age2 = max(df$age))
expect_true(1 == count(df1))
expect_equal(columns(df1), c("age2"))
gd <- groupBy(df, "name")
expect_true(inherits(gd, "GroupedData"))
df2 <- count(gd)
expect_true(inherits(df2, "DataFrame"))
expect_true(3 == count(df2))
df3 <- agg(gd, age = "sum")
expect_true(inherits(df3, "DataFrame"))
expect_true(3 == count(df3))
df3 <- agg(gd, age = sum(df$age))
expect_true(inherits(df3, "DataFrame"))
expect_true(3 == count(df3))
expect_equal(columns(df3), c("name", "age"))
df4 <- sum(gd, "age")
expect_true(inherits(df4, "DataFrame"))
expect_true(3 == count(df4))
expect_true(3 == count(mean(gd, "age")))
expect_true(3 == count(max(gd, "age")))
})
test_that("sortDF() and orderBy() on a DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
sorted <- sortDF(df, df$age)
expect_true(collect(sorted)[1,2] == "Michael")
sorted2 <- sortDF(df, "name")
expect_true(collect(sorted2)[2,"age"] == 19)
sorted3 <- orderBy(df, asc(df$age))
expect_true(is.na(first(sorted3)$age))
expect_true(collect(sorted3)[2, "age"] == 19)
sorted4 <- orderBy(df, desc(df$name))
expect_true(first(sorted4)$name == "Michael")
expect_true(collect(sorted4)[3,"name"] == "Andy")
})
test_that("filter() on a DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
filtered <- filter(df, "age > 20")
expect_true(count(filtered) == 1)
expect_true(collect(filtered)$name == "Andy")
filtered2 <- where(df, df$name != "Michael")
expect_true(count(filtered2) == 2)
expect_true(collect(filtered2)$age[2] == 19)
})
test_that("join() on a DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}",
"{\"name\":\"Andy\", \"test\": \"no\"}",
"{\"name\":\"Justin\", \"test\": \"yes\"}",
"{\"name\":\"Bob\", \"test\": \"yes\"}")
jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp")
writeLines(mockLines2, jsonPath2)
df2 <- jsonFile(sqlCtx, jsonPath2)
joined <- join(df, df2)
expect_equal(names(joined), c("age", "name", "name", "test"))
expect_true(count(joined) == 12)
joined2 <- join(df, df2, df$name == df2$name)
expect_equal(names(joined2), c("age", "name", "name", "test"))
expect_true(count(joined2) == 3)
joined3 <- join(df, df2, df$name == df2$name, "right_outer")
expect_equal(names(joined3), c("age", "name", "name", "test"))
expect_true(count(joined3) == 4)
expect_true(is.na(collect(orderBy(joined3, joined3$age))$age[2]))
joined4 <- select(join(df, df2, df$name == df2$name, "outer"),
alias(df$age + 5, "newAge"), df$name, df2$test)
expect_equal(names(joined4), c("newAge", "name", "test"))
expect_true(count(joined4) == 4)
expect_equal(collect(orderBy(joined4, joined4$name))$newAge[3], 24)
})
test_that("toJSON() returns an RDD of the correct values", {
df <- jsonFile(sqlCtx, jsonPath)
testRDD <- toJSON(df)
expect_true(inherits(testRDD, "RDD"))
expect_true(SparkR:::getSerializedMode(testRDD) == "string")
expect_equal(collect(testRDD)[[1]], mockLines[1])
})
test_that("showDF()", {
df <- jsonFile(sqlCtx, jsonPath)
expect_output(showDF(df), "age name \nnull Michael\n30 Andy \n19 Justin ")
})
test_that("isLocal()", {
df <- jsonFile(sqlCtx, jsonPath)
expect_false(isLocal(df))
})
test_that("unionAll(), subtract(), and intersect() on a DataFrame", {
df <- jsonFile(sqlCtx, jsonPath)
lines <- c("{\"name\":\"Bob\", \"age\":24}",
"{\"name\":\"Andy\", \"age\":30}",
"{\"name\":\"James\", \"age\":35}")
jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp")
writeLines(lines, jsonPath2)
df2 <- loadDF(sqlCtx, jsonPath2, "json")
unioned <- sortDF(unionAll(df, df2), df$age)
expect_true(inherits(unioned, "DataFrame"))
expect_true(count(unioned) == 6)
expect_true(first(unioned)$name == "Michael")
subtracted <- sortDF(subtract(df, df2), desc(df$age))
expect_true(inherits(unioned, "DataFrame"))
expect_true(count(subtracted) == 2)
expect_true(first(subtracted)$name == "Justin")
intersected <- sortDF(intersect(df, df2), df$age)
expect_true(inherits(unioned, "DataFrame"))
expect_true(count(intersected) == 1)
expect_true(first(intersected)$name == "Andy")
})
test_that("withColumn() and withColumnRenamed()", {
df <- jsonFile(sqlCtx, jsonPath)
newDF <- withColumn(df, "newAge", df$age + 2)
expect_true(length(columns(newDF)) == 3)
expect_true(columns(newDF)[3] == "newAge")
expect_true(first(filter(newDF, df$name != "Michael"))$newAge == 32)
newDF2 <- withColumnRenamed(df, "age", "newerAge")
expect_true(length(columns(newDF2)) == 2)
expect_true(columns(newDF2)[1] == "newerAge")
})
test_that("saveDF() on DataFrame and works with parquetFile", {
df <- jsonFile(sqlCtx, jsonPath)
saveDF(df, parquetPath, "parquet", mode="overwrite")
parquetDF <- parquetFile(sqlCtx, parquetPath)
expect_true(inherits(parquetDF, "DataFrame"))
expect_equal(count(df), count(parquetDF))
})
test_that("parquetFile works with multiple input paths", {
df <- jsonFile(sqlCtx, jsonPath)
saveDF(df, parquetPath, "parquet", mode="overwrite")
parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet")
saveDF(df, parquetPath2, "parquet", mode="overwrite")
parquetDF <- parquetFile(sqlCtx, parquetPath, parquetPath2)
expect_true(inherits(parquetDF, "DataFrame"))
expect_true(count(parquetDF) == count(df)*2)
})
unlink(parquetPath)
unlink(jsonPath)

View file

@ -0,0 +1,67 @@
#
# 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.
#
context("tests RDD function take()")
# Mock data
numVector <- c(-10:97)
numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10)
strVector <- c("Dexter Morgan: I suppose I should be upset, even feel",
"violated, but I'm not. No, in fact, I think this is a friendly",
"message, like \"Hey, wanna play?\" and yes, I want to play. ",
"I really, really do.")
strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ",
"other times it helps me control the chaos.",
"Dexter Morgan: Harry and Dorris Morgan did a wonderful job ",
"raising me. But they're both dead now. I didn't kill them. Honest.")
# JavaSparkContext handle
jsc <- sparkR.init()
test_that("take() gives back the original elements in correct count and order", {
numVectorRDD <- parallelize(jsc, numVector, 10)
# case: number of elements to take is less than the size of the first partition
expect_equal(take(numVectorRDD, 1), as.list(head(numVector, n = 1)))
# case: number of elements to take is the same as the size of the first partition
expect_equal(take(numVectorRDD, 11), as.list(head(numVector, n = 11)))
# case: number of elements to take is greater than all elements
expect_equal(take(numVectorRDD, length(numVector)), as.list(numVector))
expect_equal(take(numVectorRDD, length(numVector) + 1), as.list(numVector))
numListRDD <- parallelize(jsc, numList, 1)
numListRDD2 <- parallelize(jsc, numList, 4)
expect_equal(take(numListRDD, 3), take(numListRDD2, 3))
expect_equal(take(numListRDD, 5), take(numListRDD2, 5))
expect_equal(take(numListRDD, 1), as.list(head(numList, n = 1)))
expect_equal(take(numListRDD2, 999), numList)
strVectorRDD <- parallelize(jsc, strVector, 2)
strVectorRDD2 <- parallelize(jsc, strVector, 3)
expect_equal(take(strVectorRDD, 4), as.list(strVector))
expect_equal(take(strVectorRDD2, 2), as.list(head(strVector, n = 2)))
strListRDD <- parallelize(jsc, strList, 4)
strListRDD2 <- parallelize(jsc, strList, 1)
expect_equal(take(strListRDD, 3), as.list(head(strList, n = 3)))
expect_equal(take(strListRDD2, 1), as.list(head(strList, n = 1)))
expect_true(length(take(strListRDD, 0)) == 0)
expect_true(length(take(strVectorRDD, 0)) == 0)
expect_true(length(take(numListRDD, 0)) == 0)
expect_true(length(take(numVectorRDD, 0)) == 0)
})

View file

@ -0,0 +1,162 @@
#
# 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.
#
context("the textFile() function")
# JavaSparkContext handle
sc <- sparkR.init()
mockFile = c("Spark is pretty.", "Spark is awesome.")
test_that("textFile() on a local file returns an RDD", {
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName)
rdd <- textFile(sc, fileName)
expect_true(inherits(rdd, "RDD"))
expect_true(count(rdd) > 0)
expect_true(count(rdd) == 2)
unlink(fileName)
})
test_that("textFile() followed by a collect() returns the same content", {
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName)
rdd <- textFile(sc, fileName)
expect_equal(collect(rdd), as.list(mockFile))
unlink(fileName)
})
test_that("textFile() word count works as expected", {
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName)
rdd <- textFile(sc, fileName)
words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] })
wordCount <- lapply(words, function(word) { list(word, 1L) })
counts <- reduceByKey(wordCount, "+", 2L)
output <- collect(counts)
expected <- list(list("pretty.", 1), list("is", 2), list("awesome.", 1),
list("Spark", 2))
expect_equal(sortKeyValueList(output), sortKeyValueList(expected))
unlink(fileName)
})
test_that("several transformations on RDD created by textFile()", {
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName)
rdd <- textFile(sc, fileName) # RDD
for (i in 1:10) {
# PipelinedRDD initially created from RDD
rdd <- lapply(rdd, function(x) paste(x, x))
}
collect(rdd)
unlink(fileName)
})
test_that("textFile() followed by a saveAsTextFile() returns the same content", {
fileName1 <- tempfile(pattern="spark-test", fileext=".tmp")
fileName2 <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName1)
rdd <- textFile(sc, fileName1)
saveAsTextFile(rdd, fileName2)
rdd <- textFile(sc, fileName2)
expect_equal(collect(rdd), as.list(mockFile))
unlink(fileName1)
unlink(fileName2)
})
test_that("saveAsTextFile() on a parallelized list works as expected", {
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
l <- list(1, 2, 3)
rdd <- parallelize(sc, l)
saveAsTextFile(rdd, fileName)
rdd <- textFile(sc, fileName)
expect_equal(collect(rdd), lapply(l, function(x) {toString(x)}))
unlink(fileName)
})
test_that("textFile() and saveAsTextFile() word count works as expected", {
fileName1 <- tempfile(pattern="spark-test", fileext=".tmp")
fileName2 <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName1)
rdd <- textFile(sc, fileName1)
words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] })
wordCount <- lapply(words, function(word) { list(word, 1L) })
counts <- reduceByKey(wordCount, "+", 2L)
saveAsTextFile(counts, fileName2)
rdd <- textFile(sc, fileName2)
output <- collect(rdd)
expected <- list(list("awesome.", 1), list("Spark", 2),
list("pretty.", 1), list("is", 2))
expectedStr <- lapply(expected, function(x) { toString(x) })
expect_equal(sortKeyValueList(output), sortKeyValueList(expectedStr))
unlink(fileName1)
unlink(fileName2)
})
test_that("textFile() on multiple paths", {
fileName1 <- tempfile(pattern="spark-test", fileext=".tmp")
fileName2 <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines("Spark is pretty.", fileName1)
writeLines("Spark is awesome.", fileName2)
rdd <- textFile(sc, c(fileName1, fileName2))
expect_true(count(rdd) == 2)
unlink(fileName1)
unlink(fileName2)
})
test_that("Pipelined operations on RDDs created using textFile", {
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName)
rdd <- textFile(sc, fileName)
lengths <- lapply(rdd, function(x) { length(x) })
expect_equal(collect(lengths), list(1, 1))
lengthsPipelined <- lapply(lengths, function(x) { x + 10 })
expect_equal(collect(lengthsPipelined), list(11, 11))
lengths30 <- lapply(lengthsPipelined, function(x) { x + 20 })
expect_equal(collect(lengths30), list(31, 31))
lengths20 <- lapply(lengths, function(x) { x + 20 })
expect_equal(collect(lengths20), list(21, 21))
unlink(fileName)
})

View file

@ -0,0 +1,137 @@
#
# 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.
#
context("functions in utils.R")
# JavaSparkContext handle
sc <- sparkR.init()
test_that("convertJListToRList() gives back (deserializes) the original JLists
of strings and integers", {
# It's hard to manually create a Java List using rJava, since it does not
# support generics well. Instead, we rely on collect() returning a
# JList.
nums <- as.list(1:10)
rdd <- parallelize(sc, nums, 1L)
jList <- callJMethod(rdd@jrdd, "collect")
rList <- convertJListToRList(jList, flatten = TRUE)
expect_equal(rList, nums)
strs <- as.list("hello", "spark")
rdd <- parallelize(sc, strs, 2L)
jList <- callJMethod(rdd@jrdd, "collect")
rList <- convertJListToRList(jList, flatten = TRUE)
expect_equal(rList, strs)
})
test_that("serializeToBytes on RDD", {
# File content
mockFile <- c("Spark is pretty.", "Spark is awesome.")
fileName <- tempfile(pattern="spark-test", fileext=".tmp")
writeLines(mockFile, fileName)
text.rdd <- textFile(sc, fileName)
expect_true(getSerializedMode(text.rdd) == "string")
ser.rdd <- serializeToBytes(text.rdd)
expect_equal(collect(ser.rdd), as.list(mockFile))
expect_true(getSerializedMode(ser.rdd) == "byte")
unlink(fileName)
})
test_that("cleanClosure on R functions", {
y <- c(1, 2, 3)
g <- function(x) { x + 1 }
f <- function(x) { g(x) + y }
newF <- cleanClosure(f)
env <- environment(newF)
expect_equal(length(ls(env)), 2) # y, g
actual <- get("y", envir = env, inherits = FALSE)
expect_equal(actual, y)
actual <- get("g", envir = env, inherits = FALSE)
expect_equal(actual, g)
# Test for nested enclosures and package variables.
env2 <- new.env()
funcEnv <- new.env(parent = env2)
f <- function(x) { log(g(x) + y) }
environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv
newF <- cleanClosure(f)
env <- environment(newF)
expect_equal(length(ls(env)), 2) # "min" should not be included
actual <- get("y", envir = env, inherits = FALSE)
expect_equal(actual, y)
actual <- get("g", envir = env, inherits = FALSE)
expect_equal(actual, g)
base <- c(1, 2, 3)
l <- list(field = matrix(1))
field <- matrix(2)
defUse <- 3
g <- function(x) { x + y }
f <- function(x) {
defUse <- base::as.integer(x) + 1 # Test for access operators `::`.
lapply(x, g) + 1 # Test for capturing function call "g"'s closure as a argument of lapply.
l$field[1,1] <- 3 # Test for access operators `$`.
res <- defUse + l$field[1,] # Test for def-use chain of "defUse", and "" symbol.
f(res) # Test for recursive calls.
}
newF <- cleanClosure(f)
env <- environment(newF)
expect_equal(length(ls(env)), 3) # Only "g", "l" and "f". No "base", "field" or "defUse".
expect_true("g" %in% ls(env))
expect_true("l" %in% ls(env))
expect_true("f" %in% ls(env))
expect_equal(get("l", envir = env, inherits = FALSE), l)
# "y" should be in the environemnt of g.
newG <- get("g", envir = env, inherits = FALSE)
env <- environment(newG)
expect_equal(length(ls(env)), 1)
actual <- get("y", envir = env, inherits = FALSE)
expect_equal(actual, y)
# Test for function (and variable) definitions.
f <- function(x) {
g <- function(y) { y * 2 }
g(x)
}
newF <- cleanClosure(f)
env <- environment(newF)
expect_equal(length(ls(env)), 0) # "y" and "g" should not be included.
# Test for overriding variables in base namespace (Issue: SparkR-196).
nums <- as.list(1:10)
rdd <- parallelize(sc, nums, 2L)
t = 4 # Override base::t in .GlobalEnv.
f <- function(x) { x > t }
newF <- cleanClosure(f)
env <- environment(newF)
expect_equal(ls(env), "t")
expect_equal(get("t", envir = env, inherits = FALSE), t)
actual <- collect(lapply(rdd, f))
expected <- as.list(c(rep(FALSE, 4), rep(TRUE, 6)))
expect_equal(actual, expected)
# Test for broadcast variables.
a <- matrix(nrow=10, ncol=10, data=rnorm(100))
aBroadcast <- broadcast(sc, a)
normMultiply <- function(x) { norm(aBroadcast$value) * x }
newnormMultiply <- SparkR:::cleanClosure(normMultiply)
env <- environment(newnormMultiply)
expect_equal(ls(env), "aBroadcast")
expect_equal(get("aBroadcast", envir = env, inherits = FALSE), aBroadcast)
})

View file

@ -0,0 +1,52 @@
#
# 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.
#
# Worker daemon
rLibDir <- Sys.getenv("SPARKR_RLIBDIR")
script <- paste(rLibDir, "SparkR/worker/worker.R", sep = "/")
# preload SparkR package, speedup worker
.libPaths(c(rLibDir, .libPaths()))
suppressPackageStartupMessages(library(SparkR))
port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT"))
inputCon <- socketConnection(port = port, open = "rb", blocking = TRUE, timeout = 3600)
while (TRUE) {
ready <- socketSelect(list(inputCon))
if (ready) {
port <- SparkR:::readInt(inputCon)
# There is a small chance that it could be interrupted by signal, retry one time
if (length(port) == 0) {
port <- SparkR:::readInt(inputCon)
if (length(port) == 0) {
cat("quitting daemon\n")
quit(save = "no")
}
}
p <- parallel:::mcfork()
if (inherits(p, "masterProcess")) {
close(inputCon)
Sys.setenv(SPARKR_WORKER_PORT = port)
source(script)
# Set SIGUSR1 so that child can exit
tools::pskill(Sys.getpid(), tools::SIGUSR1)
parallel:::mcexit(0L)
}
}
}

128
R/pkg/inst/worker/worker.R Normal file
View file

@ -0,0 +1,128 @@
#
# 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.
#
# Worker class
rLibDir <- Sys.getenv("SPARKR_RLIBDIR")
# Set libPaths to include SparkR package as loadNamespace needs this
# TODO: Figure out if we can avoid this by not loading any objects that require
# SparkR namespace
.libPaths(c(rLibDir, .libPaths()))
suppressPackageStartupMessages(library(SparkR))
port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT"))
inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb")
outputCon <- socketConnection(port = port, blocking = TRUE, open = "wb")
# read the index of the current partition inside the RDD
partition <- SparkR:::readInt(inputCon)
deserializer <- SparkR:::readString(inputCon)
serializer <- SparkR:::readString(inputCon)
# Include packages as required
packageNames <- unserialize(SparkR:::readRaw(inputCon))
for (pkg in packageNames) {
suppressPackageStartupMessages(require(as.character(pkg), character.only=TRUE))
}
# read function dependencies
funcLen <- SparkR:::readInt(inputCon)
computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen))
env <- environment(computeFunc)
parent.env(env) <- .GlobalEnv # Attach under global environment.
# Read and set broadcast variables
numBroadcastVars <- SparkR:::readInt(inputCon)
if (numBroadcastVars > 0) {
for (bcast in seq(1:numBroadcastVars)) {
bcastId <- SparkR:::readInt(inputCon)
value <- unserialize(SparkR:::readRaw(inputCon))
setBroadcastValue(bcastId, value)
}
}
# If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int
# as number of partitions to create.
numPartitions <- SparkR:::readInt(inputCon)
isEmpty <- SparkR:::readInt(inputCon)
if (isEmpty != 0) {
if (numPartitions == -1) {
if (deserializer == "byte") {
# Now read as many characters as described in funcLen
data <- SparkR:::readDeserialize(inputCon)
} else if (deserializer == "string") {
data <- as.list(readLines(inputCon))
} else if (deserializer == "row") {
data <- SparkR:::readDeserializeRows(inputCon)
}
output <- computeFunc(partition, data)
if (serializer == "byte") {
SparkR:::writeRawSerialize(outputCon, output)
} else if (serializer == "row") {
SparkR:::writeRowSerialize(outputCon, output)
} else {
SparkR:::writeStrings(outputCon, output)
}
} else {
if (deserializer == "byte") {
# Now read as many characters as described in funcLen
data <- SparkR:::readDeserialize(inputCon)
} else if (deserializer == "string") {
data <- readLines(inputCon)
} else if (deserializer == "row") {
data <- SparkR:::readDeserializeRows(inputCon)
}
res <- new.env()
# Step 1: hash the data to an environment
hashTupleToEnvir <- function(tuple) {
# NOTE: execFunction is the hash function here
hashVal <- computeFunc(tuple[[1]])
bucket <- as.character(hashVal %% numPartitions)
acc <- res[[bucket]]
# Create a new accumulator
if (is.null(acc)) {
acc <- SparkR:::initAccumulator()
}
SparkR:::addItemToAccumulator(acc, tuple)
res[[bucket]] <- acc
}
invisible(lapply(data, hashTupleToEnvir))
# Step 2: write out all of the environment as key-value pairs.
for (name in ls(res)) {
SparkR:::writeInt(outputCon, 2L)
SparkR:::writeInt(outputCon, as.integer(name))
# Truncate the accumulator list to the number of elements we have
length(res[[name]]$data) <- res[[name]]$counter
SparkR:::writeRawSerialize(outputCon, res[[name]]$data)
}
}
}
# End of output
if (serializer %in% c("byte", "row")) {
SparkR:::writeInt(outputCon, 0L)
}
close(outputCon)
close(inputCon)

27
R/pkg/src/Makefile Normal file
View file

@ -0,0 +1,27 @@
#
# 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.
#
all: sharelib
sharelib: string_hash_code.c
R CMD SHLIB -o SparkR.so string_hash_code.c
clean:
rm -f *.o
rm -f *.so
.PHONY: all clean

27
R/pkg/src/Makefile.win Normal file
View file

@ -0,0 +1,27 @@
#
# 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.
#
all: sharelib
sharelib: string_hash_code.c
R CMD SHLIB -o SparkR.dll string_hash_code.c
clean:
rm -f *.o
rm -f *.dll
.PHONY: all clean

View file

@ -0,0 +1,49 @@
/*
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.
*/
/*
* A C function for R extension which implements the Java String hash algorithm.
* Refer to http://en.wikipedia.org/wiki/Java_hashCode%28%29#The_java.lang.String_hash_function
*
*/
#include <R.h>
#include <Rinternals.h>
/* for compatibility with R before 3.1 */
#ifndef IS_SCALAR
#define IS_SCALAR(x, type) (TYPEOF(x) == (type) && XLENGTH(x) == 1)
#endif
SEXP stringHashCode(SEXP string) {
const char* str;
R_xlen_t len, i;
int hashCode = 0;
if (!IS_SCALAR(string, STRSXP)) {
error("invalid input");
}
str = CHAR(asChar(string));
len = XLENGTH(asChar(string));
for (i = 0; i < len; i++) {
hashCode = (hashCode << 5) - hashCode + *str++;
}
return ScalarInteger(hashCode);
}

21
R/pkg/tests/run-all.R Normal file
View file

@ -0,0 +1,21 @@
#
# 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.
#
library(testthat)
library(SparkR)
test_package("SparkR")

39
R/run-tests.sh Executable file
View file

@ -0,0 +1,39 @@
#!/bin/bash
#
# 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.
#
FWDIR="$(cd `dirname $0`; pwd)"
FAILED=0
LOGFILE=$FWDIR/unit-tests.out
rm -f $LOGFILE
SPARK_TESTING=1 $FWDIR/../bin/sparkR --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
FAILED=$((PIPESTATUS[0]||$FAILED))
if [[ $FAILED != 0 ]]; then
cat $LOGFILE
echo -en "\033[31m" # Red
echo "Had test failures; see logs."
echo -en "\033[0m" # No color
exit -1
else
echo -en "\033[32m" # Green
echo "Tests passed."
echo -en "\033[0m" # No color
fi

39
bin/sparkR Executable file
View file

@ -0,0 +1,39 @@
#!/bin/bash
#
# 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.
#
# Figure out where Spark is installed
export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)"
source "$SPARK_HOME"/bin/load-spark-env.sh
function usage() {
if [ -n "$1" ]; then
echo $1
fi
echo "Usage: ./bin/sparkR [options]" 1>&2
"$SPARK_HOME"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2
exit $2
}
export -f usage
if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then
usage
fi
exec "$SPARK_HOME"/bin/spark-submit sparkr-shell-main "$@"

23
bin/sparkR.cmd Normal file
View file

@ -0,0 +1,23 @@
@echo off
rem
rem Licensed to the Apache Software Foundation (ASF) under one or more
rem contributor license agreements. See the NOTICE file distributed with
rem this work for additional information regarding copyright ownership.
rem The ASF licenses this file to You under the Apache License, Version 2.0
rem (the "License"); you may not use this file except in compliance with
rem the License. You may obtain a copy of the License at
rem
rem http://www.apache.org/licenses/LICENSE-2.0
rem
rem Unless required by applicable law or agreed to in writing, software
rem distributed under the License is distributed on an "AS IS" BASIS,
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
rem This is the entry point for running SparkR. To avoid polluting the
rem environment, it just launches a new cmd to do the real work.
cmd /V /E /C %~dp0sparkR2.cmd %*

26
bin/sparkR2.cmd Normal file
View file

@ -0,0 +1,26 @@
@echo off
rem
rem Licensed to the Apache Software Foundation (ASF) under one or more
rem contributor license agreements. See the NOTICE file distributed with
rem this work for additional information regarding copyright ownership.
rem The ASF licenses this file to You under the Apache License, Version 2.0
rem (the "License"); you may not use this file except in compliance with
rem the License. You may obtain a copy of the License at
rem
rem http://www.apache.org/licenses/LICENSE-2.0
rem
rem Unless required by applicable law or agreed to in writing, software
rem distributed under the License is distributed on an "AS IS" BASIS,
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
rem Figure out where the Spark framework is installed
set SPARK_HOME=%~dp0..
call %SPARK_HOME%\bin\load-spark-env.cmd
call %SPARK_HOME%\bin\spark-submit2.cmd sparkr-shell-main %*

View file

@ -442,4 +442,55 @@
</resources>
</build>
<profiles>
<profile>
<id>Windows</id>
<activation>
<os>
<family>Windows</family>
</os>
</activation>
<properties>
<path.separator>\</path.separator>
<script.extension>.bat</script.extension>
</properties>
</profile>
<profile>
<id>unix</id>
<activation>
<os>
<family>unix</family>
</os>
</activation>
<properties>
<path.separator>/</path.separator>
<script.extension>.sh</script.extension>
</properties>
</profile>
<profile>
<id>sparkr</id>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>exec-maven-plugin</artifactId>
<version>1.3.2</version>
<executions>
<execution>
<id>sparkr-pkg</id>
<phase>compile</phase>
<goals>
<goal>exec</goal>
</goals>
</execution>
</executions>
<configuration>
<executable>..${path.separator}R${path.separator}install-dev${script.extension}</executable>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View file

@ -0,0 +1,145 @@
/*
* 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.api.r
import java.io.{DataOutputStream, File, FileOutputStream, IOException}
import java.net.{InetSocketAddress, ServerSocket}
import java.util.concurrent.TimeUnit
import io.netty.bootstrap.ServerBootstrap
import io.netty.channel.{ChannelFuture, ChannelInitializer, EventLoopGroup}
import io.netty.channel.nio.NioEventLoopGroup
import io.netty.channel.socket.SocketChannel
import io.netty.channel.socket.nio.NioServerSocketChannel
import io.netty.handler.codec.LengthFieldBasedFrameDecoder
import io.netty.handler.codec.bytes.{ByteArrayDecoder, ByteArrayEncoder}
import org.apache.spark.Logging
/**
* Netty-based backend server that is used to communicate between R and Java.
*/
private[spark] class RBackend {
private[this] var channelFuture: ChannelFuture = null
private[this] var bootstrap: ServerBootstrap = null
private[this] var bossGroup: EventLoopGroup = null
def init(): Int = {
bossGroup = new NioEventLoopGroup(2)
val workerGroup = bossGroup
val handler = new RBackendHandler(this)
bootstrap = new ServerBootstrap()
.group(bossGroup, workerGroup)
.channel(classOf[NioServerSocketChannel])
bootstrap.childHandler(new ChannelInitializer[SocketChannel]() {
def initChannel(ch: SocketChannel): Unit = {
ch.pipeline()
.addLast("encoder", new ByteArrayEncoder())
.addLast("frameDecoder",
// maxFrameLength = 2G
// lengthFieldOffset = 0
// lengthFieldLength = 4
// lengthAdjustment = 0
// initialBytesToStrip = 4, i.e. strip out the length field itself
new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
.addLast("decoder", new ByteArrayDecoder())
.addLast("handler", handler)
}
})
channelFuture = bootstrap.bind(new InetSocketAddress(0))
channelFuture.syncUninterruptibly()
channelFuture.channel().localAddress().asInstanceOf[InetSocketAddress].getPort()
}
def run(): Unit = {
channelFuture.channel.closeFuture().syncUninterruptibly()
}
def close(): Unit = {
if (channelFuture != null) {
// close is a local operation and should finish within milliseconds; timeout just to be safe
channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS)
channelFuture = null
}
if (bootstrap != null && bootstrap.group() != null) {
bootstrap.group().shutdownGracefully()
}
if (bootstrap != null && bootstrap.childGroup() != null) {
bootstrap.childGroup().shutdownGracefully()
}
bootstrap = null
}
}
private[spark] object RBackend extends Logging {
def main(args: Array[String]): Unit = {
if (args.length < 1) {
System.err.println("Usage: RBackend <tempFilePath>")
System.exit(-1)
}
val sparkRBackend = new RBackend()
try {
// bind to random port
val boundPort = sparkRBackend.init()
val serverSocket = new ServerSocket(0, 1)
val listenPort = serverSocket.getLocalPort()
// tell the R process via temporary file
val path = args(0)
val f = new File(path + ".tmp")
val dos = new DataOutputStream(new FileOutputStream(f))
dos.writeInt(boundPort)
dos.writeInt(listenPort)
dos.close()
f.renameTo(new File(path))
// wait for the end of stdin, then exit
new Thread("wait for socket to close") {
setDaemon(true)
override def run(): Unit = {
// any un-catched exception will also shutdown JVM
val buf = new Array[Byte](1024)
// shutdown JVM if R does not connect back in 10 seconds
serverSocket.setSoTimeout(10000)
try {
val inSocket = serverSocket.accept()
serverSocket.close()
// wait for the end of socket, closed if R process die
inSocket.getInputStream().read(buf)
} finally {
sparkRBackend.close()
System.exit(0)
}
}
}.start()
sparkRBackend.run()
} catch {
case e: IOException =>
logError("Server shutting down: failed with exception ", e)
sparkRBackend.close()
System.exit(1)
}
System.exit(0)
}
}

View file

@ -0,0 +1,223 @@
/*
* 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.api.r
import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream}
import scala.collection.mutable.HashMap
import io.netty.channel.ChannelHandler.Sharable
import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler}
import org.apache.spark.Logging
import org.apache.spark.api.r.SerDe._
/**
* Handler for RBackend
* TODO: This is marked as sharable to get a handle to RBackend. Is it safe to re-use
* this across connections ?
*/
@Sharable
private[r] class RBackendHandler(server: RBackend)
extends SimpleChannelInboundHandler[Array[Byte]] with Logging {
override def channelRead0(ctx: ChannelHandlerContext, msg: Array[Byte]): Unit = {
val bis = new ByteArrayInputStream(msg)
val dis = new DataInputStream(bis)
val bos = new ByteArrayOutputStream()
val dos = new DataOutputStream(bos)
// First bit is isStatic
val isStatic = readBoolean(dis)
val objId = readString(dis)
val methodName = readString(dis)
val numArgs = readInt(dis)
if (objId == "SparkRHandler") {
methodName match {
case "stopBackend" =>
writeInt(dos, 0)
writeType(dos, "void")
server.close()
case "rm" =>
try {
val t = readObjectType(dis)
assert(t == 'c')
val objToRemove = readString(dis)
JVMObjectTracker.remove(objToRemove)
writeInt(dos, 0)
writeObject(dos, null)
} catch {
case e: Exception =>
logError(s"Removing $objId failed", e)
writeInt(dos, -1)
}
case _ => dos.writeInt(-1)
}
} else {
handleMethodCall(isStatic, objId, methodName, numArgs, dis, dos)
}
val reply = bos.toByteArray
ctx.write(reply)
}
override def channelReadComplete(ctx: ChannelHandlerContext): Unit = {
ctx.flush()
}
override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = {
// Close the connection when an exception is raised.
cause.printStackTrace()
ctx.close()
}
def handleMethodCall(
isStatic: Boolean,
objId: String,
methodName: String,
numArgs: Int,
dis: DataInputStream,
dos: DataOutputStream): Unit = {
var obj: Object = null
try {
val cls = if (isStatic) {
Class.forName(objId)
} else {
JVMObjectTracker.get(objId) match {
case None => throw new IllegalArgumentException("Object not found " + objId)
case Some(o) =>
obj = o
o.getClass
}
}
val args = readArgs(numArgs, dis)
val methods = cls.getMethods
val selectedMethods = methods.filter(m => m.getName == methodName)
if (selectedMethods.length > 0) {
val methods = selectedMethods.filter { x =>
matchMethod(numArgs, args, x.getParameterTypes)
}
if (methods.isEmpty) {
logWarning(s"cannot find matching method ${cls}.$methodName. "
+ s"Candidates are:")
selectedMethods.foreach { method =>
logWarning(s"$methodName(${method.getParameterTypes.mkString(",")})")
}
throw new Exception(s"No matched method found for $cls.$methodName")
}
val ret = methods.head.invoke(obj, args:_*)
// Write status bit
writeInt(dos, 0)
writeObject(dos, ret.asInstanceOf[AnyRef])
} else if (methodName == "<init>") {
// methodName should be "<init>" for constructor
val ctor = cls.getConstructors.filter { x =>
matchMethod(numArgs, args, x.getParameterTypes)
}.head
val obj = ctor.newInstance(args:_*)
writeInt(dos, 0)
writeObject(dos, obj.asInstanceOf[AnyRef])
} else {
throw new IllegalArgumentException("invalid method " + methodName + " for object " + objId)
}
} catch {
case e: Exception =>
logError(s"$methodName on $objId failed", e)
writeInt(dos, -1)
}
}
// Read a number of arguments from the data input stream
def readArgs(numArgs: Int, dis: DataInputStream): Array[java.lang.Object] = {
(0 until numArgs).map { arg =>
readObject(dis)
}.toArray
}
// Checks if the arguments passed in args matches the parameter types.
// NOTE: Currently we do exact match. We may add type conversions later.
def matchMethod(
numArgs: Int,
args: Array[java.lang.Object],
parameterTypes: Array[Class[_]]): Boolean = {
if (parameterTypes.length != numArgs) {
return false
}
for (i <- 0 to numArgs - 1) {
val parameterType = parameterTypes(i)
var parameterWrapperType = parameterType
// Convert native parameters to Object types as args is Array[Object] here
if (parameterType.isPrimitive) {
parameterWrapperType = parameterType match {
case java.lang.Integer.TYPE => classOf[java.lang.Integer]
case java.lang.Double.TYPE => classOf[java.lang.Double]
case java.lang.Boolean.TYPE => classOf[java.lang.Boolean]
case _ => parameterType
}
}
if (!parameterWrapperType.isInstance(args(i))) {
return false
}
}
true
}
}
/**
* Helper singleton that tracks JVM objects returned to R.
* This is useful for referencing these objects in RPC calls.
*/
private[r] object JVMObjectTracker {
// TODO: This map should be thread-safe if we want to support multiple
// connections at the same time
private[this] val objMap = new HashMap[String, Object]
// TODO: We support only one connection now, so an integer is fine.
// Investigate using use atomic integer in the future.
private[this] var objCounter: Int = 0
def getObject(id: String): Object = {
objMap(id)
}
def get(id: String): Option[Object] = {
objMap.get(id)
}
def put(obj: Object): String = {
val objId = objCounter.toString
objCounter = objCounter + 1
objMap.put(objId, obj)
objId
}
def remove(id: String): Option[Object] = {
objMap.remove(id)
}
}

View file

@ -0,0 +1,450 @@
/*
* 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.api.r
import java.io._
import java.net.ServerSocket
import java.util.{Map => JMap}
import scala.collection.JavaConversions._
import scala.io.Source
import scala.reflect.ClassTag
import scala.util.Try
import org.apache.spark._
import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
import org.apache.spark.util.Utils
private abstract class BaseRRDD[T: ClassTag, U: ClassTag](
parent: RDD[T],
numPartitions: Int,
func: Array[Byte],
deserializer: String,
serializer: String,
packageNames: Array[Byte],
rLibDir: String,
broadcastVars: Array[Broadcast[Object]])
extends RDD[U](parent) with Logging {
override def getPartitions: Array[Partition] = parent.partitions
override def compute(partition: Partition, context: TaskContext): Iterator[U] = {
// The parent may be also an RRDD, so we should launch it first.
val parentIterator = firstParent[T].iterator(partition, context)
// we expect two connections
val serverSocket = new ServerSocket(0, 2)
val listenPort = serverSocket.getLocalPort()
// The stdout/stderr is shared by multiple tasks, because we use one daemon
// to launch child process as worker.
val errThread = RRDD.createRWorker(rLibDir, listenPort)
// We use two sockets to separate input and output, then it's easy to manage
// the lifecycle of them to avoid deadlock.
// TODO: optimize it to use one socket
// the socket used to send out the input of task
serverSocket.setSoTimeout(10000)
val inSocket = serverSocket.accept()
startStdinThread(inSocket.getOutputStream(), parentIterator, partition.index)
// the socket used to receive the output of task
val outSocket = serverSocket.accept()
val inputStream = new BufferedInputStream(outSocket.getInputStream)
val dataStream = openDataStream(inputStream)
serverSocket.close()
try {
return new Iterator[U] {
def next(): U = {
val obj = _nextObj
if (hasNext) {
_nextObj = read()
}
obj
}
var _nextObj = read()
def hasNext(): Boolean = {
val hasMore = (_nextObj != null)
if (!hasMore) {
dataStream.close()
}
hasMore
}
}
} catch {
case e: Exception =>
throw new SparkException("R computation failed with\n " + errThread.getLines())
}
}
/**
* Start a thread to write RDD data to the R process.
*/
private def startStdinThread[T](
output: OutputStream,
iter: Iterator[T],
partition: Int): Unit = {
val env = SparkEnv.get
val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
val stream = new BufferedOutputStream(output, bufferSize)
new Thread("writer for R") {
override def run(): Unit = {
try {
SparkEnv.set(env)
val dataOut = new DataOutputStream(stream)
dataOut.writeInt(partition)
SerDe.writeString(dataOut, deserializer)
SerDe.writeString(dataOut, serializer)
dataOut.writeInt(packageNames.length)
dataOut.write(packageNames)
dataOut.writeInt(func.length)
dataOut.write(func)
dataOut.writeInt(broadcastVars.length)
broadcastVars.foreach { broadcast =>
// TODO(shivaram): Read a Long in R to avoid this cast
dataOut.writeInt(broadcast.id.toInt)
// TODO: Pass a byte array from R to avoid this cast ?
val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]]
dataOut.writeInt(broadcastByteArr.length)
dataOut.write(broadcastByteArr)
}
dataOut.writeInt(numPartitions)
if (!iter.hasNext) {
dataOut.writeInt(0)
} else {
dataOut.writeInt(1)
}
val printOut = new PrintStream(stream)
def writeElem(elem: Any): Unit = {
if (deserializer == SerializationFormats.BYTE) {
val elemArr = elem.asInstanceOf[Array[Byte]]
dataOut.writeInt(elemArr.length)
dataOut.write(elemArr)
} else if (deserializer == SerializationFormats.ROW) {
dataOut.write(elem.asInstanceOf[Array[Byte]])
} else if (deserializer == SerializationFormats.STRING) {
printOut.println(elem)
}
}
for (elem <- iter) {
elem match {
case (key, value) =>
writeElem(key)
writeElem(value)
case _ =>
writeElem(elem)
}
}
stream.flush()
} catch {
// TODO: We should propogate this error to the task thread
case e: Exception =>
logError("R Writer thread got an exception", e)
} finally {
Try(output.close())
}
}
}.start()
}
protected def openDataStream(input: InputStream): Closeable
protected def read(): U
}
/**
* Form an RDD[(Int, Array[Byte])] from key-value pairs returned from R.
* This is used by SparkR's shuffle operations.
*/
private class PairwiseRRDD[T: ClassTag](
parent: RDD[T],
numPartitions: Int,
hashFunc: Array[Byte],
deserializer: String,
packageNames: Array[Byte],
rLibDir: String,
broadcastVars: Array[Object])
extends BaseRRDD[T, (Int, Array[Byte])](
parent, numPartitions, hashFunc, deserializer,
SerializationFormats.BYTE, packageNames, rLibDir,
broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) {
private var dataStream: DataInputStream = _
override protected def openDataStream(input: InputStream): Closeable = {
dataStream = new DataInputStream(input)
dataStream
}
override protected def read(): (Int, Array[Byte]) = {
try {
val length = dataStream.readInt()
length match {
case length if length == 2 =>
val hashedKey = dataStream.readInt()
val contentPairsLength = dataStream.readInt()
val contentPairs = new Array[Byte](contentPairsLength)
dataStream.readFully(contentPairs)
(hashedKey, contentPairs)
case _ => null // End of input
}
} catch {
case eof: EOFException => {
throw new SparkException("R worker exited unexpectedly (crashed)", eof)
}
}
}
lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this)
}
/**
* An RDD that stores serialized R objects as Array[Byte].
*/
private class RRDD[T: ClassTag](
parent: RDD[T],
func: Array[Byte],
deserializer: String,
serializer: String,
packageNames: Array[Byte],
rLibDir: String,
broadcastVars: Array[Object])
extends BaseRRDD[T, Array[Byte]](
parent, -1, func, deserializer, serializer, packageNames, rLibDir,
broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) {
private var dataStream: DataInputStream = _
override protected def openDataStream(input: InputStream): Closeable = {
dataStream = new DataInputStream(input)
dataStream
}
override protected def read(): Array[Byte] = {
try {
val length = dataStream.readInt()
length match {
case length if length > 0 =>
val obj = new Array[Byte](length)
dataStream.readFully(obj, 0, length)
obj
case _ => null
}
} catch {
case eof: EOFException => {
throw new SparkException("R worker exited unexpectedly (crashed)", eof)
}
}
}
lazy val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
}
/**
* An RDD that stores R objects as Array[String].
*/
private class StringRRDD[T: ClassTag](
parent: RDD[T],
func: Array[Byte],
deserializer: String,
packageNames: Array[Byte],
rLibDir: String,
broadcastVars: Array[Object])
extends BaseRRDD[T, String](
parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, rLibDir,
broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) {
private var dataStream: BufferedReader = _
override protected def openDataStream(input: InputStream): Closeable = {
dataStream = new BufferedReader(new InputStreamReader(input))
dataStream
}
override protected def read(): String = {
try {
dataStream.readLine()
} catch {
case e: IOException => {
throw new SparkException("R worker exited unexpectedly (crashed)", e)
}
}
}
lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this)
}
private[r] class BufferedStreamThread(
in: InputStream,
name: String,
errBufferSize: Int) extends Thread(name) with Logging {
val lines = new Array[String](errBufferSize)
var lineIdx = 0
override def run() {
for (line <- Source.fromInputStream(in).getLines) {
synchronized {
lines(lineIdx) = line
lineIdx = (lineIdx + 1) % errBufferSize
}
logInfo(line)
}
}
def getLines(): String = synchronized {
(0 until errBufferSize).filter { x =>
lines((x + lineIdx) % errBufferSize) != null
}.map { x =>
lines((x + lineIdx) % errBufferSize)
}.mkString("\n")
}
}
private[r] object RRDD {
// Because forking processes from Java is expensive, we prefer to launch
// a single R daemon (daemon.R) and tell it to fork new workers for our tasks.
// This daemon currently only works on UNIX-based systems now, so we should
// also fall back to launching workers (worker.R) directly.
private[this] var errThread: BufferedStreamThread = _
private[this] var daemonChannel: DataOutputStream = _
def createSparkContext(
master: String,
appName: String,
sparkHome: String,
jars: Array[String],
sparkEnvirMap: JMap[Object, Object],
sparkExecutorEnvMap: JMap[Object, Object]): JavaSparkContext = {
val sparkConf = new SparkConf().setAppName(appName)
.setSparkHome(sparkHome)
.setJars(jars)
// Override `master` if we have a user-specified value
if (master != "") {
sparkConf.setMaster(master)
} else {
// If conf has no master set it to "local" to maintain
// backwards compatibility
sparkConf.setIfMissing("spark.master", "local")
}
for ((name, value) <- sparkEnvirMap) {
sparkConf.set(name.asInstanceOf[String], value.asInstanceOf[String])
}
for ((name, value) <- sparkExecutorEnvMap) {
sparkConf.setExecutorEnv(name.asInstanceOf[String], value.asInstanceOf[String])
}
new JavaSparkContext(sparkConf)
}
/**
* Start a thread to print the process's stderr to ours
*/
private def startStdoutThread(proc: Process): BufferedStreamThread = {
val BUFFER_SIZE = 100
val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE)
thread.setDaemon(true)
thread.start()
thread
}
private def createRProcess(rLibDir: String, port: Int, script: String): BufferedStreamThread = {
val rCommand = "Rscript"
val rOptions = "--vanilla"
val rExecScript = rLibDir + "/SparkR/worker/" + script
val pb = new ProcessBuilder(List(rCommand, rOptions, rExecScript))
// Unset the R_TESTS environment variable for workers.
// This is set by R CMD check as startup.Rs
// (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R)
// and confuses worker script which tries to load a non-existent file
pb.environment().put("R_TESTS", "")
pb.environment().put("SPARKR_RLIBDIR", rLibDir)
pb.environment().put("SPARKR_WORKER_PORT", port.toString)
pb.redirectErrorStream(true) // redirect stderr into stdout
val proc = pb.start()
val errThread = startStdoutThread(proc)
errThread
}
/**
* ProcessBuilder used to launch worker R processes.
*/
def createRWorker(rLibDir: String, port: Int): BufferedStreamThread = {
val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true)
if (!Utils.isWindows && useDaemon) {
synchronized {
if (daemonChannel == null) {
// we expect one connections
val serverSocket = new ServerSocket(0, 1)
val daemonPort = serverSocket.getLocalPort
errThread = createRProcess(rLibDir, daemonPort, "daemon.R")
// the socket used to send out the input of task
serverSocket.setSoTimeout(10000)
val sock = serverSocket.accept()
daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream))
serverSocket.close()
}
try {
daemonChannel.writeInt(port)
daemonChannel.flush()
} catch {
case e: IOException =>
// daemon process died
daemonChannel.close()
daemonChannel = null
errThread = null
// fail the current task, retry by scheduler
throw e
}
errThread
}
} else {
createRProcess(rLibDir, port, "worker.R")
}
}
/**
* Create an RRDD given a sequence of byte arrays. Used to create RRDD when `parallelize` is
* called from R.
*/
def createRDDFromArray(jsc: JavaSparkContext, arr: Array[Array[Byte]]): JavaRDD[Array[Byte]] = {
JavaRDD.fromRDD(jsc.sc.parallelize(arr, arr.length))
}
}

View file

@ -0,0 +1,340 @@
/*
* 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.api.r
import java.io.{DataInputStream, DataOutputStream}
import java.sql.{Date, Time}
import scala.collection.JavaConversions._
/**
* Utility functions to serialize, deserialize objects to / from R
*/
private[spark] object SerDe {
// Type mapping from R to Java
//
// NULL -> void
// integer -> Int
// character -> String
// logical -> Boolean
// double, numeric -> Double
// raw -> Array[Byte]
// Date -> Date
// POSIXlt/POSIXct -> Time
//
// list[T] -> Array[T], where T is one of above mentioned types
// environment -> Map[String, T], where T is a native type
// jobj -> Object, where jobj is an object created in the backend
def readObjectType(dis: DataInputStream): Char = {
dis.readByte().toChar
}
def readObject(dis: DataInputStream): Object = {
val dataType = readObjectType(dis)
readTypedObject(dis, dataType)
}
def readTypedObject(
dis: DataInputStream,
dataType: Char): Object = {
dataType match {
case 'n' => null
case 'i' => new java.lang.Integer(readInt(dis))
case 'd' => new java.lang.Double(readDouble(dis))
case 'b' => new java.lang.Boolean(readBoolean(dis))
case 'c' => readString(dis)
case 'e' => readMap(dis)
case 'r' => readBytes(dis)
case 'l' => readList(dis)
case 'D' => readDate(dis)
case 't' => readTime(dis)
case 'j' => JVMObjectTracker.getObject(readString(dis))
case _ => throw new IllegalArgumentException(s"Invalid type $dataType")
}
}
def readBytes(in: DataInputStream): Array[Byte] = {
val len = readInt(in)
val out = new Array[Byte](len)
val bytesRead = in.readFully(out)
out
}
def readInt(in: DataInputStream): Int = {
in.readInt()
}
def readDouble(in: DataInputStream): Double = {
in.readDouble()
}
def readString(in: DataInputStream): String = {
val len = in.readInt()
val asciiBytes = new Array[Byte](len)
in.readFully(asciiBytes)
assert(asciiBytes(len - 1) == 0)
val str = new String(asciiBytes.dropRight(1).map(_.toChar))
str
}
def readBoolean(in: DataInputStream): Boolean = {
val intVal = in.readInt()
if (intVal == 0) false else true
}
def readDate(in: DataInputStream): Date = {
Date.valueOf(readString(in))
}
def readTime(in: DataInputStream): Time = {
val t = in.readDouble()
new Time((t * 1000L).toLong)
}
def readBytesArr(in: DataInputStream): Array[Array[Byte]] = {
val len = readInt(in)
(0 until len).map(_ => readBytes(in)).toArray
}
def readIntArr(in: DataInputStream): Array[Int] = {
val len = readInt(in)
(0 until len).map(_ => readInt(in)).toArray
}
def readDoubleArr(in: DataInputStream): Array[Double] = {
val len = readInt(in)
(0 until len).map(_ => readDouble(in)).toArray
}
def readBooleanArr(in: DataInputStream): Array[Boolean] = {
val len = readInt(in)
(0 until len).map(_ => readBoolean(in)).toArray
}
def readStringArr(in: DataInputStream): Array[String] = {
val len = readInt(in)
(0 until len).map(_ => readString(in)).toArray
}
def readList(dis: DataInputStream): Array[_] = {
val arrType = readObjectType(dis)
arrType match {
case 'i' => readIntArr(dis)
case 'c' => readStringArr(dis)
case 'd' => readDoubleArr(dis)
case 'b' => readBooleanArr(dis)
case 'j' => readStringArr(dis).map(x => JVMObjectTracker.getObject(x))
case 'r' => readBytesArr(dis)
case _ => throw new IllegalArgumentException(s"Invalid array type $arrType")
}
}
def readMap(in: DataInputStream): java.util.Map[Object, Object] = {
val len = readInt(in)
if (len > 0) {
val keysType = readObjectType(in)
val keysLen = readInt(in)
val keys = (0 until keysLen).map(_ => readTypedObject(in, keysType))
val valuesType = readObjectType(in)
val valuesLen = readInt(in)
val values = (0 until valuesLen).map(_ => readTypedObject(in, valuesType))
mapAsJavaMap(keys.zip(values).toMap)
} else {
new java.util.HashMap[Object, Object]()
}
}
// Methods to write out data from Java to R
//
// Type mapping from Java to R
//
// void -> NULL
// Int -> integer
// String -> character
// Boolean -> logical
// Double -> double
// Long -> double
// Array[Byte] -> raw
// Date -> Date
// Time -> POSIXct
//
// Array[T] -> list()
// Object -> jobj
def writeType(dos: DataOutputStream, typeStr: String): Unit = {
typeStr match {
case "void" => dos.writeByte('n')
case "character" => dos.writeByte('c')
case "double" => dos.writeByte('d')
case "integer" => dos.writeByte('i')
case "logical" => dos.writeByte('b')
case "date" => dos.writeByte('D')
case "time" => dos.writeByte('t')
case "raw" => dos.writeByte('r')
case "list" => dos.writeByte('l')
case "jobj" => dos.writeByte('j')
case _ => throw new IllegalArgumentException(s"Invalid type $typeStr")
}
}
def writeObject(dos: DataOutputStream, value: Object): Unit = {
if (value == null) {
writeType(dos, "void")
} else {
value.getClass.getName match {
case "java.lang.String" =>
writeType(dos, "character")
writeString(dos, value.asInstanceOf[String])
case "long" | "java.lang.Long" =>
writeType(dos, "double")
writeDouble(dos, value.asInstanceOf[Long].toDouble)
case "double" | "java.lang.Double" =>
writeType(dos, "double")
writeDouble(dos, value.asInstanceOf[Double])
case "int" | "java.lang.Integer" =>
writeType(dos, "integer")
writeInt(dos, value.asInstanceOf[Int])
case "boolean" | "java.lang.Boolean" =>
writeType(dos, "logical")
writeBoolean(dos, value.asInstanceOf[Boolean])
case "java.sql.Date" =>
writeType(dos, "date")
writeDate(dos, value.asInstanceOf[Date])
case "java.sql.Time" =>
writeType(dos, "time")
writeTime(dos, value.asInstanceOf[Time])
case "[B" =>
writeType(dos, "raw")
writeBytes(dos, value.asInstanceOf[Array[Byte]])
// TODO: Types not handled right now include
// byte, char, short, float
// Handle arrays
case "[Ljava.lang.String;" =>
writeType(dos, "list")
writeStringArr(dos, value.asInstanceOf[Array[String]])
case "[I" =>
writeType(dos, "list")
writeIntArr(dos, value.asInstanceOf[Array[Int]])
case "[J" =>
writeType(dos, "list")
writeDoubleArr(dos, value.asInstanceOf[Array[Long]].map(_.toDouble))
case "[D" =>
writeType(dos, "list")
writeDoubleArr(dos, value.asInstanceOf[Array[Double]])
case "[Z" =>
writeType(dos, "list")
writeBooleanArr(dos, value.asInstanceOf[Array[Boolean]])
case "[[B" =>
writeType(dos, "list")
writeBytesArr(dos, value.asInstanceOf[Array[Array[Byte]]])
case otherName =>
// Handle array of objects
if (otherName.startsWith("[L")) {
val objArr = value.asInstanceOf[Array[Object]]
writeType(dos, "list")
writeType(dos, "jobj")
dos.writeInt(objArr.length)
objArr.foreach(o => writeJObj(dos, o))
} else {
writeType(dos, "jobj")
writeJObj(dos, value)
}
}
}
}
def writeInt(out: DataOutputStream, value: Int): Unit = {
out.writeInt(value)
}
def writeDouble(out: DataOutputStream, value: Double): Unit = {
out.writeDouble(value)
}
def writeBoolean(out: DataOutputStream, value: Boolean): Unit = {
val intValue = if (value) 1 else 0
out.writeInt(intValue)
}
def writeDate(out: DataOutputStream, value: Date): Unit = {
writeString(out, value.toString)
}
def writeTime(out: DataOutputStream, value: Time): Unit = {
out.writeDouble(value.getTime.toDouble / 1000.0)
}
// NOTE: Only works for ASCII right now
def writeString(out: DataOutputStream, value: String): Unit = {
val len = value.length
out.writeInt(len + 1) // For the \0
out.writeBytes(value)
out.writeByte(0)
}
def writeBytes(out: DataOutputStream, value: Array[Byte]): Unit = {
out.writeInt(value.length)
out.write(value)
}
def writeJObj(out: DataOutputStream, value: Object): Unit = {
val objId = JVMObjectTracker.put(value)
writeString(out, objId)
}
def writeIntArr(out: DataOutputStream, value: Array[Int]): Unit = {
writeType(out, "integer")
out.writeInt(value.length)
value.foreach(v => out.writeInt(v))
}
def writeDoubleArr(out: DataOutputStream, value: Array[Double]): Unit = {
writeType(out, "double")
out.writeInt(value.length)
value.foreach(v => out.writeDouble(v))
}
def writeBooleanArr(out: DataOutputStream, value: Array[Boolean]): Unit = {
writeType(out, "logical")
out.writeInt(value.length)
value.foreach(v => writeBoolean(out, v))
}
def writeStringArr(out: DataOutputStream, value: Array[String]): Unit = {
writeType(out, "character")
out.writeInt(value.length)
value.foreach(v => writeString(out, v))
}
def writeBytesArr(out: DataOutputStream, value: Array[Array[Byte]]): Unit = {
writeType(out, "raw")
out.writeInt(value.length)
value.foreach(v => writeBytes(out, v))
}
}
private[r] object SerializationFormats {
val BYTE = "byte"
val STRING = "string"
val ROW = "row"
}

View file

@ -0,0 +1,92 @@
/*
* 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.deploy
import java.io._
import java.util.concurrent.{Semaphore, TimeUnit}
import scala.collection.JavaConversions._
import org.apache.hadoop.fs.Path
import org.apache.spark.api.r.RBackend
import org.apache.spark.util.RedirectThread
/**
* Main class used to launch SparkR applications using spark-submit. It executes R as a
* subprocess and then has it connect back to the JVM to access system properties etc.
*/
object RRunner {
def main(args: Array[String]): Unit = {
val rFile = PythonRunner.formatPath(args(0))
val otherArgs = args.slice(1, args.length)
// Time to wait for SparkR backend to initialize in seconds
val backendTimeout = sys.env.getOrElse("SPARKR_BACKEND_TIMEOUT", "120").toInt
val rCommand = "Rscript"
// Check if the file path exists.
// If not, change directory to current working directory for YARN cluster mode
val rF = new File(rFile)
val rFileNormalized = if (!rF.exists()) {
new Path(rFile).getName
} else {
rFile
}
// Launch a SparkR backend server for the R process to connect to; this will let it see our
// Java system properties etc.
val sparkRBackend = new RBackend()
@volatile var sparkRBackendPort = 0
val initialized = new Semaphore(0)
val sparkRBackendThread = new Thread("SparkR backend") {
override def run() {
sparkRBackendPort = sparkRBackend.init()
initialized.release()
sparkRBackend.run()
}
}
sparkRBackendThread.start()
// Wait for RBackend initialization to finish
if (initialized.tryAcquire(backendTimeout, TimeUnit.SECONDS)) {
// Launch R
val returnCode = try {
val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs)
val env = builder.environment()
env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString)
val sparkHome = System.getenv("SPARK_HOME")
env.put("R_PROFILE_USER",
Seq(sparkHome, "R", "lib", "SparkR", "profile", "general.R").mkString(File.separator))
builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize
val process = builder.start()
new RedirectThread(process.getInputStream, System.out, "redirect R output").start()
process.waitFor()
} finally {
sparkRBackend.close()
}
System.exit(returnCode)
} else {
System.err.println("SparkR backend did not initialize in " + backendTimeout + " seconds")
System.exit(-1)
}
}
}

View file

@ -77,6 +77,7 @@ object SparkSubmit {
// Special primary resource names that represent shells rather than application jars.
private val SPARK_SHELL = "spark-shell"
private val PYSPARK_SHELL = "pyspark-shell"
private val SPARKR_SHELL = "sparkr-shell"
private val CLASS_NOT_FOUND_EXIT_STATUS = 101
@ -284,6 +285,13 @@ object SparkSubmit {
}
}
// Require all R files to be local
if (args.isR && !isYarnCluster) {
if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) {
printErrorAndExit(s"Only local R files are supported: $args.primaryResource")
}
}
// The following modes are not supported or applicable
(clusterManager, deployMode) match {
case (MESOS, CLUSTER) =>
@ -291,6 +299,9 @@ object SparkSubmit {
case (STANDALONE, CLUSTER) if args.isPython =>
printErrorAndExit("Cluster deploy mode is currently not supported for python " +
"applications on standalone clusters.")
case (STANDALONE, CLUSTER) if args.isR =>
printErrorAndExit("Cluster deploy mode is currently not supported for R " +
"applications on standalone clusters.")
case (_, CLUSTER) if isShell(args.primaryResource) =>
printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.")
case (_, CLUSTER) if isSqlShell(args.mainClass) =>
@ -317,11 +328,32 @@ object SparkSubmit {
}
}
// In yarn-cluster mode for a python app, add primary resource and pyFiles to files
// that can be distributed with the job
if (args.isPython && isYarnCluster) {
args.files = mergeFileLists(args.files, args.primaryResource)
args.files = mergeFileLists(args.files, args.pyFiles)
// If we're running a R app, set the main class to our specific R runner
if (args.isR && deployMode == CLIENT) {
if (args.primaryResource == SPARKR_SHELL) {
args.mainClass = "org.apache.spark.api.r.RBackend"
} else {
// If a R file is provided, add it to the child arguments and list of files to deploy.
// Usage: RRunner <main R file> [app arguments]
args.mainClass = "org.apache.spark.deploy.RRunner"
args.childArgs = ArrayBuffer(args.primaryResource) ++ args.childArgs
args.files = mergeFileLists(args.files, args.primaryResource)
}
}
if (isYarnCluster) {
// In yarn-cluster mode for a python app, add primary resource and pyFiles to files
// that can be distributed with the job
if (args.isPython) {
args.files = mergeFileLists(args.files, args.primaryResource)
args.files = mergeFileLists(args.files, args.pyFiles)
}
// In yarn-cluster mode for a R app, add primary resource to files
// that can be distributed with the job
if (args.isR) {
args.files = mergeFileLists(args.files, args.primaryResource)
}
}
// Special flag to avoid deprecation warnings at the client
@ -405,8 +437,8 @@ object SparkSubmit {
// Add the application jar automatically so the user doesn't have to call sc.addJar
// For YARN cluster mode, the jar is already distributed on each node as "app.jar"
// For python files, the primary resource is already distributed as a regular file
if (!isYarnCluster && !args.isPython) {
// For python and R files, the primary resource is already distributed as a regular file
if (!isYarnCluster && !args.isPython && !args.isR) {
var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty)
if (isUserJar(args.primaryResource)) {
jars = jars ++ Seq(args.primaryResource)
@ -447,6 +479,10 @@ object SparkSubmit {
childArgs += ("--py-files", pyFilesNames)
}
childArgs += ("--class", "org.apache.spark.deploy.PythonRunner")
} else if (args.isR) {
val mainFile = new Path(args.primaryResource).getName
childArgs += ("--primary-r-file", mainFile)
childArgs += ("--class", "org.apache.spark.deploy.RRunner")
} else {
if (args.primaryResource != SPARK_INTERNAL) {
childArgs += ("--jar", args.primaryResource)
@ -591,15 +627,15 @@ object SparkSubmit {
/**
* Return whether the given primary resource represents a user jar.
*/
private def isUserJar(primaryResource: String): Boolean = {
!isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource)
private[deploy] def isUserJar(res: String): Boolean = {
!isShell(res) && !isPython(res) && !isInternal(res) && !isR(res)
}
/**
* Return whether the given primary resource represents a shell.
*/
private[deploy] def isShell(primaryResource: String): Boolean = {
primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL
private[deploy] def isShell(res: String): Boolean = {
(res == SPARK_SHELL || res == PYSPARK_SHELL || res == SPARKR_SHELL)
}
/**
@ -619,12 +655,19 @@ object SparkSubmit {
/**
* Return whether the given primary resource requires running python.
*/
private[deploy] def isPython(primaryResource: String): Boolean = {
primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL
private[deploy] def isPython(res: String): Boolean = {
res != null && res.endsWith(".py") || res == PYSPARK_SHELL
}
private[deploy] def isInternal(primaryResource: String): Boolean = {
primaryResource == SPARK_INTERNAL
/**
* Return whether the given primary resource requires running R.
*/
private[deploy] def isR(res: String): Boolean = {
res != null && res.endsWith(".R") || res == SPARKR_SHELL
}
private[deploy] def isInternal(res: String): Boolean = {
res == SPARK_INTERNAL
}
/**

View file

@ -59,6 +59,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
var verbose: Boolean = false
var isPython: Boolean = false
var pyFiles: String = null
var isR: Boolean = false
var action: SparkSubmitAction = null
val sparkProperties: HashMap[String, String] = new HashMap[String, String]()
var proxyUser: String = null
@ -158,7 +159,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
.getOrElse(sparkProperties.get("spark.executor.instances").orNull)
// Try to set main class from JAR if no --class argument is given
if (mainClass == null && !isPython && primaryResource != null) {
if (mainClass == null && !isPython && !isR && primaryResource != null) {
val uri = new URI(primaryResource)
val uriScheme = uri.getScheme()
@ -211,9 +212,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
printUsageAndExit(-1)
}
if (primaryResource == null) {
SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python file)")
SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python or R file)")
}
if (mainClass == null && !isPython) {
if (mainClass == null && SparkSubmit.isUserJar(primaryResource)) {
SparkSubmit.printErrorAndExit("No main class set in JAR; please specify one with --class")
}
if (pyFiles != null && !isPython) {
@ -414,6 +415,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
opt
}
isPython = SparkSubmit.isPython(opt)
isR = SparkSubmit.isR(opt)
false
}

View file

@ -236,3 +236,18 @@ echo "========================================================================="
CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS
./python/run-tests
echo ""
echo "========================================================================="
echo "Running SparkR tests"
echo "========================================================================="
CURRENT_BLOCK=$BLOCK_SPARKR_UNIT_TESTS
if [ $(command -v R) ]; then
./R/install-dev.sh
./R/run-tests.sh
else
echo "Ignoring SparkR tests as R was not found in PATH"
fi

View file

@ -25,3 +25,4 @@ readonly BLOCK_BUILD=14
readonly BLOCK_MIMA=15
readonly BLOCK_SPARK_UNIT_TESTS=16
readonly BLOCK_PYSPARK_UNIT_TESTS=17
readonly BLOCK_SPARKR_UNIT_TESTS=18

View file

@ -210,6 +210,8 @@ done
failing_test="Spark unit tests"
elif [ "$test_result" -eq "$BLOCK_PYSPARK_UNIT_TESTS" ]; then
failing_test="PySpark unit tests"
elif [ "$test_result" -eq "$BLOCK_SPARKR_UNIT_TESTS" ]; then
failing_test="SparkR unit tests"
else
failing_test="some tests"
fi

View file

@ -58,13 +58,19 @@ phase, use the following sytax:
We use Sphinx to generate Python API docs, so you will need to install it by running
`sudo pip install sphinx`.
## API Docs (Scaladoc and Sphinx)
## knitr, devtools
SparkR documentation is written using `roxygen2` and we use `knitr`, `devtools` to generate
documentation. To install these packages you can run `install.packages(c("knitr", "devtools"))` from a
R console.
## API Docs (Scaladoc, Sphinx, roxygen2)
You can build just the Spark scaladoc by running `build/sbt unidoc` from the SPARK_PROJECT_ROOT directory.
Similarly, you can build just the PySpark docs by running `make html` from the
SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as
public in `__init__.py`.
public in `__init__.py`. The SparkR docs can be built by running SPARK_PROJECT_ROOT/R/create-docs.sh.
When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various
Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a
@ -72,5 +78,5 @@ jekyll plugin to run `build/sbt unidoc` before building the site so if you haven
may take some time as it generates all of the scaladoc. The jekyll plugin also generates the
PySpark docs [Sphinx](http://sphinx-doc.org/).
NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1
NOTE: To skip the step of building and copying over the Scala, Python, R API docs, run `SKIP_API=1
jekyll`.

View file

@ -84,6 +84,7 @@
<li><a href="api/scala/index.html#org.apache.spark.package">Scala</a></li>
<li><a href="api/java/index.html">Java</a></li>
<li><a href="api/python/index.html">Python</a></li>
<li><a href="api/R/index.html">R</a></li>
</ul>
</li>

View file

@ -78,5 +78,18 @@ if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1')
puts "cp -r python/docs/_build/html/. docs/api/python"
cp_r("python/docs/_build/html/.", "docs/api/python")
cd("..")
# Build SparkR API docs
puts "Moving to R directory and building roxygen docs."
cd("R")
puts `./create-docs.sh`
puts "Moving back into home dir."
cd("../")
puts "Making directory api/R"
mkdir_p "docs/api/R"
puts "cp -r R/pkg/html/. docs/api/R"
cp_r("R/pkg/html/.", "docs/api/R")
end

View file

@ -0,0 +1,93 @@
#
# 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.
#
library(SparkR)
# Logistic regression in Spark.
# Note: unlike the example in Scala, a point here is represented as a vector of
# doubles.
parseVectors <- function(lines) {
lines <- strsplit(as.character(lines) , " ", fixed = TRUE)
list(matrix(as.numeric(unlist(lines)), ncol = length(lines[[1]])))
}
dist.fun <- function(P, C) {
apply(
C,
1,
function(x) {
colSums((t(P) - x)^2)
}
)
}
closestPoint <- function(P, C) {
max.col(-dist.fun(P, C))
}
# Main program
args <- commandArgs(trailing = TRUE)
if (length(args) != 3) {
print("Usage: kmeans <file> <K> <convergeDist>")
q("no")
}
sc <- sparkR.init(appName = "RKMeans")
K <- as.integer(args[[2]])
convergeDist <- as.double(args[[3]])
lines <- textFile(sc, args[[1]])
points <- cache(lapplyPartition(lines, parseVectors))
# kPoints <- take(points, K)
kPoints <- do.call(rbind, takeSample(points, FALSE, K, 16189L))
tempDist <- 1.0
while (tempDist > convergeDist) {
closest <- lapplyPartition(
lapply(points,
function(p) {
cp <- closestPoint(p, kPoints);
mapply(list, unique(cp), split.data.frame(cbind(1, p), cp), SIMPLIFY=FALSE)
}),
function(x) {do.call(c, x)
})
pointStats <- reduceByKey(closest,
function(p1, p2) {
t(colSums(rbind(p1, p2)))
},
2L)
newPoints <- do.call(
rbind,
collect(lapply(pointStats,
function(tup) {
point.sum <- tup[[2]][, -1]
point.count <- tup[[2]][, 1]
point.sum/point.count
})))
D <- dist.fun(kPoints, newPoints)
tempDist <- sum(D[cbind(1:3, max.col(-D))])
kPoints <- newPoints
cat("Finished iteration (delta = ", tempDist, ")\n")
}
cat("Final centers:\n")
writeLines(unlist(lapply(kPoints, paste, collapse = " ")))

View file

@ -0,0 +1,107 @@
#
# 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.
#
# Instructions: https://github.com/amplab-extras/SparkR-pkg/wiki/SparkR-Example:-Digit-Recognition-on-EC2
library(SparkR)
library(Matrix)
args <- commandArgs(trailing = TRUE)
# number of random features; default to 1100
D <- ifelse(length(args) > 0, as.integer(args[[1]]), 1100)
# number of partitions for training dataset
trainParts <- 12
# dimension of digits
d <- 784
# number of test examples
NTrain <- 60000
# number of training examples
NTest <- 10000
# scale of features
gamma <- 4e-4
sc <- sparkR.init(appName = "SparkR-LinearSolver")
# You can also use HDFS path to speed things up:
# hdfs://<master>/train-mnist-dense-with-labels.data
file <- textFile(sc, "/data/train-mnist-dense-with-labels.data", trainParts)
W <- gamma * matrix(nrow=D, ncol=d, data=rnorm(D*d))
b <- 2 * pi * matrix(nrow=D, ncol=1, data=runif(D))
broadcastW <- broadcast(sc, W)
broadcastB <- broadcast(sc, b)
includePackage(sc, Matrix)
numericLines <- lapplyPartitionsWithIndex(file,
function(split, part) {
matList <- sapply(part, function(line) {
as.numeric(strsplit(line, ",", fixed=TRUE)[[1]])
}, simplify=FALSE)
mat <- Matrix(ncol=d+1, data=unlist(matList, F, F),
sparse=T, byrow=T)
mat
})
featureLabels <- cache(lapplyPartition(
numericLines,
function(part) {
label <- part[,1]
mat <- part[,-1]
ones <- rep(1, nrow(mat))
features <- cos(
mat %*% t(value(broadcastW)) + (matrix(ncol=1, data=ones) %*% t(value(broadcastB))))
onesMat <- Matrix(ones)
featuresPlus <- cBind(features, onesMat)
labels <- matrix(nrow=nrow(mat), ncol=10, data=-1)
for (i in 1:nrow(mat)) {
labels[i, label[i]] <- 1
}
list(label=labels, features=featuresPlus)
}))
FTF <- Reduce("+", collect(lapplyPartition(featureLabels,
function(part) {
t(part$features) %*% part$features
}), flatten=F))
FTY <- Reduce("+", collect(lapplyPartition(featureLabels,
function(part) {
t(part$features) %*% part$label
}), flatten=F))
# solve for the coefficient matrix
C <- solve(FTF, FTY)
test <- Matrix(as.matrix(read.csv("/data/test-mnist-dense-with-labels.data",
header=F), sparse=T))
testData <- test[,-1]
testLabels <- matrix(ncol=1, test[,1])
err <- 0
# contstruct the feature maps for all examples from this digit
featuresTest <- cos(testData %*% t(value(broadcastW)) +
(matrix(ncol=1, data=rep(1, NTest)) %*% t(value(broadcastB))))
featuresTest <- cBind(featuresTest, Matrix(rep(1, NTest)))
# extract the one vs. all assignment
results <- featuresTest %*% C
labelsGot <- apply(results, 1, which.max)
err <- sum(testLabels != labelsGot) / nrow(testLabels)
cat("\nFinished running. The error rate is: ", err, ".\n")

View file

@ -0,0 +1,62 @@
#
# 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.
#
library(SparkR)
args <- commandArgs(trailing = TRUE)
if (length(args) != 3) {
print("Usage: logistic_regression <file> <iters> <dimension>")
q("no")
}
# Initialize Spark context
sc <- sparkR.init(appName = "LogisticRegressionR")
iterations <- as.integer(args[[2]])
D <- as.integer(args[[3]])
readPartition <- function(part){
part = strsplit(part, " ", fixed = T)
list(matrix(as.numeric(unlist(part)), ncol = length(part[[1]])))
}
# Read data points and convert each partition to a matrix
points <- cache(lapplyPartition(textFile(sc, args[[1]]), readPartition))
# Initialize w to a random value
w <- runif(n=D, min = -1, max = 1)
cat("Initial w: ", w, "\n")
# Compute logistic regression gradient for a matrix of data points
gradient <- function(partition) {
partition = partition[[1]]
Y <- partition[, 1] # point labels (first column of input file)
X <- partition[, -1] # point coordinates
# For each point (x, y), compute gradient function
dot <- X %*% w
logit <- 1 / (1 + exp(-Y * dot))
grad <- t(X) %*% ((logit - 1) * Y)
list(grad)
}
for (i in 1:iterations) {
cat("On iteration ", i, "\n")
w <- w - reduce(lapplyPartition(points, gradient), "+")
}
cat("Final w: ", w, "\n")

46
examples/src/main/r/pi.R Normal file
View file

@ -0,0 +1,46 @@
#
# 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.
#
library(SparkR)
args <- commandArgs(trailing = TRUE)
sc <- sparkR.init(appName = "PiR")
slices <- ifelse(length(args) > 1, as.integer(args[[2]]), 2)
n <- 100000 * slices
piFunc <- function(elem) {
rands <- runif(n = 2, min = -1, max = 1)
val <- ifelse((rands[1]^2 + rands[2]^2) < 1, 1.0, 0.0)
val
}
piFuncVec <- function(elems) {
message(length(elems))
rands1 <- runif(n = length(elems), min = -1, max = 1)
rands2 <- runif(n = length(elems), min = -1, max = 1)
val <- ifelse((rands1^2 + rands2^2) < 1, 1.0, 0.0)
sum(val)
}
rdd <- parallelize(sc, 1:n, slices)
count <- reduce(lapplyPartition(rdd, piFuncVec), sum)
cat("Pi is roughly", 4.0 * count / n, "\n")
cat("Num elements in RDD ", count(rdd), "\n")

View file

@ -0,0 +1,42 @@
#
# 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.
#
library(SparkR)
args <- commandArgs(trailing = TRUE)
if (length(args) != 1) {
print("Usage: wordcount <file>")
q("no")
}
# Initialize Spark context
sc <- sparkR.init(appName = "RwordCount")
lines <- textFile(sc, args[[1]])
words <- flatMap(lines,
function(line) {
strsplit(line, " ")[[1]]
})
wordCount <- lapply(words, function(word) { list(word, 1L) })
counts <- reduceByKey(wordCount, "+", 2L)
output <- collect(counts)
for (wordcount in output) {
cat(wordcount[[1]], ": ", wordcount[[2]], "\n")
}

View file

@ -274,14 +274,14 @@ class CommandBuilderUtils {
}
/**
* Quotes a string so that it can be used in a command string and be parsed back into a single
* argument by python's "shlex.split()" function.
*
* Quotes a string so that it can be used in a command string.
* Basically, just add simple escapes. E.g.:
* original single argument : ab "cd" ef
* after: "ab \"cd\" ef"
*
* This can be parsed back into a single argument by python's "shlex.split()" function.
*/
static String quoteForPython(String s) {
static String quoteForCommandString(String s) {
StringBuilder quoted = new StringBuilder().append('"');
for (int i = 0; i < s.length(); i++) {
int cp = s.codePointAt(i);

View file

@ -17,14 +17,9 @@
package org.apache.spark.launcher;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.*;
import static org.apache.spark.launcher.CommandBuilderUtils.*;
@ -53,6 +48,20 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
*/
static final String PYSPARK_SHELL_RESOURCE = "pyspark-shell";
/**
* Name of the app resource used to identify the SparkR shell. The command line parser expects
* the resource name to be the very first argument to spark-submit in this case.
*
* NOTE: this cannot be "sparkr-shell" since that identifies the SparkR shell to SparkSubmit
* (see sparkR.R), and can cause this code to enter into an infinite loop.
*/
static final String SPARKR_SHELL = "sparkr-shell-main";
/**
* This is the actual resource name that identifies the SparkR shell to SparkSubmit.
*/
static final String SPARKR_SHELL_RESOURCE = "sparkr-shell";
/**
* This map must match the class names for available special classes, since this modifies the way
* command line parsing works. This maps the class name to the resource to use when calling
@ -87,6 +96,10 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
this.allowsMixedArguments = true;
appResource = PYSPARK_SHELL_RESOURCE;
submitArgs = args.subList(1, args.size());
} else if (args.size() > 0 && args.get(0).equals(SPARKR_SHELL)) {
this.allowsMixedArguments = true;
appResource = SPARKR_SHELL_RESOURCE;
submitArgs = args.subList(1, args.size());
} else {
this.allowsMixedArguments = false;
}
@ -98,6 +111,8 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
public List<String> buildCommand(Map<String, String> env) throws IOException {
if (PYSPARK_SHELL_RESOURCE.equals(appResource)) {
return buildPySparkShellCommand(env);
} else if (SPARKR_SHELL_RESOURCE.equals(appResource)) {
return buildSparkRCommand(env);
} else {
return buildSparkSubmitCommand(env);
}
@ -213,26 +228,14 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
return buildCommand(env);
}
// When launching the pyspark shell, the spark-submit arguments should be stored in the
// PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable
// set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS.
checkArgument(appArgs.isEmpty(), "pyspark does not support any application options.");
Properties props = loadPropertiesFile();
mergeEnvPathList(env, getLibPathEnvName(),
firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props));
// Store spark-submit arguments in an environment variable, since there's no way to pass
// them to shell.py on the comand line.
StringBuilder submitArgs = new StringBuilder();
for (String arg : buildSparkSubmitArgs()) {
if (submitArgs.length() > 0) {
submitArgs.append(" ");
}
submitArgs.append(quoteForPython(arg));
}
env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString());
// When launching the pyspark shell, the spark-submit arguments should be stored in the
// PYSPARK_SUBMIT_ARGS env variable.
constructEnvVarArgs(env, "PYSPARK_SUBMIT_ARGS");
// The executable is the PYSPARK_DRIVER_PYTHON env variable set by the pyspark script,
// followed by PYSPARK_DRIVER_PYTHON_OPTS.
List<String> pyargs = new ArrayList<String>();
pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python"));
String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS");
@ -243,6 +246,44 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
return pyargs;
}
private List<String> buildSparkRCommand(Map<String, String> env) throws IOException {
if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".R")) {
appResource = appArgs.get(0);
appArgs.remove(0);
return buildCommand(env);
}
// When launching the SparkR shell, store the spark-submit arguments in the SPARKR_SUBMIT_ARGS
// env variable.
constructEnvVarArgs(env, "SPARKR_SUBMIT_ARGS");
// Set shell.R as R_PROFILE_USER to load the SparkR package when the shell comes up.
String sparkHome = System.getenv("SPARK_HOME");
env.put("R_PROFILE_USER",
join(File.separator, sparkHome, "R", "lib", "SparkR", "profile", "shell.R"));
List<String> args = new ArrayList<String>();
args.add(firstNonEmpty(System.getenv("SPARKR_DRIVER_R"), "R"));
return args;
}
private void constructEnvVarArgs(
Map<String, String> env,
String submitArgsEnvVariable) throws IOException {
Properties props = loadPropertiesFile();
mergeEnvPathList(env, getLibPathEnvName(),
firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props));
StringBuilder submitArgs = new StringBuilder();
for (String arg : buildSparkSubmitArgs()) {
if (submitArgs.length() > 0) {
submitArgs.append(" ");
}
submitArgs.append(quoteForCommandString(arg));
}
env.put(submitArgsEnvVariable, submitArgs.toString());
}
private boolean isClientMode(Properties userProps) {
String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER));
// Default master is "local[*]", so assume client mode in that case.

View file

@ -79,9 +79,9 @@ public class CommandBuilderUtilsSuite {
@Test
public void testPythonArgQuoting() {
assertEquals("\"abc\"", quoteForPython("abc"));
assertEquals("\"a b c\"", quoteForPython("a b c"));
assertEquals("\"a \\\"b\\\" c\"", quoteForPython("a \"b\" c"));
assertEquals("\"abc\"", quoteForCommandString("abc"));
assertEquals("\"a b c\"", quoteForCommandString("a b c"));
assertEquals("\"a \\\"b\\\" c\"", quoteForCommandString("a \"b\" c"));
}
private void testOpt(String opts, List<String> expected) {

View file

@ -1749,5 +1749,8 @@
<profile>
<id>parquet-provided</id>
</profile>
<profile>
<id>sparkr</id>
</profile>
</profiles>
</project>

View file

@ -34,7 +34,7 @@ import org.apache.spark.sql.types.NumericType
@Experimental
class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) {
private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = {
private[sql] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = {
val namedGroupingExprs = groupingExprs.map {
case expr: NamedExpression => expr
case expr: Expression => Alias(expr, expr.prettyString)()

View file

@ -0,0 +1,127 @@
/*
* 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.api.r
import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream}
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.api.r.SerDe
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
import org.apache.spark.sql.types.{DataType, StructType}
import org.apache.spark.sql.{Column, DataFrame, GroupedData, Row, SQLContext, SaveMode}
private[r] object SQLUtils {
def createSQLContext(jsc: JavaSparkContext): SQLContext = {
new SQLContext(jsc)
}
def getJavaSparkContext(sqlCtx: SQLContext): JavaSparkContext = {
new JavaSparkContext(sqlCtx.sparkContext)
}
def toSeq[T](arr: Array[T]): Seq[T] = {
arr.toSeq
}
def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = {
val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
val num = schema.fields.size
val rowRDD = rdd.map(bytesToRow)
sqlContext.createDataFrame(rowRDD, schema)
}
// A helper to include grouping columns in Agg()
def aggWithGrouping(gd: GroupedData, exprs: Column*): DataFrame = {
val aggExprs = exprs.map { col =>
col.expr match {
case expr: NamedExpression => expr
case expr: Expression => Alias(expr, expr.simpleString)()
}
}
gd.toDF(aggExprs)
}
def dfToRowRDD(df: DataFrame): JavaRDD[Array[Byte]] = {
df.map(r => rowToRBytes(r))
}
private[this] def bytesToRow(bytes: Array[Byte]): Row = {
val bis = new ByteArrayInputStream(bytes)
val dis = new DataInputStream(bis)
val num = SerDe.readInt(dis)
Row.fromSeq((0 until num).map { i =>
SerDe.readObject(dis)
}.toSeq)
}
private[this] def rowToRBytes(row: Row): Array[Byte] = {
val bos = new ByteArrayOutputStream()
val dos = new DataOutputStream(bos)
SerDe.writeInt(dos, row.length)
(0 until row.length).map { idx =>
val obj: Object = row(idx).asInstanceOf[Object]
SerDe.writeObject(dos, obj)
}
bos.toByteArray()
}
def dfToCols(df: DataFrame): Array[Array[Byte]] = {
// localDF is Array[Row]
val localDF = df.collect()
val numCols = df.columns.length
// dfCols is Array[Array[Any]]
val dfCols = convertRowsToColumns(localDF, numCols)
dfCols.map { col =>
colToRBytes(col)
}
}
def convertRowsToColumns(localDF: Array[Row], numCols: Int): Array[Array[Any]] = {
(0 until numCols).map { colIdx =>
localDF.map { row =>
row(colIdx)
}
}.toArray
}
def colToRBytes(col: Array[Any]): Array[Byte] = {
val numRows = col.length
val bos = new ByteArrayOutputStream()
val dos = new DataOutputStream(bos)
SerDe.writeInt(dos, numRows)
col.map { item =>
val obj: Object = item.asInstanceOf[Object]
SerDe.writeObject(dos, obj)
}
bos.toByteArray()
}
def saveMode(mode: String): SaveMode = {
mode match {
case "append" => SaveMode.Append
case "overwrite" => SaveMode.Overwrite
case "error" => SaveMode.ErrorIfExists
case "ignore" => SaveMode.Ignore
}
}
}

View file

@ -469,6 +469,9 @@ private[spark] class ApplicationMaster(
System.setProperty("spark.submit.pyFiles",
PythonRunner.formatPaths(args.pyFiles).mkString(","))
}
if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
// TODO(davies): add R dependencies here
}
val mainMethod = userClassLoader.loadClass(args.userClass)
.getMethod("main", classOf[Array[String]])

View file

@ -25,6 +25,7 @@ class ApplicationMasterArguments(val args: Array[String]) {
var userJar: String = null
var userClass: String = null
var primaryPyFile: String = null
var primaryRFile: String = null
var pyFiles: String = null
var userArgs: Seq[String] = Seq[String]()
var executorMemory = 1024
@ -54,6 +55,10 @@ class ApplicationMasterArguments(val args: Array[String]) {
primaryPyFile = value
args = tail
case ("--primary-r-file") :: value :: tail =>
primaryRFile = value
args = tail
case ("--py-files") :: value :: tail =>
pyFiles = value
args = tail
@ -79,6 +84,11 @@ class ApplicationMasterArguments(val args: Array[String]) {
}
}
if (primaryPyFile != null && primaryRFile != null) {
System.err.println("Cannot have primary-py-file and primary-r-file at the same time")
System.exit(-1)
}
userArgs = userArgsBuffer.readOnly
}
@ -92,6 +102,7 @@ class ApplicationMasterArguments(val args: Array[String]) {
| --jar JAR_PATH Path to your application's JAR file
| --class CLASS_NAME Name of your application's main class
| --primary-py-file A main Python file
| --primary-r-file A main R file
| --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to
| place on the PYTHONPATH for Python apps.
| --args ARGS Arguments to be passed to your application's main class.

View file

@ -491,6 +491,12 @@ private[spark] class Client(
} else {
Nil
}
val primaryRFile =
if (args.primaryRFile != null) {
Seq("--primary-r-file", args.primaryRFile)
} else {
Nil
}
val amClass =
if (isClusterMode) {
Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName
@ -500,12 +506,15 @@ private[spark] class Client(
if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) {
args.userArgs = ArrayBuffer(args.primaryPyFile, args.pyFiles) ++ args.userArgs
}
if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs
}
val userArgs = args.userArgs.flatMap { arg =>
Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg))
}
val amArgs =
Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ userArgs ++
Seq(
Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ primaryRFile ++
userArgs ++ Seq(
"--executor-memory", args.executorMemory.toString + "m",
"--executor-cores", args.executorCores.toString,
"--num-executors ", args.numExecutors.toString)

View file

@ -32,6 +32,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
var userClass: String = null
var pyFiles: String = null
var primaryPyFile: String = null
var primaryRFile: String = null
var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]()
var executorMemory = 1024 // MB
var executorCores = 1
@ -150,6 +151,10 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
primaryPyFile = value
args = tail
case ("--primary-r-file") :: value :: tail =>
primaryRFile = value
args = tail
case ("--args" | "--arg") :: value :: tail =>
if (args(0) == "--args") {
println("--args is deprecated. Use --arg instead.")
@ -228,6 +233,11 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
throw new IllegalArgumentException(getUsageMessage(args))
}
}
if (primaryPyFile != null && primaryRFile != null) {
throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" +
" at the same time")
}
}
private def getUsageMessage(unknownParam: List[String] = null): String = {
@ -240,6 +250,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
| mode)
| --class CLASS_NAME Name of your application's main class (required)
| --primary-py-file A main Python file
| --primary-r-file A main R file
| --arg ARG Argument to be passed to your application's main class.
| Multiple invocations are possible, each will be passed in order.
| --num-executors NUM Number of executors to start (Default: 2)