From 73430c643aba95cbd9f8b7ce7546692893699d79 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 31 Jan 2015 00:06:50 -0800 Subject: [PATCH 001/121] Make SparkR work on paths with spaces on Windows --- pkg/R/sparkR.R | 21 ++++++++++++++++----- pkg/R/sparkRClient.R | 2 ++ 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 7feb45e8081c5..d0d36ad514b64 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -4,7 +4,6 @@ assemblyJarName <- "sparkr-assembly-0.1.jar" sparkR.onLoad <- function(libname, pkgname) { assemblyJarPath <- paste(libname, "/SparkR/", assemblyJarName, sep = "") - assemblyJarPath <- gsub(" ", "\\ ", assemblyJarPath, fixed = T) packageStartupMessage("[SparkR] Initializing with classpath ", assemblyJarPath, "\n") .sparkREnv$libname <- libname @@ -87,9 +86,16 @@ sparkR.init <- function( } sparkMem <- Sys.getenv("SPARK_MEM", "512m") - jars <- c(as.character(.sparkREnv$assemblyJarPath), as.character(sparkJars)) - - cp <- paste0(jars, collapse = ":") + jars <- suppressWarnings( + normalizePath(c(as.character(.sparkREnv$assemblyJarPath), as.character(sparkJars)))) + + # Classpath separator is ";" on Windows + if (.Platform$OS.type == "unix") { + collapseChar <- ":" + } else { + collapseChar <- ";" + } + cp <- paste0(jars, collapse = collapseChar) yarn_conf_dir <- Sys.getenv("YARN_CONF_DIR", "") if (yarn_conf_dir != "") { @@ -125,7 +131,12 @@ sparkR.init <- function( } nonEmptyJars <- Filter(function(x) { x != "" }, jars) - localJarPaths <- sapply(nonEmptyJars, function(j) { paste("file://", j, sep = "") }) + # URIs don't work very well on Windows, so just use paths. + localJarPaths <- if (.Platform$OS.type == "unix") { + sapply(nonEmptyJars, function(j) { paste("file://", j, sep = "") }) + } else { + nonEmptyJars + } assign( ".sparkRjsc", diff --git a/pkg/R/sparkRClient.R b/pkg/R/sparkRClient.R index e46f151eaf371..955ca4348b373 100644 --- a/pkg/R/sparkRClient.R +++ b/pkg/R/sparkRClient.R @@ -33,6 +33,8 @@ launchBackend <- function( } else { java_bin <- java_bin_name } + # Quote the classpath to make sure it handles spaces on Windows + classPath <- shQuote(classPath) combinedArgs <- paste(javaOpts, "-cp", classPath, mainClass, args, sep = " ") cat("Launching java with command ", java_bin, " ", combinedArgs, "\n") invisible(system2(java_bin, combinedArgs, wait = F)) From add97f5354ae83d4b87ff1e373a3c298cf589d55 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 31 Jan 2015 10:18:06 -0800 Subject: [PATCH 002/121] Use URL encode to create valid URIs for jars --- pkg/R/sparkR.R | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index d0d36ad514b64..0f683a6ffe7a9 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -131,12 +131,7 @@ sparkR.init <- function( } nonEmptyJars <- Filter(function(x) { x != "" }, jars) - # URIs don't work very well on Windows, so just use paths. - localJarPaths <- if (.Platform$OS.type == "unix") { - sapply(nonEmptyJars, function(j) { paste("file://", j, sep = "") }) - } else { - nonEmptyJars - } + localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file://", j, sep = "")) }) assign( ".sparkRjsc", From 06d99f0f2ce19c86ddd81c6ded69f30761b47228 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 31 Jan 2015 10:35:39 -0800 Subject: [PATCH 003/121] Fix URI to have right number of slashes --- pkg/R/sparkR.R | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 0f683a6ffe7a9..649b2f5942785 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -90,10 +90,13 @@ sparkR.init <- function( normalizePath(c(as.character(.sparkREnv$assemblyJarPath), 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 <- "////" } cp <- paste0(jars, collapse = collapseChar) @@ -131,7 +134,7 @@ sparkR.init <- function( } nonEmptyJars <- Filter(function(x) { x != "" }, jars) - localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file://", j, sep = "")) }) + localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) assign( ".sparkRjsc", From 179aa75afb2e5993cea5a3c86d363eed31d5ae8e Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 1 Feb 2015 06:28:27 +0000 Subject: [PATCH 004/121] 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 --- pkg/R/sparkRClient.R | 2 +- pkg/inst/worker/worker.R | 19 +++ .../edu/berkeley/cs/amplab/sparkr/RRDD.scala | 123 ++++++++++-------- 3 files changed, 86 insertions(+), 58 deletions(-) diff --git a/pkg/R/sparkRClient.R b/pkg/R/sparkRClient.R index e46f151eaf371..5721f33c71c9e 100644 --- a/pkg/R/sparkRClient.R +++ b/pkg/R/sparkRClient.R @@ -2,7 +2,7 @@ # Creates a SparkR client connection object # if one doesn't already exist -connectBackend <- function(hostname, port, timeout = 60) { +connectBackend <- function(hostname, port, timeout = 6000) { if (exists(".sparkRcon", envir = .sparkREnv)) { cat("SparkRBackend client connection already exists\n") return(get(".sparkRcon", envir = .sparkREnv)) diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index c5457adcbc54d..6d88b287b369b 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -1,5 +1,7 @@ # Worker class +begin <- proc.time()[3] + # NOTE: We use "stdin" to get the process stdin instead of the command line inputConStdin <- file("stdin", open = "rb") @@ -65,6 +67,8 @@ numPartitions <- SparkR:::readInt(inputCon) isEmpty <- SparkR:::readInt(inputCon) +metadataEnd <- proc.time()[3] + if (isEmpty != 0) { if (numPartitions == -1) { @@ -74,12 +78,15 @@ if (isEmpty != 0) { } else { data <- readLines(inputCon) } + dataReadEnd <- proc.time()[3] output <- do.call(execFunctionName, list(splitIndex, data)) + computeEnd <- proc.time()[3] if (isOutputSerialized) { SparkR:::writeRawSerialize(outputCon, output) } else { SparkR:::writeStrings(outputCon, output) } + writeEnd <- proc.time()[3] } else { if (isInputSerialized) { # Now read as many characters as described in funcLen @@ -88,6 +95,7 @@ if (isEmpty != 0) { data <- readLines(inputCon) } + dataReadEnd <- proc.time()[3] res <- new.env() # Step 1: hash the data to an environment @@ -105,6 +113,8 @@ if (isEmpty != 0) { } invisible(lapply(data, hashTupleToEnvir)) + computeEnd <- proc.time()[3] + # Step 2: write out all of the environment as key-value pairs. for (name in ls(res)) { SparkR:::writeInt(outputCon, 2L) @@ -113,6 +123,7 @@ if (isEmpty != 0) { length(res[[name]]$data) <- res[[name]]$counter SparkR:::writeRawSerialize(outputCon, res[[name]]$data) } + writeEnd <- proc.time()[3] } } @@ -128,5 +139,13 @@ unlink(inFileName) # Restore stdout sink() +end <- proc.time()[3] + +cat("stats: total ", (end-begin), "\n", file=stderr()) +cat("stats: metadata ", (metadataEnd-begin), "\n", file=stderr()) +cat("stats: input read ", (dataReadEnd-metadataEnd), "\n", file=stderr()) +cat("stats: compute ", (computeEnd-dataReadEnd), "\n", file=stderr()) +cat("stats: output write ", (writeEnd-computeEnd), "\n", file=stderr()) + # Finally print the name of the output file cat(outputFileName, "\n") diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 89eb10b23a293..e39d2acf8e8a3 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -117,68 +117,77 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for R") { override def run() { - SparkEnv.set(env) - val streamStd = new BufferedOutputStream(proc.getOutputStream, bufferSize) - val printOutStd = new PrintStream(streamStd) - printOutStd.println(tempFileName) - printOutStd.println(rLibDir) - printOutStd.println(tempFileIn.getAbsolutePath()) - printOutStd.flush() - - streamStd.close() - - val stream = new BufferedOutputStream(new FileOutputStream(tempFileIn), bufferSize) - val printOut = new PrintStream(stream) - val dataOut = new DataOutputStream(stream) - - dataOut.writeInt(splitIndex) - - dataOut.writeInt(func.length) - dataOut.write(func, 0, func.length) - - // R worker process input serialization flag - dataOut.writeInt(if (parentSerialized) 1 else 0) - // R worker process output serialization flag - dataOut.writeInt(if (dataSerialized) 1 else 0) - - dataOut.writeInt(packageNames.length) - dataOut.write(packageNames, 0, packageNames.length) - - dataOut.writeInt(functionDependencies.length) - dataOut.write(functionDependencies, 0, functionDependencies.length) - - 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, 0, broadcastByteArr.length) - } - - dataOut.writeInt(numPartitions) + try { + SparkEnv.set(env) + val stream = new BufferedOutputStream(new FileOutputStream(tempFileIn), bufferSize) + val printOut = new PrintStream(stream) + val dataOut = new DataOutputStream(stream) + + dataOut.writeInt(splitIndex) + + dataOut.writeInt(func.length) + dataOut.write(func, 0, func.length) + + // R worker process input serialization flag + dataOut.writeInt(if (parentSerialized) 1 else 0) + // R worker process output serialization flag + dataOut.writeInt(if (dataSerialized) 1 else 0) + + dataOut.writeInt(packageNames.length) + dataOut.write(packageNames, 0, packageNames.length) + + dataOut.writeInt(functionDependencies.length) + dataOut.write(functionDependencies, 0, functionDependencies.length) + + 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, 0, broadcastByteArr.length) + } - if (!iter.hasNext) { - dataOut.writeInt(0) - } else { - dataOut.writeInt(1) - } + dataOut.writeInt(numPartitions) - for (elem <- iter) { - if (parentSerialized) { - val elemArr = elem.asInstanceOf[Array[Byte]] - dataOut.writeInt(elemArr.length) - dataOut.write(elemArr, 0, elemArr.length) + if (!iter.hasNext) { + dataOut.writeInt(0) } else { - printOut.println(elem) + dataOut.writeInt(1) + } + + for (elem <- iter) { + if (parentSerialized) { + val elemArr = elem.asInstanceOf[Array[Byte]] + dataOut.writeInt(elemArr.length) + dataOut.write(elemArr, 0, elemArr.length) + } else { + printOut.println(elem) + } } - } - printOut.flush() - dataOut.flush() - stream.flush() - stream.close() + printOut.flush() + dataOut.flush() + stream.flush() + stream.close() + + // NOTE: We need to write out the temp file before writing out the + // file name to stdin. Otherwise the R process could read partial state + val streamStd = new BufferedOutputStream(proc.getOutputStream, bufferSize) + val printOutStd = new PrintStream(streamStd) + printOutStd.println(tempFileName) + printOutStd.println(rLibDir) + printOutStd.println(tempFileIn.getAbsolutePath()) + printOutStd.flush() + + streamStd.close() + } catch { + // TODO: We should propogate this error to the task thread + case e: Exception => + System.err.println("R Writer thread got an exception " + e) + e.printStackTrace() + } } }.start() From 60da1df9b3175c49322247bcf53999db1f922d91 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 1 Feb 2015 06:38:58 +0000 Subject: [PATCH 005/121] Initialize timing variables --- pkg/inst/worker/worker.R | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index 6d88b287b369b..e339391752cbb 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -68,6 +68,9 @@ numPartitions <- SparkR:::readInt(inputCon) isEmpty <- SparkR:::readInt(inputCon) metadataEnd <- proc.time()[3] +dataReadEnd <- metadataEnd +computeEnd <- dataReadEnd +writeEnd <- computeEnd if (isEmpty != 0) { From f34bb88bf4ba4371eeb15d0a92658e2d947ef32f Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 2 Feb 2015 10:08:43 -0800 Subject: [PATCH 006/121] Remove profiling information from this PR --- pkg/inst/worker/worker.R | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index e339391752cbb..c5457adcbc54d 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -1,7 +1,5 @@ # Worker class -begin <- proc.time()[3] - # NOTE: We use "stdin" to get the process stdin instead of the command line inputConStdin <- file("stdin", open = "rb") @@ -67,11 +65,6 @@ numPartitions <- SparkR:::readInt(inputCon) isEmpty <- SparkR:::readInt(inputCon) -metadataEnd <- proc.time()[3] -dataReadEnd <- metadataEnd -computeEnd <- dataReadEnd -writeEnd <- computeEnd - if (isEmpty != 0) { if (numPartitions == -1) { @@ -81,15 +74,12 @@ if (isEmpty != 0) { } else { data <- readLines(inputCon) } - dataReadEnd <- proc.time()[3] output <- do.call(execFunctionName, list(splitIndex, data)) - computeEnd <- proc.time()[3] if (isOutputSerialized) { SparkR:::writeRawSerialize(outputCon, output) } else { SparkR:::writeStrings(outputCon, output) } - writeEnd <- proc.time()[3] } else { if (isInputSerialized) { # Now read as many characters as described in funcLen @@ -98,7 +88,6 @@ if (isEmpty != 0) { data <- readLines(inputCon) } - dataReadEnd <- proc.time()[3] res <- new.env() # Step 1: hash the data to an environment @@ -116,8 +105,6 @@ if (isEmpty != 0) { } invisible(lapply(data, hashTupleToEnvir)) - computeEnd <- proc.time()[3] - # Step 2: write out all of the environment as key-value pairs. for (name in ls(res)) { SparkR:::writeInt(outputCon, 2L) @@ -126,7 +113,6 @@ if (isEmpty != 0) { length(res[[name]]$data) <- res[[name]]$counter SparkR:::writeRawSerialize(outputCon, res[[name]]$data) } - writeEnd <- proc.time()[3] } } @@ -142,13 +128,5 @@ unlink(inFileName) # Restore stdout sink() -end <- proc.time()[3] - -cat("stats: total ", (end-begin), "\n", file=stderr()) -cat("stats: metadata ", (metadataEnd-begin), "\n", file=stderr()) -cat("stats: input read ", (dataReadEnd-metadataEnd), "\n", file=stderr()) -cat("stats: compute ", (computeEnd-dataReadEnd), "\n", file=stderr()) -cat("stats: output write ", (writeEnd-computeEnd), "\n", file=stderr()) - # Finally print the name of the output file cat(outputFileName, "\n") From a95823edc2b384288a0fccaa033bfb103ce8a823 Mon Sep 17 00:00:00 2001 From: hqzizania Date: Wed, 4 Feb 2015 09:35:43 +0800 Subject: [PATCH 007/121] modified: pkg/R/RDD.R --- pkg/NAMESPACE | 1 + pkg/R/RDD.R | 24 ++++++++++++++++++++++++ pkg/man/collect-methods.Rd | 21 ++++++++++++++++++++- 3 files changed, 45 insertions(+), 1 deletion(-) diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 6977dd2d43330..44ac05043392e 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -6,6 +6,7 @@ exportMethods( "checkpoint", "cogroup", "collect", + "collectAsMap", "collectPartition", "combineByKey", "count", diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 66e616f252533..c3099375fca5b 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -358,6 +358,7 @@ setMethod("collect", convertJListToRList(collected, flatten) }) + #' @rdname collect-methods #' @export #' @description @@ -382,6 +383,29 @@ setMethod("collectPartition", convertJListToRList(jList, flatten = TRUE) }) +#' @rdname collect-methods +#' @export +#' @description +#' \code{collectAsMap} returns a named list as a map that contains all of the elements +#' in a key-value pair RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2),list(3, 4)), 2L) +#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#'} +setGeneric("collectAsMap", function(rdd) { standardGeneric("collectAsMap") }) + +#' @rdname collect-methods +#' @aliases collectAsMap,RDD-method +setMethod("collectAsMap", + signature(rdd = "RDD"), + function(rdd) { + pairList <- collect(rdd) + map <- new.env() + lapply(pairList, function(x) { assign(as.character(x[[1]]), x[[2]], envir = map) }) + as.list(map) + }) #' Look up elements of a key in an RDD #' diff --git a/pkg/man/collect-methods.Rd b/pkg/man/collect-methods.Rd index 57d9f0481c839..b11b5e9791060 100644 --- a/pkg/man/collect-methods.Rd +++ b/pkg/man/collect-methods.Rd @@ -1,8 +1,11 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand +% Generated by roxygen2 (4.1.0): do not edit by hand +% Please edit documentation in R/RDD.R \docType{methods} \name{collect} \alias{collect} \alias{collect,RDD-method} +\alias{collectAsMap} +\alias{collectAsMap,RDD-method} \alias{collectPartition} \alias{collectPartition,RDD,integer-method} \alias{collectPartition,integer,RDD-method} @@ -15,6 +18,10 @@ collect(rdd, ...) collectPartition(rdd, partitionId) \S4method{collectPartition}{RDD,integer}(rdd, partitionId) + +collectAsMap(rdd) + +\S4method{collectAsMap}{RDD}(rdd) } \arguments{ \item{rdd}{The RDD to collect} @@ -24,15 +31,22 @@ collectPartition(rdd, partitionId) \item{flatten}{FALSE if the list should not flattened} \item{partitionId}{the partition to collect (starts from 0)} + +\item{rdd}{The RDD to collect as a map} } \value{ a list containing elements in the RDD + +a named list containing elements in the RDD } \description{ \code{collect} returns a list that contains all of the elements in this RDD. \code{collectPartition} returns a list that contains all of the elements in the specified partition of the RDD. + +\code{collectAsMap} returns a named list as a map that contains all of the elements +in this key-value pair RDD. } \examples{ \dontrun{ @@ -41,5 +55,10 @@ rdd <- parallelize(sc, 1:10, 2L) collect(rdd) # list from 1 to 10 collectPartition(rdd, 0L) # list from 1 to 5 } +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list(list(1, 2),list(3, 4)), 2L) +collectAsMap(rdd) # list(`1` = 2, `3` = 4) +} } From 491409142ed7dcb7ad475b513b786f2e92305911 Mon Sep 17 00:00:00 2001 From: hqzizania Date: Wed, 4 Feb 2015 13:46:15 +0800 Subject: [PATCH 008/121] modified: pkg/inst/tests/test_rdd.R --- pkg/inst/tests/test_rdd.R | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 7c2599f51e7e5..9b25948feeb5b 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -373,3 +373,22 @@ test_that("fullOuterJoin() on pairwise RDDs", { 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("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")) +}) + From 5d69f0ae76b83e957d5ce0221110ce3a1640ede9 Mon Sep 17 00:00:00 2001 From: hqzizania Date: Wed, 4 Feb 2015 14:01:00 +0800 Subject: [PATCH 009/121] modified: pkg/R/RDD.R --- pkg/R/RDD.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index c3099375fca5b..789f36d172c91 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -391,7 +391,7 @@ setMethod("collectPartition", #' @examples #'\dontrun{ #' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2),list(3, 4)), 2L) +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) #' collectAsMap(rdd) # list(`1` = 2, `3` = 4) #'} setGeneric("collectAsMap", function(rdd) { standardGeneric("collectAsMap") }) From 9767e8ec07bb45cab90818c050fb9f9ce2b7f565 Mon Sep 17 00:00:00 2001 From: hqzizania Date: Wed, 4 Feb 2015 14:21:50 +0800 Subject: [PATCH 010/121] modified: pkg/man/collect-methods.Rd --- pkg/man/collect-methods.Rd | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/pkg/man/collect-methods.Rd b/pkg/man/collect-methods.Rd index b11b5e9791060..e9b5dcb0e3734 100644 --- a/pkg/man/collect-methods.Rd +++ b/pkg/man/collect-methods.Rd @@ -31,13 +31,9 @@ collectAsMap(rdd) \item{flatten}{FALSE if the list should not flattened} \item{partitionId}{the partition to collect (starts from 0)} - -\item{rdd}{The RDD to collect as a map} } \value{ a list containing elements in the RDD - -a named list containing elements in the RDD } \description{ \code{collect} returns a list that contains all of the elements in this RDD. @@ -46,7 +42,7 @@ a named list containing elements in the RDD in the specified partition of the RDD. \code{collectAsMap} returns a named list as a map that contains all of the elements -in this key-value pair RDD. +in a key-value pair RDD. } \examples{ \dontrun{ @@ -57,7 +53,7 @@ collectPartition(rdd, 0L) # list from 1 to 5 } \dontrun{ sc <- sparkR.init() -rdd <- parallelize(sc, list(list(1, 2),list(3, 4)), 2L) +rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) collectAsMap(rdd) # list(`1` = 2, `3` = 4) } } From d9da4519fa9efa5db769a43cb75d296a94d44a74 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Wed, 4 Feb 2015 21:46:49 +0800 Subject: [PATCH 011/121] [SPARKR-150] phase 1: implement sortBy() and sortByKey(). --- pkg/NAMESPACE | 2 + pkg/R/RDD.R | 106 ++++++++++++++++++++++++++++++++++++++ pkg/R/utils.R | 4 +- pkg/inst/tests/test_rdd.R | 14 +++++ pkg/man/sortBy.Rd | 36 +++++++++++++ pkg/man/sortByKey.Rd | 34 ++++++++++++ 6 files changed, 194 insertions(+), 2 deletions(-) create mode 100644 pkg/man/sortBy.Rd create mode 100644 pkg/man/sortByKey.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 6977dd2d43330..ac90c36734e6d 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -44,6 +44,8 @@ exportMethods( "sampleRDD", "saveAsTextFile", "saveAsObjectFile", + "sortBy", + "sortByKey", "take", "takeSample", "unionRDD", diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 66e616f252533..b788f13119cb6 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1240,6 +1240,40 @@ setMethod("flatMapValues", flatMap(X, flatMapFunc) }) +#' Sort an RDD by the given key function. +#' +#' @param rdd An RDD to be sorted. +#' @param func A function used to compute the sort key for each element. +#' @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 elements are sorted. +#' @rdname sortBy +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(3, 2, 1)) +#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#'} +setGeneric("sortBy", function(rdd, func, ascending, numPartitions) { standardGeneric("sortBy") }) + +setClassUnion("missingOrLogical", c("missing", "logical")) +#' @rdname sortBy +#' @aliases sortBy,RDD,RDD-method +setMethod("sortBy", + signature(rdd = "RDD", func = "function", + ascending = "missingOrLogical", numPartitions = "missingOrInteger"), + function(rdd, func, ascending, numPartitions) { + if (missing(ascending)) { + ascending = TRUE + } + if (missing(numPartitions)) { + numPartitions = SparkR::numPartitions(rdd) + } + + values(sortByKey(keyBy(rdd, func), ascending, numPartitions)) + }) + ############ Shuffle Functions ############ #' Partition an RDD by key @@ -1796,6 +1830,78 @@ setMethod("cogroup", group.func) }) +#' Sort an (k, v) pair RDD by k. +#' +#' @param rdd An (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. +#' @rdname sortByKey +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(3, 3), list(2, 2), list(1, 1))) +#' collect(sortByKey(rdd)) # list (list(1, 1), list(2, 2), list(3, 3)) +#'} +setGeneric("sortByKey", function(rdd, ascending, numPartitions) { standardGeneric("sortByKey") }) + +#' @rdname sortByKey +#' @aliases sortByKey,RDD,RDD-method +setMethod("sortByKey", + signature(rdd = "RDD", ascending = "missingOrLogical", numPartitions = "missingOrInteger"), + function(rdd, ascending, numPartitions) { + if (missing(ascending)) { + ascending = TRUE + } + if (missing(numPartitions)) { + numPartitions = SparkR::numPartitions(rdd) + } + + rangeBounds <- list() + + if (numPartitions > 1) { + rddSize <- count(rdd) + # constant from Spark's RangePartitioner + maxSampleSize <- numPartitions * 20 + fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) + + samples <- collect(keys(sampleRDD(rdd, FALSE, fraction, 1L))) + + # Note: the built-in R sort() function only 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 + + 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(rdd, numPartitions, rangePartitionFunc) + lapplyPartition(newRDD, partitionFunc) + }) + # TODO: Consider caching the name in the RDD's environment #' Return an RDD's name. #' diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 7c2a153b8b55d..8bb77463602b4 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -197,9 +197,9 @@ initAccumulator <- function() { # Utility function to sort a list of key value pairs # Used in unit tests -sortKeyValueList <- function(kv_list) { +sortKeyValueList <- function(kv_list, decreasing = FALSE) { keys <- sapply(kv_list, function(x) x[[1]]) - kv_list[order(keys)] + kv_list[order(keys, decreasing = decreasing)] } # Utility function to generate compact R lists from grouped rdd diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 7c2599f51e7e5..5f675fcd689ba 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -254,6 +254,12 @@ test_that("keyBy on RDDs", { expect_equal(actual, lapply(nums, function(x) { list(func(x), x) })) }) +test_that("sortBy() on RDDs", { + sortedRdd <- sortBy(rdd, function(x) { x }, ascending = FALSE) + actual <- collect(sortedRdd) + expect_equal(actual, as.list(sort(nums, decreasing = TRUE))) +}) + test_that("keys() on RDDs", { keys <- keys(intRdd) actual <- collect(keys) @@ -373,3 +379,11 @@ test_that("fullOuterJoin() on pairwise RDDs", { 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)) +}) \ No newline at end of file diff --git a/pkg/man/sortBy.Rd b/pkg/man/sortBy.Rd new file mode 100644 index 0000000000000..d3a231c745240 --- /dev/null +++ b/pkg/man/sortBy.Rd @@ -0,0 +1,36 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{sortBy} +\alias{sortBy} +\alias{sortBy,RDD,RDD-method} +\alias{sortBy,RDD,function,missingOrLogical,missingOrInteger-method} +\title{Sort an RDD by the given key function.} +\usage{ +sortBy(rdd, func, ascending, numPartitions) + +\S4method{sortBy}{RDD,`function`,missingOrLogical,missingOrInteger}(rdd, func, + ascending, numPartitions) +} +\arguments{ +\item{rdd}{An RDD to be sorted.} + +\item{func}{A function used to compute the sort key for each element.} + +\item{ascending}{A flag to indicate whether the sorting is ascending or descending.} + +\item{numPartitions}{Number of partitions to create.} +} +\value{ +An RDD where all elements are sorted. +} +\description{ +Sort an RDD by the given key function. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list(3, 2, 1)) +collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +} +} + diff --git a/pkg/man/sortByKey.Rd b/pkg/man/sortByKey.Rd new file mode 100644 index 0000000000000..b58dd5bf22ce6 --- /dev/null +++ b/pkg/man/sortByKey.Rd @@ -0,0 +1,34 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{sortByKey} +\alias{sortByKey} +\alias{sortByKey,RDD,RDD-method} +\alias{sortByKey,RDD,missingOrLogical,missingOrInteger-method} +\title{Sort an (k, v) pair RDD by k.} +\usage{ +sortByKey(rdd, ascending, numPartitions) + +\S4method{sortByKey}{RDD,missingOrLogical,missingOrInteger}(rdd, ascending, + numPartitions) +} +\arguments{ +\item{rdd}{An (k, v) pair RDD to be sorted.} + +\item{ascending}{A flag to indicate whether the sorting is ascending or descending.} + +\item{numPartitions}{Number of partitions to create.} +} +\value{ +An RDD where all (k, v) pair elements are sorted. +} +\description{ +Sort an (k, v) pair RDD by k. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list(list(3, 3), list(2, 2), list(1, 1))) +collect(sortByKey(rdd)) # list (list(1, 1), list(2, 2), list(3, 3)) +} +} + From 0c53d6c4952ba6c87b2fc0837a17339062351df6 Mon Sep 17 00:00:00 2001 From: dputler Date: Wed, 4 Feb 2015 09:00:49 -0800 Subject: [PATCH 012/121] Data frames now coerced to lists, and messages issued for a data frame or matrix on how they are parallelized --- pkg/R/context.R | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/pkg/R/context.R b/pkg/R/context.R index d77c8d04a48a8..3c4b5d278a06a 100644 --- a/pkg/R/context.R +++ b/pkg/R/context.R @@ -85,11 +85,14 @@ 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)) { - if (!is.vector(coll)) { + 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) } @@ -109,7 +112,6 @@ parallelize <- function(sc, coll, numSlices = 1) { RDD(jrdd, TRUE) } - #' Include this specified package on all workers #' #' This function can be used to include a package on all workers before the From 91b2fd6deaf1d49cdc6aeec8d1c5fb4d686c0319 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 5 Feb 2015 11:09:29 +0800 Subject: [PATCH 013/121] Add more test cases. --- pkg/inst/tests/test_rdd.R | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 5f675fcd689ba..4efa17a77b242 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -258,6 +258,11 @@ test_that("sortBy() on RDDs", { sortedRdd <- sortBy(rdd, function(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 }) + actual <- collect(sortedRdd2) + expect_equal(actual, as.list(nums)) }) test_that("keys() on RDDs", { @@ -386,4 +391,10 @@ test_that("sortByKey() on pairwise RDDs", { 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) }) \ No newline at end of file From 804355977ba387a036b9c5355156cf547e20a738 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 5 Feb 2015 12:12:47 +0800 Subject: [PATCH 014/121] Add a TODO to use binary search in the range partitioner. --- pkg/R/RDD.R | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index b788f13119cb6..74bd6bfad364c 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1883,6 +1883,7 @@ setMethod("sortByKey", 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 } From 345f1b8230943583163e88f3da758f187dfff47a Mon Sep 17 00:00:00 2001 From: dputler Date: Wed, 4 Feb 2015 22:17:23 -0800 Subject: [PATCH 015/121] [SPARKR-195] Implemented project style guidelines for if-else statements --- pkg/R/context.R | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/pkg/R/context.R b/pkg/R/context.R index 3c4b5d278a06a..db765fafe463d 100644 --- a/pkg/R/context.R +++ b/pkg/R/context.R @@ -85,14 +85,17 @@ 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.")) + 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) } From 193f5fe526eaf4af620fd625d754130a13bd06f6 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Thu, 5 Feb 2015 02:49:00 -0500 Subject: [PATCH 016/121] loop 1-12 test pass. --- pkg/R/RDD.R | 18 ++++++++++++++++- pkg/R/utils.R | 42 ++++++++++++++++++++++++++++++++++++++++ pkg/inst/worker/worker.R | 18 +++++++++++++---- 3 files changed, 73 insertions(+), 5 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 66e616f252533..6b101714f9d71 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -18,6 +18,7 @@ setClass("RDD", setClass("PipelinedRDD", slots = list(prev = "RDD", func = "function", + funcAccum = "environment", prev_jrdd = "jobj"), contains = "RDD") @@ -52,6 +53,8 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) # NOTE: We use prev_serialized to track if prev_jrdd is serialized # prev_serialized is used during the delayed computation of JRDD in getJRDD .Object@prev <- prev + + .Object@funcAccum <- initAccumulator() isPipelinable <- function(rdd) { e <- rdd@env @@ -70,10 +73,12 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) func(split, prev@func(split, iterator)) } .Object@func <- pipelinedFunc + .Object@funcAccum <- cloneAccumulator(prev@funcAccum) .Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline # Get if the prev_jrdd was serialized from the parent RDD .Object@env$prev_serialized <- prev@env$prev_serialized } + addItemToAccumulator(.Object@funcAccum, func) .Object }) @@ -121,6 +126,7 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), function(name) { get(name, .broadcastNames) }) depsBin <- getDependencies(computeFunc) + depsBin <- serialize(rdd@funcAccum, NULL, ascii = TRUE) prev_jrdd <- rdd@prev_jrdd @@ -526,6 +532,11 @@ setMethod("countByKey", setMethod("lapply", signature(X = "RDD", FUN = "function"), function(X, FUN) { + env.fun <- new.env(parent=.GlobalEnv) + clean.closure(FUN, env.fun) + # need to add ref variables to en.fun environment + environment(FUN) <- env.fun + func <- function(split, iterator) { lapply(iterator, FUN) } @@ -646,7 +657,12 @@ setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { - closureCapturingFunc <- function(split, part) { + env.fun <- new.env(parent=.GlobalEnv) + clean.closure(FUN, env.fun) + # need to add ref variables to en.fun environment + environment(FUN) <- env.fun + + closureCapturingFunc <- function(split, part) { FUN(split, part) } PipelinedRDD(X, closureCapturingFunc) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 7c2a153b8b55d..bf4fa48ee5de4 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -195,6 +195,14 @@ initAccumulator <- function() { acc } +cloneAccumulator <- function(acc) { + newAcc <- initAccumulator() + newAcc$size <- acc$size + newAcc$data <- acc$data + newAcc$counter <- acc$counter + newAcc +} + # Utility function to sort a list of key value pairs # Used in unit tests sortKeyValueList <- function(kv_list) { @@ -259,3 +267,37 @@ joinTaggedList <- function(tagged_list, cnull) { lists <- genCompactLists(tagged_list, cnull) mergeCompactLists(lists[[1]], lists[[2]]) } + +closure.process <- function( + node, + func, + env +) { + nlen <- length(node) + if(nlen > 1) { + for(i in 1:nlen) { + closure.process(node[[i]], func, env) + } + } else if(nlen == 1) { + if(mode(node) == 'name') { + cnode <- as.character(node) + if(!cnode %in% names(as.list(args(func)))) { + func.env <- environment(func) + if(exists(cnode, envir=func.env, inherits=F)) { + assign(cnode, get(cnode, envir=func.env), envir=env) + } + } + } + } +} + +clean.closure <- function( + func, + env +) { + if(mode(func) != 'function' || mode(env) != 'environment') + stop('parameter type mismatch...') + func.body <- body(func) + + closure.process(func.body, func, env) +} diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index c5457adcbc54d..136db6960d88b 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -43,10 +43,20 @@ for (pkg in packageNames) { # read function dependencies depsLen <- SparkR:::readInt(inputCon) -if (depsLen > 0) { - execFunctionDeps <- SparkR:::readRawLen(inputCon, depsLen) - # load the dependencies into current environment - load(rawConnection(execFunctionDeps, open='rb')) +funcAccum <- unserialize(SparkR:::readRawLen(inputCon, depsLen)) +# testfilename <- "/home//ubuntu/Desktop/testfile" +# cat(typeof(funcAccum), '\n', file = testfilename) +# cat(funcAccum$size, '\n', file = testfilename, append = T) +# cat(typeof(funcAccum$data[[1]]), '\n', file = testfilename, append = T) +# cat(as.character(body(funcAccum$data[[1]])), file = testfilename, append = T) +if (funcAccum$size > 0) { + computeFunc <- function(split, part) { + res <- part + for (i in 1:funcAccum$counter) { + res <- funcAccum$data[[i]](split, res) + } + res + } } # Read and set broadcast variables From 7ca651263fcef6c1e8d2352fcc1639a7cf056512 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 5 Feb 2015 23:10:57 -0800 Subject: [PATCH 017/121] First cut of SparkRRunner --- pkg/R/sparkR.R | 15 ++-- .../cs/amplab/sparkr/SparkRRunner.scala | 81 +++++++++++++++++++ 2 files changed, 91 insertions(+), 5 deletions(-) create mode 100644 pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index ea9129d3bd40f..2767b8a233772 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -97,11 +97,16 @@ sparkR.init <- function( if (yarn_conf_dir != "") { cp <- paste(cp, yarn_conf_dir, sep = ":") } - launchBackend(classPath = cp, - mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", - args = as.character(sparkRBackendPort), - javaOpts = paste("-Xmx", sparkMem, sep = "")) - Sys.sleep(2) # Wait for backend to come up + sparkRExistingPort <- Sys.getenv("SPARKR_BACKEND_PORT", "") + if (sparkRExistingPort != "") { + sparkRBackendPort <- sparkRExistingPort + } else { + launchBackend(classPath = cp, + mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", + args = as.character(sparkRBackendPort), + javaOpts = paste("-Xmx", sparkMem, sep = "")) + Sys.sleep(2) # Wait for backend to come up + } .sparkREnv$sparkRBackendPort <- sparkRBackendPort connectBackend("localhost", sparkRBackendPort) # Connect to it diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala new file mode 100644 index 0000000000000..84dd4bd775cc2 --- /dev/null +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala @@ -0,0 +1,81 @@ +package edu.berkeley.cs.amplab.sparkr + +import java.io._ +import java.net.URI + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ + +/** + * 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 SparkRRunner { + def main(args: Array[String]) { + val rFile = args(0) + + val otherArgs = args.slice(1, args.length) + // TODO: Can we get this from SparkConf ? + val sparkRBackendPort = sys.env.getOrElse("SPARKR_BACKEND_PORT", "12345").toInt + val rCommand = "Rscript" + + // val formattedPythonFile = formatPath(pythonFile) + // TODO: Normalize path ? + val rFileNormalized = 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 SparkRBackend() + val sparkRBackendThread = new Thread() { + override def run() { + sparkRBackend.init(sparkRBackendPort) + sparkRBackend.run() + } + + def stopBackend() { + sparkRBackend.close() + } + } + + sparkRBackendThread.start() + + // Launch R + val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) + val env = builder.environment() + env.put("SPARKR_BACKEND_PORT", "" + sparkRBackendPort) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + + val returnCode = process.waitFor() + sparkRBackendThread.stopBackend() + System.exit(returnCode) + } + + private class RedirectThread( + in: InputStream, + out: OutputStream, + name: String, + propagateEof: Boolean = false) + extends Thread(name) { + + setDaemon(true) + override def run() { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + try { + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } finally { + if (propagateEof) { + out.close() + } + } + } + } +} From b082a35e5d9aac2a0ffc033b871a0b7588be291b Mon Sep 17 00:00:00 2001 From: lythesia Date: Fri, 6 Feb 2015 16:36:34 +0800 Subject: [PATCH 018/121] add reduceByKeyLocally --- pkg/NAMESPACE | 1 + pkg/R/RDD.R | 134 ++++++++++++++++++++-------------- pkg/R/utils.R | 29 ++++++++ pkg/inst/tests/test_rdd.R | 13 ++++ pkg/man/reduceByKeyLocally.Rd | 41 +++++++++++ 5 files changed, 164 insertions(+), 54 deletions(-) create mode 100644 pkg/man/reduceByKeyLocally.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 44ac05043392e..e023cd87fcedf 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -41,6 +41,7 @@ exportMethods( "persist", "reduce", "reduceByKey", + "reduceByKeyLocally", "rightOuterJoin", "sampleRDD", "saveAsTextFile", diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 789f36d172c91..fc62d92835736 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1382,26 +1382,18 @@ setMethod("groupByKey", groupVals <- function(part) { vals <- new.env() keys <- new.env() + pred <- function(item) exists(item$hash, keys) # Each item in the partition is list of (K, V) lapply(part, function(item) { - hashVal <- as.character(hashCode(item[[1]])) - if (exists(hashVal, vals)) { - acc <- vals[[hashVal]] - acc[[length(acc) + 1]] <- item[[2]] - vals[[hashVal]] <- acc - } else { - vals[[hashVal]] <- list(item[[2]]) - keys[[hashVal]] <- item[[1]] - } + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, + function(vs, v) c(vs, list(v)), + function(x) list(x)) }) # Every key in the environment contains a list # Convert that to list(K, Seq[V]) - grouped <- lapply(ls(vals), - function(name) { - list(keys[[name]], vals[[name]]) - }) - grouped + convertEnvsToList(keys, vals) } lapplyPartition(shuffled, groupVals) }) @@ -1442,28 +1434,79 @@ setMethod("reduceByKey", reduceVals <- function(part) { vals <- new.env() keys <- new.env() + pred <- function(item) exists(item$hash, keys) lapply(part, function(item) { - hashVal <- as.character(hashCode(item[[1]])) - if (exists(hashVal, vals)) { - vals[[hashVal]] <- do.call( - combineFunc, list(vals[[hashVal]], item[[2]])) - } else { - vals[[hashVal]] <- item[[2]] - keys[[hashVal]] <- item[[1]] - } + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, combineFunc, function(x) x) }) - combined <- lapply(ls(vals), - function(name) { - list(keys[[name]], vals[[name]]) - }) - combined + convertEnvsToList(keys, vals) } locallyReduced <- lapplyPartition(rdd, 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 master as R list. +#' +#' @param rdd 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 An list where each element is list(K, V') where V' is the merged +#' value +#' @rdname reduceByKeyLocally +#' @seealso reduceByKey +#' @export +#' @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[[1]] # Should be a list(1, 6) +#'} +setGeneric("reduceByKeyLocally", + function(rdd, combineFunc) { + standardGeneric("reduceByKeyLocally") + }) + +#' @rdname reduceByKeyLocally +#' @aliases reduceByKeyLocally,RDD,integer-method +setMethod("reduceByKeyLocally", + signature(rdd = "RDD", combineFunc = "ANY"), + function(rdd, 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, function(x) x) + }) + 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, function(x) x) + }) + accum + } + reduced <- mapPartitions(rdd, 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 @@ -1513,46 +1556,29 @@ setMethod("combineByKey", combineLocally <- function(part) { combiners <- new.env() keys <- new.env() + pred <- function(item) exists(item$hash, keys) lapply(part, function(item) { - k <- as.character(item[[1]]) - if (!exists(k, keys)) { - combiners[[k]] <- do.call(createCombiner, - list(item[[2]])) - keys[[k]] <- item[[1]] - } else { - combiners[[k]] <- do.call(mergeValue, - list(combiners[[k]], - item[[2]])) - } - }) - lapply(ls(keys), function(k) { - list(keys[[k]], combiners[[k]]) + item$hash <- as.character(item[[1]]) + updateOrCreatePair(item, keys, combiners, pred, mergeValue, createCombiner) }) + convertEnvsToList(keys, combiners) } locallyCombined <- lapplyPartition(rdd, 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) { - k <- as.character(item[[1]]) - if (!exists(k, combiners)) { - combiners[[k]] <- item[[2]] - keys[[k]] <- item[[1]] - } else { - combiners[[k]] <- do.call(mergeCombiners, - list(combiners[[k]], - item[[2]])) - } - }) - lapply(ls(keys), function(k) { - list(keys[[k]], combiners[[k]]) + item$hash <- as.character(item[[1]]) + updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, + function(x) x) }) + convertEnvsToList(keys, combiners) } - combined <-lapplyPartition(shuffled, mergeAfterShuffle) - combined + lapplyPartition(shuffled, mergeAfterShuffle) }) ############ Binary Functions ############# diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 7c2a153b8b55d..cb5797d981ba7 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -259,3 +259,32 @@ 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 +# item key-val pair +# keys/vals env of key/value with hashes +# pred predicate function +# update_fn update or merge function for existing pair, similar with `mergeVal` @combineByKey +# create_fn create function for new pair, similar with `createCombiner` @combinebykey +updateOrCreatePair <- function(item, keys, vals, pred, update_fn, create_fn) { + # assum hashval bind to `$hash`, key/val with index 1/2 + hashVal <- item$hash + key <- item[[1]] + val <- item[[2]] + if (pred(item)) { + assign(hashVal, do.call(update_fn, list(get(hashVal, envir=vals), val)), envir=vals) + } else { + assign(hashVal, do.call(create_fn, 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]]) + }) +} diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 9b25948feeb5b..5d6b128cd51b1 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -229,6 +229,19 @@ test_that("flatMapValues() on pairwise RDDs", { 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) diff --git a/pkg/man/reduceByKeyLocally.Rd b/pkg/man/reduceByKeyLocally.Rd new file mode 100644 index 0000000000000..2bf671f7e300c --- /dev/null +++ b/pkg/man/reduceByKeyLocally.Rd @@ -0,0 +1,41 @@ +% Generated by roxygen2 (4.1.0): do not edit by hand +% Please edit documentation in R/RDD.R +\docType{methods} +\name{reduceByKeyLocally} +\alias{reduceByKeyLocally} +\alias{reduceByKeyLocally,RDD,integer-method} +\alias{reduceByKeyLocally,RDD-method} +\title{Merge values by key locally} +\usage{ +reduceByKeyLocally(rdd, combineFunc) + +\S4method{reduceByKeyLocally}{RDD}(rdd, combineFunc) +} +\arguments{ +\item{rdd}{The RDD to reduce by key. Should be an RDD where each element is +list(K, V) or c(K, V).} + +\item{combineFunc}{The associative reduce function to use.} +} +\value{ +An list where each element is list(K, V') where V' is the merged + value +} +\description{ +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 master as R list. +} +\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[[1]] # Should be a list(1, 6) +} +} +\seealso{ +reduceByKey +} + From 6ad4fc36bde14e9cdbab4224cec466b9e2aaf73f Mon Sep 17 00:00:00 2001 From: hlin09 Date: Fri, 6 Feb 2015 14:13:14 -0500 Subject: [PATCH 019/121] add docs --- pkg/R/RDD.R | 22 ++++++++++++++++++---- pkg/inst/tests/test_rdd.R | 5 ++++- pkg/inst/worker/worker.R | 38 +++++++++++++++++++++++--------------- 3 files changed, 45 insertions(+), 20 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 6b101714f9d71..750e0b8eec399 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -18,7 +18,8 @@ setClass("RDD", setClass("PipelinedRDD", slots = list(prev = "RDD", func = "function", - funcAccum = "environment", + # Accumulator to store function lineage + funcAccum = "environment", prev_jrdd = "jobj"), contains = "RDD") @@ -54,6 +55,7 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) # prev_serialized is used during the delayed computation of JRDD in getJRDD .Object@prev <- prev + # We use funcAccum to store the lineage of function closures in a PipelinedRDD. .Object@funcAccum <- initAccumulator() isPipelinable <- function(rdd) { @@ -125,7 +127,9 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - depsBin <- getDependencies(computeFunc) + # Sorry, I use "depsBin" to serialize the funcAccum for now. + # Instead of serializing the entire environment, we can serialize + # a sequence of functions (with its closure). depsBin <- serialize(rdd@funcAccum, NULL, ascii = TRUE) prev_jrdd <- rdd@prev_jrdd @@ -532,9 +536,12 @@ setMethod("countByKey", setMethod("lapply", signature(X = "RDD", FUN = "function"), function(X, FUN) { + # Creats a new the closure (environment) for the FUN to capture + # free variables. env.fun <- new.env(parent=.GlobalEnv) + # need to add free variables to env.fun environment clean.closure(FUN, env.fun) - # need to add ref variables to en.fun environment + # FUN now have new environment env.fun, with all values it needs. environment(FUN) <- env.fun func <- function(split, iterator) { @@ -1298,8 +1305,15 @@ setMethod("partitionBy", #} depsBinArr <- getDependencies(partitionFunc) + env.fun <- new.env(parent=.GlobalEnv) + clean.closure(partitionFunc, env.fun) + # need to add ref variables to en.fun environment + environment(partitionFunc) <- env.fun + funcAccum <- initAccumulator() + addItemToAccumulator(funcAccum, partitionFunc) + depsBinArr <- serialize(funcAccum, NULL, ascii = TRUE) - serializedHashFuncBytes <- serialize(as.character(substitute(partitionFunc)), + serializedHashFuncBytes <- serialize("computeFunc", #as.character(substitute(partitionFunc)), connection = NULL, ascii = TRUE) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 7c2599f51e7e5..bcf4cd4e1bc51 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -86,7 +86,10 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { part <- as.list(unlist(part) * split + i) }) rdd2 <- lapply(rdd2, function(x) x + x) - collect(rdd2) + actual <- collect(rdd2) + expected <- + expect_equal(actual, list(24, 24, 24, 24, 24, + 168, 170, 172, 174, 176)) }) test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index 136db6960d88b..40fc9a8881645 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -41,22 +41,11 @@ for (pkg in packageNames) { suppressPackageStartupMessages(require(as.character(pkg), character.only=TRUE)) } -# read function dependencies +# Read the function lineage. depsLen <- SparkR:::readInt(inputCon) -funcAccum <- unserialize(SparkR:::readRawLen(inputCon, depsLen)) -# testfilename <- "/home//ubuntu/Desktop/testfile" -# cat(typeof(funcAccum), '\n', file = testfilename) -# cat(funcAccum$size, '\n', file = testfilename, append = T) -# cat(typeof(funcAccum$data[[1]]), '\n', file = testfilename, append = T) -# cat(as.character(body(funcAccum$data[[1]])), file = testfilename, append = T) -if (funcAccum$size > 0) { - computeFunc <- function(split, part) { - res <- part - for (i in 1:funcAccum$counter) { - res <- funcAccum$data[[i]](split, res) - } - res - } +if (depsLen > 0) { + # function lineage stored in an Accumulator + funcAccum <- unserialize(SparkR:::readRawLen(inputCon, depsLen)) } # Read and set broadcast variables @@ -73,6 +62,25 @@ if (numBroadcastVars > 0) { # as number of partitions to create. numPartitions <- SparkR:::readInt(inputCon) +# Build up the execFunction from the Accumulator +if (exists("funcAccum")) { + if (funcAccum$size > 0) { + if (numPartitions == -1) { + # Regular RDDs: build nested functions. + computeFunc <- function(split, part) { + res <- part + for (i in 1:funcAccum$counter) { + res <- funcAccum$data[[i]](split, res) + } + res + } + } else { + # Pairwise RDDs. + computeFunc <- funcAccum$data[[1]] + } + } +} + isEmpty <- SparkR:::readInt(inputCon) if (isEmpty != 0) { From a46516506cca8d625d983edd264330476c8774ef Mon Sep 17 00:00:00 2001 From: hlin09 Date: Fri, 6 Feb 2015 14:25:19 -0500 Subject: [PATCH 020/121] More docs --- pkg/R/utils.R | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index bf4fa48ee5de4..a16a0204720eb 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -268,6 +268,8 @@ joinTaggedList <- function(tagged_list, cnull) { mergeCompactLists(lists[[1]], lists[[2]]) } +# Recursively examine variables in functions to decide if their values should +# be included in the new function environment. closure.process <- function( node, func, @@ -281,7 +283,7 @@ closure.process <- function( } else if(nlen == 1) { if(mode(node) == 'name') { cnode <- as.character(node) - if(!cnode %in% names(as.list(args(func)))) { + if(!cnode %in% names(as.list(args(func)))) { # Not a function argument func.env <- environment(func) if(exists(cnode, envir=func.env, inherits=F)) { assign(cnode, get(cnode, envir=func.env), envir=env) @@ -291,6 +293,7 @@ closure.process <- function( } } +# Get function dependencies. clean.closure <- function( func, env From 24a7f13f272484c3b781c84adfc19c5bc17d2859 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Fri, 6 Feb 2015 14:27:54 -0500 Subject: [PATCH 021/121] More docs --- pkg/inst/tests/test_rdd.R | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index bcf4cd4e1bc51..0140c24cfec10 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -87,9 +87,9 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { }) rdd2 <- lapply(rdd2, function(x) x + x) actual <- collect(rdd2) - expected <- - expect_equal(actual, list(24, 24, 24, 24, 24, - 168, 170, 172, 174, 176)) + 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()", { From fd836db89e785843be4fcd7868215770e00ffcd4 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Fri, 6 Feb 2015 14:47:41 -0500 Subject: [PATCH 022/121] fix tests. --- pkg/inst/tests/test_rdd.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 0140c24cfec10..dbebe6fa979c8 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -89,7 +89,7 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { actual <- collect(rdd2) expected <- list(24, 24, 24, 24, 24, 168, 170, 172, 174, 176) - expect_equal(actual, expected)) + expect_equal(actual, expected) }) test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { From 25639cf25182a2f93e0ff7c3a76ca2844da0d29b Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 6 Feb 2015 11:55:36 -0800 Subject: [PATCH 023/121] Replace tabs with spaces --- pkg/R/context.R | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/pkg/R/context.R b/pkg/R/context.R index db765fafe463d..095ad5528ee7f 100644 --- a/pkg/R/context.R +++ b/pkg/R/context.R @@ -85,17 +85,17 @@ 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.")) - } - } + 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) } From 343b6ab95459a0b36c4cef1fe5d83734471316d0 Mon Sep 17 00:00:00 2001 From: Oscar Olmedo Date: Fri, 6 Feb 2015 18:57:37 -0800 Subject: [PATCH 024/121] Export sparkR.stop Closes #156 from oscaroboto/master --- README.md | 5 +++++ pkg/NAMESPACE | 1 + pkg/R/sparkR.R | 47 ++++++++++++++++++++++++++------------------ pkg/R/sparkRClient.R | 6 ++++-- 4 files changed, 38 insertions(+), 21 deletions(-) diff --git a/README.md b/README.md index e928359819e97..795b0a1477305 100644 --- a/README.md +++ b/README.md @@ -84,7 +84,12 @@ pass the variable `spark.executor.memory` to the SparkContext constructor. sc <- sparkR.init(master="spark://:7077", sparkEnvir=list(spark.executor.memory="1g")) +Finally, to stop the cluster run + sparkR.stop() + +sparkR.stop() can be invoked to terminate a SparkContext created previously via sparkR.init(). Then you can call sparR.init() again to create a new SparkContext that may have different configurations. + ## Examples, Unit tests SparkR comes with several sample programs in the `examples` directory. diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 44ac05043392e..c542e9034da30 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -65,6 +65,7 @@ export( "setCheckpointDir" ) export("sparkR.init") +export("sparkR.stop") export("print.jobj") useDynLib(SparkR, stringHashCode) importFrom(methods, setGeneric, setMethod, setOldClass) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index ea9129d3bd40f..2218170fef104 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -23,30 +23,39 @@ connExists <- function(env) { # Stop the Spark context. # Also terminates the backend this R session is connected to -sparkR.stop <- function(env) { - cat("Stopping SparkR\n") +sparkR.stop <- function(env = .sparkREnv) { if (!connExists(env)) { # When the workspace is saved in R, the connections are closed # *before* the finalizer is run. In these cases, we reconnect # to the backend, so we can shut it down. - connectBackend("localhost", .sparkREnv$sparkRBackendPort) - } - - if (exists(".sparkRjsc", envir = env)) { - sc <- get(".sparkRjsc", envir = env) - callJMethod(sc, "stop") + tryCatch({ + connectBackend("localhost", .sparkREnv$sparkRBackendPort) + }, error = function(err) { + cat("Error in Connection: Use sparkR.init() to restart SparkR\n") + }, warning = function(war) { + cat("No Connection Found: Use sparkR.init() to restart SparkR\n") + }) + } + + 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) + } + + callJStatic("SparkRHandler", "stopBackend") + # Also close the connection and remove it from our env + conn <- get(".sparkRCon", env) + close(conn) + rm(".sparkRCon", envir = env) + # Finally, sleep for 1 sec to let backend finish exiting. + # Without this we get port conflicts in RStudio when we try to 'Restart R'. + Sys.sleep(1) } - - callJStatic("SparkRHandler", "stopBackend") - # Also close the connection and remove it from our env - conn <- get(".sparkRCon", env) - close(conn) - rm(".sparkRCon", envir = env) - - # Finally, sleep for 1 sec to let backend finish exiting. - # Without this we get port conflicts in RStudio when we try to 'Restart R'. - Sys.sleep(1) + } #' Initialize a new Spark Context. @@ -84,7 +93,7 @@ sparkR.init <- function( sparkRBackendPort = 12345) { if (exists(".sparkRjsc", envir = .sparkREnv)) { - cat("Re-using existing Spark Context. Please restart R to create a new Spark Context\n") + 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)) } diff --git a/pkg/R/sparkRClient.R b/pkg/R/sparkRClient.R index 5721f33c71c9e..61ddf03575325 100644 --- a/pkg/R/sparkRClient.R +++ b/pkg/R/sparkRClient.R @@ -4,8 +4,10 @@ # if one doesn't already exist connectBackend <- function(hostname, port, timeout = 6000) { if (exists(".sparkRcon", envir = .sparkREnv)) { - cat("SparkRBackend client connection already exists\n") - return(get(".sparkRcon", envir = .sparkREnv)) + if (isOpen(env[[".sparkRCon"]])) { + cat("SparkRBackend client connection already exists\n") + return(get(".sparkRcon", envir = .sparkREnv)) + } } con <- socketConnection(host = hostname, port = port, server = FALSE, From ba6f04443e9685d3b025a9f6511236740100a7fd Mon Sep 17 00:00:00 2001 From: lythesia Date: Sat, 7 Feb 2015 15:37:07 +0800 Subject: [PATCH 025/121] fixes for reduceByKeyLocally --- pkg/R/RDD.R | 33 ++++++++++++++++++++++----------- pkg/R/utils.R | 24 ++++++++++++------------ pkg/man/reduceByKeyLocally.Rd | 7 +++---- 3 files changed, 37 insertions(+), 27 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index fc62d92835736..6b6fc5a1757d0 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1388,9 +1388,22 @@ setMethod("groupByKey", function(item) { item$hash <- as.character(hashCode(item[[1]])) updateOrCreatePair(item, keys, vals, pred, - function(vs, v) c(vs, list(v)), - function(x) list(x)) + function(acc, x) { + addItemToAccumulator(acc, x) + acc + }, + function(x) { + acc <- initAccumulator() + addItemToAccumulator(acc, x) + acc + }) }) + # extract out data field + vals <- eapply(vals, + function(x) { + length(x$data) <- x$counter + x$data + }) # Every key in the environment contains a list # Convert that to list(K, Seq[V]) convertEnvsToList(keys, vals) @@ -1438,7 +1451,7 @@ setMethod("reduceByKey", lapply(part, function(item) { item$hash <- as.character(hashCode(item[[1]])) - updateOrCreatePair(item, keys, vals, pred, combineFunc, function(x) x) + updateOrCreatePair(item, keys, vals, pred, combineFunc, identity) }) convertEnvsToList(keys, vals) } @@ -1451,13 +1464,12 @@ setMethod("reduceByKey", #' #' 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 master as R list. +#' results immediately to the driver as an R list. #' #' @param rdd 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 An list where each element is list(K, V') where V' is the merged -#' value +#' @return A list of elements of type list(K, V') where V' is the merged value for each key #' @rdname reduceByKeyLocally #' @seealso reduceByKey #' @export @@ -1467,7 +1479,7 @@ setMethod("reduceByKey", #' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) #' rdd <- parallelize(sc, pairs) #' reduced <- reduceByKeyLocally(rdd, "+") -#' reduced[[1]] # Should be a list(1, 6) +#' reduced # list(list(1, 6), list(1.1, 3)) #'} setGeneric("reduceByKeyLocally", function(rdd, combineFunc) { @@ -1486,7 +1498,7 @@ setMethod("reduceByKeyLocally", lapply(part, function(item) { item$hash <- as.character(hashCode(item[[1]])) - updateOrCreatePair(item, keys, vals, pred, combineFunc, function(x) x) + updateOrCreatePair(item, keys, vals, pred, combineFunc, identity) }) list(list(keys, vals)) # return hash to avoid re-compute in merge } @@ -1498,7 +1510,7 @@ setMethod("reduceByKeyLocally", function(name) { item <- list(x[[1]][[name]], x[[2]][[name]]) item$hash <- name - updateOrCreatePair(item, accum[[1]], accum[[2]], pred, combineFunc, function(x) x) + updateOrCreatePair(item, accum[[1]], accum[[2]], pred, combineFunc, identity) }) accum } @@ -1573,8 +1585,7 @@ setMethod("combineByKey", lapply(part, function(item) { item$hash <- as.character(item[[1]]) - updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, - function(x) x) + updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, identity) }) convertEnvsToList(keys, combiners) } diff --git a/pkg/R/utils.R b/pkg/R/utils.R index cb5797d981ba7..820866ad80fe1 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -263,20 +263,20 @@ joinTaggedList <- function(tagged_list, cnull) { # Utility function to reduce a key-value list with predicate # Used in *ByKey functions # param -# item key-val pair +# pair key-value pair # keys/vals env of key/value with hashes -# pred predicate function -# update_fn update or merge function for existing pair, similar with `mergeVal` @combineByKey -# create_fn create function for new pair, similar with `createCombiner` @combinebykey -updateOrCreatePair <- function(item, keys, vals, pred, update_fn, create_fn) { - # assum hashval bind to `$hash`, key/val with index 1/2 - hashVal <- item$hash - key <- item[[1]] - val <- item[[2]] - if (pred(item)) { - assign(hashVal, do.call(update_fn, list(get(hashVal, envir=vals), val)), envir=vals) +# 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(create_fn, list(val)), envir=vals) + assign(hashVal, do.call(createFn, list(val)), envir = vals) assign(hashVal, key, envir=keys) } } diff --git a/pkg/man/reduceByKeyLocally.Rd b/pkg/man/reduceByKeyLocally.Rd index 2bf671f7e300c..f13c296065649 100644 --- a/pkg/man/reduceByKeyLocally.Rd +++ b/pkg/man/reduceByKeyLocally.Rd @@ -18,13 +18,12 @@ list(K, V) or c(K, V).} \item{combineFunc}{The associative reduce function to use.} } \value{ -An list where each element is list(K, V') where V' is the merged - value +A list of elements of type list(K, V') where V' is the merged value for each key } \description{ 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 master as R list. +results immediately to the driver as an R list. } \examples{ \dontrun{ @@ -32,7 +31,7 @@ sc <- sparkR.init() pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) rdd <- parallelize(sc, pairs) reduced <- reduceByKeyLocally(rdd, "+") -reduced[[1]] # Should be a list(1, 6) +reduced # list(list(1, 6), list(1.1, 3)) } } \seealso{ From f5038c062988b1d7fe6b6c6275b9f767dbc94689 Mon Sep 17 00:00:00 2001 From: lythesia Date: Sun, 8 Feb 2015 11:49:18 +0800 Subject: [PATCH 026/121] pull out anonymous functions in groupByKey --- pkg/R/RDD.R | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 6b6fc5a1757d0..f321f874c8abf 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1383,20 +1383,21 @@ setMethod("groupByKey", vals <- new.env() keys <- new.env() pred <- function(item) exists(item$hash, keys) + appendList <- function(acc, x) { + addItemToAccumulator(acc, x) + acc + } + makeList <- function(x) { + acc <- initAccumulator() + addItemToAccumulator(acc, x) + 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, - function(acc, x) { - addItemToAccumulator(acc, x) - acc - }, - function(x) { - acc <- initAccumulator() - addItemToAccumulator(acc, x) - acc - }) + appendList, makeList) }) # extract out data field vals <- eapply(vals, From de2bfb3f05a6a33694b1d740ee1227d8d1dd5418 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Mon, 9 Feb 2015 15:42:14 +0800 Subject: [PATCH 027/121] Fix minor comments and add more test cases. --- pkg/R/RDD.R | 12 ++++++------ pkg/inst/tests/test_rdd.R | 41 ++++++++++++++++++++++++++++++++++++--- pkg/man/sortByKey.Rd | 10 +++++----- 3 files changed, 49 insertions(+), 14 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 74bd6bfad364c..ade3b59fc4388 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1830,9 +1830,9 @@ setMethod("cogroup", group.func) }) -#' Sort an (k, v) pair RDD by k. +#' Sort a (k, v) pair RDD by k. #' -#' @param rdd An (k, v) pair RDD to be sorted. +#' @param rdd 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. @@ -1841,8 +1841,8 @@ setMethod("cogroup", #' @examples #'\dontrun{ #' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(3, 3), list(2, 2), list(1, 1))) -#' collect(sortByKey(rdd)) # list (list(1, 1), list(2, 2), list(3, 3)) +#' 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)) #'} setGeneric("sortByKey", function(rdd, ascending, numPartitions) { standardGeneric("sortByKey") }) @@ -1868,7 +1868,7 @@ setMethod("sortByKey", samples <- collect(keys(sampleRDD(rdd, FALSE, fraction, 1L))) - # Note: the built-in R sort() function only atomic vectors + # Note: the built-in R sort() function only works on atomic vectors samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending) if (length(samples) > 0) { @@ -1883,7 +1883,7 @@ setMethod("sortByKey", rangePartitionFunc <- function(key) { partition <- 0 - // TODO: Use binary search instead of linear search, similar with Spark + # TODO: Use binary search instead of linear search, similar with Spark while (partition < length(rangeBounds) && key > rangeBounds[[partition + 1]]) { partition <- partition + 1 } diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 4efa17a77b242..5331ff9ad5f76 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -255,12 +255,12 @@ test_that("keyBy on RDDs", { }) test_that("sortBy() on RDDs", { - sortedRdd <- sortBy(rdd, function(x) { x }, ascending = FALSE) + 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 }) + sortedRdd2 <- sortBy(rdd2, function(x) { x * x }) actual <- collect(sortedRdd2) expect_equal(actual, as.list(nums)) }) @@ -397,4 +397,39 @@ test_that("sortByKey() on pairwise RDDs", { sortedRdd2 <- sortByKey(numPairsRdd2) actual <- collect(sortedRdd2) expect_equal(actual, numPairs) -}) \ No newline at end of file + + # 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) +}) diff --git a/pkg/man/sortByKey.Rd b/pkg/man/sortByKey.Rd index b58dd5bf22ce6..b39aff6ca8757 100644 --- a/pkg/man/sortByKey.Rd +++ b/pkg/man/sortByKey.Rd @@ -4,7 +4,7 @@ \alias{sortByKey} \alias{sortByKey,RDD,RDD-method} \alias{sortByKey,RDD,missingOrLogical,missingOrInteger-method} -\title{Sort an (k, v) pair RDD by k.} +\title{Sort a (k, v) pair RDD by k.} \usage{ sortByKey(rdd, ascending, numPartitions) @@ -12,7 +12,7 @@ sortByKey(rdd, ascending, numPartitions) numPartitions) } \arguments{ -\item{rdd}{An (k, v) pair RDD to be sorted.} +\item{rdd}{A (k, v) pair RDD to be sorted.} \item{ascending}{A flag to indicate whether the sorting is ascending or descending.} @@ -22,13 +22,13 @@ sortByKey(rdd, ascending, numPartitions) An RDD where all (k, v) pair elements are sorted. } \description{ -Sort an (k, v) pair RDD by k. +Sort a (k, v) pair RDD by k. } \examples{ \dontrun{ sc <- sparkR.init() -rdd <- parallelize(sc, list(list(3, 3), list(2, 2), list(1, 1))) -collect(sortByKey(rdd)) # list (list(1, 1), list(2, 2), list(3, 3)) +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)) } } From 7feac3899e1db7d471bda19aa44c068d5cc86cb4 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Mon, 9 Feb 2015 18:40:28 +0800 Subject: [PATCH 028/121] Use default arguments for sortBy() and sortKeyBy(). --- pkg/R/RDD.R | 37 +++++++++++++++---------------------- 1 file changed, 15 insertions(+), 22 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index ade3b59fc4388..4d1c85a73359a 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1255,22 +1255,18 @@ setMethod("flatMapValues", #' rdd <- parallelize(sc, list(3, 2, 1)) #' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) #'} -setGeneric("sortBy", function(rdd, func, ascending, numPartitions) { standardGeneric("sortBy") }) +setGeneric("sortBy", function(rdd, + func, + ascending = TRUE, + numPartitions = 1L) { + standardGeneric("sortBy") + }) -setClassUnion("missingOrLogical", c("missing", "logical")) #' @rdname sortBy #' @aliases sortBy,RDD,RDD-method setMethod("sortBy", - signature(rdd = "RDD", func = "function", - ascending = "missingOrLogical", numPartitions = "missingOrInteger"), - function(rdd, func, ascending, numPartitions) { - if (missing(ascending)) { - ascending = TRUE - } - if (missing(numPartitions)) { - numPartitions = SparkR::numPartitions(rdd) - } - + signature(rdd = "RDD", func = "function"), + function(rdd, func, ascending = TRUE, numPartitions = SparkR::numPartitions(rdd)) { values(sortByKey(keyBy(rdd, func), ascending, numPartitions)) }) @@ -1844,20 +1840,17 @@ setMethod("cogroup", #' 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)) #'} -setGeneric("sortByKey", function(rdd, ascending, numPartitions) { standardGeneric("sortByKey") }) +setGeneric("sortByKey", function(rdd, + ascending = TRUE, + numPartitions = 1L) { + standardGeneric("sortByKey") + }) #' @rdname sortByKey #' @aliases sortByKey,RDD,RDD-method setMethod("sortByKey", - signature(rdd = "RDD", ascending = "missingOrLogical", numPartitions = "missingOrInteger"), - function(rdd, ascending, numPartitions) { - if (missing(ascending)) { - ascending = TRUE - } - if (missing(numPartitions)) { - numPartitions = SparkR::numPartitions(rdd) - } - + signature(rdd = "RDD"), + function(rdd, ascending = TRUE, numPartitions = SparkR::numPartitions(rdd)) { rangeBounds <- list() if (numPartitions > 1) { From 8398f2ec8fa592fa8af0697ba625090711fde349 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 9 Feb 2015 21:15:16 -0800 Subject: [PATCH 029/121] Add sparkR-submit helper script Also adjust R file path for YARN cluster mode --- .../cs/amplab/sparkr/SparkRRunner.scala | 14 ++++- sparkR-submit | 56 +++++++++++++++++++ 2 files changed, 67 insertions(+), 3 deletions(-) create mode 100755 sparkR-submit diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala index 84dd4bd775cc2..4ccf247e0eeb1 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala @@ -6,6 +6,8 @@ import java.net.URI import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ +import org.apache.hadoop.fs.Path + /** * 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. @@ -19,9 +21,15 @@ object SparkRRunner { val sparkRBackendPort = sys.env.getOrElse("SPARKR_BACKEND_PORT", "12345").toInt val rCommand = "Rscript" - // val formattedPythonFile = formatPath(pythonFile) - // TODO: Normalize path ? - val rFileNormalized = rFile + // 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. diff --git a/sparkR-submit b/sparkR-submit new file mode 100755 index 0000000000000..fbbbd78b10b61 --- /dev/null +++ b/sparkR-submit @@ -0,0 +1,56 @@ +#!/bin/bash +# To use sparkR-submit, we assume the SparkR package in yarn-cluster mode +# we assume that it has been installed to a standard location using +# R CMD INSTALL pkg/ + +FWDIR="$(cd `dirname $0`; pwd)" + +export PROJECT_HOME="$FWDIR" + +export SPARKR_JAR_FILE="$FWDIR/lib/SparkR/sparkr-assembly-0.1.jar" + +# Exit if the user hasn't set SPARK_HOME +if [ ! -f "$SPARK_HOME/bin/spark-submit" ]; then + echo "SPARK_HOME must be set to use sparkR-submit" + exit 1 +fi + +source "$SPARK_HOME/bin/utils.sh" + +function usage() { + echo "Usage: ./sparkR-submit [options]" 1>&2 + "$SPARK_HOME"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +} + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage +fi + + +# Add SparkR to .libPaths +# If we are running an R program, only set libPaths and use Rscript + +export R_PROFILE_USER="/tmp/sparkR.profile" + +cat > /tmp/sparkR.profile << EOF + .First <- function() { + projecHome <- Sys.getenv("PROJECT_HOME") + .libPaths(c(paste(projecHome,"/lib", sep=""), .libPaths())) + Sys.setenv(NOAWT=1) +} +EOF + +# Build up arguments list manually to preserve quotes and backslashes. +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" + +# If a R file is provided, directly run spark-submit. +if [[ "$1" =~ \.R$ ]]; then + primary="$1" + shift + gatherSparkSubmitOpts "$@" + exec "$FWDIR"/bin/spark-submit --class edu.berkeley.cs.amplab.sparkr.SparkRRunner --files "$primary" "${SUBMISSION_OPTS[@]}" "$SPARKR_JAR_FILE" "$primary" "${APPLICATION_OPTS[@]}" +else + echo "sparkR-submit can only be used to run R programs. Please use sparkR to launch a shell" +fi From 050390b79bcdb2675523e12743c4a42ee33a7d52 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 9 Feb 2015 21:40:27 -0800 Subject: [PATCH 030/121] Fix bugs in inferring R file --- sparkR-submit | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sparkR-submit b/sparkR-submit index fbbbd78b10b61..814625f145985 100755 --- a/sparkR-submit +++ b/sparkR-submit @@ -46,11 +46,11 @@ SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" # If a R file is provided, directly run spark-submit. -if [[ "$1" =~ \.R$ ]]; then - primary="$1" +if [[ "${APPLICATION_OPTS[0]}" =~ \.R$ ]]; then + primary="${APPLICATION_OPTS[0]}" shift - gatherSparkSubmitOpts "$@" - exec "$FWDIR"/bin/spark-submit --class edu.berkeley.cs.amplab.sparkr.SparkRRunner --files "$primary" "${SUBMISSION_OPTS[@]}" "$SPARKR_JAR_FILE" "$primary" "${APPLICATION_OPTS[@]}" + # Set the main class to SparkRRunner and add the primary R file to --files to make sure its copied to the cluster + exec "$SPARK_HOME"/bin/spark-submit --class edu.berkeley.cs.amplab.sparkr.SparkRRunner --files "$primary" "${SUBMISSION_OPTS[@]}" "$SPARKR_JAR_FILE" "$primary" "${APPLICATION_OPTS[@]:1}" else - echo "sparkR-submit can only be used to run R programs. Please use sparkR to launch a shell" + echo "sparkR-submit can only be used to run R programs. Please use sparkR to launch a shell." fi From 63e62ed5c5c94370267cf87eab4c874cbf75eb12 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Tue, 10 Feb 2015 19:09:17 +0800 Subject: [PATCH 031/121] [SPARKR-150] phase 2: implement takeOrdered() and top(). --- pkg/NAMESPACE | 2 ++ pkg/R/RDD.R | 73 +++++++++++++++++++++++++++++++++++++++ pkg/inst/tests/test_rdd.R | 24 +++++++++++++ pkg/man/takeOrdered.Rd | 31 +++++++++++++++++ pkg/man/top.Rd | 31 +++++++++++++++++ 5 files changed, 161 insertions(+) create mode 100644 pkg/man/takeOrdered.Rd create mode 100644 pkg/man/top.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 812e85238e9c6..b629b01ab2140 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -49,7 +49,9 @@ exportMethods( "sortBy", "sortByKey", "take", + "takeOrdered", "takeSample", + "top", "unionRDD", "unpersist", "value", diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 3f6d96251365b..2e884d35af716 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1294,6 +1294,79 @@ setMethod("sortBy", values(sortByKey(keyBy(rdd, func), ascending, numPartitions)) }) +# Helper function to get first N elements from an RDD in the specified order. +# Param: +# rdd An RDD. +# num Number of elements to return. +# ascending A flag to indicate whether the sorting is ascending or descending. +# Return: +# A list of the first N elements from the RDD in the specified order. +# +takeOrderedElem <- function(rdd, num, ascending = TRUE) { + if (num <= 0L) { + return(list()) + } + + partitionFunc <- function(part) { + if (num < length(part)) { + # R limitation: order works only on primitive types! + ord <- order(unlist(part, recursive = FALSE), decreasing = !ascending) + part[ord[1:num]] + } else { + part + } + } + + newRdd <- mapPartitions(rdd, partitionFunc) + take(sortBy(newRdd, function(x) { x }, ascending = ascending), num) +} + +#' Returns the first N elements from an RDD in ascending order. +#' +#' @param rdd An RDD. +#' @param num Number of elements to return. +#' @return The first N elements from the RDD in ascending order. +#' @rdname takeOrdered +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#'} +setGeneric("takeOrdered", function(rdd, num) { standardGeneric("takeOrdered") }) + +#' @rdname takeOrdered +#' @aliases takeOrdered,RDD,RDD-method +setMethod("takeOrdered", + signature(rdd = "RDD", num = "integer"), + function(rdd, num) { + takeOrderedElem(rdd, num) + }) + +#' Returns the top N elements from an RDD. +#' +#' @param rdd An RDD. +#' @param num Number of elements to return. +#' @return The top N elements from the RDD. +#' @rdname top +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#'} +setGeneric("top", function(rdd, num) { standardGeneric("top") }) + +#' @rdname top +#' @aliases top,RDD,RDD-method +setMethod("top", + signature(rdd = "RDD", num = "integer"), + function(rdd, num) { + takeOrderedElem(rdd, num, FALSE) + }) + ############ Shuffle Functions ############ #' Partition an RDD by key diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 2f48db61020fd..fa6c112ba7a51 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -278,6 +278,30 @@ test_that("sortBy() on RDDs", { 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("keys() on RDDs", { keys <- keys(intRdd) actual <- collect(keys) diff --git a/pkg/man/takeOrdered.Rd b/pkg/man/takeOrdered.Rd new file mode 100644 index 0000000000000..9ae2137abed21 --- /dev/null +++ b/pkg/man/takeOrdered.Rd @@ -0,0 +1,31 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{takeOrdered} +\alias{takeOrdered} +\alias{takeOrdered,RDD,RDD-method} +\alias{takeOrdered,RDD,integer-method} +\title{Returns the first N elements from an RDD in ascending order.} +\usage{ +takeOrdered(rdd, num) + +\S4method{takeOrdered}{RDD,integer}(rdd, num) +} +\arguments{ +\item{rdd}{An RDD.} + +\item{num}{Number of elements to return.} +} +\value{ +The first N elements from the RDD in ascending order. +} +\description{ +Returns the first N elements from an RDD in ascending order. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +} +} + diff --git a/pkg/man/top.Rd b/pkg/man/top.Rd new file mode 100644 index 0000000000000..627a43fd4ff71 --- /dev/null +++ b/pkg/man/top.Rd @@ -0,0 +1,31 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{top} +\alias{top} +\alias{top,RDD,RDD-method} +\alias{top,RDD,integer-method} +\title{Returns the top N elements from an RDD.} +\usage{ +top(rdd, num) + +\S4method{top}{RDD,integer}(rdd, num) +} +\arguments{ +\item{rdd}{An RDD.} + +\item{num}{Number of elements to return.} +} +\value{ +The top N elements from the RDD. +} +\description{ +Returns the top N elements from an RDD. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +} +} + From f4573c17ee0a895a99a12b289e86925baf99836f Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Wed, 11 Feb 2015 11:29:28 +0800 Subject: [PATCH 032/121] Use reduce() instead of sortBy().take() to get the ordered elements. --- pkg/R/RDD.R | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 2e884d35af716..3ffe017e15182 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1311,14 +1311,21 @@ takeOrderedElem <- function(rdd, num, ascending = TRUE) { if (num < length(part)) { # R limitation: order works only on primitive types! ord <- order(unlist(part, recursive = FALSE), decreasing = !ascending) - part[ord[1:num]] + list(part[ord[1:num]]) } else { - part + list(part) } } + + reduceFunc <- function(elems, part) { + newElems <- append(elems, part) + # R limitation: order works only on primitive types! + ord <- order(unlist(newElems, recursive = FALSE), decreasing = !ascending) + newElems[ord[1:num]] + } newRdd <- mapPartitions(rdd, partitionFunc) - take(sortBy(newRdd, function(x) { x }, ascending = ascending), num) + reduce(newRdd, reduceFunc) } #' Returns the first N elements from an RDD in ascending order. From 769087804897dcafeaf16075ddbcb33888b3eaa3 Mon Sep 17 00:00:00 2001 From: lythesia Date: Wed, 11 Feb 2015 13:53:14 +0800 Subject: [PATCH 033/121] separate out pair RDD functions --- pkg/R/RDD.R | 790 ------------------------------------- pkg/R/pairRDD.R | 799 ++++++++++++++++++++++++++++++++++++++ pkg/man/fullOuterJoin.Rd | 44 --- pkg/man/join-methods.Rd | 129 ++++++ pkg/man/join.Rd | 38 -- pkg/man/leftOuterJoin.Rd | 39 -- pkg/man/rightOuterJoin.Rd | 39 -- 7 files changed, 928 insertions(+), 950 deletions(-) create mode 100644 pkg/R/pairRDD.R delete mode 100644 pkg/man/fullOuterJoin.Rd create mode 100644 pkg/man/join-methods.Rd delete mode 100644 pkg/man/join.Rd delete mode 100644 pkg/man/leftOuterJoin.Rd delete mode 100644 pkg/man/rightOuterJoin.Rd diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 3f6d96251365b..434991b518e1e 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -407,38 +407,6 @@ setMethod("collectAsMap", as.list(map) }) -#' Look up elements of a key in an RDD -#' -#' @description -#' \code{lookup} returns a list of values in this RDD for key key. -#' -#' @param rdd The RDD to collect -#' @param key The key to look up for -#' @return a list of values in this RDD for key key -#' @rdname lookup -#' @export -#' @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) -#'} -setGeneric("lookup", function(rdd, key) { standardGeneric("lookup") }) - -#' @rdname lookup -#' @aliases lookup,RDD-method -setMethod("lookup", - signature(rdd = "RDD", key = "ANY"), - function(rdd, key) { - partitionFunc <- function(part) { - filtered <- part[unlist(lapply(part, function(x) { identical(key, x[[1]]) }))] - lapply(filtered, function(x) { x[[2]] }) - } - valsRDD <- lapplyPartition(rdd, partitionFunc) - collect(valsRDD) - }) - #' Return the number of elements in the RDD. #' #' @param x The RDD to count @@ -503,32 +471,6 @@ setMethod("countByValue", collect(reduceByKey(ones, `+`, numPartitions(rdd))) }) -#' 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 rdd The RDD to count keys. -#' @return list of (key, count) pairs, where count is number of each key in rdd. -#' @rdname countByKey -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) -#' countByKey(rdd) # ("a", 2L), ("b", 1L) -#'} -setGeneric("countByKey", function(rdd) { standardGeneric("countByKey") }) - -#' @rdname countByKey -#' @aliases countByKey,RDD-method -setMethod("countByKey", - signature(rdd = "RDD"), - function(rdd) { - keys <- lapply(rdd, function(item) { item[[1]] }) - countByValue(keys) - }) - #' Apply a function to all elements #' #' This function creates a new RDD by applying the given transformation to all @@ -1156,114 +1098,6 @@ setMethod("saveAsTextFile", callJMethod(getJRDD(stringRdd, dataSerialization = FALSE), "saveAsTextFile", path)) }) -#' Return an RDD with the keys of each tuple. -#' -#' @param rdd The RDD from which the keys of each tuple is returned. -#' @rdname keys -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(keys(rdd)) # list(1, 3) -#'} -setGeneric("keys", function(rdd) { standardGeneric("keys") }) - -#' @rdname keys -#' @aliases keys,RDD -setMethod("keys", - signature(rdd = "RDD"), - function(rdd) { - func <- function(x) { - x[[1]] - } - lapply(rdd, func) - }) - -#' Return an RDD with the values of each tuple. -#' -#' @param rdd The RDD from which the values of each tuple is returned. -#' @rdname values -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(values(rdd)) # list(2, 4) -#'} -setGeneric("values", function(rdd) { standardGeneric("values") }) - -#' @rdname values -#' @aliases values,RDD -setMethod("values", - signature(rdd = "RDD"), - function(rdd) { - func <- function(x) { - x[[2]] - } - lapply(rdd, 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. -#' @rdname mapValues -#' @export -#' @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), ...) -#'} -setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) - -#' @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. -#' @rdname flatMapValues -#' @export -#' @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)) -#'} -setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) - -#' @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) - }) - #' Sort an RDD by the given key function. #' #' @param rdd An RDD to be sorted. @@ -1294,335 +1128,6 @@ setMethod("sortBy", values(sortByKey(keyBy(rdd, func), ascending, numPartitions)) }) -############ 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 rdd 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. -#' @rdname partitionBy -#' @export -#' @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) -#'} -setGeneric("partitionBy", - function(rdd, numPartitions, ...) { - standardGeneric("partitionBy") - }) - -#' @rdname partitionBy -#' @aliases partitionBy,RDD,integer-method -setMethod("partitionBy", - signature(rdd = "RDD", numPartitions = "integer"), - function(rdd, numPartitions, partitionFunc = hashCode) { - - #if (missing(partitionFunc)) { - # partitionFunc <- hashCode - #} - - depsBinArr <- getDependencies(partitionFunc) - - serializedHashFuncBytes <- serialize(as.character(substitute(partitionFunc)), - connection = NULL, - ascii = TRUE) - - packageNamesArr <- serialize(.sparkREnv$.packages, - connection = NULL, - ascii = TRUE) - broadcastArr <- lapply(ls(.broadcastNames), function(name) { - get(name, .broadcastNames) }) - jrdd <- getJRDD(rdd) - - # 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("edu.berkeley.cs.amplab.sparkr.PairwiseRRDD", - callJMethod(jrdd, "rdd"), - as.integer(numPartitions), - serializedHashFuncBytes, - rdd@env$serialized, - depsBinArr, - 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, serialized = TRUE) - }) - -#' 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 rdd 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 -#' @rdname groupByKey -#' @export -#' @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)) -#'} -setGeneric("groupByKey", - function(rdd, numPartitions) { - standardGeneric("groupByKey") - }) - -#' @rdname groupByKey -#' @aliases groupByKey,RDD,integer-method -setMethod("groupByKey", - signature(rdd = "RDD", numPartitions = "integer"), - function(rdd, numPartitions) { - shuffled <- partitionBy(rdd, numPartitions) - groupVals <- function(part) { - vals <- new.env() - keys <- new.env() - pred <- function(item) exists(item$hash, keys) - appendList <- function(acc, x) { - addItemToAccumulator(acc, x) - acc - } - makeList <- function(x) { - acc <- initAccumulator() - addItemToAccumulator(acc, x) - 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(x) { - length(x$data) <- x$counter - x$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 rdd 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 -#' @rdname reduceByKey -#' @seealso groupByKey -#' @export -#' @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) -#'} -setGeneric("reduceByKey", - function(rdd, combineFunc, numPartitions) { - standardGeneric("reduceByKey") - }) - -#' @rdname reduceByKey -#' @aliases reduceByKey,RDD,integer-method -setMethod("reduceByKey", - signature(rdd = "RDD", combineFunc = "ANY", numPartitions = "integer"), - function(rdd, 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(rdd, 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 rdd 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 -#' @rdname reduceByKeyLocally -#' @seealso reduceByKey -#' @export -#' @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)) -#'} -setGeneric("reduceByKeyLocally", - function(rdd, combineFunc) { - standardGeneric("reduceByKeyLocally") - }) - -#' @rdname reduceByKeyLocally -#' @aliases reduceByKeyLocally,RDD,integer-method -setMethod("reduceByKeyLocally", - signature(rdd = "RDD", combineFunc = "ANY"), - function(rdd, 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(rdd, 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 rdd 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 -#' -#' @rdname combineByKey -#' @seealso groupByKey, reduceByKey -#' @export -#' @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) -#'} -setGeneric("combineByKey", - function(rdd, createCombiner, mergeValue, mergeCombiners, numPartitions) { - standardGeneric("combineByKey") - }) - -#' @rdname combineByKey -#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method -setMethod("combineByKey", - signature(rdd = "RDD", createCombiner = "ANY", mergeValue = "ANY", - mergeCombiners = "ANY", numPartitions = "integer"), - function(rdd, 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(rdd, 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) - }) - ############ Binary Functions ############# #' Return the union RDD of two RDDs. @@ -1663,301 +1168,6 @@ setMethod("unionRDD", union.rdd }) -#' Join two RDDs -#' -#' 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 rdd1 An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param rdd2 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. -#' @rdname join -#' @export -#' @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)) -#'} -setGeneric("join", function(rdd1, rdd2, numPartitions) { standardGeneric("join") }) - -#' @rdname join -#' @aliases join,RDD,RDD-method -setMethod("join", - signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), - function(rdd1, rdd2, numPartitions) { - rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) - rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) - - doJoin <- function(v) { - joinTaggedList(v, list(FALSE, FALSE)) - } - - joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), numPartitions), doJoin) - }) - -#' Left outer join two RDDs -#' -#' 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 rdd1 An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param rdd2 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 rdd1, 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. -#' @rdname leftOuterJoin -#' @export -#' @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))) -#'} -setGeneric("leftOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("leftOuterJoin") }) - -#' @rdname leftOuterJoin -#' @aliases leftOuterJoin,RDD,RDD-method -setMethod("leftOuterJoin", - signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), - function(rdd1, rdd2, numPartitions) { - rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) - rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) - - doJoin <- function(v) { - joinTaggedList(v, list(FALSE, TRUE)) - } - - joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), numPartitions), doJoin) - }) - -#' Right outer join two RDDs -#' -#' 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 rdd1 An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param rdd2 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 rdd2, the resulting RDD will either contain -#' all pairs (k, (v, w)) for (k, v) in rdd1, or the pair (k, (NULL, w)) -#' if no elements in rdd1 have key k. -#' @rdname rightOuterJoin -#' @export -#' @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))) -#'} -setGeneric("rightOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("rightOuterJoin") }) - -#' @rdname rightOuterJoin -#' @aliases rightOuterJoin,RDD,RDD-method -setMethod("rightOuterJoin", - signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), - function(rdd1, rdd2, numPartitions) { - rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) - rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) - - doJoin <- function(v) { - joinTaggedList(v, list(TRUE, FALSE)) - } - - joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), numPartitions), doJoin) - }) - -#' Full outer join two RDDs -#' -#' 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 rdd1 An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param rdd2 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 rdd1 and (k, w) in rdd2, the resulting RDD -#' will contain all pairs (k, (v, w)) for both (k, v) in rdd1 and and -#' (k, w) in rdd2, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements -#' in rdd1/rdd2 have key k. -#' @rdname fullOuterJoin -#' @export -#' @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)), -#'} -setGeneric("fullOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("fullOuterJoin") }) - -#' @rdname fullOuterJoin -#' @aliases fullOuterJoin,RDD,RDD-method - -setMethod("fullOuterJoin", - signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), - function(rdd1, rdd2, numPartitions) { - rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) - rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) - - doJoin <- function(v) { - joinTaggedList(v, list(TRUE, TRUE)) - } - - joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), 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. -#' @rdname cogroup -#' @export -#' @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())) -#'} -setGeneric("cogroup", - function(..., numPartitions) { standardGeneric("cogroup") }, - signature = "...") - -#' @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 <- SparkR:::initAccumulator() - } - SparkR:::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 rdd 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. -#' @rdname sortByKey -#' @export -#' @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)) -#'} -setGeneric("sortByKey", function(rdd, - ascending = TRUE, - numPartitions = 1L) { - standardGeneric("sortByKey") - }) - -#' @rdname sortByKey -#' @aliases sortByKey,RDD,RDD-method -setMethod("sortByKey", - signature(rdd = "RDD"), - function(rdd, ascending = TRUE, numPartitions = SparkR::numPartitions(rdd)) { - rangeBounds <- list() - - if (numPartitions > 1) { - rddSize <- count(rdd) - # constant from Spark's RangePartitioner - maxSampleSize <- numPartitions * 20 - fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) - - samples <- collect(keys(sampleRDD(rdd, 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(rdd, numPartitions, rangePartitionFunc) - lapplyPartition(newRDD, partitionFunc) - }) - # TODO: Consider caching the name in the RDD's environment #' Return an RDD's name. #' diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R new file mode 100644 index 0000000000000..f7497c5f8138d --- /dev/null +++ b/pkg/R/pairRDD.R @@ -0,0 +1,799 @@ +# Pair RDD in R implemented in S4 OO system. + +############ 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 rdd The RDD to collect +#' @param key The key to look up for +#' @return a list of values in this RDD for key key +#' @rdname lookup +#' @export +#' @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) +#'} +setGeneric("lookup", function(rdd, key) { standardGeneric("lookup") }) + +#' @rdname lookup +#' @aliases lookup,RDD-method +setMethod("lookup", + signature(rdd = "RDD", key = "ANY"), + function(rdd, key) { + partitionFunc <- function(part) { + filtered <- part[unlist(lapply(part, function(x) { identical(key, x[[1]]) }))] + lapply(filtered, function(x) { x[[2]] }) + } + valsRDD <- lapplyPartition(rdd, 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 rdd The RDD to count keys. +#' @return list of (key, count) pairs, where count is number of each key in rdd. +#' @rdname countByKey +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +#' countByKey(rdd) # ("a", 2L), ("b", 1L) +#'} +setGeneric("countByKey", function(rdd) { standardGeneric("countByKey") }) + +#' @rdname countByKey +#' @aliases countByKey,RDD-method +setMethod("countByKey", + signature(rdd = "RDD"), + function(rdd) { + keys <- lapply(rdd, function(item) { item[[1]] }) + countByValue(keys) + }) + +#' Return an RDD with the keys of each tuple. +#' +#' @param rdd The RDD from which the keys of each tuple is returned. +#' @rdname keys +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(keys(rdd)) # list(1, 3) +#'} +setGeneric("keys", function(rdd) { standardGeneric("keys") }) + +#' @rdname keys +#' @aliases keys,RDD +setMethod("keys", + signature(rdd = "RDD"), + function(rdd) { + func <- function(x) { + x[[1]] + } + lapply(rdd, func) + }) + +#' Return an RDD with the values of each tuple. +#' +#' @param rdd The RDD from which the values of each tuple is returned. +#' @rdname values +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(values(rdd)) # list(2, 4) +#'} +setGeneric("values", function(rdd) { standardGeneric("values") }) + +#' @rdname values +#' @aliases values,RDD +setMethod("values", + signature(rdd = "RDD"), + function(rdd) { + func <- function(x) { + x[[2]] + } + lapply(rdd, 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. +#' @rdname mapValues +#' @export +#' @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), ...) +#'} +setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) + +#' @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. +#' @rdname flatMapValues +#' @export +#' @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)) +#'} +setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) + +#' @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 rdd 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. +#' @rdname partitionBy +#' @export +#' @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) +#'} +setGeneric("partitionBy", + function(rdd, numPartitions, ...) { + standardGeneric("partitionBy") + }) + +#' @rdname partitionBy +#' @aliases partitionBy,RDD,integer-method +setMethod("partitionBy", + signature(rdd = "RDD", numPartitions = "integer"), + function(rdd, numPartitions, partitionFunc = hashCode) { + + #if (missing(partitionFunc)) { + # partitionFunc <- hashCode + #} + + depsBinArr <- getDependencies(partitionFunc) + + serializedHashFuncBytes <- serialize(as.character(substitute(partitionFunc)), + connection = NULL, + ascii = TRUE) + + packageNamesArr <- serialize(.sparkREnv$.packages, + connection = NULL, + ascii = TRUE) + broadcastArr <- lapply(ls(.broadcastNames), function(name) { + get(name, .broadcastNames) }) + jrdd <- getJRDD(rdd) + + # 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("edu.berkeley.cs.amplab.sparkr.PairwiseRRDD", + callJMethod(jrdd, "rdd"), + as.integer(numPartitions), + serializedHashFuncBytes, + rdd@env$serialized, + depsBinArr, + 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, serialized = TRUE) + }) + +#' 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 rdd 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 +#' @rdname groupByKey +#' @export +#' @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)) +#'} +setGeneric("groupByKey", + function(rdd, numPartitions) { + standardGeneric("groupByKey") + }) + +#' @rdname groupByKey +#' @aliases groupByKey,RDD,integer-method +setMethod("groupByKey", + signature(rdd = "RDD", numPartitions = "integer"), + function(rdd, numPartitions) { + shuffled <- partitionBy(rdd, numPartitions) + groupVals <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + appendList <- function(acc, x) { + addItemToAccumulator(acc, x) + acc + } + makeList <- function(x) { + acc <- initAccumulator() + addItemToAccumulator(acc, x) + 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(x) { + length(x$data) <- x$counter + x$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 rdd 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 +#' @rdname reduceByKey +#' @seealso groupByKey +#' @export +#' @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) +#'} +setGeneric("reduceByKey", + function(rdd, combineFunc, numPartitions) { + standardGeneric("reduceByKey") + }) + +#' @rdname reduceByKey +#' @aliases reduceByKey,RDD,integer-method +setMethod("reduceByKey", + signature(rdd = "RDD", combineFunc = "ANY", numPartitions = "integer"), + function(rdd, 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(rdd, 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 rdd 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 +#' @rdname reduceByKeyLocally +#' @seealso reduceByKey +#' @export +#' @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)) +#'} +setGeneric("reduceByKeyLocally", + function(rdd, combineFunc) { + standardGeneric("reduceByKeyLocally") + }) + +#' @rdname reduceByKeyLocally +#' @aliases reduceByKeyLocally,RDD,integer-method +setMethod("reduceByKeyLocally", + signature(rdd = "RDD", combineFunc = "ANY"), + function(rdd, 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(rdd, 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 rdd 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 +#' +#' @rdname combineByKey +#' @seealso groupByKey, reduceByKey +#' @export +#' @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) +#'} +setGeneric("combineByKey", + function(rdd, createCombiner, mergeValue, mergeCombiners, numPartitions) { + standardGeneric("combineByKey") + }) + +#' @rdname combineByKey +#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method +setMethod("combineByKey", + signature(rdd = "RDD", createCombiner = "ANY", mergeValue = "ANY", + mergeCombiners = "ANY", numPartitions = "integer"), + function(rdd, 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(rdd, 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) + }) + +############ 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 rdd1 An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param rdd2 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. +#' @rdname join-methods +#' @export +#' @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)) +#'} +setGeneric("join", function(rdd1, rdd2, numPartitions) { standardGeneric("join") }) + +#' @rdname join-methods +#' @aliases join,RDD,RDD-method +setMethod("join", + signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), + function(rdd1, rdd2, numPartitions) { + rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) + rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(FALSE, FALSE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), 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 rdd1 An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param rdd2 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 rdd1, 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. +#' @rdname join-methods +#' @export +#' @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))) +#'} +setGeneric("leftOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("leftOuterJoin") }) + +#' @rdname join-methods +#' @aliases leftOuterJoin,RDD,RDD-method +setMethod("leftOuterJoin", + signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), + function(rdd1, rdd2, numPartitions) { + rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) + rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(FALSE, TRUE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), 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 rdd1 An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param rdd2 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 rdd2, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, v) in rdd1, or the pair (k, (NULL, w)) +#' if no elements in rdd1 have key k. +#' @rdname join-methods +#' @export +#' @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))) +#'} +setGeneric("rightOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("rightOuterJoin") }) + +#' @rdname join-methods +#' @aliases rightOuterJoin,RDD,RDD-method +setMethod("rightOuterJoin", + signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), + function(rdd1, rdd2, numPartitions) { + rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) + rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(TRUE, FALSE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), 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 rdd1 An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param rdd2 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 rdd1 and (k, w) in rdd2, the resulting RDD +#' will contain all pairs (k, (v, w)) for both (k, v) in rdd1 and and +#' (k, w) in rdd2, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +#' in rdd1/rdd2 have key k. +#' @rdname join-methods +#' @export +#' @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)), +#'} +setGeneric("fullOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("fullOuterJoin") }) + +#' @rdname join-methods +#' @aliases fullOuterJoin,RDD,RDD-method + +setMethod("fullOuterJoin", + signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), + function(rdd1, rdd2, numPartitions) { + rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) + rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(TRUE, TRUE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), 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. +#' @rdname cogroup +#' @export +#' @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())) +#'} +setGeneric("cogroup", + function(..., numPartitions) { standardGeneric("cogroup") }, + signature = "...") + +#' @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 <- SparkR:::initAccumulator() + } + SparkR:::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 rdd 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. +#' @rdname sortByKey +#' @export +#' @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)) +#'} +setGeneric("sortByKey", function(rdd, + ascending = TRUE, + numPartitions = 1L) { + standardGeneric("sortByKey") + }) + +#' @rdname sortByKey +#' @aliases sortByKey,RDD,RDD-method +setMethod("sortByKey", + signature(rdd = "RDD"), + function(rdd, ascending = TRUE, numPartitions = SparkR::numPartitions(rdd)) { + rangeBounds <- list() + + if (numPartitions > 1) { + rddSize <- count(rdd) + # constant from Spark's RangePartitioner + maxSampleSize <- numPartitions * 20 + fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) + + samples <- collect(keys(sampleRDD(rdd, 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(rdd, numPartitions, rangePartitionFunc) + lapplyPartition(newRDD, partitionFunc) + }) + diff --git a/pkg/man/fullOuterJoin.Rd b/pkg/man/fullOuterJoin.Rd deleted file mode 100644 index eef2391e323fe..0000000000000 --- a/pkg/man/fullOuterJoin.Rd +++ /dev/null @@ -1,44 +0,0 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand -\docType{methods} -\name{fullOuterJoin} -\alias{fullOuterJoin} -\alias{fullOuterJoin,RDD,RDD,integer-method} -\alias{fullOuterJoin,RDD,RDD-method} -\title{Full outer join two RDDs} -\usage{ -fullOuterJoin(rdd1, rdd2, numPartitions) - -\S4method{fullOuterJoin}{RDD,RDD,integer}(rdd1, rdd2, numPartitions) -} -\arguments{ -\item{rdd1}{An RDD to be joined. Should be an RDD where each element is -list(K, V).} - -\item{rdd2}{An RDD to be joined. Should be an RDD where each element is -list(K, V).} - -\item{numPartitions}{Number of partitions to create.} -} -\value{ -For each element (k, v) in rdd1 and (k, w) in rdd2, the resulting RDD - will contain all pairs (k, (v, w)) for both (k, v) in rdd1 and and - (k, w) in rdd2, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements - in rdd1/rdd2 have key k. -} -\description{ -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. -} -\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(3, list(3, NULL)), - # list(2, list(NULL, 4))) -} -} - diff --git a/pkg/man/join-methods.Rd b/pkg/man/join-methods.Rd new file mode 100644 index 0000000000000..11229a4e61ef9 --- /dev/null +++ b/pkg/man/join-methods.Rd @@ -0,0 +1,129 @@ +% Generated by roxygen2 (4.1.0): do not edit by hand +% Please edit documentation in R/pairRDD.R +\docType{methods} +\name{join} +\alias{fullOuterJoin} +\alias{fullOuterJoin,RDD,RDD,integer-method} +\alias{fullOuterJoin,RDD,RDD-method} +\alias{join} +\alias{join,RDD,RDD,integer-method} +\alias{join,RDD,RDD-method} +\alias{leftOuterJoin} +\alias{leftOuterJoin,RDD,RDD,integer-method} +\alias{leftOuterJoin,RDD,RDD-method} +\alias{rightOuterJoin} +\alias{rightOuterJoin,RDD,RDD,integer-method} +\alias{rightOuterJoin,RDD,RDD-method} +\title{Join two RDDs} +\usage{ +join(rdd1, rdd2, numPartitions) + +\S4method{join}{RDD,RDD,integer}(rdd1, rdd2, numPartitions) + +leftOuterJoin(rdd1, rdd2, numPartitions) + +\S4method{leftOuterJoin}{RDD,RDD,integer}(rdd1, rdd2, numPartitions) + +rightOuterJoin(rdd1, rdd2, numPartitions) + +\S4method{rightOuterJoin}{RDD,RDD,integer}(rdd1, rdd2, numPartitions) + +fullOuterJoin(rdd1, rdd2, numPartitions) + +\S4method{fullOuterJoin}{RDD,RDD,integer}(rdd1, rdd2, numPartitions) +} +\arguments{ +\item{rdd1}{An RDD to be joined. Should be an RDD where each element is +list(K, V).} + +\item{rdd2}{An RDD to be joined. Should be an RDD where each element is +list(K, V).} + +\item{numPartitions}{Number of partitions to create.} + +\item{rdd1}{An RDD to be joined. Should be an RDD where each element is +list(K, V).} + +\item{rdd2}{An RDD to be joined. Should be an RDD where each element is +list(K, V).} + +\item{numPartitions}{Number of partitions to create.} + +\item{rdd1}{An RDD to be joined. Should be an RDD where each element is +list(K, V).} + +\item{rdd2}{An RDD to be joined. Should be an RDD where each element is +list(K, V).} + +\item{numPartitions}{Number of partitions to create.} + +\item{rdd1}{An RDD to be joined. Should be an RDD where each element is +list(K, V).} + +\item{rdd2}{An RDD to be joined. Should be an RDD where each element is +list(K, V).} + +\item{numPartitions}{Number of partitions to create.} +} +\value{ +a new RDD containing all pairs of elements with matching keys in + two input RDDs. + +For each element (k, v) in rdd1, 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. + +For each element (k, w) in rdd2, the resulting RDD will either contain + all pairs (k, (v, w)) for (k, v) in rdd1, or the pair (k, (NULL, w)) + if no elements in rdd1 have key k. + +For each element (k, v) in rdd1 and (k, w) in rdd2, the resulting RDD + will contain all pairs (k, (v, w)) for both (k, v) in rdd1 and and + (k, w) in rdd2, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements + in rdd1/rdd2 have key k. +} +\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. + +\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. + +\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. + +\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. +} +\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)) +} +\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))) +} +\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))) +} +\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)), +} +} + diff --git a/pkg/man/join.Rd b/pkg/man/join.Rd deleted file mode 100644 index f406d90ec6a46..0000000000000 --- a/pkg/man/join.Rd +++ /dev/null @@ -1,38 +0,0 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand -\docType{methods} -\name{join} -\alias{join} -\alias{join,RDD,RDD,integer-method} -\alias{join,RDD,RDD-method} -\title{Join two RDDs} -\usage{ -join(rdd1, rdd2, numPartitions) - -\S4method{join}{RDD,RDD,integer}(rdd1, rdd2, numPartitions) -} -\arguments{ -\item{rdd1}{An RDD to be joined. Should be an RDD where each element is -list(K, V).} - -\item{rdd2}{An RDD to be joined. Should be an RDD where each element is -list(K, V).} - -\item{numPartitions}{Number of partitions to create.} -} -\value{ -a new RDD containing all pairs of elements with matching keys in - two input RDDs. -} -\description{ -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. -} -\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)) -} -} - diff --git a/pkg/man/leftOuterJoin.Rd b/pkg/man/leftOuterJoin.Rd deleted file mode 100644 index d04f4e6f8ea35..0000000000000 --- a/pkg/man/leftOuterJoin.Rd +++ /dev/null @@ -1,39 +0,0 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand -\docType{methods} -\name{leftOuterJoin} -\alias{leftOuterJoin} -\alias{leftOuterJoin,RDD,RDD,integer-method} -\alias{leftOuterJoin,RDD,RDD-method} -\title{Left outer join two RDDs} -\usage{ -leftOuterJoin(rdd1, rdd2, numPartitions) - -\S4method{leftOuterJoin}{RDD,RDD,integer}(rdd1, rdd2, numPartitions) -} -\arguments{ -\item{rdd1}{An RDD to be joined. Should be an RDD where each element is -list(K, V).} - -\item{rdd2}{An RDD to be joined. Should be an RDD where each element is -list(K, V).} - -\item{numPartitions}{Number of partitions to create.} -} -\value{ -For each element (k, v) in rdd1, 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. -} -\description{ -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. -} -\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))) -} -} - diff --git a/pkg/man/rightOuterJoin.Rd b/pkg/man/rightOuterJoin.Rd deleted file mode 100644 index 9e72859b6df17..0000000000000 --- a/pkg/man/rightOuterJoin.Rd +++ /dev/null @@ -1,39 +0,0 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand -\docType{methods} -\name{rightOuterJoin} -\alias{rightOuterJoin} -\alias{rightOuterJoin,RDD,RDD,integer-method} -\alias{rightOuterJoin,RDD,RDD-method} -\title{Right outer join two RDDs} -\usage{ -rightOuterJoin(rdd1, rdd2, numPartitions) - -\S4method{rightOuterJoin}{RDD,RDD,integer}(rdd1, rdd2, numPartitions) -} -\arguments{ -\item{rdd1}{An RDD to be joined. Should be an RDD where each element is -list(K, V).} - -\item{rdd2}{An RDD to be joined. Should be an RDD where each element is -list(K, V).} - -\item{numPartitions}{Number of partitions to create.} -} -\value{ -For each element (k, w) in rdd2, the resulting RDD will either contain - all pairs (k, (v, w)) for (k, v) in rdd1, or the pair (k, (NULL, w)) - if no elements in rdd1 have key k. -} -\description{ -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. -} -\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))) -} -} - From d96866486ac32da277bff4a5f234be1c413b70ee Mon Sep 17 00:00:00 2001 From: lythesia Date: Thu, 12 Feb 2015 12:21:21 +0800 Subject: [PATCH 034/121] fix comment --- pkg/R/pairRDD.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R index f7497c5f8138d..eef111d309263 100644 --- a/pkg/R/pairRDD.R +++ b/pkg/R/pairRDD.R @@ -1,4 +1,4 @@ -# Pair RDD in R implemented in S4 OO system. +# Operations supported on RDDs contains pairs (i.e key, value) ############ Actions and Transformations ############ From 5f29551bf8f1171482b9dabc17ecdf9569fa35bf Mon Sep 17 00:00:00 2001 From: hqzizania Date: Thu, 12 Feb 2015 16:26:20 +0800 Subject: [PATCH 035/121] modified: pkg/R/RDD.R modified: pkg/R/context.R --- pkg/R/RDD.R | 13 +++++-------- pkg/R/context.R | 2 +- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 3f6d96251365b..d914377b620cc 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -110,12 +110,10 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), computeFunc <- function(split, part) { rdd@func(split, part) } - serializedFuncArr <- serialize("computeFunc", connection = NULL, - ascii = TRUE) + serializedFuncArr <- serialize("computeFunc", connection = NULL) packageNamesArr <- serialize(.sparkREnv[[".packages"]], - connection = NULL, - ascii = TRUE) + connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) @@ -1338,12 +1336,11 @@ setMethod("partitionBy", depsBinArr <- getDependencies(partitionFunc) serializedHashFuncBytes <- serialize(as.character(substitute(partitionFunc)), - connection = NULL, - ascii = TRUE) + connection = NULL) packageNamesArr <- serialize(.sparkREnv$.packages, - connection = NULL, - ascii = TRUE) + connection = NULL) + broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) jrdd <- getJRDD(rdd) diff --git a/pkg/R/context.R b/pkg/R/context.R index 095ad5528ee7f..e05c1cd71da40 100644 --- a/pkg/R/context.R +++ b/pkg/R/context.R @@ -179,7 +179,7 @@ includePackage <- function(sc, pkg) { #'} broadcast <- function(sc, object) { objName <- as.character(substitute(object)) - serializedObj <- serialize(object, connection = NULL, ascii = TRUE) + serializedObj <- serialize(object, connection = NULL) jBroadcast <- callJMethod(sc, "broadcast", serializedObj) id <- as.character(callJMethod(jBroadcast, "id")) From 94066bff5f1b238a6ed5b9906ef05b3a3d76bb12 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Fri, 13 Feb 2015 20:03:30 +0800 Subject: [PATCH 036/121] [SPARKR-153] phase 1: implement fold() and aggregate(). --- pkg/NAMESPACE | 2 ++ pkg/R/RDD.R | 65 +++++++++++++++++++++++++++++++++++++++ pkg/inst/tests/test_rdd.R | 22 +++++++++++++ pkg/man/aggregateRDD.Rd | 40 ++++++++++++++++++++++++ pkg/man/fold.Rd | 34 ++++++++++++++++++++ 5 files changed, 163 insertions(+) create mode 100644 pkg/man/aggregateRDD.Rd create mode 100644 pkg/man/fold.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index b629b01ab2140..1fa1336674750 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -2,6 +2,7 @@ exportClasses("RDD") exportClasses("Broadcast") exportMethods( + "aggregateRDD", "cache", "checkpoint", "cogroup", @@ -17,6 +18,7 @@ exportMethods( "filterRDD", "flatMap", "flatMapValues", + "fold", "foreach", "foreachPartition", "fullOuterJoin", diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 3ffe017e15182..39f054b9ba67a 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1374,6 +1374,71 @@ setMethod("top", takeOrderedElem(rdd, num, FALSE) }) +#' Fold an RDD using a given associative function and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using a given associative function and a neutral "zero value". +#' +#' @param rdd An RDD. +#' @param zeroValue A neutral "zero value". +#' @param op An associative function for the folding operation. +#' @return The folding result. +#' @rdname fold +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +#' fold(rdd, 0, "+") # 15 +#'} +setGeneric("fold", function(rdd, zeroValue, op) { standardGeneric("fold") }) + +#' @rdname fold +#' @aliases fold,RDD,RDD-method +setMethod("fold", + signature(rdd = "RDD", zeroValue = "ANY", op = "ANY"), + function(rdd, zeroValue, op) { + aggregateRDD(rdd, zeroValue, op, op) + }) + +#' Aggregate an RDD using the given combine functions and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using given combine functions and a neutral "zero value". +#' +#' @param rdd An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the RDD elements. It may return a different +#' result type from the type of the RDD elements. +#' @param combOp A function to aggregate results of seqOp. +#' @return The aggregation result. +#' @rdname aggregateRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' 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]]) } +#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#'} +setGeneric("aggregateRDD", function(rdd, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) + +#' @rdname aggregateRDD +#' @aliases aggregateRDD,RDD,RDD-method +setMethod("aggregateRDD", + signature(rdd = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), + function(rdd, zeroValue, seqOp, combOp) { + partitionFunc <- function(part) { + Reduce(seqOp, part, zeroValue) + } + + partitionList <- collect(lapplyPartition(rdd, partitionFunc), + flatten = FALSE) + Reduce(combOp, partitionList, zeroValue) + }) + ############ Shuffle Functions ############ #' Partition an RDD by key diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index fa6c112ba7a51..dff91179c6495 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -302,6 +302,28 @@ test_that("top() on RDDs", { 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("aggregate() 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("keys() on RDDs", { keys <- keys(intRdd) actual <- collect(keys) diff --git a/pkg/man/aggregateRDD.Rd b/pkg/man/aggregateRDD.Rd new file mode 100644 index 0000000000000..b64936aa8482f --- /dev/null +++ b/pkg/man/aggregateRDD.Rd @@ -0,0 +1,40 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{aggregateRDD} +\alias{aggregateRDD} +\alias{aggregateRDD,RDD,RDD-method} +\alias{aggregateRDD,RDD-method} +\title{Aggregate an RDD using the given combine functions and a neutral "zero value".} +\usage{ +aggregateRDD(rdd, zeroValue, seqOp, combOp) + +\S4method{aggregateRDD}{RDD}(rdd, zeroValue, seqOp, combOp) +} +\arguments{ +\item{rdd}{An RDD.} + +\item{zeroValue}{A neutral "zero value".} + +\item{seqOp}{A function to aggregate the RDD elements. It may return a different +result type from the type of the RDD elements.} + +\item{combOp}{A function to aggregate results of seqOp.} +} +\value{ +The aggregation result. +} +\description{ +Aggregate the elements of each partition, and then the results for all the +partitions, using given combine functions and a neutral "zero value". +} +\examples{ +\dontrun{ +sc <- sparkR.init() +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]]) } +aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +} +} + diff --git a/pkg/man/fold.Rd b/pkg/man/fold.Rd new file mode 100644 index 0000000000000..e0d22d23e3b79 --- /dev/null +++ b/pkg/man/fold.Rd @@ -0,0 +1,34 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{fold} +\alias{fold} +\alias{fold,RDD,RDD-method} +\alias{fold,RDD-method} +\title{Fold an RDD using a given associative function and a neutral "zero value".} +\usage{ +fold(rdd, zeroValue, op) + +\S4method{fold}{RDD}(rdd, zeroValue, op) +} +\arguments{ +\item{rdd}{An RDD.} + +\item{zeroValue}{A neutral "zero value".} + +\item{op}{An associative function for the folding operation.} +} +\value{ +The folding result. +} +\description{ +Aggregate the elements of each partition, and then the results for all the +partitions, using a given associative function and a neutral "zero value". +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +fold(rdd, 0, "+") # 15 +} +} + From 9d335a9caab36b029c6f802454470fdd225d4d78 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Fri, 13 Feb 2015 19:00:19 -0500 Subject: [PATCH 037/121] Makes git to ignore Eclipse meta files. --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index 7d027d3bf143e..f1b641883a31a 100644 --- a/.gitignore +++ b/.gitignore @@ -20,3 +20,6 @@ work/ SparkR-pkg.Rproj *.o *.so +# Eclipse Meta Files +.project +.classpath From 141723eeed1cd032931e112a3d11c7f57c9c72c9 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Sun, 15 Feb 2015 10:25:11 +0800 Subject: [PATCH 038/121] fix comments. --- pkg/R/RDD.R | 2 ++ pkg/inst/tests/test_rdd.R | 2 +- pkg/man/aggregateRDD.Rd | 3 +++ pkg/man/fold.Rd | 3 +++ 4 files changed, 9 insertions(+), 1 deletion(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 39f054b9ba67a..b23e13db31b61 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1384,6 +1384,7 @@ setMethod("top", #' @param op An associative function for the folding operation. #' @return The folding result. #' @rdname fold +#' @seealso reduce #' @export #' @examples #'\dontrun{ @@ -1413,6 +1414,7 @@ setMethod("fold", #' @param combOp A function to aggregate results of seqOp. #' @return The aggregation result. #' @rdname aggregateRDD +#' @seealso reduce #' @export #' @examples #'\dontrun{ diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index dff91179c6495..89d7890fbf685 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -311,7 +311,7 @@ test_that("fold() on RDDs", { expect_equal(actual, 0) }) -test_that("aggregate() on RDDs", { +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) } diff --git a/pkg/man/aggregateRDD.Rd b/pkg/man/aggregateRDD.Rd index b64936aa8482f..16a0fbe4cb3b3 100644 --- a/pkg/man/aggregateRDD.Rd +++ b/pkg/man/aggregateRDD.Rd @@ -37,4 +37,7 @@ combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) } } +\seealso{ +reduce +} diff --git a/pkg/man/fold.Rd b/pkg/man/fold.Rd index e0d22d23e3b79..f2253d3f34fa9 100644 --- a/pkg/man/fold.Rd +++ b/pkg/man/fold.Rd @@ -31,4 +31,7 @@ rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) fold(rdd, 0, "+") # 15 } } +\seealso{ +reduce +} From 7fcb46a83d593ae49dc2a059f841675cedb9114c Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Mon, 16 Feb 2015 10:44:20 +0800 Subject: [PATCH 039/121] Remove partitionBy() in RDD. --- pkg/R/RDD.R | 155 ++++++++++++---------------------------------------- 1 file changed, 36 insertions(+), 119 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index bb5e759c2a261..c54abae2666a2 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1275,87 +1275,51 @@ setMethod("aggregateRDD", Reduce(combOp, partitionList, zeroValue) }) -############ 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 rdd 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. +# TODO: Consider caching the name in the RDD's environment +#' Return an RDD's name. #' -#' @param partitionFunc The partition function to use. Uses a default hashCode -#' function if not provided -#' @return An RDD partitioned using the specified partitioner. -#' @rdname partitionBy +#' @param rdd The RDD whose name is returned. +#' @rdname name #' @export #' @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) +#' rdd <- parallelize(sc, list(1,2,3)) +#' name(rdd) # NULL (if not set before) #'} -setGeneric("partitionBy", - function(rdd, numPartitions, ...) { - standardGeneric("partitionBy") - }) - -#' @rdname partitionBy -#' @aliases partitionBy,RDD,integer-method -setMethod("partitionBy", - signature(rdd = "RDD", numPartitions = "integer"), - function(rdd, numPartitions, partitionFunc = hashCode) { - - #if (missing(partitionFunc)) { - # partitionFunc <- hashCode - #} - - depsBinArr <- getDependencies(partitionFunc) +setGeneric("name", function(rdd) { standardGeneric("name") }) - serializedHashFuncBytes <- serialize(as.character(substitute(partitionFunc)), - connection = NULL, - ascii = TRUE) +#' @rdname name +#' @aliases name,RDD +setMethod("name", + signature(rdd = "RDD"), + function(rdd) { + callJMethod(getJRDD(rdd), "name") + }) - packageNamesArr <- serialize(.sparkREnv$.packages, - connection = NULL, - ascii = TRUE) - broadcastArr <- lapply(ls(.broadcastNames), function(name) { - get(name, .broadcastNames) }) - jrdd <- getJRDD(rdd) +#' Set an RDD's name. +#' +#' @param rdd The RDD whose name is to be set. +#' @param name The RDD name to be set. +#' @return a new RDD renamed. +#' @rdname setName +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' setName(rdd, "myRDD") +#' name(rdd) # "myRDD" +#'} +setGeneric("setName", function(rdd, name) { standardGeneric("setName") }) - # 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("edu.berkeley.cs.amplab.sparkr.PairwiseRRDD", - callJMethod(jrdd, "rdd"), - as.integer(numPartitions), - serializedHashFuncBytes, - rdd@env$serialized, - depsBinArr, - 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, serialized = TRUE) +#' @rdname setName +#' @aliases setName,RDD +setMethod("setName", + signature(rdd = "RDD", name = "character"), + function(rdd, name) { + callJMethod(getJRDD(rdd), "setName", name) + rdd }) ############ Binary Functions ############# @@ -1397,50 +1361,3 @@ setMethod("unionRDD", } union.rdd }) - -# TODO: Consider caching the name in the RDD's environment -#' Return an RDD's name. -#' -#' @param rdd The RDD whose name is returned. -#' @rdname name -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' name(rdd) # NULL (if not set before) -#'} -setGeneric("name", function(rdd) { standardGeneric("name") }) - -#' @rdname name -#' @aliases name,RDD -setMethod("name", - signature(rdd = "RDD"), - function(rdd) { - callJMethod(getJRDD(rdd), "name") - }) - -#' Set an RDD's name. -#' -#' @param rdd The RDD whose name is to be set. -#' @param name The RDD name to be set. -#' @return a new RDD renamed. -#' @rdname setName -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' setName(rdd, "myRDD") -#' name(rdd) # "myRDD" -#'} -setGeneric("setName", function(rdd, name) { standardGeneric("setName") }) - -#' @rdname setName -#' @aliases setName,RDD -setMethod("setName", - signature(rdd = "RDD", name = "character"), - function(rdd, name) { - callJMethod(getJRDD(rdd), "setName", name) - rdd - }) From 67fbc60af65fd1b11f8d0d6e3a47fb185b7b94e4 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 15 Feb 2015 23:44:59 -0800 Subject: [PATCH 040/121] Add support for SparkR shell to use spark-submit This ensures that SparkConf options are read in both in batch and interactive modes --- pkg/R/sparkR.R | 21 ++++++-- pkg/R/sparkRClient.R | 27 +++++++++++ .../edu/berkeley/cs/amplab/sparkr/RRDD.scala | 13 ++++- sparkR | 2 +- sparkR-submit | 48 ++++++++++++++++--- 5 files changed, 96 insertions(+), 15 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 2767b8a233772..453d41c2ff74e 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -74,7 +74,7 @@ sparkR.stop <- function(env) { #'} sparkR.init <- function( - master = "local", + master = "", appName = "SparkR", sparkHome = Sys.getenv("SPARK_HOME"), sparkEnvir = list(), @@ -101,10 +101,21 @@ sparkR.init <- function( if (sparkRExistingPort != "") { sparkRBackendPort <- sparkRExistingPort } else { - launchBackend(classPath = cp, - mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", - args = as.character(sparkRBackendPort), - javaOpts = paste("-Xmx", sparkMem, sep = "")) + if (Sys.getenv("SPARKR_USE_SPARK_SUBMIT", "") == "") { + launchBackend(classPath = cp, + mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", + args = as.character(sparkRBackendPort), + javaOpts = paste("-Xmx", sparkMem, sep = "")) + } else { + # TODO: We should deprecate sparkJars and ask users to add it to the + # command line (using --jars) which is picked up by SparkSubmit + launchBackendSparkSubmit( + mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", + args = as.character(sparkRBackendPort), + appJar = .sparkREnv$assemblyJarPath, + sparkHome = sparkHome, + sparkSubmitOpts = Sys.getenv("SPARKR_SUBMIT_ARGS", "")) + } Sys.sleep(2) # Wait for backend to come up } .sparkREnv$sparkRBackendPort <- sparkRBackendPort diff --git a/pkg/R/sparkRClient.R b/pkg/R/sparkRClient.R index 5721f33c71c9e..fb536e56fe3f3 100644 --- a/pkg/R/sparkRClient.R +++ b/pkg/R/sparkRClient.R @@ -37,3 +37,30 @@ launchBackend <- function( cat("Launching java with command ", java_bin, " ", combinedArgs, "\n") invisible(system2(java_bin, combinedArgs, wait = F)) } + +launchBackendSparkSubmit <- function( + mainClass, + args, + appJar, + sparkHome, + sparkSubmitOpts) { + if (.Platform$OS.type == "unix") { + spark_submit_bin_name = "spark-submit" + } else { + spark_submit_bin_name = "spark-submit.cmd" + } + + if (sparkHome != "") { + spark_submit_bin <- file.path(sparkHome, "bin", spark_submit_bin_name) + } else { + spark_submit_bin <- spark_submit_bin_name + } + + # Since this function is only used while launching R shell using spark-submit, + # the format we need to construct is + # spark-submit --class + + combinedArgs <- paste("--class", mainClass, sparkSubmitOpts, appJar, args, sep = " ") + cat("Launching java with spark-submit command ", spark_submit_bin, " ", combinedArgs, "\n") + invisible(system2(spark_submit_bin, combinedArgs, wait = F)) +} diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index e39d2acf8e8a3..74d5837fb9384 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -357,10 +357,19 @@ object RRDD { sparkEnvirMap: JMap[Object, Object], sparkExecutorEnvMap: JMap[Object, Object]): JavaSparkContext = { - val sparkConf = new SparkConf().setMaster(master) - .setAppName(appName) + 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]) } diff --git a/sparkR b/sparkR index 01ed365fd4b15..916523d57f846 100755 --- a/sparkR +++ b/sparkR @@ -28,7 +28,7 @@ cat > /tmp/sparkR.profile << EOF Sys.setenv(NOAWT=1) .libPaths(c(paste(projecHome,"/lib", sep=""), .libPaths())) require(SparkR) - sc <- sparkR.init(Sys.getenv("MASTER", unset = "local")) + sc <- sparkR.init(Sys.getenv("MASTER", unset = "")) assign("sc", sc, envir=.GlobalEnv) cat("\n Welcome to SparkR!") cat("\n Spark context is available as sc\n") diff --git a/sparkR-submit b/sparkR-submit index 814625f145985..2c4d9bf376bd3 100755 --- a/sparkR-submit +++ b/sparkR-submit @@ -33,6 +33,26 @@ fi export R_PROFILE_USER="/tmp/sparkR.profile" +# Build up arguments list manually to preserve quotes and backslashes. +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" + +SPARKR_SUBMIT_ARGS="" +whitespace="[[:space:]]" +for i in "${SUBMISSION_OPTS[@]}" +do + if [[ $i =~ \" ]]; then i=$(echo $i | sed 's/\"/\\\"/g'); fi + if [[ $i =~ $whitespace ]]; then i=\"$i\"; fi + SPARKR_SUBMIT_ARGS="$SPARKR_SUBMIT_ARGS $i" +done +export SPARKR_SUBMIT_ARGS +export SPARKR_USE_SPARK_SUBMIT=1 + +NUM_APPLICATION_OPTS=${#APPLICATION_OPTS[@]} + +# If a R file is provided, directly run spark-submit. +if [[ $NUM_APPLICATION_OPTS -gt 0 && "${APPLICATION_OPTS[0]}" =~ \.R$ ]]; then + cat > /tmp/sparkR.profile << EOF .First <- function() { projecHome <- Sys.getenv("PROJECT_HOME") @@ -41,16 +61,30 @@ cat > /tmp/sparkR.profile << EOF } EOF -# Build up arguments list manually to preserve quotes and backslashes. -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" - -# If a R file is provided, directly run spark-submit. -if [[ "${APPLICATION_OPTS[0]}" =~ \.R$ ]]; then primary="${APPLICATION_OPTS[0]}" shift # Set the main class to SparkRRunner and add the primary R file to --files to make sure its copied to the cluster + echo "Running $SPARK_HOME/bin/spark-submit --class edu.berkeley.cs.amplab.sparkr.SparkRRunner --files $primary ${SUBMISSION_OPTS[@]} $SPARKR_JAR_FILE $primary" "${APPLICATION_OPTS[@]:1}" exec "$SPARK_HOME"/bin/spark-submit --class edu.berkeley.cs.amplab.sparkr.SparkRRunner --files "$primary" "${SUBMISSION_OPTS[@]}" "$SPARKR_JAR_FILE" "$primary" "${APPLICATION_OPTS[@]:1}" else - echo "sparkR-submit can only be used to run R programs. Please use sparkR to launch a shell." + + # If we don't have an R file to run, initialize context and run R +cat > /tmp/sparkR.profile << EOF +.First <- function() { + projecHome <- Sys.getenv("PROJECT_HOME") + Sys.setenv(NOAWT=1) + .libPaths(c(paste(projecHome,"/lib", sep=""), .libPaths())) + require(SparkR) + sc <- sparkR.init(Sys.getenv("MASTER", unset = "")) + assign("sc", sc, envir=.GlobalEnv) + cat("\n Welcome to SparkR!") + cat("\n Spark context is available as sc\n") +} +EOF + + # Add SPARKR_JAR, main class etc. to SPARKR_SUBMIT_ARGS + export SPARKR_SUBMIT_ARGS + + R + fi From 0cda231ef9bcd0866a0c8f20a966bb424c880954 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Mon, 16 Feb 2015 16:51:34 +0800 Subject: [PATCH 041/121] [SPARKR-153] phase 2: implement aggregateByKey() and foldByKey(). --- pkg/NAMESPACE | 2 + pkg/R/pairRDD.R | 82 +++++++++++++++++++++++++++++++++++ pkg/inst/tests/test_shuffle.R | 71 ++++++++++++++++++++++++++++++ pkg/man/aggregateByKey.Rd | 50 +++++++++++++++++++++ pkg/man/foldByKey.Rd | 38 ++++++++++++++++ 5 files changed, 243 insertions(+) create mode 100644 pkg/man/aggregateByKey.Rd create mode 100644 pkg/man/foldByKey.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 1fa1336674750..007d5662def28 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -2,6 +2,7 @@ exportClasses("RDD") exportClasses("Broadcast") exportMethods( + "aggregateByKey", "aggregateRDD", "cache", "checkpoint", @@ -19,6 +20,7 @@ exportMethods( "flatMap", "flatMapValues", "fold", + "foldByKey", "foreach", "foreachPartition", "fullOuterJoin", diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R index eef111d309263..6f2401adfcaa0 100644 --- a/pkg/R/pairRDD.R +++ b/pkg/R/pairRDD.R @@ -497,6 +497,88 @@ setMethod("combineByKey", 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 rdd 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. +#' @rdname aggregateByKey +#' @seealso foldByKey, combineByKey +#' @export +#' @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))) +#'} +setGeneric("aggregateByKey", + function(rdd, zeroValue, seqOp, combOp, numPartitions) { + standardGeneric("aggregateByKey") + }) + +#' @rdname aggregateByKey +#' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method +setMethod("aggregateByKey", + signature(rdd = "RDD", zeroValue = "ANY", seqOp = "ANY", + combOp = "ANY", numPartitions = "integer"), + function(rdd, zeroValue, seqOp, combOp, numPartitions) { + createCombiner <- function(v) { + do.call(seqOp, list(zeroValue, v)) + } + + combineByKey(rdd, 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 rdd 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. +#' @rdname foldByKey +#' @seealso aggregateByKey, combineByKey +#' @export +#' @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)) +#'} +setGeneric("foldByKey", + function(rdd, zeroValue, func, numPartitions) { + standardGeneric("foldByKey") + }) + +#' @rdname foldByKey +#' @aliases foldByKey,RDD,ANY,ANY,integer-method +setMethod("foldByKey", + signature(rdd = "RDD", zeroValue = "ANY", + func = "ANY", numPartitions = "integer"), + function(rdd, zeroValue, func, numPartitions) { + aggregateByKey(rdd, zeroValue, func, func, numPartitions) + }) + ############ Binary Functions ############# #' Join two RDDs diff --git a/pkg/inst/tests/test_shuffle.R b/pkg/inst/tests/test_shuffle.R index 3683287fd0423..8df9b439a08cd 100644 --- a/pkg/inst/tests/test_shuffle.R +++ b/pkg/inst/tests/test_shuffle.R @@ -70,6 +70,77 @@ test_that("combineByKey for doubles", { 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 } diff --git a/pkg/man/aggregateByKey.Rd b/pkg/man/aggregateByKey.Rd new file mode 100644 index 0000000000000..7bcbd5cc69d12 --- /dev/null +++ b/pkg/man/aggregateByKey.Rd @@ -0,0 +1,50 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{aggregateByKey} +\alias{aggregateByKey} +\alias{aggregateByKey,RDD,ANY,ANY,ANY,integer-method} +\title{Aggregate a pair RDD by each key.} +\usage{ +aggregateByKey(rdd, zeroValue, seqOp, combOp, numPartitions) + +\S4method{aggregateByKey}{RDD,ANY,ANY,ANY,integer}(rdd, zeroValue, seqOp, + combOp, numPartitions) +} +\arguments{ +\item{rdd}{An RDD.} + +\item{zeroValue}{A neutral "zero value".} + +\item{seqOp}{A function to aggregate the values of each key. It may return +a different result type from the type of the values.} + +\item{combOp}{A function to aggregate results of seqOp.} +} +\value{ +An RDD containing the aggregation result. +} +\description{ +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. +} +\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))) +} +} +\seealso{ +foldByKey, combineByKey +} + diff --git a/pkg/man/foldByKey.Rd b/pkg/man/foldByKey.Rd new file mode 100644 index 0000000000000..a2822c51a8ff4 --- /dev/null +++ b/pkg/man/foldByKey.Rd @@ -0,0 +1,38 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{foldByKey} +\alias{foldByKey} +\alias{foldByKey,RDD,ANY,ANY,integer-method} +\title{Fold a pair RDD by each key.} +\usage{ +foldByKey(rdd, zeroValue, func, numPartitions) + +\S4method{foldByKey}{RDD,ANY,ANY,integer}(rdd, zeroValue, func, numPartitions) +} +\arguments{ +\item{rdd}{An RDD.} + +\item{zeroValue}{A neutral "zero value".} + +\item{func}{An associative function for folding values of each key.} +} +\value{ +An RDD containing the aggregation result. +} +\description{ +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.). +} +\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)) +} +} +\seealso{ +aggregateByKey, combineByKey +} + From 5292be71de95b4861a8eb31f59752ef949e98d2e Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 16 Feb 2015 16:05:11 -0500 Subject: [PATCH 042/121] Adds support of pipeRDD(). --- pkg/NAMESPACE | 1 + pkg/R/RDD.R | 37 +++++++++++++++++++++++++++++++++++++ pkg/inst/tests/test_rdd.R | 12 ++++++++++++ pkg/man/pipeRDD.Rd | 34 ++++++++++++++++++++++++++++++++++ 4 files changed, 84 insertions(+) create mode 100644 pkg/man/pipeRDD.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 1fa1336674750..dbe05ea3f8df2 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -41,6 +41,7 @@ exportMethods( "numPartitions", "partitionBy", "persist", + "pipeRDD", "reduce", "reduceByKey", "reduceByKeyLocally", diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index c54abae2666a2..8af5d3608f295 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1275,6 +1275,43 @@ setMethod("aggregateRDD", Reduce(combOp, partitionList, zeroValue) }) +#' Pipes elements to a forked externel process. +#' +#' The same as 'pipe()' in Spark. +#' +#' @param rdd The RDD whose elements are piped to the forked externel process. +#' @param command The command to fork an externel process. +#' @param env A named list to set environment variables of the externel process. +#' @return A new RDD created by piping all elements to a forked externel process. +#' @rdname pipeRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' collect(pipeRDD(rdd, "more") +#' Output: c("1", "2", ..., "10") +#'} +setGeneric("pipeRDD", function(rdd, command, env = list()) { + standardGeneric("pipeRDD") +}) + +#' @rdname pipeRDD +#' @aliases pipeRDD,RDD,character-method +setMethod("pipeRDD", + signature(rdd = "RDD", command = "character"), + function(rdd, command, env = list()) { + func <- function(part) { + trim.trailing.func <- function(x) { + sub("[\r\n]*$", "", toString(x)) + } + input <- unlist(lapply(part, trim.trailing.func)) + res <- system2(command, stdout = TRUE, input = input, env = env) + lapply(res, trim.trailing.func) + } + lapplyPartition(rdd, func) + }) + # TODO: Consider caching the name in the RDD's environment #' Return an RDD's name. #' diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 89d7890fbf685..fa2a2959c6913 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -336,6 +336,18 @@ test_that("values() on RDDs", { 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) + + 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("join() on pairwise RDDs", { rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) diff --git a/pkg/man/pipeRDD.Rd b/pkg/man/pipeRDD.Rd new file mode 100644 index 0000000000000..e930fe0a65a84 --- /dev/null +++ b/pkg/man/pipeRDD.Rd @@ -0,0 +1,34 @@ +% Generated by roxygen2 (4.1.0): do not edit by hand +% Please edit documentation in R/RDD.R +\docType{methods} +\name{pipeRDD} +\alias{pipeRDD} +\alias{pipeRDD,RDD,character-method} +\title{Pipes elements to a forked externel process.} +\usage{ +pipeRDD(rdd, command, env = list()) + +\S4method{pipeRDD}{RDD,character}(rdd, command, env = list()) +} +\arguments{ +\item{rdd}{The RDD whose elements are piped to the forked externel process.} + +\item{command}{The command to fork an externel process.} + +\item{env}{A named list to set environment variables of the externel process.} +} +\value{ +A new RDD created by piping all elements to a forked externel process. +} +\description{ +The same as 'pipe()' in Spark. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, 1:10) +collect(pipeRDD(rdd, "more") +Output: c("1", "2", ..., "10") +} +} + From 1f5a6ac052a91784f4343d7ac0d40fca88ce1cc0 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Tue, 17 Feb 2015 22:57:37 -0500 Subject: [PATCH 043/121] fixed comments --- pkg/R/RDD.R | 10 +++++----- pkg/inst/tests/test_rdd.R | 5 +++++ pkg/man/pipeRDD.Rd | 10 +++++----- 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 8af5d3608f295..2629db7d128c8 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1275,14 +1275,14 @@ setMethod("aggregateRDD", Reduce(combOp, partitionList, zeroValue) }) -#' Pipes elements to a forked externel process. +#' Pipes elements to a forked external process. #' #' The same as 'pipe()' in Spark. #' -#' @param rdd The RDD whose elements are piped to the forked externel process. -#' @param command The command to fork an externel process. -#' @param env A named list to set environment variables of the externel process. -#' @return A new RDD created by piping all elements to a forked externel process. +#' @param rdd The RDD whose elements are piped to the forked external process. +#' @param command The command to fork an external process. +#' @param env A named list to set environment variables of the external process. +#' @return A new RDD created by piping all elements to a forked external process. #' @rdname pipeRDD #' @export #' @examples diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index fa2a2959c6913..f0c00d71d076c 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -341,6 +341,11 @@ test_that("pipeRDD() on RDDs", { 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")) diff --git a/pkg/man/pipeRDD.Rd b/pkg/man/pipeRDD.Rd index e930fe0a65a84..0964d3415ecb8 100644 --- a/pkg/man/pipeRDD.Rd +++ b/pkg/man/pipeRDD.Rd @@ -4,21 +4,21 @@ \name{pipeRDD} \alias{pipeRDD} \alias{pipeRDD,RDD,character-method} -\title{Pipes elements to a forked externel process.} +\title{Pipes elements to a forked external process.} \usage{ pipeRDD(rdd, command, env = list()) \S4method{pipeRDD}{RDD,character}(rdd, command, env = list()) } \arguments{ -\item{rdd}{The RDD whose elements are piped to the forked externel process.} +\item{rdd}{The RDD whose elements are piped to the forked external process.} -\item{command}{The command to fork an externel process.} +\item{command}{The command to fork an external process.} -\item{env}{A named list to set environment variables of the externel process.} +\item{env}{A named list to set environment variables of the external process.} } \value{ -A new RDD created by piping all elements to a forked externel process. +A new RDD created by piping all elements to a forked external process. } \description{ The same as 'pipe()' in Spark. From 86fc639a8ddb8e872b1f5cd7392ebd3f896d22c8 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 17 Feb 2015 23:29:51 -0800 Subject: [PATCH 044/121] Move sparkR-submit into pkg/inst --- sparkR-submit => pkg/inst/sparkR-submit | 34 ++----------------------- 1 file changed, 2 insertions(+), 32 deletions(-) rename sparkR-submit => pkg/inst/sparkR-submit (65%) diff --git a/sparkR-submit b/pkg/inst/sparkR-submit similarity index 65% rename from sparkR-submit rename to pkg/inst/sparkR-submit index 2c4d9bf376bd3..a5311fd73dc92 100755 --- a/sparkR-submit +++ b/pkg/inst/sparkR-submit @@ -7,7 +7,7 @@ FWDIR="$(cd `dirname $0`; pwd)" export PROJECT_HOME="$FWDIR" -export SPARKR_JAR_FILE="$FWDIR/lib/SparkR/sparkr-assembly-0.1.jar" +export SPARKR_JAR_FILE="$FWDIR/sparkr-assembly-0.1.jar" # Exit if the user hasn't set SPARK_HOME if [ ! -f "$SPARK_HOME/bin/spark-submit" ]; then @@ -27,12 +27,6 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage fi - -# Add SparkR to .libPaths -# If we are running an R program, only set libPaths and use Rscript - -export R_PROFILE_USER="/tmp/sparkR.profile" - # Build up arguments list manually to preserve quotes and backslashes. SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" @@ -53,14 +47,6 @@ NUM_APPLICATION_OPTS=${#APPLICATION_OPTS[@]} # If a R file is provided, directly run spark-submit. if [[ $NUM_APPLICATION_OPTS -gt 0 && "${APPLICATION_OPTS[0]}" =~ \.R$ ]]; then -cat > /tmp/sparkR.profile << EOF - .First <- function() { - projecHome <- Sys.getenv("PROJECT_HOME") - .libPaths(c(paste(projecHome,"/lib", sep=""), .libPaths())) - Sys.setenv(NOAWT=1) -} -EOF - primary="${APPLICATION_OPTS[0]}" shift # Set the main class to SparkRRunner and add the primary R file to --files to make sure its copied to the cluster @@ -68,23 +54,7 @@ EOF exec "$SPARK_HOME"/bin/spark-submit --class edu.berkeley.cs.amplab.sparkr.SparkRRunner --files "$primary" "${SUBMISSION_OPTS[@]}" "$SPARKR_JAR_FILE" "$primary" "${APPLICATION_OPTS[@]:1}" else - # If we don't have an R file to run, initialize context and run R -cat > /tmp/sparkR.profile << EOF -.First <- function() { - projecHome <- Sys.getenv("PROJECT_HOME") - Sys.setenv(NOAWT=1) - .libPaths(c(paste(projecHome,"/lib", sep=""), .libPaths())) - require(SparkR) - sc <- sparkR.init(Sys.getenv("MASTER", unset = "")) - assign("sc", sc, envir=.GlobalEnv) - cat("\n Welcome to SparkR!") - cat("\n Spark context is available as sc\n") -} -EOF - - # Add SPARKR_JAR, main class etc. to SPARKR_SUBMIT_ARGS - export SPARKR_SUBMIT_ARGS - + # If we don't have an R file to run, run R shell R fi From 22a19ac516be71b585246d0ebba76f20edc31995 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 18 Feb 2015 14:34:32 -0800 Subject: [PATCH 045/121] Use a semaphore to wait for backend to initalize Also pick a random port to avoid collisions --- .../berkeley/cs/amplab/sparkr/SparkRRunner.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala index 4ccf247e0eeb1..1a4b7355b4f01 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala @@ -2,6 +2,7 @@ package edu.berkeley.cs.amplab.sparkr import java.io._ import java.net.URI +import java.util.concurrent.Semaphore import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ @@ -17,8 +18,12 @@ object SparkRRunner { val rFile = args(0) val otherArgs = args.slice(1, args.length) + + // Pick a non-privileged port + val randomPort = scala.util.Random.nextInt(65536 - 1024) + 1024 + // TODO: Can we get this from SparkConf ? - val sparkRBackendPort = sys.env.getOrElse("SPARKR_BACKEND_PORT", "12345").toInt + val sparkRBackendPort = sys.env.getOrElse("SPARKR_BACKEND_PORT", randomPort.toString).toInt val rCommand = "Rscript" // Check if the file path exists. @@ -35,8 +40,12 @@ object SparkRRunner { // Java system properties etc. val sparkRBackend = new SparkRBackend() val sparkRBackendThread = new Thread() { + val finishedInit = new Semaphore(1) + finishedInit.acquire() + override def run() { sparkRBackend.init(sparkRBackendPort) + finishedInit.release() sparkRBackend.run() } @@ -46,6 +55,8 @@ object SparkRRunner { } sparkRBackendThread.start() + // Wait for SparkRBackend initialization to finish + sparkRBackendThread.finishedInit.acquire() // Launch R val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) From bc04cf439fe642c7ebbc7baededdacf89c4abf89 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 18 Feb 2015 18:24:44 -0800 Subject: [PATCH 046/121] Use SPARKR_BACKEND_PORT in sparkR.R as default Change SparkRRunner to use EXISTING_SPARKR_BACKEND_PORT to differentiate between the two --- pkg/R/sparkR.R | 5 +++-- .../scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala | 7 ++----- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index f4a2582c22041..02ae0c6ae490a 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -90,7 +90,7 @@ sparkR.init <- function( sparkExecutorEnv = list(), sparkJars = "", sparkRLibDir = "", - sparkRBackendPort = 12345) { + sparkRBackendPort = as.integer(Sys.getenv("SPARKR_BACKEND_PORT", "12345"))) { 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") @@ -106,7 +106,8 @@ sparkR.init <- function( if (yarn_conf_dir != "") { cp <- paste(cp, yarn_conf_dir, sep = ":") } - sparkRExistingPort <- Sys.getenv("SPARKR_BACKEND_PORT", "") + + sparkRExistingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "") if (sparkRExistingPort != "") { sparkRBackendPort <- sparkRExistingPort } else { diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala index 1a4b7355b4f01..6b4e781494464 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala @@ -19,11 +19,8 @@ object SparkRRunner { val otherArgs = args.slice(1, args.length) - // Pick a non-privileged port - val randomPort = scala.util.Random.nextInt(65536 - 1024) + 1024 - // TODO: Can we get this from SparkConf ? - val sparkRBackendPort = sys.env.getOrElse("SPARKR_BACKEND_PORT", randomPort.toString).toInt + val sparkRBackendPort = sys.env.getOrElse("SPARKR_BACKEND_PORT", "12345").toInt val rCommand = "Rscript" // Check if the file path exists. @@ -61,7 +58,7 @@ object SparkRRunner { // Launch R val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) val env = builder.environment() - env.put("SPARKR_BACKEND_PORT", "" + sparkRBackendPort) + env.put("EXISTING_SPARKR_BACKEND_PORT", "" + sparkRBackendPort) builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize val process = builder.start() From 749e2d08831b85fe70603a985e3e28f49def51fb Mon Sep 17 00:00:00 2001 From: Hossein Date: Wed, 18 Feb 2015 22:56:25 -0800 Subject: [PATCH 047/121] Updated README --- README.md | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index 795b0a1477305..7d0f1195a8e72 100644 --- a/README.md +++ b/README.md @@ -10,15 +10,9 @@ R. ### Requirements SparkR requires Scala 2.10 and Spark version >= 0.9.0. Current build by default uses -Apache Spark 1.1.0. You can also build SparkR against a +Apache Spark 1.3.0. You can also build SparkR against a different Spark version (>= 0.9.0) by modifying `pkg/src/build.sbt`. -SparkR also requires the R package `rJava` to be installed. To install `rJava`, -you can run the following command in R: - - install.packages("rJava") - - ### Package installation To develop SparkR, you can build the scala package and the R package using @@ -29,7 +23,7 @@ If you wish to try out the package directly from github, you can use [`install_g library(devtools) install_github("amplab-extras/SparkR-pkg", subdir="pkg") -SparkR by default uses Apache Spark 1.1.0. You can switch to a different Spark +SparkR by default uses Apache Spark 1.3.0. You can switch to a different Spark version by setting the environment variable `SPARK_VERSION`. For example, to use Apache Spark 1.2.0, you can run @@ -97,7 +91,7 @@ To run one of them, use `./sparkR `. For example: ./sparkR examples/pi.R local[2] -You can also run the unit-tests for SparkR by running +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. ./run-tests.sh @@ -110,7 +104,7 @@ Instructions for running SparkR on EC2 can be found in the Currently, SparkR supports running on YARN with the `yarn-client` mode. These steps show how to build SparkR with YARN support and run SparkR programs on a YARN cluster: ``` -# assumes Java, R, rJava, yarn, spark etc. are installed on the whole cluster. +# assumes Java, R, yarn, spark etc. are installed on the whole cluster. cd SparkR-pkg/ USE_YARN=1 SPARK_YARN_VERSION=2.4.0 SPARK_HADOOP_VERSION=2.4.0 ./install-dev.sh ``` From 4cd7d3f8966102742d75cc60a9e93308b44026cf Mon Sep 17 00:00:00 2001 From: lythesia Date: Thu, 19 Feb 2015 21:51:44 +0800 Subject: [PATCH 048/121] retry backend connection --- pkg/R/sparkR.R | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 2218170fef104..7d42842dc8847 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -110,9 +110,19 @@ sparkR.init <- function( mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", args = as.character(sparkRBackendPort), javaOpts = paste("-Xmx", sparkMem, sep = "")) - Sys.sleep(2) # Wait for backend to come up + + cat("Waiting JVM bring up ...\n") + while(TRUE) { + if(!connExists(.sparkREnv)) { + Sys.sleep(1) + cat(".") + connectBackend("localhost", sparkRBackendPort) # Connect to it + } else { + cat(" ok.\n") + break + } + } .sparkREnv$sparkRBackendPort <- sparkRBackendPort - connectBackend("localhost", sparkRBackendPort) # Connect to it if (nchar(sparkHome) != 0) { sparkHome <- normalizePath(sparkHome) From ba2b72bf56a8fd964e210ea34c320e3cb7eb5436 Mon Sep 17 00:00:00 2001 From: Hossein Date: Thu, 19 Feb 2015 10:35:07 -0800 Subject: [PATCH 049/121] Spark 1.1.0 is default --- README.md | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 7d0f1195a8e72..6d6b097222ade 100644 --- a/README.md +++ b/README.md @@ -10,7 +10,7 @@ R. ### Requirements SparkR requires Scala 2.10 and Spark version >= 0.9.0. Current build by default uses -Apache Spark 1.3.0. You can also build SparkR against a +Apache Spark 1.1.0. You can also build SparkR against a different Spark version (>= 0.9.0) by modifying `pkg/src/build.sbt`. ### Package installation @@ -23,11 +23,11 @@ If you wish to try out the package directly from github, you can use [`install_g library(devtools) install_github("amplab-extras/SparkR-pkg", subdir="pkg") -SparkR by default uses Apache Spark 1.3.0. You can switch to a different Spark +SparkR by default uses Apache Spark 1.1.0. You can switch to a different Spark version by setting the environment variable `SPARK_VERSION`. For example, to -use Apache Spark 1.2.0, you can run +use Apache Spark 1.3.0, you can run - SPARK_VERSION=1.2.0 ./install-dev.sh + SPARK_VERSION=1.3.0 ./install-dev.sh SparkR by default links to Hadoop 1.0.4. To use SparkR with other Hadoop versions, you will need to rebuild SparkR with the same version that [Spark is @@ -91,8 +91,9 @@ To run one of them, use `./sparkR `. For example: ./sparkR examples/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. +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")' ./run-tests.sh ## Running on EC2 From e6ad12d93852674c63fc69a8301963a6c395eb18 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 19 Feb 2015 12:35:45 -0800 Subject: [PATCH 050/121] Update comment describing sparkR-submit --- pkg/inst/sparkR-submit | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/inst/sparkR-submit b/pkg/inst/sparkR-submit index a5311fd73dc92..d8d0873bcf245 100755 --- a/pkg/inst/sparkR-submit +++ b/pkg/inst/sparkR-submit @@ -1,7 +1,7 @@ #!/bin/bash -# To use sparkR-submit, we assume the SparkR package in yarn-cluster mode -# we assume that it has been installed to a standard location using -# R CMD INSTALL pkg/ +# This script launches SparkR through spark-submit. This accepts +# the same set of options as spark-submit and requires SPARK_HOME +# to be set. FWDIR="$(cd `dirname $0`; pwd)" From f9268d922e24b88643a74b4a4c6a03e7024525ec Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 19 Feb 2015 15:58:17 -0800 Subject: [PATCH 051/121] Fix code review comments --- pkg/R/sparkRClient.R | 12 ++++++------ .../edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala | 5 ++--- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/pkg/R/sparkRClient.R b/pkg/R/sparkRClient.R index 4fff9634f4674..99c7aa5c5727c 100644 --- a/pkg/R/sparkRClient.R +++ b/pkg/R/sparkRClient.R @@ -47,15 +47,15 @@ launchBackendSparkSubmit <- function( sparkHome, sparkSubmitOpts) { if (.Platform$OS.type == "unix") { - spark_submit_bin_name = "spark-submit" + sparkSubmitBinName = "spark-submit" } else { - spark_submit_bin_name = "spark-submit.cmd" + sparkSubmitBinName = "spark-submit.cmd" } if (sparkHome != "") { - spark_submit_bin <- file.path(sparkHome, "bin", spark_submit_bin_name) + sparkSubmitBin <- file.path(sparkHome, "bin", sparkSubmitBinName) } else { - spark_submit_bin <- spark_submit_bin_name + sparkSubmitBin <- sparkSubmitBinName } # Since this function is only used while launching R shell using spark-submit, @@ -63,6 +63,6 @@ launchBackendSparkSubmit <- function( # spark-submit --class combinedArgs <- paste("--class", mainClass, sparkSubmitOpts, appJar, args, sep = " ") - cat("Launching java with spark-submit command ", spark_submit_bin, " ", combinedArgs, "\n") - invisible(system2(spark_submit_bin, combinedArgs, wait = F)) + cat("Launching java with spark-submit command ", sparkSubmitBin, " ", combinedArgs, "\n") + invisible(system2(sparkSubmitBin, combinedArgs, wait = F)) } diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala index 6b4e781494464..6c4607574c4d1 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala @@ -37,8 +37,7 @@ object SparkRRunner { // Java system properties etc. val sparkRBackend = new SparkRBackend() val sparkRBackendThread = new Thread() { - val finishedInit = new Semaphore(1) - finishedInit.acquire() + val finishedInit = new Semaphore(0) override def run() { sparkRBackend.init(sparkRBackendPort) @@ -58,7 +57,7 @@ object SparkRRunner { // Launch R val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) val env = builder.environment() - env.put("EXISTING_SPARKR_BACKEND_PORT", "" + sparkRBackendPort) + env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString) builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize val process = builder.start() From 88bf97f48dc0de9464f5e771d99ddaac8d86617c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 19 Feb 2015 16:45:03 -0800 Subject: [PATCH 052/121] Create SparkContext for R shell launch --- pkg/inst/sparkR-submit | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/pkg/inst/sparkR-submit b/pkg/inst/sparkR-submit index d8d0873bcf245..9c451ab8e3712 100755 --- a/pkg/inst/sparkR-submit +++ b/pkg/inst/sparkR-submit @@ -54,7 +54,21 @@ if [[ $NUM_APPLICATION_OPTS -gt 0 && "${APPLICATION_OPTS[0]}" =~ \.R$ ]]; then exec "$SPARK_HOME"/bin/spark-submit --class edu.berkeley.cs.amplab.sparkr.SparkRRunner --files "$primary" "${SUBMISSION_OPTS[@]}" "$SPARKR_JAR_FILE" "$primary" "${APPLICATION_OPTS[@]:1}" else + export R_PROFILE_USER="/tmp/sparkR.profile" + # If we don't have an R file to run, run R shell +cat > /tmp/sparkR.profile << EOF +.First <- function() { + projecHome <- Sys.getenv("PROJECT_HOME") + Sys.setenv(NOAWT=1) + .libPaths(c(paste(projecHome,"/..", sep=""), .libPaths())) + require(SparkR) + sc <- sparkR.init() + assign("sc", sc, envir=.GlobalEnv) + cat("\n Welcome to SparkR!") + cat("\n Spark context is available as sc\n") +} +EOF R fi From 179ab38810e42917db332c385ada9d27da42f6bb Mon Sep 17 00:00:00 2001 From: lythesia Date: Fri, 20 Feb 2015 12:02:47 +0800 Subject: [PATCH 053/121] add try counts and increase time interval --- pkg/R/sparkR.R | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 7d42842dc8847..16eb825f08a57 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -90,7 +90,8 @@ sparkR.init <- function( sparkExecutorEnv = list(), sparkJars = "", sparkRLibDir = "", - sparkRBackendPort = 12345) { + sparkRBackendPort = 12345, + sparkRRetryCount = 6) { 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") @@ -111,18 +112,28 @@ sparkR.init <- function( args = as.character(sparkRBackendPort), javaOpts = paste("-Xmx", sparkMem, sep = "")) - cat("Waiting JVM bring up ...\n") - while(TRUE) { + .sparkREnv$sparkRBackendPort <- sparkRBackendPort + cat("Waiting for JVM to come up...\n") + tries <- 0 + while(tries < sparkRRetryCount) { if(!connExists(.sparkREnv)) { - Sys.sleep(1) - cat(".") - connectBackend("localhost", sparkRBackendPort) # Connect to it + Sys.sleep(2 ^ tries) + tryCatch({ + connectBackend("localhost", .sparkREnv$sparkRBackendPort) + }, error = function(err) { + cat("Error in Connection, retrying...\n") + }, warning = function(war) { + cat("No Connection Found, retrying...\n") + }) + tries <- tries + 1 } else { - cat(" ok.\n") + cat("Connect ok.\n") break } } - .sparkREnv$sparkRBackendPort <- sparkRBackendPort + if (tries == sparkRRetryCount) { + stop(sprintf("Failed to connect JVM after %d tries.\n", sparkRRetryCount)) + } if (nchar(sparkHome) != 0) { sparkHome <- normalizePath(sparkHome) From 910e3bef359dd74ce53bf39009004f1161ddc8f8 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 20 Feb 2015 10:41:54 -0800 Subject: [PATCH 054/121] Add a timeout for initialization Also move sparkRBackend.stop into a finally block --- .../cs/amplab/sparkr/SparkRRunner.scala | 37 ++++++++++++------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala index 6c4607574c4d1..fb356f89b2d1d 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala @@ -3,6 +3,7 @@ package edu.berkeley.cs.amplab.sparkr import java.io._ import java.net.URI import java.util.concurrent.Semaphore +import java.util.concurrent.TimeUnit import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ @@ -19,6 +20,8 @@ object SparkRRunner { 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 // TODO: Can we get this from SparkConf ? val sparkRBackendPort = sys.env.getOrElse("SPARKR_BACKEND_PORT", "12345").toInt val rCommand = "Rscript" @@ -52,20 +55,26 @@ object SparkRRunner { sparkRBackendThread.start() // Wait for SparkRBackend initialization to finish - sparkRBackendThread.finishedInit.acquire() - - // Launch R - val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) - val env = builder.environment() - env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString) - builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize - val process = builder.start() - - new RedirectThread(process.getInputStream, System.out, "redirect output").start() - - val returnCode = process.waitFor() - sparkRBackendThread.stopBackend() - System.exit(returnCode) + if (sparkRBackendThread.finishedInit.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) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + + process.waitFor() + } finally { + sparkRBackendThread.stopBackend() + } + System.exit(returnCode) + } else { + System.err.println("SparkR backend did not initialize in " + backendTimeout + " seconds") + System.exit(-1) + } } private class RedirectThread( From 57e005baf4a7c6a550eb81a169efe1651ca1effe Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sat, 21 Feb 2015 21:37:25 -0500 Subject: [PATCH 055/121] Revert "fix tests." This reverts commit fd836db89e785843be4fcd7868215770e00ffcd4. --- pkg/inst/tests/test_rdd.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index dbebe6fa979c8..0140c24cfec10 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -89,7 +89,7 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { actual <- collect(rdd2) expected <- list(24, 24, 24, 24, 24, 168, 170, 172, 174, 176) - expect_equal(actual, expected) + expect_equal(actual, expected)) }) test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { From 8e4b3da2d89631a43eb42bd7c7a3e9fcbf67ec13 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sat, 21 Feb 2015 21:38:50 -0500 Subject: [PATCH 056/121] Revert "More docs" This reverts commit 24a7f13f272484c3b781c84adfc19c5bc17d2859. --- pkg/inst/tests/test_rdd.R | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 0140c24cfec10..bcf4cd4e1bc51 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -87,9 +87,9 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { }) 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)) + expected <- + expect_equal(actual, list(24, 24, 24, 24, 24, + 168, 170, 172, 174, 176)) }) test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { From f8ef0ab4be9d37cc27e3e9720417d8d19e683c53 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sat, 21 Feb 2015 21:38:57 -0500 Subject: [PATCH 057/121] Revert "More docs" This reverts commit a46516506cca8d625d983edd264330476c8774ef. --- pkg/R/utils.R | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index a16a0204720eb..bf4fa48ee5de4 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -268,8 +268,6 @@ joinTaggedList <- function(tagged_list, cnull) { mergeCompactLists(lists[[1]], lists[[2]]) } -# Recursively examine variables in functions to decide if their values should -# be included in the new function environment. closure.process <- function( node, func, @@ -283,7 +281,7 @@ closure.process <- function( } else if(nlen == 1) { if(mode(node) == 'name') { cnode <- as.character(node) - if(!cnode %in% names(as.list(args(func)))) { # Not a function argument + if(!cnode %in% names(as.list(args(func)))) { func.env <- environment(func) if(exists(cnode, envir=func.env, inherits=F)) { assign(cnode, get(cnode, envir=func.env), envir=env) @@ -293,7 +291,6 @@ closure.process <- function( } } -# Get function dependencies. clean.closure <- function( func, env From fddb9ccf6070ec89c8acf177e63579e2e8318137 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sat, 21 Feb 2015 21:39:00 -0500 Subject: [PATCH 058/121] Revert "add docs" This reverts commit 6ad4fc36bde14e9cdbab4224cec466b9e2aaf73f. --- pkg/R/RDD.R | 22 ++++------------------ pkg/inst/tests/test_rdd.R | 5 +---- pkg/inst/worker/worker.R | 38 +++++++++++++++----------------------- 3 files changed, 20 insertions(+), 45 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 750e0b8eec399..6b101714f9d71 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -18,8 +18,7 @@ setClass("RDD", setClass("PipelinedRDD", slots = list(prev = "RDD", func = "function", - # Accumulator to store function lineage - funcAccum = "environment", + funcAccum = "environment", prev_jrdd = "jobj"), contains = "RDD") @@ -55,7 +54,6 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) # prev_serialized is used during the delayed computation of JRDD in getJRDD .Object@prev <- prev - # We use funcAccum to store the lineage of function closures in a PipelinedRDD. .Object@funcAccum <- initAccumulator() isPipelinable <- function(rdd) { @@ -127,9 +125,7 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - # Sorry, I use "depsBin" to serialize the funcAccum for now. - # Instead of serializing the entire environment, we can serialize - # a sequence of functions (with its closure). + depsBin <- getDependencies(computeFunc) depsBin <- serialize(rdd@funcAccum, NULL, ascii = TRUE) prev_jrdd <- rdd@prev_jrdd @@ -536,12 +532,9 @@ setMethod("countByKey", setMethod("lapply", signature(X = "RDD", FUN = "function"), function(X, FUN) { - # Creats a new the closure (environment) for the FUN to capture - # free variables. env.fun <- new.env(parent=.GlobalEnv) - # need to add free variables to env.fun environment clean.closure(FUN, env.fun) - # FUN now have new environment env.fun, with all values it needs. + # need to add ref variables to en.fun environment environment(FUN) <- env.fun func <- function(split, iterator) { @@ -1305,15 +1298,8 @@ setMethod("partitionBy", #} depsBinArr <- getDependencies(partitionFunc) - env.fun <- new.env(parent=.GlobalEnv) - clean.closure(partitionFunc, env.fun) - # need to add ref variables to en.fun environment - environment(partitionFunc) <- env.fun - funcAccum <- initAccumulator() - addItemToAccumulator(funcAccum, partitionFunc) - depsBinArr <- serialize(funcAccum, NULL, ascii = TRUE) - serializedHashFuncBytes <- serialize("computeFunc", #as.character(substitute(partitionFunc)), + serializedHashFuncBytes <- serialize(as.character(substitute(partitionFunc)), connection = NULL, ascii = TRUE) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index bcf4cd4e1bc51..7c2599f51e7e5 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -86,10 +86,7 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { part <- as.list(unlist(part) * split + i) }) rdd2 <- lapply(rdd2, function(x) x + x) - actual <- collect(rdd2) - expected <- - expect_equal(actual, list(24, 24, 24, 24, 24, - 168, 170, 172, 174, 176)) + collect(rdd2) }) test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index 40fc9a8881645..136db6960d88b 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -41,11 +41,22 @@ for (pkg in packageNames) { suppressPackageStartupMessages(require(as.character(pkg), character.only=TRUE)) } -# Read the function lineage. +# read function dependencies depsLen <- SparkR:::readInt(inputCon) -if (depsLen > 0) { - # function lineage stored in an Accumulator - funcAccum <- unserialize(SparkR:::readRawLen(inputCon, depsLen)) +funcAccum <- unserialize(SparkR:::readRawLen(inputCon, depsLen)) +# testfilename <- "/home//ubuntu/Desktop/testfile" +# cat(typeof(funcAccum), '\n', file = testfilename) +# cat(funcAccum$size, '\n', file = testfilename, append = T) +# cat(typeof(funcAccum$data[[1]]), '\n', file = testfilename, append = T) +# cat(as.character(body(funcAccum$data[[1]])), file = testfilename, append = T) +if (funcAccum$size > 0) { + computeFunc <- function(split, part) { + res <- part + for (i in 1:funcAccum$counter) { + res <- funcAccum$data[[i]](split, res) + } + res + } } # Read and set broadcast variables @@ -62,25 +73,6 @@ if (numBroadcastVars > 0) { # as number of partitions to create. numPartitions <- SparkR:::readInt(inputCon) -# Build up the execFunction from the Accumulator -if (exists("funcAccum")) { - if (funcAccum$size > 0) { - if (numPartitions == -1) { - # Regular RDDs: build nested functions. - computeFunc <- function(split, part) { - res <- part - for (i in 1:funcAccum$counter) { - res <- funcAccum$data[[i]](split, res) - } - res - } - } else { - # Pairwise RDDs. - computeFunc <- funcAccum$data[[1]] - } - } -} - isEmpty <- SparkR:::readInt(inputCon) if (isEmpty != 0) { From 8be02de8416dcdccbf805d437eadfa30207f9940 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sat, 21 Feb 2015 21:39:01 -0500 Subject: [PATCH 059/121] Revert "loop 1-12 test pass." This reverts commit 193f5fe526eaf4af620fd625d754130a13bd06f6. --- pkg/R/RDD.R | 18 +---------------- pkg/R/utils.R | 42 ---------------------------------------- pkg/inst/worker/worker.R | 18 ++++------------- 3 files changed, 5 insertions(+), 73 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 6b101714f9d71..66e616f252533 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -18,7 +18,6 @@ setClass("RDD", setClass("PipelinedRDD", slots = list(prev = "RDD", func = "function", - funcAccum = "environment", prev_jrdd = "jobj"), contains = "RDD") @@ -53,8 +52,6 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) # NOTE: We use prev_serialized to track if prev_jrdd is serialized # prev_serialized is used during the delayed computation of JRDD in getJRDD .Object@prev <- prev - - .Object@funcAccum <- initAccumulator() isPipelinable <- function(rdd) { e <- rdd@env @@ -73,12 +70,10 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) func(split, prev@func(split, iterator)) } .Object@func <- pipelinedFunc - .Object@funcAccum <- cloneAccumulator(prev@funcAccum) .Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline # Get if the prev_jrdd was serialized from the parent RDD .Object@env$prev_serialized <- prev@env$prev_serialized } - addItemToAccumulator(.Object@funcAccum, func) .Object }) @@ -126,7 +121,6 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), function(name) { get(name, .broadcastNames) }) depsBin <- getDependencies(computeFunc) - depsBin <- serialize(rdd@funcAccum, NULL, ascii = TRUE) prev_jrdd <- rdd@prev_jrdd @@ -532,11 +526,6 @@ setMethod("countByKey", setMethod("lapply", signature(X = "RDD", FUN = "function"), function(X, FUN) { - env.fun <- new.env(parent=.GlobalEnv) - clean.closure(FUN, env.fun) - # need to add ref variables to en.fun environment - environment(FUN) <- env.fun - func <- function(split, iterator) { lapply(iterator, FUN) } @@ -657,12 +646,7 @@ setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { - env.fun <- new.env(parent=.GlobalEnv) - clean.closure(FUN, env.fun) - # need to add ref variables to en.fun environment - environment(FUN) <- env.fun - - closureCapturingFunc <- function(split, part) { + closureCapturingFunc <- function(split, part) { FUN(split, part) } PipelinedRDD(X, closureCapturingFunc) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index bf4fa48ee5de4..7c2a153b8b55d 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -195,14 +195,6 @@ initAccumulator <- function() { acc } -cloneAccumulator <- function(acc) { - newAcc <- initAccumulator() - newAcc$size <- acc$size - newAcc$data <- acc$data - newAcc$counter <- acc$counter - newAcc -} - # Utility function to sort a list of key value pairs # Used in unit tests sortKeyValueList <- function(kv_list) { @@ -267,37 +259,3 @@ joinTaggedList <- function(tagged_list, cnull) { lists <- genCompactLists(tagged_list, cnull) mergeCompactLists(lists[[1]], lists[[2]]) } - -closure.process <- function( - node, - func, - env -) { - nlen <- length(node) - if(nlen > 1) { - for(i in 1:nlen) { - closure.process(node[[i]], func, env) - } - } else if(nlen == 1) { - if(mode(node) == 'name') { - cnode <- as.character(node) - if(!cnode %in% names(as.list(args(func)))) { - func.env <- environment(func) - if(exists(cnode, envir=func.env, inherits=F)) { - assign(cnode, get(cnode, envir=func.env), envir=env) - } - } - } - } -} - -clean.closure <- function( - func, - env -) { - if(mode(func) != 'function' || mode(env) != 'environment') - stop('parameter type mismatch...') - func.body <- body(func) - - closure.process(func.body, func, env) -} diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index 136db6960d88b..c5457adcbc54d 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -43,20 +43,10 @@ for (pkg in packageNames) { # read function dependencies depsLen <- SparkR:::readInt(inputCon) -funcAccum <- unserialize(SparkR:::readRawLen(inputCon, depsLen)) -# testfilename <- "/home//ubuntu/Desktop/testfile" -# cat(typeof(funcAccum), '\n', file = testfilename) -# cat(funcAccum$size, '\n', file = testfilename, append = T) -# cat(typeof(funcAccum$data[[1]]), '\n', file = testfilename, append = T) -# cat(as.character(body(funcAccum$data[[1]])), file = testfilename, append = T) -if (funcAccum$size > 0) { - computeFunc <- function(split, part) { - res <- part - for (i in 1:funcAccum$counter) { - res <- funcAccum$data[[i]](split, res) - } - res - } +if (depsLen > 0) { + execFunctionDeps <- SparkR:::readRawLen(inputCon, depsLen) + # load the dependencies into current environment + load(rawConnection(execFunctionDeps, open='rb')) } # Read and set broadcast variables From 21d4a97c2fb4015e4b27bd7d46ade7a8cf4f7d48 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 23 Feb 2015 00:02:51 -0500 Subject: [PATCH 060/121] Adds cleanClosure to capture the function closures. --- pkg/R/utils.R | 91 +++++++++++++++++++++++++++++++++++++ pkg/inst/tests/test_utils.R | 35 ++++++++++++++ 2 files changed, 126 insertions(+) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 778ae67def047..64c43eccd4eb9 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -288,3 +288,94 @@ convertEnvsToList <- function(keys, vals) { list(keys[[name]], vals[[name]]) }) } + +# 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. +# argNames The argument names of the function. +# newEnv A new function environment to store necessary function dependencies. +processClosure <- function(node, oldEnv, argNames, newEnv) { + nodeLen <- length(node) + if (nodeLen == 0) { + return + } + if (nodeLen > 1 && typeof(node) == "language") { + # Recursive case: current AST node is an internal node, check for its children. + nodeChar <- as.character(node[[1]]) + switch(nodeChar, + "{" = { # Start of a function body. + for (i in 2:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + }, + "<-" = { # Assignment. + defVar <- node[[2]] + if (length(defVar) == 1 && typeof(defVar) == "symbol") { + # Add the defined variable name into .defVars. + assign(".defVars", + c(get(".defVars", envir = .sparkREnv), as.character(defVar)), + envir = .sparkREnv) + } + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + }, + "function" = { # Function definition. + newArgs <- names(node[[2]]) + argNames <- c(argNames, newArgs) # Add parameter names. + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + }, + { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + }) + } else if (nodeLen == 1 && typeof(node) == "symbol") { + # Base case: current AST node is a leaf node and a symbol. + nodeChar <- as.character(node) + if (!nodeChar %in% argNames && # Not a function parameter or function local variable. + !nodeChar %in% get(".defVars", envir = .sparkREnv)) { + 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 below the global, as they are + # assumed to be loaded on workers. + while (!identical(func.env, topEnv) && !isNamespace(func.env)) { + # Set parameter 'inherits' to FALSE since we do not need to search in + # attached package environments. + if (exists(nodeChar, envir=func.env, inherits = FALSE)) { + assign(nodeChar, get(nodeChar, envir=func.env), envir = newEnv) + break + } else { + # 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 a new environment. +# param +# func A function whose closure needs to be captured. +# newEnv A new function environment to store necessary function dependencies. +cleanClosure <- function(func, newEnv) { + if (is.function(func) && is.environment(newEnv)) { + # .defVars is a character vector of variables names defined in the function. + assign(".defVars", c(), envir = .sparkREnv) + func.body <- body(func) + oldEnv <- environment(func) + argNames <- names(as.list(args(func))) + argsNames <- argNames[-length(argNames)] # Remove the ending NULL in pairlist. + # Recursively examine variables in the function body. + processClosure(func.body, oldEnv, argNames, newEnv) + } +} diff --git a/pkg/inst/tests/test_utils.R b/pkg/inst/tests/test_utils.R index 0a112254c7df1..7317d0421d092 100644 --- a/pkg/inst/tests/test_utils.R +++ b/pkg/inst/tests/test_utils.R @@ -35,3 +35,38 @@ test_that("reserialize on RDD", { unlink(fileName) }) + +test_that("clean.closure on R functions", { + y <- c(1, 2, 3) + g <- function(x) { x + 1 } + f <- function(x) { g(x) + y } + env <- new.env() + cleanClosure(f, env) + expect_equal(length(ls(env)), 2) # y, g + actual <- get("y", envir = env) + expect_equal(actual, y) + actual <- get("g", envir = env) + expect_equal(actual, g) + + # Check for nested enclosures and package variables. + env2 <- new.env() + funcEnv <- new.env(parent = env2) + f <- function(x) { min(g(x) + y) } + environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv + env <- new.env() + SparkR:::cleanClosure(f, env) + expect_equal(length(ls(env)), 2) # "min" should not be included + actual <- get("y", envir = env) + expect_equal(actual, y) + actual <- get("g", envir = env) + expect_equal(actual, g) + + # Test for function (and variable) definitions. + f <- function(x) { + g <- function(y) { y * 2 } + g(x) + } + env <- new.env() + SparkR:::cleanClosure(f, env) + expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. +}) From 04c4b6593b1a39f2dfb75f917531d4e618c25d1b Mon Sep 17 00:00:00 2001 From: lythesia Date: Tue, 24 Feb 2015 23:40:31 +0800 Subject: [PATCH 061/121] add repartition/coalesce --- pkg/NAMESPACE | 2 ++ pkg/R/RDD.R | 54 +++++++++++++++++++++++++++++++++++++++ pkg/inst/tests/test_rdd.R | 14 ++++++++++ pkg/man/coalesce.Rd | 33 ++++++++++++++++++++++++ pkg/man/repartition.Rd | 41 +++++++++++++++++++++++++++++ 5 files changed, 144 insertions(+) create mode 100644 pkg/man/coalesce.Rd create mode 100644 pkg/man/repartition.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 1cfcc0a6da674..26511b8c0c29d 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -6,6 +6,7 @@ exportMethods( "aggregateRDD", "cache", "checkpoint", + "coalesce", "cogroup", "collect", "collectAsMap", @@ -47,6 +48,7 @@ exportMethods( "reduce", "reduceByKey", "reduceByKeyLocally", + "repartition", "rightOuterJoin", "sampleRDD", "saveAsTextFile", diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 67fb28f219e27..76f09f44d1ada 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1039,6 +1039,60 @@ setMethod("keyBy", lapply(rdd, apply.func) }) +#' Return a new RDD that has exactly numPartitions partitions. +#' Can increase or decrease the level of parallelism in this RDD. Internally, +#' this uses a shuffle to redistribute data. +#' If you are decreasing the number of partitions in this RDD, consider using +#' coalesce, which can avoid performing a shuffle. +#' +#' @param rdd The RDD. +#' @param numPartitions Number of partitions to create. +#' @rdname repartition +#' @seealso coalesce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +#' numPartitions(rdd) # 4 +#' numPartitions(repartition(rdd, 2L)) # 2 +#'} +setGeneric("repartition", function(rdd, numPartitions) { standardGeneric("repartition") }) + +#' @rdname repartition +#' @aliases repartition,RDD +setMethod("repartition", + signature(rdd = "RDD", numPartitions = "integer"), + function(rdd, numPartitions) { + jrdd <- callJMethod(getJRDD(rdd), "repartition", numPartitions) + RDD(jrdd) + }) + +#' Return a new RDD that is reduced into numPartitions partitions. +#' +#' @param rdd The RDD. +#' @param numPartitions Number of partitions to create. +#' @rdname coalesce +#' @seealso repartition +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +#' numPartitions(rdd) # 3 +#' numPartitions(coalesce(rdd, 1L)) # 1 +#'} +setGeneric("coalesce", function(rdd, numPartitions, ...) { standardGeneric("coalesce") }) + +#' @rdname coalesce +#' @aliases coalesce,RDD +setMethod("coalesce", + signature(rdd = "RDD", numPartitions = "integer"), + function(rdd, numPartitions, shuffle = FALSE) { + jrdd <- callJMethod(getJRDD(rdd), "coalesce", numPartitions, shuffle) + RDD(jrdd) + }) + #' Save this RDD as a SequenceFile of serialized objects. #' #' @param rdd The RDD to save diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index f0c00d71d076c..ffa749171bf16 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -267,6 +267,20 @@ test_that("keyBy on RDDs", { expect_equal(actual, lapply(nums, function(x) { list(func(x), x) })) }) +test_that("repartition/coalesce on RDDs", { + rdd <- parallelize(sc, 1:10, 3L) + expect_equal(numPartitions(rdd), 3L) + + nrdd <- repartition(rdd, 2L) + expect_equal(numPartitions(nrdd), 2L) + + nrdd2 <- repartition(rdd, 5L) + expect_equal(numPartitions(nrdd2), 5L) + + nrdd3 <- coalesce(rdd, 1L) + expect_equal(numPartitions(nrdd3), 1L) +}) + test_that("sortBy() on RDDs", { sortedRdd <- sortBy(rdd, function(x) { x * x }, ascending = FALSE) actual <- collect(sortedRdd) diff --git a/pkg/man/coalesce.Rd b/pkg/man/coalesce.Rd new file mode 100644 index 0000000000000..f308e9a853d8e --- /dev/null +++ b/pkg/man/coalesce.Rd @@ -0,0 +1,33 @@ +% Generated by roxygen2 (4.1.0): do not edit by hand +% Please edit documentation in R/RDD.R +\docType{methods} +\name{coalesce} +\alias{coalesce} +\alias{coalesce,RDD} +\alias{coalesce,RDD,integer-method} +\title{Return a new RDD that is reduced into numPartitions partitions.} +\usage{ +coalesce(rdd, numPartitions, ...) + +\S4method{coalesce}{RDD,integer}(rdd, numPartitions, shuffle = FALSE) +} +\arguments{ +\item{rdd}{The RDD.} + +\item{numPartitions}{Number of partitions to create.} +} +\description{ +Return a new RDD that is reduced into numPartitions partitions. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +numPartitions(rdd) # 3 +numPartitions(coalesce(rdd, 1L)) # 1 +} +} +\seealso{ +repartition +} + diff --git a/pkg/man/repartition.Rd b/pkg/man/repartition.Rd new file mode 100644 index 0000000000000..1bbf672ec8231 --- /dev/null +++ b/pkg/man/repartition.Rd @@ -0,0 +1,41 @@ +% Generated by roxygen2 (4.1.0): do not edit by hand +% Please edit documentation in R/RDD.R +\docType{methods} +\name{repartition} +\alias{repartition} +\alias{repartition,RDD} +\alias{repartition,RDD,integer-method} +\title{Return a new RDD that has exactly numPartitions partitions. +Can increase or decrease the level of parallelism in this RDD. Internally, +this uses a shuffle to redistribute data. +If you are decreasing the number of partitions in this RDD, consider using +coalesce, which can avoid performing a shuffle.} +\usage{ +repartition(rdd, numPartitions) + +\S4method{repartition}{RDD,integer}(rdd, numPartitions) +} +\arguments{ +\item{rdd}{The RDD.} + +\item{numPartitions}{Number of partitions to create.} +} +\description{ +Return a new RDD that has exactly numPartitions partitions. +Can increase or decrease the level of parallelism in this RDD. Internally, +this uses a shuffle to redistribute data. +If you are decreasing the number of partitions in this RDD, consider using +coalesce, which can avoid performing a shuffle. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +numPartitions(rdd) # 4 +numPartitions(repartition(rdd, 2L)) # 2 +} +} +\seealso{ +coalesce +} + From 4e712e18682d5fa80039867b4a66e52d332556d2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 25 Feb 2015 00:33:01 -0800 Subject: [PATCH 062/121] use random port in backend --- pkg/R/sparkR.R | 43 ++++++++----------- pkg/R/sparkRClient.R | 2 +- .../cs/amplab/sparkr/SparkRBackend.scala | 24 +++++++---- .../cs/amplab/sparkr/SparkRRunner.scala | 20 +++------ 4 files changed, 40 insertions(+), 49 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 8a3fca68713e9..4e89d823af876 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -88,9 +88,7 @@ sparkR.init <- function( sparkEnvir = list(), sparkExecutorEnv = list(), sparkJars = "", - sparkRLibDir = "", - sparkRBackendPort = as.integer(Sys.getenv("SPARKR_BACKEND_PORT", "12345")), - sparkRRetryCount = 6) { + 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") @@ -121,45 +119,38 @@ sparkR.init <- function( if (sparkRExistingPort != "") { sparkRBackendPort <- sparkRExistingPort } else { + # TODO: test the random port and retry, or use httpuv:::startServer + callbackPort = sample(40000, 1) + 10000 if (Sys.getenv("SPARKR_USE_SPARK_SUBMIT", "") == "") { launchBackend(classPath = cp, mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", - args = as.character(sparkRBackendPort), + args = as.character(callbackPort), javaOpts = paste("-Xmx", sparkMem, sep = "")) } else { # TODO: We should deprecate sparkJars and ask users to add it to the # command line (using --jars) which is picked up by SparkSubmit launchBackendSparkSubmit( mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", - args = as.character(sparkRBackendPort), + args = as.character(callbackPort), appJar = .sparkREnv$assemblyJarPath, sparkHome = sparkHome, sparkSubmitOpts = Sys.getenv("SPARKR_SUBMIT_ARGS", "")) } + sock <- socketConnection(port = callbackPort, server = TRUE, open = 'rb', + blocking = TRUE, timeout = 10) + sparkRBackendPort <- readInt(sock) + if (length(sparkRBackendPort) == 0) { + stop("JVM failed to launch") + } + close(sock) } .sparkREnv$sparkRBackendPort <- sparkRBackendPort - cat("Waiting for JVM to come up...\n") - tries <- 0 - while (tries < sparkRRetryCount) { - if (!connExists(.sparkREnv)) { - Sys.sleep(2 ^ tries) - tryCatch({ - connectBackend("localhost", .sparkREnv$sparkRBackendPort) - }, error = function(err) { - cat("Error in Connection, retrying...\n") - }, warning = function(war) { - cat("No Connection Found, retrying...\n") - }) - tries <- tries + 1 - } else { - cat("Connection ok.\n") - break - } - } - if (tries == sparkRRetryCount) { - stop(sprintf("Failed to connect JVM after %d tries.\n", sparkRRetryCount)) - } + tryCatch({ + connectBackend("localhost", sparkRBackendPort) + }, error = function(err) { + stop("Failed to connect JVM\n") + }) if (nchar(sparkHome) != 0) { sparkHome <- normalizePath(sparkHome) diff --git a/pkg/R/sparkRClient.R b/pkg/R/sparkRClient.R index 1a747031586d2..4a06bb3364b50 100644 --- a/pkg/R/sparkRClient.R +++ b/pkg/R/sparkRClient.R @@ -2,7 +2,7 @@ # Creates a SparkR client connection object # if one doesn't already exist -connectBackend <- function(hostname, port, timeout = 6000) { +connectBackend <- function(hostname, port, timeout = 6) { if (exists(".sparkRcon", envir = .sparkREnv)) { if (isOpen(env[[".sparkRCon"]])) { cat("SparkRBackend client connection already exists\n") diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala index e6c606a519c3f..111b7cf678d03 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala @@ -1,7 +1,7 @@ package edu.berkeley.cs.amplab.sparkr -import java.io.IOException -import java.net.InetSocketAddress +import java.io.{DataOutputStream, IOException} +import java.net.{InetSocketAddress, Socket} import java.util.concurrent.TimeUnit import io.netty.bootstrap.ServerBootstrap @@ -24,9 +24,7 @@ class SparkRBackend { var bootstrap: ServerBootstrap = null var bossGroup: EventLoopGroup = null - def init(port: Int) { - val socketAddr = new InetSocketAddress(port) - + def init(): Int = { bossGroup = new NioEventLoopGroup(SparkRConf.numServerThreads) val workerGroup = bossGroup val handler = new SparkRBackendHandler(this) @@ -51,9 +49,9 @@ class SparkRBackend { } }) - channelFuture = bootstrap.bind(socketAddr) + channelFuture = bootstrap.bind(new InetSocketAddress(0)) channelFuture.syncUninterruptibly() - println("SparkR Backend server started on port :" + port) + channelFuture.channel().localAddress().asInstanceOf[InetSocketAddress].getPort() } def run() = { @@ -85,13 +83,21 @@ object SparkRBackend { } val sparkRBackend = new SparkRBackend() try { - sparkRBackend.init(args(0).toInt) + // bind to random port + val boundPort = sparkRBackend.init() + val callbackPort = args(0).toInt + val callbackSocket = new Socket("localhost", callbackPort) + val dos = new DataOutputStream(callbackSocket.getOutputStream) + dos.writeInt(boundPort) + dos.close() + callbackSocket.close() + sparkRBackend.run() } catch { case e: IOException => System.err.println("Server shutting down: failed with exception ", e) sparkRBackend.close() - System.exit(0) + System.exit(1) } System.exit(0) } diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala index fb356f89b2d1d..1b27f78844551 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRRunner.scala @@ -22,8 +22,6 @@ object SparkRRunner { // Time to wait for SparkR backend to initialize in seconds val backendTimeout = sys.env.getOrElse("SPARKR_BACKEND_TIMEOUT", "120").toInt - // TODO: Can we get this from SparkConf ? - val sparkRBackendPort = sys.env.getOrElse("SPARKR_BACKEND_PORT", "12345").toInt val rCommand = "Rscript" // Check if the file path exists. @@ -39,23 +37,19 @@ object SparkRRunner { // 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 SparkRBackend() - val sparkRBackendThread = new Thread() { - val finishedInit = new Semaphore(0) - + @volatile var sparkRBackendPort = 0 + val initialized = new Semaphore(0) + val sparkRBackendThread = new Thread("SparkR backend") { override def run() { - sparkRBackend.init(sparkRBackendPort) - finishedInit.release() + sparkRBackendPort = sparkRBackend.init() + initialized.release() sparkRBackend.run() } - - def stopBackend() { - sparkRBackend.close() - } } sparkRBackendThread.start() // Wait for SparkRBackend initialization to finish - if (sparkRBackendThread.finishedInit.tryAcquire(backendTimeout, TimeUnit.SECONDS)) { + if (initialized.tryAcquire(backendTimeout, TimeUnit.SECONDS)) { // Launch R val returnCode = try { val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) @@ -68,7 +62,7 @@ object SparkRRunner { process.waitFor() } finally { - sparkRBackendThread.stopBackend() + sparkRBackend.close() } System.exit(returnCode) } else { From 7651d84c2982fc5a567cdafdf1ca657733b4ca52 Mon Sep 17 00:00:00 2001 From: lythesia Date: Wed, 25 Feb 2015 21:14:21 +0800 Subject: [PATCH 063/121] fix coalesce --- pkg/R/RDD.R | 29 +++++++++++++++++++++++------ 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 76f09f44d1ada..9287a380ee768 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1062,10 +1062,9 @@ setGeneric("repartition", function(rdd, numPartitions) { standardGeneric("repart #' @rdname repartition #' @aliases repartition,RDD setMethod("repartition", - signature(rdd = "RDD", numPartitions = "integer"), + signature(rdd = "RDD", numPartitions = "numeric"), function(rdd, numPartitions) { - jrdd <- callJMethod(getJRDD(rdd), "repartition", numPartitions) - RDD(jrdd) + coalesce(rdd, numPartitions, TRUE) }) #' Return a new RDD that is reduced into numPartitions partitions. @@ -1087,10 +1086,28 @@ setGeneric("coalesce", function(rdd, numPartitions, ...) { standardGeneric("coal #' @rdname coalesce #' @aliases coalesce,RDD setMethod("coalesce", - signature(rdd = "RDD", numPartitions = "integer"), + signature(rdd = "RDD", numPartitions = "numeric"), function(rdd, numPartitions, shuffle = FALSE) { - jrdd <- callJMethod(getJRDD(rdd), "coalesce", numPartitions, shuffle) - RDD(jrdd) + if(as.integer(numPartitions) != numPartitions) { + warning("Number of partitions should be an integer. Coercing it to integer.") + } + numPartitionsm <- as.integer(numPartitions) + if (shuffle || numPartitions > SparkR::numPartitions(rdd)) { + func <- function(s, part) { + set.seed(s) # split as seed + lapply(part, + function(v) { + k <- as.integer(runif(1, 0, numPartitions)) + list(k, v) + }) + } + shuffled <- lapplyPartitionsWithIndex(rdd, func) + reparted <- partitionBy(shuffled, numPartitions) + values(reparted) + } else { + jrdd <- callJMethod(getJRDD(rdd), "coalesce", numPartitions, shuffle) + RDD(jrdd) + } }) #' Save this RDD as a SequenceFile of serialized objects. From 982f342a26f499c590fbb92835205f12e57f430a Mon Sep 17 00:00:00 2001 From: lythesia Date: Wed, 25 Feb 2015 21:33:28 +0800 Subject: [PATCH 064/121] fix numeric issue --- pkg/R/RDD.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 9287a380ee768..6d69f3e0cc7a0 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1091,7 +1091,7 @@ setMethod("coalesce", if(as.integer(numPartitions) != numPartitions) { warning("Number of partitions should be an integer. Coercing it to integer.") } - numPartitionsm <- as.integer(numPartitions) + numPartitions <- as.integer(numPartitions) if (shuffle || numPartitions > SparkR::numPartitions(rdd)) { func <- function(s, part) { set.seed(s) # split as seed From c652b4c2f1858af5343a71ef81f63eff684e769b Mon Sep 17 00:00:00 2001 From: cafreeman Date: Wed, 25 Feb 2015 16:22:36 -0600 Subject: [PATCH 065/121] Update method signatures to use generic arg Replace the `rdd` argument in all of the S4 methods with `x`. This will allow us to standardize the code as other Spark components get added and we need to set up multiple dispatch on S4 methods. In any cases where `x` was used as a generic iterator, I've replaced it with `i` except in a few cases where a different letter made sense. For example, in some of the pair functions, we now use `function(k)` and `function(v)` for the key/value functions. --- pkg/R/RDD.R | 348 ++++++++++++++++++++++++------------------------ pkg/R/pairRDD.R | 232 ++++++++++++++++---------------- 2 files changed, 290 insertions(+), 290 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 67fb28f219e27..608ebe1ee2673 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -97,18 +97,18 @@ PipelinedRDD <- function(prev, func) { # The jrdd accessor function. -setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) -setMethod("getJRDD", signature(rdd = "RDD"), function(rdd) rdd@jrdd ) -setMethod("getJRDD", signature(rdd = "PipelinedRDD"), - function(rdd, dataSerialization = TRUE) { - if (!is.null(rdd@env$jrdd_val)) { - return(rdd@env$jrdd_val) +setGeneric("getJRDD", function(x, ...) { standardGeneric("getJRDD") }) +setMethod("getJRDD", signature(x = "RDD"), function(x) x@jrdd ) +setMethod("getJRDD", signature(x = "PipelinedRDD"), + function(x, dataSerialization = TRUE) { + if (!is.null(x@env$jrdd_val)) { + return(x@env$jrdd_val) } # TODO: This is to handle anonymous functions. Find out a # better way to do this. computeFunc <- function(split, part) { - rdd@func(split, part) + x@func(split, part) } serializedFuncArr <- serialize("computeFunc", connection = NULL) @@ -120,13 +120,13 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), depsBin <- getDependencies(computeFunc) - prev_jrdd <- rdd@prev_jrdd + prev_jrdd <- x@prev_jrdd if (dataSerialization) { rddRef <- newJObject("edu.berkeley.cs.amplab.sparkr.RRDD", callJMethod(prev_jrdd, "rdd"), serializedFuncArr, - rdd@env$prev_serialized, + x@env$prev_serialized, depsBin, packageNamesArr, as.character(.sparkREnv[["libname"]]), @@ -136,7 +136,7 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), rddRef <- newJObject("edu.berkeley.cs.amplab.sparkr.StringRRDD", callJMethod(prev_jrdd, "rdd"), serializedFuncArr, - rdd@env$prev_serialized, + x@env$prev_serialized, depsBin, packageNamesArr, as.character(.sparkREnv[["libname"]]), @@ -144,9 +144,9 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), callJMethod(prev_jrdd, "classTag")) } # Save the serialization flag after we create a RRDD - rdd@env$serialized <- dataSerialization - rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() - rdd@env$jrdd_val + x@env$serialized <- dataSerialization + x@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() + x@env$jrdd_val }) @@ -170,7 +170,7 @@ setValidity("RDD", #' #' Persist this RDD with the default storage level (MEMORY_ONLY). #' -#' @param rdd The RDD to cache +#' @param x The RDD to cache #' @rdname cache-methods #' @export #' @examples @@ -179,16 +179,16 @@ setValidity("RDD", #' rdd <- parallelize(sc, 1:10, 2L) #' cache(rdd) #'} -setGeneric("cache", function(rdd) { standardGeneric("cache") }) +setGeneric("cache", function(x) { standardGeneric("cache") }) #' @rdname cache-methods #' @aliases cache,RDD-method setMethod("cache", - signature(rdd = "RDD"), - function(rdd) { - callJMethod(getJRDD(rdd), "cache") - rdd@env$isCached <- TRUE - rdd + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "cache") + x@env$isCached <- TRUE + x }) #' Persist an RDD @@ -197,7 +197,7 @@ setMethod("cache", #' supported storage levels, refer to #' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. #' -#' @param rdd The RDD to persist +#' @param x The RDD to persist #' @param newLevel The new storage level to be assigned #' @rdname persist #' @export @@ -207,13 +207,13 @@ setMethod("cache", #' rdd <- parallelize(sc, 1:10, 2L) #' persist(rdd, "MEMORY_AND_DISK") #'} -setGeneric("persist", function(rdd, newLevel) { standardGeneric("persist") }) +setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) #' @rdname persist #' @aliases persist,RDD-method setMethod("persist", - signature(rdd = "RDD", newLevel = "character"), - function(rdd, newLevel = c("DISK_ONLY", + signature(x = "RDD", newLevel = "character"), + function(x, newLevel = c("DISK_ONLY", "DISK_ONLY_2", "MEMORY_AND_DISK", "MEMORY_AND_DISK_2", @@ -238,9 +238,9 @@ setMethod("persist", "MEMORY_ONLY_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER_2"), "OFF_HEAP" = callJStatic("org.apache.spark.storage.StorageLevel", "OFF_HEAP")) - callJMethod(getJRDD(rdd), "persist", storageLevel) - rdd@env$isCached <- TRUE - rdd + callJMethod(getJRDD(x), "persist", storageLevel) + x@env$isCached <- TRUE + x }) #' Unpersist an RDD @@ -258,16 +258,16 @@ setMethod("persist", #' cache(rdd) # rdd@@env$isCached == TRUE #' unpersist(rdd) # rdd@@env$isCached == FALSE #'} -setGeneric("unpersist", function(rdd) { standardGeneric("unpersist") }) +setGeneric("unpersist", function(x) { standardGeneric("unpersist") }) #' @rdname unpersist-methods #' @aliases unpersist,RDD-method setMethod("unpersist", - signature(rdd = "RDD"), - function(rdd) { - callJMethod(getJRDD(rdd), "unpersist") - rdd@env$isCached <- FALSE - rdd + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x, "unpersist")) + x@env$isCached <- FALSE + x }) @@ -289,22 +289,22 @@ setMethod("unpersist", #' rdd <- parallelize(sc, 1:10, 2L) #' checkpoint(rdd) #'} -setGeneric("checkpoint", function(rdd) { standardGeneric("checkpoint") }) +setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) #' @rdname checkpoint-methods #' @aliases checkpoint,RDD-method setMethod("checkpoint", - signature(rdd = "RDD"), - function(rdd) { - jrdd <- getJRDD(rdd) + signature(x = "RDD"), + function(x) { + jrdd <- getJRDD(x) callJMethod(jrdd, "checkpoint") - rdd@env$isCheckpointed <- TRUE - rdd + x@env$isCheckpointed <- TRUE + x }) #' Gets the number of partitions of an RDD #' -#' @param rdd A RDD. +#' @param x A RDD. #' @return the number of partitions of rdd as an integer. #' @rdname numPartitions #' @export @@ -314,14 +314,14 @@ setMethod("checkpoint", #' rdd <- parallelize(sc, 1:10, 2L) #' numPartitions(rdd) # 2L #'} -setGeneric("numPartitions", function(rdd) { standardGeneric("numPartitions") }) +setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) #' @rdname numPartitions #' @aliases numPartitions,RDD-method setMethod("numPartitions", - signature(rdd = "RDD"), - function(rdd) { - jrdd <- getJRDD(rdd) + signature(x = "RDD"), + function(x) { + jrdd <- getJRDD(x) partitions <- callJMethod(jrdd, "splits") callJMethod(partitions, "size") }) @@ -331,7 +331,7 @@ setMethod("numPartitions", #' @description #' \code{collect} returns a list that contains all of the elements in this RDD. #' -#' @param rdd The RDD to collect +#' @param x The RDD to collect #' @param ... Other optional arguments to collect #' @param flatten FALSE if the list should not flattened #' @return a list containing elements in the RDD @@ -344,15 +344,15 @@ setMethod("numPartitions", #' collect(rdd) # list from 1 to 10 #' collectPartition(rdd, 0L) # list from 1 to 5 #'} -setGeneric("collect", function(rdd, ...) { standardGeneric("collect") }) +setGeneric("collect", function(x, ...) { standardGeneric("collect") }) #' @rdname collect-methods #' @aliases collect,RDD-method setMethod("collect", - signature(rdd = "RDD"), - function(rdd, flatten = TRUE) { + signature(x = "RDD"), + function(x, flatten = TRUE) { # Assumes a pairwise RDD is backed by a JavaPairRDD. - collected <- callJMethod(getJRDD(rdd), "collect") + collected <- callJMethod(getJRDD(x), "collect") convertJListToRList(collected, flatten) }) @@ -364,16 +364,16 @@ setMethod("collect", #' in the specified partition of the RDD. #' @param partitionId the partition to collect (starts from 0) setGeneric("collectPartition", - function(rdd, partitionId) { + function(x, partitionId) { standardGeneric("collectPartition") }) #' @rdname collect-methods #' @aliases collectPartition,integer,RDD-method setMethod("collectPartition", - signature(rdd = "RDD", partitionId = "integer"), - function(rdd, partitionId) { - jPartitionsList <- callJMethod(getJRDD(rdd), + signature(x = "RDD", partitionId = "integer"), + function(x, partitionId) { + jPartitionsList <- callJMethod(getJRDD(x), "collectPartitions", as.list(as.integer(partitionId))) @@ -392,16 +392,16 @@ setMethod("collectPartition", #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) #' collectAsMap(rdd) # list(`1` = 2, `3` = 4) #'} -setGeneric("collectAsMap", function(rdd) { standardGeneric("collectAsMap") }) +setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) #' @rdname collect-methods #' @aliases collectAsMap,RDD-method setMethod("collectAsMap", - signature(rdd = "RDD"), - function(rdd) { + signature(x = "RDD"), + function(x) { pairList <- collect(rdd) map <- new.env() - lapply(pairList, function(x) { assign(as.character(x[[1]]), x[[2]], envir = map) }) + lapply(pairList, function(i) { assign(as.character(i[[1]]), i[[2]], envir = map) }) as.list(map) }) @@ -447,7 +447,7 @@ setMethod("length", #' #' Same as countByValue in Spark. #' -#' @param rdd The RDD to count +#' @param x The RDD to count #' @return list of (value, count) pairs, where count is number of each unique #' value in rdd. #' @rdname countByValue @@ -458,15 +458,15 @@ setMethod("length", #' rdd <- parallelize(sc, c(1,2,3,2,1)) #' countByValue(rdd) # (1,2L), (2,2L), (3,1L) #'} -setGeneric("countByValue", function(rdd) { standardGeneric("countByValue") }) +setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) #' @rdname countByValue #' @aliases countByValue,RDD-method setMethod("countByValue", - signature(rdd = "RDD"), - function(rdd) { - ones <- lapply(rdd, function(item) { list(item, 1L) }) - collect(reduceByKey(ones, `+`, numPartitions(rdd))) + signature(x = "RDD"), + function(x) { + ones <- lapply(x, function(item) { list(item, 1L) }) + collect(reduceByKey(ones, `+`, numPartitions(x))) }) #' Apply a function to all elements @@ -683,26 +683,26 @@ setMethod("Filter", #' rdd <- parallelize(sc, 1:10) #' reduce(rdd, "+") # 55 #'} -setGeneric("reduce", function(rdd, func) { standardGeneric("reduce") }) +setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) #' @rdname reduce #' @aliases reduce,RDD,ANY-method setMethod("reduce", - signature(rdd = "RDD", func = "ANY"), - function(rdd, func) { + signature(x = "RDD", func = "ANY"), + function(x, func) { reducePartition <- function(part) { Reduce(func, part) } - partitionList <- collect(lapplyPartition(rdd, reducePartition), + partitionList <- collect(lapplyPartition(x, reducePartition), flatten = FALSE) Reduce(func, partitionList) }) #' Get the maximum element of an RDD. #' -#' @param rdd The RDD to get the maximum element from +#' @param x The RDD to get the maximum element from #' @export #' @rdname maximum #' @examples @@ -711,19 +711,19 @@ setMethod("reduce", #' rdd <- parallelize(sc, 1:10) #' maximum(rdd) # 10 #'} -setGeneric("maximum", function(rdd) { standardGeneric("maximum") }) +setGeneric("maximum", function(x) { standardGeneric("maximum") }) #' @rdname maximum #' @aliases maximum,RDD setMethod("maximum", - signature(rdd = "RDD"), - function(rdd) { - reduce(rdd, max) + signature(x = "RDD"), + function(x) { + reduce(x, max) }) #' Get the minimum element of an RDD. #' -#' @param rdd The RDD to get the minimum element from +#' @param x The RDD to get the minimum element from #' @export #' @rdname minimum #' @examples @@ -732,19 +732,19 @@ setMethod("maximum", #' rdd <- parallelize(sc, 1:10) #' minimum(rdd) # 1 #'} -setGeneric("minimum", function(rdd) { standardGeneric("minimum") }) +setGeneric("minimum", function(x) { standardGeneric("minimum") }) #' @rdname minimum #' @aliases minimum,RDD setMethod("minimum", - signature(rdd = "RDD"), - function(rdd) { - reduce(rdd, min) + signature(x = "RDD"), + function(x) { + reduce(x, min) }) #' Applies a function to all elements in an RDD, and force evaluation. #' -#' @param rdd The RDD to apply the function +#' @param x The RDD to apply the function #' @param func The function to be applied. #' @return invisible NULL. #' @export @@ -755,18 +755,18 @@ setMethod("minimum", #' rdd <- parallelize(sc, 1:10) #' foreach(rdd, function(x) { save(x, file=...) }) #'} -setGeneric("foreach", function(rdd, func) { standardGeneric("foreach") }) +setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) #' @rdname foreach #' @aliases foreach,RDD,function-method setMethod("foreach", - signature(rdd = "RDD", func = "function"), - function(rdd, func) { + signature(x = "RDD", func = "function"), + function(x, func) { partition.func <- function(x) { lapply(x, func) NULL } - invisible(collect(mapPartitions(rdd, partition.func))) + invisible(collect(mapPartitions(x, partition.func))) }) #' Applies a function to each partition in an RDD, and force evaluation. @@ -780,14 +780,14 @@ setMethod("foreach", #' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) #'} setGeneric("foreachPartition", - function(rdd, func) { standardGeneric("foreachPartition") }) + function(x, func) { standardGeneric("foreachPartition") }) #' @rdname foreach #' @aliases foreachPartition,RDD,function-method setMethod("foreachPartition", - signature(rdd = "RDD", func = "function"), - function(rdd, func) { - invisible(collect(mapPartitions(rdd, func))) + signature(x = "RDD", func = "function"), + function(x, func) { + invisible(collect(mapPartitions(x, func))) }) #' Take elements from an RDD. @@ -795,7 +795,7 @@ setMethod("foreachPartition", #' This function takes the first NUM elements in the RDD and #' returns them in a list. #' -#' @param rdd The RDD to take elements from +#' @param x The RDD to take elements from #' @param num Number of elements to take #' @rdname take #' @export @@ -805,17 +805,17 @@ setMethod("foreachPartition", #' rdd <- parallelize(sc, 1:10) #' take(rdd, 2L) # list(1, 2) #'} -setGeneric("take", function(rdd, num) { standardGeneric("take") }) +setGeneric("take", function(x, num) { standardGeneric("take") }) #' @rdname take #' @aliases take,RDD,numeric-method setMethod("take", - signature(rdd = "RDD", num = "numeric"), - function(rdd, num) { + signature(x = "RDD", num = "numeric"), + function(x, num) { resList <- list() index <- -1 - jrdd <- getJRDD(rdd) - numPartitions <- numPartitions(rdd) + jrdd <- getJRDD(x) + numPartitions <- numPartitions(x) # TODO(shivaram): Collect more than one partition based on size # estimates similar to the scala version of `take`. @@ -834,7 +834,7 @@ setMethod("take", elems <- convertJListToRList(partition, flatten = TRUE, logicalUpperBound = size, - serialized = rdd@env$serialized) + serialized = x@env$serialized) # TODO: Check if this append is O(n^2)? resList <- append(resList, elems) } @@ -846,7 +846,7 @@ setMethod("take", #' This function returns a new RDD containing the distinct elements in the #' given RDD. The same as `distinct()' in Spark. #' -#' @param rdd The RDD to remove duplicates from. +#' @param x The RDD to remove duplicates from. #' @param numPartitions Number of partitions to create. #' @rdname distinct #' @export @@ -857,18 +857,18 @@ setMethod("take", #' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) #'} setGeneric("distinct", - function(rdd, numPartitions) { standardGeneric("distinct") }) + function(x, numPartitions) { standardGeneric("distinct") }) setClassUnion("missingOrInteger", c("missing", "integer")) #' @rdname distinct #' @aliases distinct,RDD,missingOrInteger-method setMethod("distinct", - signature(rdd = "RDD", numPartitions = "missingOrInteger"), - function(rdd, numPartitions) { + signature(x = "RDD", numPartitions = "missingOrInteger"), + function(x, numPartitions) { if (missing(numPartitions)) { - numPartitions <- SparkR::numPartitions(rdd) + numPartitions <- SparkR::numPartitions(x) } - identical.mapped <- lapply(rdd, function(x) { list(x, NULL) }) + identical.mapped <- lapply(x, function(x) { list(x, NULL) }) reduced <- reduceByKey(identical.mapped, function(x, y) { x }, numPartitions) @@ -881,7 +881,7 @@ setMethod("distinct", #' The same as `sample()' in Spark. (We rename it due to signature #' inconsistencies with the `sample()' function in R's base package.) #' -#' @param rdd The RDD to sample elements from +#' @param x The RDD to sample elements from #' @param withReplacement Sampling with replacement or not #' @param fraction The (rough) sample target fraction #' @param seed Randomness seed value @@ -895,16 +895,16 @@ setMethod("distinct", #' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates #'} setGeneric("sampleRDD", - function(rdd, withReplacement, fraction, seed) { + function(x, withReplacement, fraction, seed) { standardGeneric("sampleRDD") }) #' @rdname sampleRDD #' @aliases sampleRDD,RDD setMethod("sampleRDD", - signature(rdd = "RDD", withReplacement = "logical", + signature(x = "RDD", withReplacement = "logical", fraction = "numeric", seed = "integer"), - function(rdd, withReplacement, fraction, seed) { + function(x, withReplacement, fraction, seed) { # The sampler: takes a partition and returns its sampled version. samplingFunc <- function(split, part) { @@ -941,13 +941,13 @@ setMethod("sampleRDD", list() } - lapplyPartitionsWithIndex(rdd, samplingFunc) + lapplyPartitionsWithIndex(x, samplingFunc) }) #' Return a list of the elements that are a sampled subset of the given RDD. #' -#' @param rdd The RDD to sample elements from +#' @param x The RDD to sample elements from #' @param withReplacement Sampling with replacement or not #' @param num Number of elements to return #' @param seed Randomness seed value @@ -963,19 +963,19 @@ setMethod("sampleRDD", #' takeSample(rdd, FALSE, 5L, 16181618L) #'} setGeneric("takeSample", - function(rdd, withReplacement, num, seed) { + function(x, withReplacement, num, seed) { standardGeneric("takeSample") }) #' @rdname takeSample #' @aliases takeSample,RDD -setMethod("takeSample", signature(rdd = "RDD", withReplacement = "logical", +setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", num = "integer", seed = "integer"), - function(rdd, withReplacement, num, seed) { + function(x, withReplacement, num, seed) { # This function is ported from RDD.scala. fraction <- 0.0 total <- 0 multiplier <- 3.0 - initialCount <- count(rdd) + initialCount <- count(x) maxSelected <- 0 MAXINT <- .Machine$integer.max @@ -997,7 +997,7 @@ setMethod("takeSample", signature(rdd = "RDD", withReplacement = "logical", } set.seed(seed) - samples <- collect(sampleRDD(rdd, withReplacement, fraction, + samples <- collect(sampleRDD(x, withReplacement, fraction, as.integer(ceiling(runif(1, -MAXINT, MAXINT))))) @@ -1005,7 +1005,7 @@ setMethod("takeSample", signature(rdd = "RDD", withReplacement = "logical", # take samples; this shouldn't happen often because we use a big # multiplier for thei initial size while (length(samples) < total) - samples <- collect(sampleRDD(rdd, withReplacement, fraction, + samples <- collect(sampleRDD(x, withReplacement, fraction, as.integer(ceiling(runif(1, -MAXINT, MAXINT))))) @@ -1016,7 +1016,7 @@ setMethod("takeSample", signature(rdd = "RDD", withReplacement = "logical", #' Creates tuples of the elements in this RDD by applying a function. #' -#' @param rdd The RDD. +#' @param x The RDD. #' @param func The function to be applied. #' @rdname keyBy #' @export @@ -1026,22 +1026,22 @@ setMethod("takeSample", signature(rdd = "RDD", withReplacement = "logical", #' rdd <- parallelize(sc, list(1, 2, 3)) #' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) #'} -setGeneric("keyBy", function(rdd, func) { standardGeneric("keyBy") }) +setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) #' @rdname keyBy #' @aliases keyBy,RDD setMethod("keyBy", - signature(rdd = "RDD", func = "function"), - function(rdd, func) { + signature(x = "RDD", func = "function"), + function(x, func) { apply.func <- function(x) { list(func(x), x) } - lapply(rdd, apply.func) + lapply(x, apply.func) }) #' Save this RDD as a SequenceFile of serialized objects. #' -#' @param rdd The RDD to save +#' @param x The RDD to save #' @param path The directory where the file is saved #' @rdname saveAsObjectFile #' @seealso objectFile @@ -1052,25 +1052,25 @@ setMethod("keyBy", #' rdd <- parallelize(sc, 1:3) #' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") #'} -setGeneric("saveAsObjectFile", function(rdd, path) { standardGeneric("saveAsObjectFile") }) +setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) #' @rdname saveAsObjectFile #' @aliases saveAsObjectFile,RDD setMethod("saveAsObjectFile", - signature(rdd = "RDD", path = "character"), - function(rdd, path) { + signature(x = "RDD", path = "character"), + function(x, path) { # If the RDD is in string format, need to serialize it before saving it because when # objectFile() is invoked to load the saved file, only serialized format is assumed. - if (!rdd@env$serialized) { - rdd <- reserialize(rdd) + if (!x@env$serialized) { + x <- reserialize(x) } # Return nothing - invisible(callJMethod(getJRDD(rdd), "saveAsObjectFile", path)) + invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) }) #' Save this RDD as a text file, using string representations of elements. #' -#' @param rdd The RDD to save +#' @param x The RDD to save #' @param path The directory where the splits of the text file are saved #' @rdname saveAsTextFile #' @export @@ -1080,17 +1080,17 @@ setMethod("saveAsObjectFile", #' rdd <- parallelize(sc, 1:3) #' saveAsTextFile(rdd, "/tmp/sparkR-tmp") #'} -setGeneric("saveAsTextFile", function(rdd, path) { standardGeneric("saveAsTextFile") }) +setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) #' @rdname saveAsTextFile #' @aliases saveAsTextFile,RDD setMethod("saveAsTextFile", - signature(rdd = "RDD", path = "character"), - function(rdd, path) { - func <- function(x) { - toString(x) + signature(x = "RDD", path = "character"), + function(x, path) { + func <- function(str) { + toString(str) } - stringRdd <- lapply(rdd, func) + stringRdd <- lapply(x, func) # Return nothing invisible( callJMethod(getJRDD(stringRdd, dataSerialization = FALSE), "saveAsTextFile", path)) @@ -1098,7 +1098,7 @@ setMethod("saveAsTextFile", #' Sort an RDD by the given key function. #' -#' @param rdd An RDD to be sorted. +#' @param x An RDD to be sorted. #' @param func A function used to compute the sort key for each element. #' @param ascending A flag to indicate whether the sorting is ascending or descending. #' @param numPartitions Number of partitions to create. @@ -1111,7 +1111,7 @@ setMethod("saveAsTextFile", #' rdd <- parallelize(sc, list(3, 2, 1)) #' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) #'} -setGeneric("sortBy", function(rdd, +setGeneric("sortBy", function(x, func, ascending = TRUE, numPartitions = 1L) { @@ -1121,20 +1121,20 @@ setGeneric("sortBy", function(rdd, #' @rdname sortBy #' @aliases sortBy,RDD,RDD-method setMethod("sortBy", - signature(rdd = "RDD", func = "function"), - function(rdd, func, ascending = TRUE, numPartitions = SparkR::numPartitions(rdd)) { - values(sortByKey(keyBy(rdd, func), ascending, numPartitions)) + signature(x = "RDD", func = "function"), + function(x, func, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { + values(sortByKey(keyBy(x, func), ascending, numPartitions)) }) # Helper function to get first N elements from an RDD in the specified order. # Param: -# rdd An RDD. +# x An RDD. # num Number of elements to return. # ascending A flag to indicate whether the sorting is ascending or descending. # Return: # A list of the first N elements from the RDD in the specified order. # -takeOrderedElem <- function(rdd, num, ascending = TRUE) { +takeOrderedElem <- function(x, num, ascending = TRUE) { if (num <= 0L) { return(list()) } @@ -1156,13 +1156,13 @@ takeOrderedElem <- function(rdd, num, ascending = TRUE) { newElems[ord[1:num]] } - newRdd <- mapPartitions(rdd, partitionFunc) + newRdd <- mapPartitions(x, partitionFunc) reduce(newRdd, reduceFunc) } #' Returns the first N elements from an RDD in ascending order. #' -#' @param rdd An RDD. +#' @param x An RDD. #' @param num Number of elements to return. #' @return The first N elements from the RDD in ascending order. #' @rdname takeOrdered @@ -1173,19 +1173,19 @@ takeOrderedElem <- function(rdd, num, ascending = TRUE) { #' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) #' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) #'} -setGeneric("takeOrdered", function(rdd, num) { standardGeneric("takeOrdered") }) +setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) #' @rdname takeOrdered #' @aliases takeOrdered,RDD,RDD-method setMethod("takeOrdered", - signature(rdd = "RDD", num = "integer"), - function(rdd, num) { - takeOrderedElem(rdd, num) + signature(x = "RDD", num = "integer"), + function(x, num) { + takeOrderedElem(x, num) }) #' Returns the top N elements from an RDD. #' -#' @param rdd An RDD. +#' @param x An RDD. #' @param num Number of elements to return. #' @return The top N elements from the RDD. #' @rdname top @@ -1196,14 +1196,14 @@ setMethod("takeOrdered", #' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) #' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) #'} -setGeneric("top", function(rdd, num) { standardGeneric("top") }) +setGeneric("top", function(x, num) { standardGeneric("top") }) #' @rdname top #' @aliases top,RDD,RDD-method setMethod("top", - signature(rdd = "RDD", num = "integer"), - function(rdd, num) { - takeOrderedElem(rdd, num, FALSE) + signature(x = "RDD", num = "integer"), + function(x, num) { + takeOrderedElem(x, num, FALSE) }) #' Fold an RDD using a given associative function and a neutral "zero value". @@ -1211,7 +1211,7 @@ setMethod("top", #' Aggregate the elements of each partition, and then the results for all the #' partitions, using a given associative function and a neutral "zero value". #' -#' @param rdd An RDD. +#' @param x An RDD. #' @param zeroValue A neutral "zero value". #' @param op An associative function for the folding operation. #' @return The folding result. @@ -1224,14 +1224,14 @@ setMethod("top", #' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) #' fold(rdd, 0, "+") # 15 #'} -setGeneric("fold", function(rdd, zeroValue, op) { standardGeneric("fold") }) +setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) #' @rdname fold #' @aliases fold,RDD,RDD-method setMethod("fold", - signature(rdd = "RDD", zeroValue = "ANY", op = "ANY"), - function(rdd, zeroValue, op) { - aggregateRDD(rdd, zeroValue, op, op) + signature(x = "RDD", zeroValue = "ANY", op = "ANY"), + function(x, zeroValue, op) { + aggregateRDD(x, zeroValue, op, op) }) #' Aggregate an RDD using the given combine functions and a neutral "zero value". @@ -1239,7 +1239,7 @@ setMethod("fold", #' Aggregate the elements of each partition, and then the results for all the #' partitions, using given combine functions and a neutral "zero value". #' -#' @param rdd An RDD. +#' @param x An RDD. #' @param zeroValue A neutral "zero value". #' @param seqOp A function to aggregate the RDD elements. It may return a different #' result type from the type of the RDD elements. @@ -1257,18 +1257,18 @@ setMethod("fold", #' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } #' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) #'} -setGeneric("aggregateRDD", function(rdd, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) +setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) #' @rdname aggregateRDD #' @aliases aggregateRDD,RDD,RDD-method setMethod("aggregateRDD", - signature(rdd = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), - function(rdd, zeroValue, seqOp, combOp) { + signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), + function(x, zeroValue, seqOp, combOp) { partitionFunc <- function(part) { Reduce(seqOp, part, zeroValue) } - partitionList <- collect(lapplyPartition(rdd, partitionFunc), + partitionList <- collect(lapplyPartition(x, partitionFunc), flatten = FALSE) Reduce(combOp, partitionList, zeroValue) }) @@ -1277,7 +1277,7 @@ setMethod("aggregateRDD", #' #' The same as 'pipe()' in Spark. #' -#' @param rdd The RDD whose elements are piped to the forked external process. +#' @param x The RDD whose elements are piped to the forked external process. #' @param command The command to fork an external process. #' @param env A named list to set environment variables of the external process. #' @return A new RDD created by piping all elements to a forked external process. @@ -1290,15 +1290,15 @@ setMethod("aggregateRDD", #' collect(pipeRDD(rdd, "more") #' Output: c("1", "2", ..., "10") #'} -setGeneric("pipeRDD", function(rdd, command, env = list()) { +setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD") }) #' @rdname pipeRDD #' @aliases pipeRDD,RDD,character-method setMethod("pipeRDD", - signature(rdd = "RDD", command = "character"), - function(rdd, command, env = list()) { + signature(x = "RDD", command = "character"), + function(x, command, env = list()) { func <- function(part) { trim.trailing.func <- function(x) { sub("[\r\n]*$", "", toString(x)) @@ -1307,13 +1307,13 @@ setMethod("pipeRDD", res <- system2(command, stdout = TRUE, input = input, env = env) lapply(res, trim.trailing.func) } - lapplyPartition(rdd, func) + lapplyPartition(x, func) }) # TODO: Consider caching the name in the RDD's environment #' Return an RDD's name. #' -#' @param rdd The RDD whose name is returned. +#' @param x The RDD whose name is returned. #' @rdname name #' @export #' @examples @@ -1322,19 +1322,19 @@ setMethod("pipeRDD", #' rdd <- parallelize(sc, list(1,2,3)) #' name(rdd) # NULL (if not set before) #'} -setGeneric("name", function(rdd) { standardGeneric("name") }) +setGeneric("name", function(x) { standardGeneric("name") }) #' @rdname name #' @aliases name,RDD setMethod("name", - signature(rdd = "RDD"), - function(rdd) { - callJMethod(getJRDD(rdd), "name") + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "name") }) #' Set an RDD's name. #' -#' @param rdd The RDD whose name is to be set. +#' @param x The RDD whose name is to be set. #' @param name The RDD name to be set. #' @return a new RDD renamed. #' @rdname setName @@ -1346,15 +1346,15 @@ setMethod("name", #' setName(rdd, "myRDD") #' name(rdd) # "myRDD" #'} -setGeneric("setName", function(rdd, name) { standardGeneric("setName") }) +setGeneric("setName", function(x, name) { standardGeneric("setName") }) #' @rdname setName #' @aliases setName,RDD setMethod("setName", - signature(rdd = "RDD", name = "character"), - function(rdd, name) { - callJMethod(getJRDD(rdd), "setName", name) - rdd + signature(x = "RDD", name = "character"), + function(x, name) { + callJMethod(getJRDD(x), "setName", name) + x }) ############ Binary Functions ############# diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R index 0c536f4478ac3..a54c1abe01b47 100644 --- a/pkg/R/pairRDD.R +++ b/pkg/R/pairRDD.R @@ -7,7 +7,7 @@ #' @description #' \code{lookup} returns a list of values in this RDD for key key. #' -#' @param rdd The RDD to collect +#' @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 #' @rdname lookup @@ -19,18 +19,18 @@ #' rdd <- parallelize(sc, pairs) #' lookup(rdd, 1) # list(1, 3) #'} -setGeneric("lookup", function(rdd, key) { standardGeneric("lookup") }) +setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) #' @rdname lookup #' @aliases lookup,RDD-method setMethod("lookup", - signature(rdd = "RDD", key = "ANY"), - function(rdd, key) { + signature(x = "RDD", key = "ANY"), + function(x, key) { partitionFunc <- function(part) { - filtered <- part[unlist(lapply(part, function(x) { identical(key, x[[1]]) }))] - lapply(filtered, function(x) { x[[2]] }) + filtered <- part[unlist(lapply(part, function(i) { identical(key, i[[1]]) }))] + lapply(filtered, function(i) { i[[2]] }) } - valsRDD <- lapplyPartition(rdd, partitionFunc) + valsRDD <- lapplyPartition(x, partitionFunc) collect(valsRDD) }) @@ -39,7 +39,7 @@ setMethod("lookup", #' #' Same as countByKey in Spark. #' -#' @param rdd The RDD to count keys. +#' @param x The RDD to count keys. #' @return list of (key, count) pairs, where count is number of each key in rdd. #' @rdname countByKey #' @export @@ -49,20 +49,20 @@ setMethod("lookup", #' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) #' countByKey(rdd) # ("a", 2L), ("b", 1L) #'} -setGeneric("countByKey", function(rdd) { standardGeneric("countByKey") }) +setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) #' @rdname countByKey #' @aliases countByKey,RDD-method setMethod("countByKey", - signature(rdd = "RDD"), - function(rdd) { - keys <- lapply(rdd, function(item) { item[[1]] }) + signature(x = "RDD"), + function(x) { + keys <- lapply(x, function(item) { item[[1]] }) countByValue(keys) }) #' Return an RDD with the keys of each tuple. #' -#' @param rdd The RDD from which the keys of each tuple is returned. +#' @param x The RDD from which the keys of each tuple is returned. #' @rdname keys #' @export #' @examples @@ -71,22 +71,22 @@ setMethod("countByKey", #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) #' collect(keys(rdd)) # list(1, 3) #'} -setGeneric("keys", function(rdd) { standardGeneric("keys") }) +setGeneric("keys", function(x) { standardGeneric("keys") }) #' @rdname keys #' @aliases keys,RDD setMethod("keys", - signature(rdd = "RDD"), - function(rdd) { - func <- function(x) { - x[[1]] + signature(x = "RDD"), + function(x) { + func <- function(k) { + k[[1]] } - lapply(rdd, func) + lapply(x, func) }) #' Return an RDD with the values of each tuple. #' -#' @param rdd The RDD from which the values of each tuple is returned. +#' @param x The RDD from which the values of each tuple is returned. #' @rdname values #' @export #' @examples @@ -95,17 +95,17 @@ setMethod("keys", #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) #' collect(values(rdd)) # list(2, 4) #'} -setGeneric("values", function(rdd) { standardGeneric("values") }) +setGeneric("values", function(x) { standardGeneric("values") }) #' @rdname values #' @aliases values,RDD setMethod("values", - signature(rdd = "RDD"), - function(rdd) { - func <- function(x) { - x[[2]] + signature(x = "RDD"), + function(x) { + func <- function(v) { + v[[2]] } - lapply(rdd, func) + lapply(x, func) }) #' Applies a function to all values of the elements, without modifying the keys. @@ -176,7 +176,7 @@ setMethod("flatMapValues", #' 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 rdd The RDD to partition. Should be an RDD where each element is +#' @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. @@ -195,15 +195,15 @@ setMethod("flatMapValues", #' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) #'} setGeneric("partitionBy", - function(rdd, numPartitions, ...) { + function(x, numPartitions, ...) { standardGeneric("partitionBy") }) #' @rdname partitionBy #' @aliases partitionBy,RDD,integer-method setMethod("partitionBy", - signature(rdd = "RDD", numPartitions = "integer"), - function(rdd, numPartitions, partitionFunc = hashCode) { + signature(x = "RDD", numPartitions = "integer"), + function(x, numPartitions, partitionFunc = hashCode) { #if (missing(partitionFunc)) { # partitionFunc <- hashCode @@ -218,7 +218,7 @@ setMethod("partitionBy", connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - jrdd <- getJRDD(rdd) + jrdd <- getJRDD(x) # We create a PairwiseRRDD that extends RDD[(Array[Byte], # Array[Byte])], where the key is the hashed split, the value is @@ -227,7 +227,7 @@ setMethod("partitionBy", callJMethod(jrdd, "rdd"), as.integer(numPartitions), serializedHashFuncBytes, - rdd@env$serialized, + x@env$serialized, depsBinArr, packageNamesArr, as.character(.sparkREnv$libname), @@ -254,7 +254,7 @@ setMethod("partitionBy", #' 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 rdd The RDD to group. Should be an RDD where each element is +#' @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)) @@ -271,27 +271,27 @@ setMethod("partitionBy", #' grouped[[1]] # Should be a list(1, list(2, 4)) #'} setGeneric("groupByKey", - function(rdd, numPartitions) { + function(x, numPartitions) { standardGeneric("groupByKey") }) #' @rdname groupByKey #' @aliases groupByKey,RDD,integer-method setMethod("groupByKey", - signature(rdd = "RDD", numPartitions = "integer"), - function(rdd, numPartitions) { - shuffled <- partitionBy(rdd, numPartitions) + 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, x) { - addItemToAccumulator(acc, x) + appendList <- function(acc, i) { + addItemToAccumulator(acc, i) acc } - makeList <- function(x) { + makeList <- function(i) { acc <- initAccumulator() - addItemToAccumulator(acc, x) + addItemToAccumulator(acc, i) acc } # Each item in the partition is list of (K, V) @@ -303,9 +303,9 @@ setMethod("groupByKey", }) # extract out data field vals <- eapply(vals, - function(x) { - length(x$data) <- x$counter - x$data + function(i) { + length(i$data) <- i$counter + i$data }) # Every key in the environment contains a list # Convert that to list(K, Seq[V]) @@ -319,7 +319,7 @@ setMethod("groupByKey", #' 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 rdd The RDD to reduce by key. Should be an RDD where each element is +#' @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. @@ -338,15 +338,15 @@ setMethod("groupByKey", #' reduced[[1]] # Should be a list(1, 6) #'} setGeneric("reduceByKey", - function(rdd, combineFunc, numPartitions) { + function(x, combineFunc, numPartitions) { standardGeneric("reduceByKey") }) #' @rdname reduceByKey #' @aliases reduceByKey,RDD,integer-method setMethod("reduceByKey", - signature(rdd = "RDD", combineFunc = "ANY", numPartitions = "integer"), - function(rdd, combineFunc, numPartitions) { + signature(x = "RDD", combineFunc = "ANY", numPartitions = "integer"), + function(x, combineFunc, numPartitions) { reduceVals <- function(part) { vals <- new.env() keys <- new.env() @@ -358,7 +358,7 @@ setMethod("reduceByKey", }) convertEnvsToList(keys, vals) } - locallyReduced <- lapplyPartition(rdd, reduceVals) + locallyReduced <- lapplyPartition(x, reduceVals) shuffled <- partitionBy(locallyReduced, numPartitions) lapplyPartition(shuffled, reduceVals) }) @@ -369,7 +369,7 @@ setMethod("reduceByKey", #' 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 rdd The RDD to reduce by key. Should be an RDD where each element is +#' @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 @@ -385,15 +385,15 @@ setMethod("reduceByKey", #' reduced # list(list(1, 6), list(1.1, 3)) #'} setGeneric("reduceByKeyLocally", - function(rdd, combineFunc) { + function(x, combineFunc) { standardGeneric("reduceByKeyLocally") }) #' @rdname reduceByKeyLocally #' @aliases reduceByKeyLocally,RDD,integer-method setMethod("reduceByKeyLocally", - signature(rdd = "RDD", combineFunc = "ANY"), - function(rdd, combineFunc) { + signature(x = "RDD", combineFunc = "ANY"), + function(x, combineFunc) { reducePart <- function(part) { vals <- new.env() keys <- new.env() @@ -417,7 +417,7 @@ setMethod("reduceByKeyLocally", }) accum } - reduced <- mapPartitions(rdd, reducePart) + reduced <- mapPartitions(x, reducePart) merged <- reduce(reduced, mergeParts) convertEnvsToList(merged[[1]], merged[[2]]) }) @@ -437,7 +437,7 @@ setMethod("reduceByKeyLocally", #' two lists). #' } #' -#' @param rdd The RDD to combine. Should be an RDD where each element is +#' @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) @@ -458,16 +458,16 @@ setMethod("reduceByKeyLocally", #' combined[[1]] # Should be a list(1, 6) #'} setGeneric("combineByKey", - function(rdd, createCombiner, mergeValue, mergeCombiners, numPartitions) { + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { standardGeneric("combineByKey") }) #' @rdname combineByKey #' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method setMethod("combineByKey", - signature(rdd = "RDD", createCombiner = "ANY", mergeValue = "ANY", + signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY", mergeCombiners = "ANY", numPartitions = "integer"), - function(rdd, createCombiner, mergeValue, mergeCombiners, numPartitions) { + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { combineLocally <- function(part) { combiners <- new.env() keys <- new.env() @@ -479,7 +479,7 @@ setMethod("combineByKey", }) convertEnvsToList(keys, combiners) } - locallyCombined <- lapplyPartition(rdd, combineLocally) + locallyCombined <- lapplyPartition(x, combineLocally) shuffled <- partitionBy(locallyCombined, numPartitions) mergeAfterShuffle <- function(part) { combiners <- new.env() @@ -506,7 +506,7 @@ setMethod("combineByKey", #' of these functions are allowed to modify and return their first argument #' instead of creating a new U. #' -#' @param rdd An RDD. +#' @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. @@ -526,21 +526,21 @@ setMethod("combineByKey", #' # list(list(1, list(3, 2)), list(2, list(7, 2))) #'} setGeneric("aggregateByKey", - function(rdd, zeroValue, seqOp, combOp, numPartitions) { + function(x, zeroValue, seqOp, combOp, numPartitions) { standardGeneric("aggregateByKey") }) #' @rdname aggregateByKey #' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method setMethod("aggregateByKey", - signature(rdd = "RDD", zeroValue = "ANY", seqOp = "ANY", + signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY", numPartitions = "integer"), - function(rdd, zeroValue, seqOp, combOp, numPartitions) { + function(x, zeroValue, seqOp, combOp, numPartitions) { createCombiner <- function(v) { do.call(seqOp, list(zeroValue, v)) } - combineByKey(rdd, createCombiner, seqOp, combOp, numPartitions) + combineByKey(x, createCombiner, seqOp, combOp, numPartitions) }) #' Fold a pair RDD by each key. @@ -550,7 +550,7 @@ setMethod("aggregateByKey", #' number of times, and must not change the result (e.g., 0 for addition, or #' 1 for multiplication.). #' -#' @param rdd An RDD. +#' @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. @@ -564,17 +564,17 @@ setMethod("aggregateByKey", #' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) #'} setGeneric("foldByKey", - function(rdd, zeroValue, func, numPartitions) { + function(x, zeroValue, func, numPartitions) { standardGeneric("foldByKey") }) #' @rdname foldByKey #' @aliases foldByKey,RDD,ANY,ANY,integer-method setMethod("foldByKey", - signature(rdd = "RDD", zeroValue = "ANY", + signature(x = "RDD", zeroValue = "ANY", func = "ANY", numPartitions = "integer"), - function(rdd, zeroValue, func, numPartitions) { - aggregateByKey(rdd, zeroValue, func, func, numPartitions) + function(x, zeroValue, func, numPartitions) { + aggregateByKey(x, zeroValue, func, func, numPartitions) }) ############ Binary Functions ############# @@ -585,9 +585,9 @@ setMethod("foldByKey", #' \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 rdd1 An RDD to be joined. Should be an RDD where each element is +#' @param x An RDD to be joined. Should be an RDD where each element is #' list(K, V). -#' @param rdd2 An RDD to be joined. Should be an RDD where each element is +#' @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 @@ -601,21 +601,21 @@ setMethod("foldByKey", #' 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)) #'} -setGeneric("join", function(rdd1, rdd2, numPartitions) { standardGeneric("join") }) +setGeneric("join", function(x, y, numPartitions) { standardGeneric("join") }) #' @rdname join-methods #' @aliases join,RDD,RDD-method setMethod("join", - signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), - function(rdd1, rdd2, numPartitions) { - rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) - rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) + 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, FALSE)) } - joined <- flatMapValues(groupByKey(unionRDD(rdd1Tagged, rdd2Tagged), numPartitions), doJoin) + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' Left outer join two RDDs @@ -624,12 +624,12 @@ setMethod("join", #' \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 rdd1 An RDD to be joined. Should be an RDD where each element is +#' @param x An RDD to be joined. Should be an RDD where each element is #' list(K, V). -#' @param rdd2 An RDD to be joined. Should be an RDD where each element is +#' @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 rdd1, the resulting RDD will either contain +#' @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. #' @rdname join-methods @@ -642,21 +642,21 @@ setMethod("join", #' leftOuterJoin(rdd1, rdd2, 2L) #' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) #'} -setGeneric("leftOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("leftOuterJoin") }) +setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) #' @rdname join-methods #' @aliases leftOuterJoin,RDD,RDD-method setMethod("leftOuterJoin", - signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), - function(rdd1, rdd2, numPartitions) { - rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) - rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) + 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(rdd1Tagged, rdd2Tagged), numPartitions), doJoin) + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' Right outer join two RDDs @@ -665,14 +665,14 @@ setMethod("leftOuterJoin", #' \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 rdd1 An RDD to be joined. Should be an RDD where each element is +#' @param x An RDD to be joined. Should be an RDD where each element is #' list(K, V). -#' @param rdd2 An RDD to be joined. Should be an RDD where each element is +#' @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 rdd2, the resulting RDD will either contain -#' all pairs (k, (v, w)) for (k, v) in rdd1, or the pair (k, (NULL, w)) -#' if no elements in rdd1 have key k. +#' @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. #' @rdname join-methods #' @export #' @examples @@ -683,21 +683,21 @@ setMethod("leftOuterJoin", #' rightOuterJoin(rdd1, rdd2, 2L) #' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) #'} -setGeneric("rightOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("rightOuterJoin") }) +setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) #' @rdname join-methods #' @aliases rightOuterJoin,RDD,RDD-method setMethod("rightOuterJoin", - signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), - function(rdd1, rdd2, numPartitions) { - rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) - rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) + 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(rdd1Tagged, rdd2Tagged), numPartitions), doJoin) + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' Full outer join two RDDs @@ -706,15 +706,15 @@ setMethod("rightOuterJoin", #' \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 rdd1 An RDD to be joined. Should be an RDD where each element is +#' @param x An RDD to be joined. Should be an RDD where each element is #' list(K, V). -#' @param rdd2 An RDD to be joined. Should be an RDD where each element is +#' @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 rdd1 and (k, w) in rdd2, the resulting RDD -#' will contain all pairs (k, (v, w)) for both (k, v) in rdd1 and and -#' (k, w) in rdd2, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements -#' in rdd1/rdd2 have key k. +#' @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. #' @rdname join-methods #' @export #' @examples @@ -727,22 +727,22 @@ setMethod("rightOuterJoin", #' # list(2, list(NULL, 4))) #' # list(3, list(3, NULL)), #'} -setGeneric("fullOuterJoin", function(rdd1, rdd2, numPartitions) { standardGeneric("fullOuterJoin") }) +setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) #' @rdname join-methods #' @aliases fullOuterJoin,RDD,RDD-method setMethod("fullOuterJoin", - signature(rdd1 = "RDD", rdd2 = "RDD", numPartitions = "integer"), - function(rdd1, rdd2, numPartitions) { - rdd1Tagged <- lapply(rdd1, function(x) { list(x[[1]], list(1L, x[[2]])) }) - rdd2Tagged <- lapply(rdd2, function(x) { list(x[[1]], list(2L, x[[2]])) }) + 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(rdd1Tagged, rdd2Tagged), numPartitions), doJoin) + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' For each key k in several RDDs, return a resulting RDD that @@ -809,7 +809,7 @@ setMethod("cogroup", #' Sort a (k, v) pair RDD by k. #' -#' @param rdd A (k, v) pair RDD to be sorted. +#' @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. @@ -821,7 +821,7 @@ setMethod("cogroup", #' 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)) #'} -setGeneric("sortByKey", function(rdd, +setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1L) { standardGeneric("sortByKey") @@ -830,17 +830,17 @@ setGeneric("sortByKey", function(rdd, #' @rdname sortByKey #' @aliases sortByKey,RDD,RDD-method setMethod("sortByKey", - signature(rdd = "RDD"), - function(rdd, ascending = TRUE, numPartitions = SparkR::numPartitions(rdd)) { + signature(x = "RDD"), + function(x, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { rangeBounds <- list() if (numPartitions > 1) { - rddSize <- count(rdd) + rddSize <- count(x) # constant from Spark's RangePartitioner maxSampleSize <- numPartitions * 20 fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) - samples <- collect(keys(sampleRDD(rdd, FALSE, fraction, 1L))) + 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) @@ -873,7 +873,7 @@ setMethod("sortByKey", sortKeyValueList(part, decreasing = !ascending) } - newRDD <- partitionBy(rdd, numPartitions, rangePartitionFunc) + newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) lapplyPartition(newRDD, partitionFunc) }) From 3294949d41c5515f785bae2ac0b566ffd8c8c5df Mon Sep 17 00:00:00 2001 From: Chris Freeman Date: Wed, 25 Feb 2015 18:19:39 -0600 Subject: [PATCH 066/121] Restore `rdd` argument to `getJRDD` --- pkg/R/RDD.R | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 608ebe1ee2673..010bbd73182dd 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -97,18 +97,18 @@ PipelinedRDD <- function(prev, func) { # The jrdd accessor function. -setGeneric("getJRDD", function(x, ...) { standardGeneric("getJRDD") }) -setMethod("getJRDD", signature(x = "RDD"), function(x) x@jrdd ) -setMethod("getJRDD", signature(x = "PipelinedRDD"), - function(x, dataSerialization = TRUE) { - if (!is.null(x@env$jrdd_val)) { - return(x@env$jrdd_val) +setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) +setMethod("getJRDD", signature(rdd = "RDD"), function(rdd) rdd@jrdd ) +setMethod("getJRDD", signature(rdd = "PipelinedRDD"), + function(rdd, dataSerialization = TRUE) { + if (!is.null(rdd@env$jrdd_val)) { + return(rdd@env$jrdd_val) } # TODO: This is to handle anonymous functions. Find out a # better way to do this. computeFunc <- function(split, part) { - x@func(split, part) + rdd@func(split, part) } serializedFuncArr <- serialize("computeFunc", connection = NULL) @@ -120,13 +120,13 @@ setMethod("getJRDD", signature(x = "PipelinedRDD"), depsBin <- getDependencies(computeFunc) - prev_jrdd <- x@prev_jrdd + prev_jrdd <- rdd@prev_jrdd if (dataSerialization) { rddRef <- newJObject("edu.berkeley.cs.amplab.sparkr.RRDD", callJMethod(prev_jrdd, "rdd"), serializedFuncArr, - x@env$prev_serialized, + rdd@env$prev_serialized, depsBin, packageNamesArr, as.character(.sparkREnv[["libname"]]), @@ -136,7 +136,7 @@ setMethod("getJRDD", signature(x = "PipelinedRDD"), rddRef <- newJObject("edu.berkeley.cs.amplab.sparkr.StringRRDD", callJMethod(prev_jrdd, "rdd"), serializedFuncArr, - x@env$prev_serialized, + rdd@env$prev_serialized, depsBin, packageNamesArr, as.character(.sparkREnv[["libname"]]), @@ -144,9 +144,9 @@ setMethod("getJRDD", signature(x = "PipelinedRDD"), callJMethod(prev_jrdd, "classTag")) } # Save the serialization flag after we create a RRDD - x@env$serialized <- dataSerialization - x@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() - x@env$jrdd_val + rdd@env$serialized <- dataSerialization + rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() + rdd@env$jrdd_val }) From 68d6de4cd2914f1dec12de3fb287cfa6d71c3142 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Thu, 26 Feb 2015 09:08:52 -0600 Subject: [PATCH 067/121] Fix typos --- pkg/R/RDD.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 010bbd73182dd..48aa66af93f42 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -265,7 +265,7 @@ setGeneric("unpersist", function(x) { standardGeneric("unpersist") }) setMethod("unpersist", signature(x = "RDD"), function(x) { - callJMethod(getJRDD(x, "unpersist")) + callJMethod(getJRDD(x), "unpersist") x@env$isCached <- FALSE x }) @@ -399,7 +399,7 @@ setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) setMethod("collectAsMap", signature(x = "RDD"), function(x) { - pairList <- collect(rdd) + pairList <- collect(x) map <- new.env() lapply(pairList, function(i) { assign(as.character(i[[1]]), i[[2]], envir = map) }) as.list(map) From f4dbb0b4a591010d0d7ee7e841174cd033756b6f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 26 Feb 2015 11:47:51 -0800 Subject: [PATCH 068/121] use socket in worker --- pkg/inst/worker/worker.R | 24 +--- .../edu/berkeley/cs/amplab/sparkr/RRDD.scala | 112 +++++++++--------- 2 files changed, 58 insertions(+), 78 deletions(-) diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index c5457adcbc54d..2ba5211ab1572 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -1,23 +1,18 @@ # Worker class -# NOTE: We use "stdin" to get the process stdin instead of the command line -inputConStdin <- file("stdin", open = "rb") +port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) -outputFileName <- readLines(inputConStdin, n = 1) -outputCon <- file(outputFileName, open="wb") +inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") +outputCon <- socketConnection(port = port, blocking = TRUE, open = "wb") # 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 -rLibDir <- readLines(inputConStdin, n = 1) +rLibDir <- readLines(inputCon, n = 1) .libPaths(c(rLibDir, .libPaths())) suppressPackageStartupMessages(library(SparkR)) -inFileName <- readLines(inputConStdin, n = 1) - -inputCon <- file(inFileName, open = "rb") - # read the index of the current partition inside the RDD splitIndex <- SparkR:::readInt(inputCon) @@ -31,10 +26,6 @@ isInputSerialized <- SparkR:::readInt(inputCon) # read the isOutputSerialized bit flag isOutputSerialized <- SparkR:::readInt(inputCon) -# Redirect stdout to stderr to prevent print statements from -# interfering with outputStream -sink(stderr()) - # Include packages as required packageNames <- unserialize(SparkR:::readRaw(inputCon)) for (pkg in packageNames) { @@ -123,10 +114,3 @@ if (isOutputSerialized) { close(outputCon) close(inputCon) -unlink(inFileName) - -# Restore stdout -sink() - -# Finally print the name of the output file -cat(outputFileName, "\n") diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 74d5837fb9384..528aa2a3984c8 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -1,6 +1,7 @@ package edu.berkeley.cs.amplab.sparkr import java.io._ +import java.net.{ServerSocket} import java.util.{Map => JMap} import scala.collection.JavaConversions._ @@ -12,6 +13,8 @@ import org.apache.spark.api.java.{JavaSparkContext, JavaRDD, JavaPairRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import scala.util.Try + private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( parent: RDD[T], numPartitions: Int, @@ -27,21 +30,32 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[U] = { - val parentIterator = firstParent[T].iterator(split, context) + val serverSocket = new ServerSocket(0, 2) + val listenPort = serverSocket.getLocalPort() - val pb = rWorkerProcessBuilder() + val pb = rWorkerProcessBuilder(listenPort) + pb.redirectErrorStream() // redirect stderr into stdout val proc = pb.start() + val errThread = startStdoutThread(proc) - val errThread = startStderrThread(proc) + // We use two socket ot 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() + val parentIterator = firstParent[T].iterator(split, context) + startStdinThread(inSocket.getOutputStream(), parentIterator, split.index) - val tempFile = startStdinThread(proc, parentIterator, split.index) + // the socket used to receive the output of task + val outSocket = serverSocket.accept() + val inputStream = new BufferedInputStream(outSocket.getInputStream) + val dataStream = openDataStream(inputStream) - // Return an iterator that read lines from the process's stdout - val inputStream = new BufferedReader(new InputStreamReader(proc.getInputStream)) + serverSocket.close() try { - val stdOutFileName = inputStream.readLine().trim() - val dataStream = openDataStream(stdOutFileName) return new Iterator[U] { def next(): U = { @@ -57,9 +71,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( def hasNext(): Boolean = { val hasMore = (_nextObj != null) if (!hasMore) { - // Delete the temporary file we created as we are done reading it dataStream.close() - tempFile.delete() } hasMore } @@ -73,7 +85,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( /** * ProcessBuilder used to launch worker R processes. */ - private def rWorkerProcessBuilder() = { + private def rWorkerProcessBuilder(port: Int) = { val rCommand = "Rscript" val rOptions = "--vanilla" val rExecScript = rLibDir + "/SparkR/worker/worker.R" @@ -82,47 +94,42 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( // 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("R_TESTS", "") + pb.environment().put("SPARKR_WORKER_PORT", port.toString) pb } /** * Start a thread to print the process's stderr to ours */ - private def startStderrThread(proc: Process): BufferedStreamThread = { - val ERR_BUFFER_SIZE = 100 - val errThread = new BufferedStreamThread(proc.getErrorStream, "stderr reader for R", - ERR_BUFFER_SIZE) - errThread.start() - errThread + 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 } /** * Start a thread to write RDD data to the R process. */ private def startStdinThread[T]( - proc: Process, + output: OutputStream, iter: Iterator[T], - splitIndex: Int) : File = { + splitIndex: Int) = { val env = SparkEnv.get - val conf = env.conf - val tempDir = RRDD.getLocalDir(conf) - val tempFile = File.createTempFile("rSpark", "out", new File(tempDir)) - val tempFileIn = File.createTempFile("rSpark", "in", new File(tempDir)) - - val tempFileName = tempFile.getAbsolutePath() val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val stream = new BufferedOutputStream(output, bufferSize) - // Start a thread to feed the process input from our parent's iterator - new Thread("stdin writer for R") { + new Thread("writer for R") { override def run() { try { SparkEnv.set(env) - val stream = new BufferedOutputStream(new FileOutputStream(tempFileIn), bufferSize) val printOut = new PrintStream(stream) - val dataOut = new DataOutputStream(stream) + printOut.println(rLibDir) + val dataOut = new DataOutputStream(stream) dataOut.writeInt(splitIndex) dataOut.writeInt(func.length) @@ -157,44 +164,33 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( dataOut.writeInt(1) } - for (elem <- iter) { - if (parentSerialized) { + if (parentSerialized) { + for (elem <- iter) { val elemArr = elem.asInstanceOf[Array[Byte]] dataOut.writeInt(elemArr.length) dataOut.write(elemArr, 0, elemArr.length) - } else { + } + } else { + for (elem <- iter) { printOut.println(elem) } } - printOut.flush() - dataOut.flush() stream.flush() - stream.close() - - // NOTE: We need to write out the temp file before writing out the - // file name to stdin. Otherwise the R process could read partial state - val streamStd = new BufferedOutputStream(proc.getOutputStream, bufferSize) - val printOutStd = new PrintStream(streamStd) - printOutStd.println(tempFileName) - printOutStd.println(rLibDir) - printOutStd.println(tempFileIn.getAbsolutePath()) - printOutStd.flush() - - streamStd.close() } catch { // TODO: We should propogate this error to the task thread case e: Exception => System.err.println("R Writer thread got an exception " + e) e.printStackTrace() + } finally { + Try(output.close()) } } }.start() - - tempFile } - protected def openDataStream(stdOutFileName: String): Closeable + protected def openDataStream(input: InputStream): Closeable + protected def read(): U } @@ -217,8 +213,8 @@ private class PairwiseRRDD[T: ClassTag]( private var dataStream: DataInputStream = _ - override protected def openDataStream(stdOutFileName: String) = { - dataStream = new DataInputStream(new FileInputStream(stdOutFileName)) + override protected def openDataStream(input: InputStream) = { + dataStream = new DataInputStream(input) dataStream } @@ -261,9 +257,9 @@ private class RRDD[T: ClassTag]( broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { private var dataStream: DataInputStream = _ - - override protected def openDataStream(stdOutFileName: String) = { - dataStream = new DataInputStream(new FileInputStream(stdOutFileName)) + + override protected def openDataStream(input: InputStream) = { + dataStream = new DataInputStream(input) dataStream } @@ -305,9 +301,8 @@ private class StringRRDD[T: ClassTag]( private var dataStream: BufferedReader = _ - override protected def openDataStream(stdOutFileName: String) = { - dataStream = new BufferedReader( - new InputStreamReader(new FileInputStream(stdOutFileName))) + override protected def openDataStream(input: InputStream) = { + dataStream = new BufferedReader(new InputStreamReader(input)) dataStream } @@ -334,6 +329,7 @@ private class BufferedStreamThread( for (line <- Source.fromInputStream(in).getLines) { lines(lineIdx) = line lineIdx = (lineIdx + 1) % errBufferSize + // TODO: user logger System.err.println(line) } } From 07aa7c03e802e43738008f4ac7a1e2ccfa6ba0fe Mon Sep 17 00:00:00 2001 From: hlin09 Date: Thu, 26 Feb 2015 15:26:10 -0500 Subject: [PATCH 069/121] Unifies the implementation of lapply with lapplyParitionsWithIndex. --- pkg/R/RDD.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 48aa66af93f42..e50dfed02f6c7 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -493,7 +493,7 @@ setMethod("lapply", func <- function(split, iterator) { lapply(iterator, FUN) } - PipelinedRDD(X, func) + lapplyPartitionsWithIndex(X, func) }) #' @rdname lapply From f4f077ca1946dd16ee80532c940faafd219f44cc Mon Sep 17 00:00:00 2001 From: hlin09 Date: Thu, 26 Feb 2015 18:34:49 -0500 Subject: [PATCH 070/121] Add recursive cleanClosure for function access. --- pkg/R/utils.R | 16 ++++++++++++---- pkg/inst/tests/test_utils.R | 26 +++++++++++++++++++------- 2 files changed, 31 insertions(+), 11 deletions(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 64c43eccd4eb9..1cba009791674 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -350,7 +350,11 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { # Set parameter 'inherits' to FALSE since we do not need to search in # attached package environments. if (exists(nodeChar, envir=func.env, inherits = FALSE)) { - assign(nodeChar, get(nodeChar, envir=func.env), envir = newEnv) + obj <- get(nodeChar, envir=func.env) + if (is.function(obj)) { + obj <- cleanClosure(obj) + } + assign(nodeChar, obj, envir = newEnv) break } else { # Continue to search in enclosure. @@ -366,9 +370,11 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { # outside a UDF, and stores them in a new environment. # param # func A function whose closure needs to be captured. -# newEnv A new function environment to store necessary function dependencies. -cleanClosure <- function(func, newEnv) { - if (is.function(func) && is.environment(newEnv)) { +# return value +# a new function that has an correct environment (closure). +cleanClosure <- function(func) { + if (is.function(func) { + newEnv <- new.env(parent = .GlobalEnv) # .defVars is a character vector of variables names defined in the function. assign(".defVars", c(), envir = .sparkREnv) func.body <- body(func) @@ -377,5 +383,7 @@ cleanClosure <- function(func, newEnv) { argsNames <- argNames[-length(argNames)] # Remove the ending NULL in pairlist. # Recursively examine variables in the function body. processClosure(func.body, oldEnv, argNames, newEnv) + environment(func) <- newEnv } + func } diff --git a/pkg/inst/tests/test_utils.R b/pkg/inst/tests/test_utils.R index 7317d0421d092..82232e8731944 100644 --- a/pkg/inst/tests/test_utils.R +++ b/pkg/inst/tests/test_utils.R @@ -36,12 +36,12 @@ test_that("reserialize on RDD", { unlink(fileName) }) -test_that("clean.closure on R functions", { +test_that("cleanClosure on R functions", { y <- c(1, 2, 3) g <- function(x) { x + 1 } f <- function(x) { g(x) + y } - env <- new.env() - cleanClosure(f, env) + newF <- cleanClosure(f) + env <- environment(newF) expect_equal(length(ls(env)), 2) # y, g actual <- get("y", envir = env) expect_equal(actual, y) @@ -53,20 +53,32 @@ test_that("clean.closure on R functions", { funcEnv <- new.env(parent = env2) f <- function(x) { min(g(x) + y) } environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv - env <- new.env() - SparkR:::cleanClosure(f, env) + newF <- SparkR:::cleanClosure(f) + env <- environment(newF) expect_equal(length(ls(env)), 2) # "min" should not be included actual <- get("y", envir = env) expect_equal(actual, y) actual <- get("g", envir = env) expect_equal(actual, g) + + g <- function(x) { x + y } + f <- function(x) { lapply(x, g) + 1 } + newF <- SparkR:::cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 1) # Only "g", "y" should be in the environemnt of g. + expect_equal(ls(env), "g") + newG <- get("g", envir = env) + env <- environment(newG) + expect_equal(length(ls(env)), 1) + actual <- get("y", envir = env) + expect_equal(actual, y) # Test for function (and variable) definitions. f <- function(x) { g <- function(y) { y * 2 } g(x) } - env <- new.env() - SparkR:::cleanClosure(f, env) + newF <- SparkR:::cleanClosure(f) + env <- environment(newF) expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. }) From 09b95126c0f6cb86d228ac20e5b4e31dff5eb68a Mon Sep 17 00:00:00 2001 From: hlin09 Date: Thu, 26 Feb 2015 19:55:20 -0500 Subject: [PATCH 071/121] add docs --- pkg/R/utils.R | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 1cba009791674..88d187461b197 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -351,7 +351,8 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { # attached package environments. if (exists(nodeChar, envir=func.env, inherits = FALSE)) { obj <- get(nodeChar, envir=func.env) - if (is.function(obj)) { + if (is.function(obj)) { + # if the node is a function call, recursively clean its closure. obj <- cleanClosure(obj) } assign(nodeChar, obj, envir = newEnv) @@ -373,7 +374,7 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { # return value # a new function that has an correct environment (closure). cleanClosure <- function(func) { - if (is.function(func) { + if (is.function(func)) { newEnv <- new.env(parent = .GlobalEnv) # .defVars is a character vector of variables names defined in the function. assign(".defVars", c(), envir = .sparkREnv) From 2ea2ecfab0d5cd3328bd343b9532ec505b7bd99a Mon Sep 17 00:00:00 2001 From: lythesia Date: Fri, 27 Feb 2015 09:32:17 +0800 Subject: [PATCH 072/121] use generic arg --- pkg/R/RDD.R | 24 ++++++++++++------------ pkg/man/coalesce.Rd | 6 +++--- pkg/man/repartition.Rd | 6 +++--- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 6d69f3e0cc7a0..111c1ffa052d2 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1045,7 +1045,7 @@ setMethod("keyBy", #' If you are decreasing the number of partitions in this RDD, consider using #' coalesce, which can avoid performing a shuffle. #' -#' @param rdd The RDD. +#' @param x The RDD. #' @param numPartitions Number of partitions to create. #' @rdname repartition #' @seealso coalesce @@ -1057,19 +1057,19 @@ setMethod("keyBy", #' numPartitions(rdd) # 4 #' numPartitions(repartition(rdd, 2L)) # 2 #'} -setGeneric("repartition", function(rdd, numPartitions) { standardGeneric("repartition") }) +setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) #' @rdname repartition #' @aliases repartition,RDD setMethod("repartition", - signature(rdd = "RDD", numPartitions = "numeric"), - function(rdd, numPartitions) { - coalesce(rdd, numPartitions, TRUE) + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions) { + coalesce(x, numPartitions, TRUE) }) #' Return a new RDD that is reduced into numPartitions partitions. #' -#' @param rdd The RDD. +#' @param x The RDD. #' @param numPartitions Number of partitions to create. #' @rdname coalesce #' @seealso repartition @@ -1081,18 +1081,18 @@ setMethod("repartition", #' numPartitions(rdd) # 3 #' numPartitions(coalesce(rdd, 1L)) # 1 #'} -setGeneric("coalesce", function(rdd, numPartitions, ...) { standardGeneric("coalesce") }) +setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) #' @rdname coalesce #' @aliases coalesce,RDD setMethod("coalesce", - signature(rdd = "RDD", numPartitions = "numeric"), - function(rdd, numPartitions, shuffle = FALSE) { + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions, shuffle = FALSE) { if(as.integer(numPartitions) != numPartitions) { warning("Number of partitions should be an integer. Coercing it to integer.") } numPartitions <- as.integer(numPartitions) - if (shuffle || numPartitions > SparkR::numPartitions(rdd)) { + if (shuffle || numPartitions > SparkR::numPartitions(x)) { func <- function(s, part) { set.seed(s) # split as seed lapply(part, @@ -1101,11 +1101,11 @@ setMethod("coalesce", list(k, v) }) } - shuffled <- lapplyPartitionsWithIndex(rdd, func) + shuffled <- lapplyPartitionsWithIndex(x, func) reparted <- partitionBy(shuffled, numPartitions) values(reparted) } else { - jrdd <- callJMethod(getJRDD(rdd), "coalesce", numPartitions, shuffle) + jrdd <- callJMethod(getJRDD(x), "coalesce", numPartitions, shuffle) RDD(jrdd) } }) diff --git a/pkg/man/coalesce.Rd b/pkg/man/coalesce.Rd index f308e9a853d8e..b3b9badb73060 100644 --- a/pkg/man/coalesce.Rd +++ b/pkg/man/coalesce.Rd @@ -7,12 +7,12 @@ \alias{coalesce,RDD,integer-method} \title{Return a new RDD that is reduced into numPartitions partitions.} \usage{ -coalesce(rdd, numPartitions, ...) +coalesce(x, numPartitions, ...) -\S4method{coalesce}{RDD,integer}(rdd, numPartitions, shuffle = FALSE) +\S4method{coalesce}{RDD,integer}(x, numPartitions, shuffle = FALSE) } \arguments{ -\item{rdd}{The RDD.} +\item{x}{The RDD.} \item{numPartitions}{Number of partitions to create.} } diff --git a/pkg/man/repartition.Rd b/pkg/man/repartition.Rd index 1bbf672ec8231..6dff0cd251152 100644 --- a/pkg/man/repartition.Rd +++ b/pkg/man/repartition.Rd @@ -11,12 +11,12 @@ this uses a shuffle to redistribute data. If you are decreasing the number of partitions in this RDD, consider using coalesce, which can avoid performing a shuffle.} \usage{ -repartition(rdd, numPartitions) +repartition(x, numPartitions) -\S4method{repartition}{RDD,integer}(rdd, numPartitions) +\S4method{repartition}{RDD,integer}(x, numPartitions) } \arguments{ -\item{rdd}{The RDD.} +\item{x}{The RDD.} \item{numPartitions}{Number of partitions to create.} } From dfa119bbdbc091af52d14710b26a7dd8a615193c Mon Sep 17 00:00:00 2001 From: hlin09 Date: Fri, 27 Feb 2015 00:18:54 -0500 Subject: [PATCH 073/121] Improve the coverage of processClosure. --- pkg/R/utils.R | 114 ++++++++++++++++++++++++++++---------------------- 1 file changed, 64 insertions(+), 50 deletions(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 88d187461b197..d04fc52f58735 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -295,7 +295,8 @@ convertEnvsToList <- function(keys, vals) { # param # node The current AST node in the traversal. # oldEnv The original function environment. -# argNames The argument names of the function. +# argNames A character vector of parameters of the function. Their values are +# passed in as arguments, and not included in the closure. # newEnv A new function environment to store necessary function dependencies. processClosure <- function(node, oldEnv, argNames, newEnv) { nodeLen <- length(node) @@ -304,39 +305,46 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { } if (nodeLen > 1 && typeof(node) == "language") { # Recursive case: current AST node is an internal node, check for its children. - nodeChar <- as.character(node[[1]]) - switch(nodeChar, - "{" = { # Start of a function body. - for (i in 2:nodeLen) { - processClosure(node[[i]], oldEnv, argNames, newEnv) - } - }, - "<-" = { # Assignment. - defVar <- node[[2]] - if (length(defVar) == 1 && typeof(defVar) == "symbol") { - # Add the defined variable name into .defVars. - assign(".defVars", - c(get(".defVars", envir = .sparkREnv), as.character(defVar)), - envir = .sparkREnv) - } - for (i in 3:nodeLen) { - processClosure(node[[i]], oldEnv, argNames, newEnv) - } - }, - "function" = { # Function definition. - newArgs <- names(node[[2]]) - argNames <- c(argNames, newArgs) # Add parameter names. - for (i in 3:nodeLen) { - processClosure(node[[i]], oldEnv, argNames, newEnv) - } - }, - { - for (i in 1:nodeLen) { - processClosure(node[[i]], oldEnv, argNames, newEnv) - } - }) - } else if (nodeLen == 1 && typeof(node) == "symbol") { - # Base case: current AST node is a leaf node and a symbol. + if (length(node[[1]]) > 1) { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + } else { # if node[[1]] is length of 1, might be an R primitive. + nodeChar <- as.character(node[[1]]) + switch(nodeChar, + "{" = { # Start of a function body. + for (i in 2:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + }, + "<-" = { # Assignment. + defVar <- node[[2]] + if (length(defVar) == 1 && typeof(defVar) == "symbol") { + # Add the defined variable name into .defVars. + assign(".defVars", + c(get(".defVars", envir = .sparkREnv), as.character(defVar)), + envir = .sparkREnv) + } + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + }, + "function" = { # Function definition. + newArgs <- names(node[[2]]) + argNames <- c(argNames, newArgs) # Add parameter names. + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + }, + { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, 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% argNames && # Not a function parameter or function local variable. !nodeChar %in% get(".defVars", envir = .sparkREnv)) { @@ -344,23 +352,29 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { 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 below the global, as they are - # assumed to be loaded on workers. - while (!identical(func.env, topEnv) && !isNamespace(func.env)) { - # Set parameter 'inherits' to FALSE since we do not need to search in - # attached package environments. - if (exists(nodeChar, envir=func.env, inherits = FALSE)) { - obj <- get(nodeChar, envir=func.env) - if (is.function(obj)) { - # if the node is a function call, recursively clean its closure. - obj <- cleanClosure(obj) + # 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") { + # Set parameter 'inherits' to FALSE since we do not need to search in + # attached package environments. + if (exists(nodeChar, envir = func.env, inherits = FALSE)) { + if (!isNamespace(func.env) || + !nodeChar %in% getNamespaceExports("SparkR")) { # Only include SparkR internals. + obj <- get(nodeChar, envir = func.env) + if (is.function(obj)) { + # if the node is a function call, recursively clean its closure. + obj <- cleanClosure(obj) + } + assign(nodeChar, obj, envir = newEnv) + break + } } - assign(nodeChar, obj, envir = newEnv) - break - } else { - # Continue to search in enclosure. - func.env <- parent.env(func.env) } + + # Continue to search in enclosure. + func.env <- parent.env(func.env) } } } @@ -372,7 +386,7 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { # param # func A function whose closure needs to be captured. # return value -# a new function that has an correct environment (closure). +# a new version of func that has an correct environment (closure). cleanClosure <- function(func) { if (is.function(func)) { newEnv <- new.env(parent = .GlobalEnv) From 50c74b1cbeab5f3d48a0ddd2655a5c846455e351 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 26 Feb 2015 21:56:21 -0800 Subject: [PATCH 074/121] address comments --- .../edu/berkeley/cs/amplab/sparkr/RRDD.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 528aa2a3984c8..54680a0708b61 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -30,6 +30,10 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[U] = { + // The parent may be also an RRDD, so we should launch it first. + val parentIterator = firstParent[T].iterator(split, context) + + // we expect two connections val serverSocket = new ServerSocket(0, 2) val listenPort = serverSocket.getLocalPort() @@ -38,14 +42,13 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( val proc = pb.start() val errThread = startStdoutThread(proc) - // We use two socket ot separate input and output, then it's easy to manage + // 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() - val parentIterator = firstParent[T].iterator(split, context) startStdinThread(inSocket.getOutputStream(), parentIterator, split.index) // the socket used to receive the output of task @@ -164,18 +167,15 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( dataOut.writeInt(1) } - if (parentSerialized) { - for (elem <- iter) { + for (elem <- iter) { + if (parentSerialized) { val elemArr = elem.asInstanceOf[Array[Byte]] dataOut.writeInt(elemArr.length) dataOut.write(elemArr, 0, elemArr.length) - } - } else { - for (elem <- iter) { + } else { printOut.println(elem) } } - stream.flush() } catch { // TODO: We should propogate this error to the task thread From e7c56d65316dfa177cfe585bcc192fdda2ebf26a Mon Sep 17 00:00:00 2001 From: lythesia Date: Fri, 27 Feb 2015 22:23:27 +0800 Subject: [PATCH 075/121] fix random partition key --- pkg/R/RDD.R | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 111c1ffa052d2..ae1f5d33e1e36 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1095,10 +1095,11 @@ setMethod("coalesce", if (shuffle || numPartitions > SparkR::numPartitions(x)) { func <- function(s, part) { set.seed(s) # split as seed - lapply(part, - function(v) { - k <- as.integer(runif(1, 0, numPartitions)) - list(k, v) + start <- as.integer(runif(1, 0, numPartitions)) + lapply(seq_along(part), + function(i) { + pos <- (start + i) %% numPartitions + list(pos, part[[i]]) }) } shuffled <- lapplyPartitionsWithIndex(x, func) From 3351afd42a9c88ef602e1ed325a8d8814064716a Mon Sep 17 00:00:00 2001 From: hlin09 Date: Fri, 27 Feb 2015 10:07:41 -0500 Subject: [PATCH 076/121] Replaces getDependencies with cleanClosure, to serialize UDFs to workers. --- pkg/R/RDD.R | 10 +++--- pkg/R/pairRDD.R | 9 ++--- pkg/R/utils.R | 34 ------------------- pkg/inst/tests/test_rdd.R | 5 ++- pkg/inst/worker/worker.R | 18 ++++------ .../edu/berkeley/cs/amplab/sparkr/RRDD.scala | 31 +++++++---------- 6 files changed, 30 insertions(+), 77 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index e50dfed02f6c7..ce03623801f79 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -110,7 +110,6 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), computeFunc <- function(split, part) { rdd@func(split, part) } - serializedFuncArr <- serialize("computeFunc", connection = NULL) packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) @@ -118,16 +117,15 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - depsBin <- getDependencies(computeFunc) + serializedFuncArr <- serialize(computeFunc, connection = NULL) prev_jrdd <- rdd@prev_jrdd if (dataSerialization) { rddRef <- newJObject("edu.berkeley.cs.amplab.sparkr.RRDD", callJMethod(prev_jrdd, "rdd"), - serializedFuncArr, rdd@env$prev_serialized, - depsBin, + serializedFuncArr, packageNamesArr, as.character(.sparkREnv[["libname"]]), broadcastArr, @@ -135,9 +133,8 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), } else { rddRef <- newJObject("edu.berkeley.cs.amplab.sparkr.StringRRDD", callJMethod(prev_jrdd, "rdd"), - serializedFuncArr, rdd@env$prev_serialized, - depsBin, + serializedFuncArr, packageNamesArr, as.character(.sparkREnv[["libname"]]), broadcastArr, @@ -610,6 +607,7 @@ setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { + FUN <- cleanClosure(FUN) closureCapturingFunc <- function(split, part) { FUN(split, part) } diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R index a54c1abe01b47..2cd5d83cf383b 100644 --- a/pkg/R/pairRDD.R +++ b/pkg/R/pairRDD.R @@ -209,10 +209,8 @@ setMethod("partitionBy", # partitionFunc <- hashCode #} - depsBinArr <- getDependencies(partitionFunc) - - serializedHashFuncBytes <- serialize(as.character(substitute(partitionFunc)), - connection = NULL) + partitionFunc <- cleanClosure(partitionFunc) + serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL) packageNamesArr <- serialize(.sparkREnv$.packages, connection = NULL) @@ -226,9 +224,8 @@ setMethod("partitionBy", pairwiseRRDD <- newJObject("edu.berkeley.cs.amplab.sparkr.PairwiseRRDD", callJMethod(jrdd, "rdd"), as.integer(numPartitions), - serializedHashFuncBytes, x@env$serialized, - depsBinArr, + serializedHashFuncBytes, packageNamesArr, as.character(.sparkREnv$libname), broadcastArr, diff --git a/pkg/R/utils.R b/pkg/R/utils.R index d04fc52f58735..523f14116f414 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -93,40 +93,6 @@ isSparkFunction <- function(name) { packageName(environment(fun)) == "SparkR" } -# Serialize the dependencies of the given function and return them as a raw -# vector. Filters out RDDs before serializing the dependencies -getDependencies <- function(name) { - varsToSave <- c() - closureEnv <- environment(name) - - currentEnv <- closureEnv - while (TRUE) { - # Don't serialize namespaces - if (!isNamespace(currentEnv)) { - varsToSave <- c(varsToSave, ls(currentEnv)) - } - - # Everything below globalenv are packages, search path stuff etc. - if (identical(currentEnv, globalenv())) - break - currentEnv <- parent.env(currentEnv) - } - filteredVars <- Filter(function(x) { !isRDD(x, closureEnv) }, varsToSave) - - # TODO: A better way to exclude variables that have been broadcast - # would be to actually list all the variables used in every function using - # `all.vars` and then walking through functions etc. - filteredVars <- Filter( - function(x) { !exists(x, .broadcastNames, inherits = FALSE) }, - filteredVars) - - rc <- rawConnection(raw(), 'wb') - save(list = filteredVars, file = rc, envir = closureEnv) - binData <- rawConnectionValue(rc) - close(rc) - binData -} - #' Compute the hashCode of an object #' #' Java-style function to compute the hashCode for the given object. Returns diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index f0c00d71d076c..3b131fca94bef 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -86,7 +86,10 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { part <- as.list(unlist(part) * split + i) }) rdd2 <- lapply(rdd2, function(x) x + x) - collect(rdd2) + 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()", { diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index c5457adcbc54d..f427f8572b5b6 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -21,10 +21,6 @@ inputCon <- file(inFileName, open = "rb") # read the index of the current partition inside the RDD splitIndex <- SparkR:::readInt(inputCon) -# read the function; if used for pairwise RRDD, this is the hash function. -execLen <- SparkR:::readInt(inputCon) -execFunctionName <- unserialize(SparkR:::readRawLen(inputCon, execLen)) - # read the isInputSerialized bit flag isInputSerialized <- SparkR:::readInt(inputCon) @@ -42,11 +38,11 @@ for (pkg in packageNames) { } # read function dependencies -depsLen <- SparkR:::readInt(inputCon) -if (depsLen > 0) { - execFunctionDeps <- SparkR:::readRawLen(inputCon, depsLen) - # load the dependencies into current environment - load(rawConnection(execFunctionDeps, open='rb')) +funcLen <- SparkR:::readInt(inputCon) +if (funcLen > 0) { + computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen)) + env <- environment(computeFunc) + parent.env(env) <- .GlobalEnv # Attach under global environment. } # Read and set broadcast variables @@ -74,7 +70,7 @@ if (isEmpty != 0) { } else { data <- readLines(inputCon) } - output <- do.call(execFunctionName, list(splitIndex, data)) + output <- computeFunc(splitIndex, data) if (isOutputSerialized) { SparkR:::writeRawSerialize(outputCon, output) } else { @@ -93,7 +89,7 @@ if (isEmpty != 0) { # Step 1: hash the data to an environment hashTupleToEnvir <- function(tuple) { # NOTE: execFunction is the hash function here - hashVal <- do.call(execFunctionName, list(tuple[[1]])) + hashVal <- computeFunc(tuple[[1]]) bucket <- as.character(hashVal %% numPartitions) acc <- res[[bucket]] # Create a new accumulator diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 74d5837fb9384..d20e40eaf4ef4 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -15,10 +15,9 @@ import org.apache.spark.rdd.RDD private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( parent: RDD[T], numPartitions: Int, - func: Array[Byte], parentSerialized: Boolean, dataSerialized: Boolean, - functionDependencies: Array[Byte], + func: Array[Byte], packageNames: Array[Byte], rLibDir: String, broadcastVars: Array[Broadcast[Object]]) @@ -125,9 +124,6 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( dataOut.writeInt(splitIndex) - dataOut.writeInt(func.length) - dataOut.write(func, 0, func.length) - // R worker process input serialization flag dataOut.writeInt(if (parentSerialized) 1 else 0) // R worker process output serialization flag @@ -136,8 +132,8 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( dataOut.writeInt(packageNames.length) dataOut.write(packageNames, 0, packageNames.length) - dataOut.writeInt(functionDependencies.length) - dataOut.write(functionDependencies, 0, functionDependencies.length) + dataOut.writeInt(func.length) + dataOut.write(func, 0, func.length) dataOut.writeInt(broadcastVars.length) broadcastVars.foreach { broadcast => @@ -205,14 +201,13 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( private class PairwiseRRDD[T: ClassTag]( parent: RDD[T], numPartitions: Int, - hashFunc: Array[Byte], parentSerialized: Boolean, - functionDependencies: Array[Byte], + hashFunc: Array[Byte], packageNames: Array[Byte], rLibDir: String, broadcastVars: Array[Object]) - extends BaseRRDD[T, (Int, Array[Byte])](parent, numPartitions, hashFunc, parentSerialized, - true, functionDependencies, packageNames, rLibDir, + extends BaseRRDD[T, (Int, Array[Byte])](parent, numPartitions, parentSerialized, + true, hashFunc, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { private var dataStream: DataInputStream = _ @@ -250,14 +245,13 @@ private class PairwiseRRDD[T: ClassTag]( */ private class RRDD[T: ClassTag]( parent: RDD[T], - func: Array[Byte], parentSerialized: Boolean, - functionDependencies: Array[Byte], + func: Array[Byte], packageNames: Array[Byte], rLibDir: String, broadcastVars: Array[Object]) - extends BaseRRDD[T, Array[Byte]](parent, -1, func, parentSerialized, - true, functionDependencies, packageNames, rLibDir, + extends BaseRRDD[T, Array[Byte]](parent, -1, parentSerialized, + true, func, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { private var dataStream: DataInputStream = _ @@ -293,14 +287,13 @@ private class RRDD[T: ClassTag]( */ private class StringRRDD[T: ClassTag]( parent: RDD[T], - func: Array[Byte], parentSerialized: Boolean, - functionDependencies: Array[Byte], + func: Array[Byte], packageNames: Array[Byte], rLibDir: String, broadcastVars: Array[Object]) - extends BaseRRDD[T, String](parent, -1, func, parentSerialized, - false, functionDependencies, packageNames, rLibDir, + extends BaseRRDD[T, String](parent, -1, parentSerialized, + false, func, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { private var dataStream: BufferedReader = _ From e776324ccbfc074aec317c898b44bd791eef4017 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 27 Feb 2015 09:52:41 -0800 Subject: [PATCH 077/121] fix import --- pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 54680a0708b61..a28d1dcd8636a 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -7,13 +7,13 @@ 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.{SparkEnv, Partition, SparkException, TaskContext, SparkConf} import org.apache.spark.api.java.{JavaSparkContext, JavaRDD, JavaPairRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import scala.util.Try private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( parent: RDD[T], From 5ef66fb8b03a635e309a5004a1b411b50f63ef9c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 27 Feb 2015 14:33:07 -0800 Subject: [PATCH 078/121] send back the port via temporary file --- pkg/R/sparkR.R | 24 ++++++++++++------- .../cs/amplab/sparkr/SparkRBackend.scala | 16 +++++++------ 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 4e89d823af876..58155a57e0e80 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -119,30 +119,38 @@ sparkR.init <- function( if (sparkRExistingPort != "") { sparkRBackendPort <- sparkRExistingPort } else { - # TODO: test the random port and retry, or use httpuv:::startServer - callbackPort = sample(40000, 1) + 10000 + path <- tempfile(pattern = "backend_port") if (Sys.getenv("SPARKR_USE_SPARK_SUBMIT", "") == "") { launchBackend(classPath = cp, mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", - args = as.character(callbackPort), + args = path, javaOpts = paste("-Xmx", sparkMem, sep = "")) } else { # TODO: We should deprecate sparkJars and ask users to add it to the # command line (using --jars) which is picked up by SparkSubmit launchBackendSparkSubmit( mainClass = "edu.berkeley.cs.amplab.sparkr.SparkRBackend", - args = as.character(callbackPort), + args = path, appJar = .sparkREnv$assemblyJarPath, sparkHome = sparkHome, sparkSubmitOpts = Sys.getenv("SPARKR_SUBMIT_ARGS", "")) } - sock <- socketConnection(port = callbackPort, server = TRUE, open = 'rb', - blocking = TRUE, timeout = 10) - sparkRBackendPort <- readInt(sock) + for (i in 1:100) { + if (file.exists(path)) { + break + } + Sys.sleep(0.1) + } + if (!file.exists(path)) { + stop("JVM is not ready after 10 seconds") + } + f <- file(path, open='rb') + sparkRBackendPort <- readInt(f) + close(f) + file.remove(path) if (length(sparkRBackendPort) == 0) { stop("JVM failed to launch") } - close(sock) } .sparkREnv$sparkRBackendPort <- sparkRBackendPort diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala index 111b7cf678d03..1f3f25859f4a7 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala @@ -1,6 +1,6 @@ package edu.berkeley.cs.amplab.sparkr -import java.io.{DataOutputStream, IOException} +import java.io.{File, FileOutputStream, DataOutputStream, IOException} import java.net.{InetSocketAddress, Socket} import java.util.concurrent.TimeUnit @@ -78,20 +78,22 @@ class SparkRBackend { object SparkRBackend { def main(args: Array[String]) { if (args.length < 1) { - System.err.println("Usage: SparkRBackend ") + System.err.println("Usage: SparkRBackend ") System.exit(-1) } val sparkRBackend = new SparkRBackend() try { // bind to random port val boundPort = sparkRBackend.init() - val callbackPort = args(0).toInt - val callbackSocket = new Socket("localhost", callbackPort) - val dos = new DataOutputStream(callbackSocket.getOutputStream) + // tell the R process via temporary file + val path = args(0) + val f = new File(path + ".tmp") + val output = new FileOutputStream(f) + val dos = new DataOutputStream(output) dos.writeInt(boundPort) dos.close() - callbackSocket.close() - + output.close() + f.renameTo(new File(path)) sparkRBackend.run() } catch { case e: IOException => From 27dd3a09ce37d8afe385ccda35b425ac5655905c Mon Sep 17 00:00:00 2001 From: lythesia Date: Sat, 28 Feb 2015 10:00:41 +0800 Subject: [PATCH 079/121] modify tests for repartition --- pkg/R/RDD.R | 4 ++-- pkg/inst/tests/test_rdd.R | 20 ++++++++++++-------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index ae1f5d33e1e36..abd57e5293a54 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1103,8 +1103,8 @@ setMethod("coalesce", }) } shuffled <- lapplyPartitionsWithIndex(x, func) - reparted <- partitionBy(shuffled, numPartitions) - values(reparted) + repartitioned <- partitionBy(shuffled, numPartitions) + values(repartitioned) } else { jrdd <- callJMethod(getJRDD(x), "coalesce", numPartitions, shuffle) RDD(jrdd) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index ffa749171bf16..50464c1cd22ba 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -268,17 +268,21 @@ test_that("keyBy on RDDs", { }) test_that("repartition/coalesce on RDDs", { - rdd <- parallelize(sc, 1:10, 3L) - expect_equal(numPartitions(rdd), 3L) + rdd <- parallelize(sc, 1:20, 4L) # each partition contains 5 elements - nrdd <- repartition(rdd, 2L) - expect_equal(numPartitions(nrdd), 2L) + # repartition + r1 <- repartition(rdd, 2) + count <- length(collectPartition(r1, 0L)) + expect_true(count >= 8 && count <= 12) - nrdd2 <- repartition(rdd, 5L) - expect_equal(numPartitions(nrdd2), 5L) + r2 <- repartition(rdd, 6) + count <- length(collectPartition(r2, 0L)) + expect_true(count >=0 && count <= 4) - nrdd3 <- coalesce(rdd, 1L) - expect_equal(numPartitions(nrdd3), 1L) + # coalesce + r3 <- coalesce(rdd, 1) + count <- length(collectPartition(r3, 0L)) + expect_equal(count, 20) }) test_that("sortBy() on RDDs", { From 7b0d070bc0fd18e26d94dfd4dbcc500963faa5bb Mon Sep 17 00:00:00 2001 From: lythesia Date: Sat, 28 Feb 2015 10:10:35 +0800 Subject: [PATCH 080/121] keep partitions check --- pkg/inst/tests/test_rdd.R | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 50464c1cd22ba..7c138dd870e2e 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -272,15 +272,18 @@ test_that("repartition/coalesce on RDDs", { # 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) }) From ad0935ef12fc6639a6ce45f1860d0f62c07ae838 Mon Sep 17 00:00:00 2001 From: lythesia Date: Sat, 28 Feb 2015 10:50:34 +0800 Subject: [PATCH 081/121] minor fixes --- pkg/R/RDD.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index abd57e5293a54..12e4691e801e2 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1088,14 +1088,14 @@ setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coales setMethod("coalesce", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions, shuffle = FALSE) { - if(as.integer(numPartitions) != numPartitions) { + if (as.integer(numPartitions) != numPartitions) { warning("Number of partitions should be an integer. Coercing it to integer.") } numPartitions <- as.integer(numPartitions) if (shuffle || numPartitions > SparkR::numPartitions(x)) { func <- function(s, part) { set.seed(s) # split as seed - start <- as.integer(runif(1, 0, numPartitions)) + start <- as.integer(sample(numPartitions, 1) - 1) lapply(seq_along(part), function(i) { pos <- (start + i) %% numPartitions From 0346e5fc907aab71aef122e6ddc1b96f93d9abbf Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 27 Feb 2015 23:05:42 -0800 Subject: [PATCH 082/121] address comment --- pkg/R/sparkRClient.R | 2 +- .../scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/pkg/R/sparkRClient.R b/pkg/R/sparkRClient.R index 4a06bb3364b50..1a747031586d2 100644 --- a/pkg/R/sparkRClient.R +++ b/pkg/R/sparkRClient.R @@ -2,7 +2,7 @@ # Creates a SparkR client connection object # if one doesn't already exist -connectBackend <- function(hostname, port, timeout = 6) { +connectBackend <- function(hostname, port, timeout = 6000) { if (exists(".sparkRcon", envir = .sparkREnv)) { if (isOpen(env[[".sparkRCon"]])) { cat("SparkRBackend client connection already exists\n") diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala index 1f3f25859f4a7..4cf7ac18f9425 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala @@ -88,11 +88,9 @@ object SparkRBackend { // tell the R process via temporary file val path = args(0) val f = new File(path + ".tmp") - val output = new FileOutputStream(f) - val dos = new DataOutputStream(output) + val dos = new DataOutputStream(new FileOutputStream(f)) dos.writeInt(boundPort) dos.close() - output.close() f.renameTo(new File(path)) sparkRBackend.run() } catch { From a00f5029279ca1e14afb4f1b63d91e946bddfd73 Mon Sep 17 00:00:00 2001 From: lythesia Date: Sat, 28 Feb 2015 15:43:58 +0800 Subject: [PATCH 083/121] fix indents --- pkg/R/RDD.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 12e4691e801e2..3a990a77ce531 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1089,11 +1089,11 @@ setMethod("coalesce", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions, shuffle = FALSE) { if (as.integer(numPartitions) != numPartitions) { - warning("Number of partitions should be an integer. Coercing it to integer.") + warning("Number of partitions should be an integer. Coercing it to integer.") } numPartitions <- as.integer(numPartitions) if (shuffle || numPartitions > SparkR::numPartitions(x)) { - func <- function(s, part) { + func <- function(s, part) { set.seed(s) # split as seed start <- as.integer(sample(numPartitions, 1) - 1) lapply(seq_along(part), From 5c72e73fb9e1971b66e359687807490a8fdc4d40 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 28 Feb 2015 00:08:51 -0800 Subject: [PATCH 084/121] wait atmost 100 seconds --- pkg/R/sparkR.R | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 58155a57e0e80..a623d84812d17 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -135,11 +135,14 @@ sparkR.init <- function( sparkHome = sparkHome, sparkSubmitOpts = Sys.getenv("SPARKR_SUBMIT_ARGS", "")) } - for (i in 1:100) { + # wait atmost 100 seconds for JVM to launch + wait <- 0.1 + for (i in 1:25) { + Sys.sleep(wait) if (file.exists(path)) { break } - Sys.sleep(0.1) + wait <- wait * 1.25 } if (!file.exists(path)) { stop("JVM is not ready after 10 seconds") From 7b7248759c228fe8b0d9418447f8e1fd7f71b723 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sun, 1 Mar 2015 12:20:37 -0500 Subject: [PATCH 085/121] Fix comments. --- pkg/R/utils.R | 84 ++++++++++++++++++------------------- pkg/inst/tests/test_utils.R | 9 ++-- pkg/inst/worker/worker.R | 8 ++-- 3 files changed, 48 insertions(+), 53 deletions(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 523f14116f414..8720b38950ca4 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -266,9 +266,7 @@ convertEnvsToList <- function(keys, vals) { # newEnv A new function environment to store necessary function dependencies. processClosure <- function(node, oldEnv, argNames, newEnv) { nodeLen <- length(node) - if (nodeLen == 0) { - return - } + if (nodeLen > 1 && typeof(node) == "language") { # Recursive case: current AST node is an internal node, check for its children. if (length(node[[1]]) > 1) { @@ -277,36 +275,35 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { } } else { # if node[[1]] is length of 1, might be an R primitive. nodeChar <- as.character(node[[1]]) - switch(nodeChar, - "{" = { # Start of a function body. - for (i in 2:nodeLen) { - processClosure(node[[i]], oldEnv, argNames, newEnv) - } - }, - "<-" = { # Assignment. - defVar <- node[[2]] - if (length(defVar) == 1 && typeof(defVar) == "symbol") { - # Add the defined variable name into .defVars. - assign(".defVars", - c(get(".defVars", envir = .sparkREnv), as.character(defVar)), - envir = .sparkREnv) - } - for (i in 3:nodeLen) { - processClosure(node[[i]], oldEnv, argNames, newEnv) - } - }, - "function" = { # Function definition. - newArgs <- names(node[[2]]) - argNames <- c(argNames, newArgs) # Add parameter names. - for (i in 3:nodeLen) { - processClosure(node[[i]], oldEnv, argNames, newEnv) - } - }, - { - for (i in 1:nodeLen) { - processClosure(node[[i]], oldEnv, argNames, newEnv) - } - }) + if (nodeChar == "{" || nodeChar == "(") { # Skip start symbol. + for (i in 2:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, 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. + assign(".defVars", + c(get(".defVars", envir = .sparkREnv), as.character(defVar)), + envir = .sparkREnv) + } else { + processClosure(node[[2]], oldEnv, argNames, newEnv) + } + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + } else if (nodeChar == "function") { # Function definition. + newArgs <- names(node[[2]]) + argNames <- c(argNames, newArgs) # Add parameter names. + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + } else { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, argNames, newEnv) + } + } } } else if (nodeLen == 1 && (typeof(node) == "symbol" || typeof(node) == "language")) { @@ -322,20 +319,19 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { # 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") { + 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 (exists(nodeChar, envir = func.env, inherits = FALSE)) { - if (!isNamespace(func.env) || - !nodeChar %in% getNamespaceExports("SparkR")) { # Only include SparkR internals. - obj <- get(nodeChar, envir = func.env) - if (is.function(obj)) { - # if the node is a function call, recursively clean its closure. - obj <- cleanClosure(obj) - } - assign(nodeChar, obj, envir = newEnv) - break + obj <- get(nodeChar, envir = func.env, inherits = FALSE) + if (is.function(obj)) { + # if the node is a function call, recursively clean its closure. + obj <- cleanClosure(obj) } + assign(nodeChar, obj, envir = newEnv) + break } } @@ -348,7 +344,7 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { # 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 a new environment. +# outside a UDF, and stores them in the function's environment. # param # func A function whose closure needs to be captured. # return value diff --git a/pkg/inst/tests/test_utils.R b/pkg/inst/tests/test_utils.R index 82232e8731944..c50350a1db958 100644 --- a/pkg/inst/tests/test_utils.R +++ b/pkg/inst/tests/test_utils.R @@ -53,7 +53,7 @@ test_that("cleanClosure on R functions", { funcEnv <- new.env(parent = env2) f <- function(x) { min(g(x) + y) } environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv - newF <- SparkR:::cleanClosure(f) + newF <- cleanClosure(f) env <- environment(newF) expect_equal(length(ls(env)), 2) # "min" should not be included actual <- get("y", envir = env) @@ -61,11 +61,12 @@ test_that("cleanClosure on R functions", { actual <- get("g", envir = env) expect_equal(actual, g) + # Test for recursive closure capture for a free variable of a function. g <- function(x) { x + y } f <- function(x) { lapply(x, g) + 1 } - newF <- SparkR:::cleanClosure(f) + newF <- cleanClosure(f) env <- environment(newF) - expect_equal(length(ls(env)), 1) # Only "g", "y" should be in the environemnt of g. + expect_equal(length(ls(env)), 1) # Only "g". "y" should be in the environemnt of g. expect_equal(ls(env), "g") newG <- get("g", envir = env) env <- environment(newG) @@ -78,7 +79,7 @@ test_that("cleanClosure on R functions", { g <- function(y) { y * 2 } g(x) } - newF <- SparkR:::cleanClosure(f) + newF <- cleanClosure(f) env <- environment(newF) expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. }) diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index f427f8572b5b6..48d6a5683e695 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -39,11 +39,9 @@ for (pkg in packageNames) { # read function dependencies funcLen <- SparkR:::readInt(inputCon) -if (funcLen > 0) { - computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen)) - env <- environment(computeFunc) - parent.env(env) <- .GlobalEnv # Attach under global environment. -} +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) From 3f57e56e3f67603bd2fda165370930fd39ad5117 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sun, 1 Mar 2015 15:43:01 -0500 Subject: [PATCH 086/121] Fix comments. --- pkg/R/utils.R | 6 +++--- pkg/inst/tests/test_utils.R | 17 +++++++++++++++-- 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 8720b38950ca4..a5cda89b80161 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -273,7 +273,7 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { for (i in 1:nodeLen) { processClosure(node[[i]], oldEnv, argNames, newEnv) } - } else { # if node[[1]] is length of 1, might be an R primitive. + } 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) { @@ -320,8 +320,8 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { 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. + (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 (exists(nodeChar, envir = func.env, inherits = FALSE)) { diff --git a/pkg/inst/tests/test_utils.R b/pkg/inst/tests/test_utils.R index c50350a1db958..970e9ab26f21c 100644 --- a/pkg/inst/tests/test_utils.R +++ b/pkg/inst/tests/test_utils.R @@ -48,10 +48,10 @@ test_that("cleanClosure on R functions", { actual <- get("g", envir = env) expect_equal(actual, g) - # Check for nested enclosures and package variables. + # Test for nested enclosures and package variables. env2 <- new.env() funcEnv <- new.env(parent = env2) - f <- function(x) { min(g(x) + y) } + f <- function(x) { log(g(x) + y) } environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv newF <- cleanClosure(f) env <- environment(newF) @@ -82,4 +82,17 @@ test_that("cleanClosure on R functions", { 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), t) + actual <- collect(lapply(rdd, f)) + expected <- as.list(c(rep(FALSE, 4), rep(TRUE, 6))) + expect_equal(actual, expected) }) From 4d36ab10389a6bccb0385a519ce0ce36dfc46696 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sun, 1 Mar 2015 16:33:53 -0500 Subject: [PATCH 087/121] Add tests for broadcast variables. --- pkg/R/utils.R | 2 ++ pkg/inst/tests/test_utils.R | 23 ++++++++++++++++------- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index a5cda89b80161..108b577f456c0 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -299,6 +299,8 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { for (i in 3:nodeLen) { processClosure(node[[i]], oldEnv, argNames, newEnv) } + } else if (nodeChar == "$") { # Skip the field. + processClosure(node[[2]], oldEnv, argNames, newEnv) } else { for (i in 1:nodeLen) { processClosure(node[[i]], oldEnv, argNames, newEnv) diff --git a/pkg/inst/tests/test_utils.R b/pkg/inst/tests/test_utils.R index 970e9ab26f21c..c549aa65708d9 100644 --- a/pkg/inst/tests/test_utils.R +++ b/pkg/inst/tests/test_utils.R @@ -43,9 +43,9 @@ test_that("cleanClosure on R functions", { newF <- cleanClosure(f) env <- environment(newF) expect_equal(length(ls(env)), 2) # y, g - actual <- get("y", envir = env) + actual <- get("y", envir = env, inherits = FALSE) expect_equal(actual, y) - actual <- get("g", envir = env) + actual <- get("g", envir = env, inherits = FALSE) expect_equal(actual, g) # Test for nested enclosures and package variables. @@ -56,9 +56,9 @@ test_that("cleanClosure on R functions", { newF <- cleanClosure(f) env <- environment(newF) expect_equal(length(ls(env)), 2) # "min" should not be included - actual <- get("y", envir = env) + actual <- get("y", envir = env, inherits = FALSE) expect_equal(actual, y) - actual <- get("g", envir = env) + actual <- get("g", envir = env, inherits = FALSE) expect_equal(actual, g) # Test for recursive closure capture for a free variable of a function. @@ -68,10 +68,10 @@ test_that("cleanClosure on R functions", { env <- environment(newF) expect_equal(length(ls(env)), 1) # Only "g". "y" should be in the environemnt of g. expect_equal(ls(env), "g") - newG <- get("g", envir = env) + newG <- get("g", envir = env, inherits = FALSE) env <- environment(newG) expect_equal(length(ls(env)), 1) - actual <- get("y", envir = env) + actual <- get("y", envir = env, inherits = FALSE) expect_equal(actual, y) # Test for function (and variable) definitions. @@ -91,8 +91,17 @@ test_that("cleanClosure on R functions", { newF <- cleanClosure(f) env <- environment(newF) expect_equal(ls(env), "t") - expect_equal(get("t", envir = 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) }) From 6e51c7ff25388bcf05776fa1ee353401b31b9443 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 1 Mar 2015 15:00:24 -0800 Subject: [PATCH 088/121] Fix stderr redirection on executors --- pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 4d69686e1a2ea..89b0bf3d74747 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -37,7 +37,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( val listenPort = serverSocket.getLocalPort() val pb = rWorkerProcessBuilder(listenPort) - pb.redirectErrorStream() // redirect stderr into stdout + pb.redirectErrorStream(true) // redirect stderr into stdout val proc = pb.start() val errThread = startStdoutThread(proc) From 8c4deaedc570c2753a2103d59aba20178d9ef777 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 1 Mar 2015 15:06:29 -0800 Subject: [PATCH 089/121] Remove unused function --- pkg/R/utils.R | 23 ----------------------- 1 file changed, 23 deletions(-) diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 108b577f456c0..6bad0abfd0e0d 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -70,29 +70,6 @@ isRDD <- function(name, env) { inherits(obj, "RDD") } -# Returns TRUE if `name` is a function in the SparkR package. -# TODO: Handle package-private functions as well ? -isSparkFunction <- function(name) { - if (is.function(name)) { - fun <- name - } else { - if (!(is.character(name) && length(name) == 1L || is.symbol(name))) { - fun <- eval.parent(substitute(substitute(name))) - if (!is.symbol(fun)) - stop(gettextf("'%s' is not a function, character or symbol", - deparse(fun)), domain = NA) - } else { - fun <- name - } - envir <- parent.frame(2) - if (!exists(as.character(fun), mode = "function", envir = envir)) { - return(FALSE) - } - fun <- get(as.character(fun), mode = "function", envir = envir) - } - packageName(environment(fun)) == "SparkR" -} - #' Compute the hashCode of an object #' #' Java-style function to compute the hashCode for the given object. Returns From f7caeb84321f04291214f17a7a6606cb3a0ddee8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 1 Mar 2015 15:11:37 -0800 Subject: [PATCH 090/121] Update SparkRBackend.scala --- .../scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala index 4cf7ac18f9425..89b2e7ada6b4c 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala @@ -78,7 +78,7 @@ class SparkRBackend { object SparkRBackend { def main(args: Array[String]) { if (args.length < 1) { - System.err.println("Usage: SparkRBackend ") + System.err.println("Usage: SparkRBackend ") System.exit(-1) } val sparkRBackend = new SparkRBackend() From 5c0bb24bd77a6e1ed4474144f14b6458cdd2c157 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Sun, 1 Mar 2015 22:20:41 -0800 Subject: [PATCH 091/121] Doc updates: build and running on YARN --- BUILDING.md | 9 +++------ README.md | 41 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 6 deletions(-) diff --git a/BUILDING.md b/BUILDING.md index c1929f94bd65e..08d9a8129009f 100644 --- a/BUILDING.md +++ b/BUILDING.md @@ -7,11 +7,8 @@ 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. Install `rJava` using `install.packages(rJava)`. If rJava fails to load due to missing jvm.dll, -you will need to add the directory containing jvm.dll to `PATH`. See this [stackoverflow post](http://stackoverflow.com/a/7604469] -for more details. -4. Download and install [Maven](http://maven.apache.org/download.html). Also include the `bin` +3. Download and install [Maven](http://maven.apache.org/download.html). Also include the `bin` directory in Maven in `PATH`. -5. Get SparkR source code either using [`git]`(http://git-scm.com/downloads) or by downloading a +4. Get SparkR source code either using [`git`](http://git-scm.com/downloads) or by downloading a source zip from github. -6. Open a command shell (`cmd`) in the SparkR directory and run `install-dev.bat` +5. Open a command shell (`cmd`) in the SparkR directory and run `install-dev.bat` diff --git a/README.md b/README.md index 6d6b097222ade..fa4655180ca73 100644 --- a/README.md +++ b/README.md @@ -46,6 +46,22 @@ the environment variable `USE_MAVEN=1`. For example If you are building SparkR from behind a proxy, you can [setup maven](https://maven.apache.org/guides/mini/guide-proxies.html) to use the right proxy server. +#### Building from source from GitHub + +Run the following within R to pull source code from GitHub and build locally. It is possible +to specify build dependencies by starting R with environment values: + +1. Start R +``` +SPARK_VERSION=1.2.0 SPARK_HADOOP_VERSION=2.5.0 R +``` + +2. Run install_github +``` +library(devtools) +install_github("repo/SparkR-pkg", ref="branchname", subdir="pkg") +``` +*note: replace repo and branchname* ## Running sparkR If you have cloned and built SparkR, you can start using it by launching the SparkR @@ -110,10 +126,23 @@ cd SparkR-pkg/ USE_YARN=1 SPARK_YARN_VERSION=2.4.0 SPARK_HADOOP_VERSION=2.4.0 ./install-dev.sh ``` +Alternatively, install_github can be use (on CDH in this case): + +``` +# assume devtools package is installed by install.packages("devtools") +USE_YARN=1 SPARK_VERSION=1.1.0 SPARK_YARN_VERSION=2.5.0-cdh5.3.0 SPARK_HADOOP_VERSION=2.5.0-cdh5.3.0 R +``` +Then within R, +``` +library(devtools) +install_github("amplab-extras/SparkR-pkg", ref="master", subdir="pkg") +``` + Before launching an application, make sure each worker node has a local copy of `lib/SparkR/sparkr-assembly-0.1.jar`. With a cluster launched with the `spark-ec2` script, do: ``` ~/spark-ec2/copy-dir ~/SparkR-pkg ``` +Or run the above installation steps on all worker node. Finally, when launching an application, the environment variable `YARN_CONF_DIR` needs to be set to the directory which contains the client-side configuration files for the Hadoop cluster (with a cluster launched with `spark-ec2`, this defaults to `/root/ephemeral-hdfs/conf/`): ``` @@ -121,6 +150,18 @@ YARN_CONF_DIR=/root/ephemeral-hdfs/conf/ MASTER=yarn-client ./sparkR YARN_CONF_DIR=/root/ephemeral-hdfs/conf/ ./sparkR examples/pi.R yarn-client ``` +### Using sparkR-submit +sparkR-submit is a script introduced to faciliate submission of SparkR jobs to a YARN cluster. +It supports the same commandline parameters as [spark-submit](http://spark.apache.org/docs/latest/running-on-yarn.html). SPARK_HOME, YARN_HOME, and JAVA_HOME must be defined. + +(On CDH 5.3.0) +``` +export SPARK_HOME=/opt/cloudera/parcels/CDH-5.3.0-1.cdh5.3.0.p0.30/lib/spark +export YARN_CONF_DIR=/etc/hadoop/conf +export JAVA_HOME=/usr/java/jdk1.7.0_67-cloudera +/usr/lib64/R/library/SparkR/sparkR-submit --master yarn-client examples/pi.R yarn-client 4 +``` + ## Report Issues/Feedback For better tracking and collaboration, issues and TODO items are reported to a dedicated [SparkR JIRA](https://sparkr.atlassian.net/browse/SPARKR/). From 03402ebdef99be680c4d0c9c475fd08702d3eb9e Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 2 Mar 2015 16:17:17 -0800 Subject: [PATCH 092/121] Updates as per feedback on sparkR-submit --- README.md | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index fa4655180ca73..027614ab74808 100644 --- a/README.md +++ b/README.md @@ -150,11 +150,16 @@ YARN_CONF_DIR=/root/ephemeral-hdfs/conf/ MASTER=yarn-client ./sparkR YARN_CONF_DIR=/root/ephemeral-hdfs/conf/ ./sparkR examples/pi.R yarn-client ``` -### Using sparkR-submit -sparkR-submit is a script introduced to faciliate submission of SparkR jobs to a YARN cluster. -It supports the same commandline parameters as [spark-submit](http://spark.apache.org/docs/latest/running-on-yarn.html). SPARK_HOME, YARN_HOME, and JAVA_HOME must be defined. +## Running on a cluster using sparkR-submit -(On CDH 5.3.0) +sparkR-submit is a script introduced to faciliate submission of SparkR jobs to a Spark supported cluster (eg. Standalone, Mesos, YARN). +It supports the same commandline parameters as [spark-submit](http://spark.apache.org/docs/latest/submitting-applications.html). SPARK_HOME and JAVA_HOME must be defined. + +On YARN, YARN_HOME must be defined. Currently, SparkR only supports [yarn-client](http://spark.apache.org/docs/latest/running-on-yarn.html) mode. + +sparkR-submit is installed with the SparkR package. By default, it can be found under the default Library (['library'](https://stat.ethz.ch/R-manual/R-devel/library/base/html/libPaths.html) subdirectory of R_HOME) + +For example, to run on YARN (CDH 5.3.0), ``` export SPARK_HOME=/opt/cloudera/parcels/CDH-5.3.0-1.cdh5.3.0.p0.30/lib/spark export YARN_CONF_DIR=/etc/hadoop/conf From 06cbc2d233e6c0da062d0984e7cb95d3d9a5a1a1 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 2 Mar 2015 17:26:14 -0800 Subject: [PATCH 093/121] launch R worker by a daemon --- pkg/inst/worker/daemon.R | 25 ++++ pkg/inst/worker/worker.R | 12 +- .../edu/berkeley/cs/amplab/sparkr/RRDD.scala | 117 ++++++++++++------ 3 files changed, 106 insertions(+), 48 deletions(-) create mode 100644 pkg/inst/worker/daemon.R diff --git a/pkg/inst/worker/daemon.R b/pkg/inst/worker/daemon.R new file mode 100644 index 0000000000000..f2fff54aaee37 --- /dev/null +++ b/pkg/inst/worker/daemon.R @@ -0,0 +1,25 @@ +# 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, blocking = TRUE, open = "rb") + +while (TRUE) { + inport <- SparkR:::readInt(inputCon) + if (length(inport) != 1) { + quit(save="no") + } + p <- parallel:::mcfork() + if (inherits(p, "masterProcess")) { + close(inputCon) + Sys.setenv(SPARKR_WORKER_PORT = inport) + source(script) + parallel:::mcexit(0) + } +} diff --git a/pkg/inst/worker/worker.R b/pkg/inst/worker/worker.R index f3fc3b2e94994..43f4221e01909 100644 --- a/pkg/inst/worker/worker.R +++ b/pkg/inst/worker/worker.R @@ -1,18 +1,16 @@ # Worker class -port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) - -inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") -outputCon <- socketConnection(port = port, blocking = TRUE, open = "wb") - +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 -rLibDir <- readLines(inputCon, n = 1) .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 splitIndex <- SparkR:::readInt(inputCon) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 89b0bf3d74747..fe9095eccd398 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -1,7 +1,7 @@ package edu.berkeley.cs.amplab.sparkr import java.io._ -import java.net.{ServerSocket} +import java.net.{Socket, ServerSocket} import java.util.{Map => JMap} import scala.collection.JavaConversions._ @@ -36,10 +36,9 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( val serverSocket = new ServerSocket(0, 2) val listenPort = serverSocket.getLocalPort() - val pb = rWorkerProcessBuilder(listenPort) - pb.redirectErrorStream(true) // redirect stderr into stdout - val proc = pb.start() - val errThread = startStdoutThread(proc) + // 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. @@ -54,7 +53,6 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( val outSocket = serverSocket.accept() val inputStream = new BufferedInputStream(outSocket.getInputStream) val dataStream = openDataStream(inputStream) - serverSocket.close() try { @@ -84,34 +82,6 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( } } - /** - * ProcessBuilder used to launch worker R processes. - */ - private def rWorkerProcessBuilder(port: Int) = { - val rCommand = "Rscript" - val rOptions = "--vanilla" - val rExecScript = rLibDir + "/SparkR/worker/worker.R" - 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_WORKER_PORT", port.toString) - pb - } - - /** - * 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 - } - /** * Start a thread to write RDD data to the R process. */ @@ -128,9 +98,6 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( override def run() { try { SparkEnv.set(env) - val printOut = new PrintStream(stream) - printOut.println(rLibDir) - val dataOut = new DataOutputStream(stream) dataOut.writeInt(splitIndex) @@ -163,6 +130,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( dataOut.writeInt(1) } + val printOut = new PrintStream(stream) for (elem <- iter) { if (parentSerialized) { val elemArr = elem.asInstanceOf[Array[Byte]] @@ -312,7 +280,7 @@ private class StringRRDD[T: ClassTag]( lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) } -private class BufferedStreamThread( +private[sparkr] class BufferedStreamThread( in: InputStream, name: String, errBufferSize: Int) extends Thread(name) { @@ -320,14 +288,16 @@ private class BufferedStreamThread( var lineIdx = 0 override def run() { for (line <- Source.fromInputStream(in).getLines) { - lines(lineIdx) = line - lineIdx = (lineIdx + 1) % errBufferSize + synchronized { + lines(lineIdx) = line + lineIdx = (lineIdx + 1) % errBufferSize + } // TODO: user logger System.err.println(line) } } - def getLines(): String = { + def getLines(): String = synchronized { (0 until errBufferSize).filter { x => lines((x + lineIdx) % errBufferSize) != null }.map { x => @@ -338,6 +308,16 @@ private class BufferedStreamThread( 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. + // TODO(davies): make it configurable + val useDaemon = !System.getProperty("os.name").startsWith("Windows") + private[this] var errThread: BufferedStreamThread = _ + private[this] var daemonSocket: Socket = _ + private[this] var daemonChannel: DataOutputStream = _ + def createSparkContext( master: String, appName: String, @@ -368,6 +348,61 @@ object RRDD { 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 + } + + def createRProcess(rLibDir: String, port: Int, script: String) = { + 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) = { + if (useDaemon) { + synchronized { + if (daemonSocket == 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) + daemonSocket = serverSocket.accept() + daemonChannel = new DataOutputStream(daemonSocket.getOutputStream) + serverSocket.close() + } + daemonChannel.writeInt(port) + daemonChannel.flush() + 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. From e2d144a798f8ef293467ed8a3eb20b6cf77dcb56 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 2 Mar 2015 17:52:10 -0800 Subject: [PATCH 094/121] Fixed small typos --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 027614ab74808..92ee42adf8363 100644 --- a/README.md +++ b/README.md @@ -152,10 +152,10 @@ YARN_CONF_DIR=/root/ephemeral-hdfs/conf/ ./sparkR examples/pi.R yarn-client ## Running on a cluster using sparkR-submit -sparkR-submit is a script introduced to faciliate submission of SparkR jobs to a Spark supported cluster (eg. Standalone, Mesos, YARN). +sparkR-submit is a script introduced to facilitate submission of SparkR jobs to a Spark supported cluster (eg. Standalone, Mesos, YARN). It supports the same commandline parameters as [spark-submit](http://spark.apache.org/docs/latest/submitting-applications.html). SPARK_HOME and JAVA_HOME must be defined. -On YARN, YARN_HOME must be defined. Currently, SparkR only supports [yarn-client](http://spark.apache.org/docs/latest/running-on-yarn.html) mode. +On YARN, YARN_HOME must be defined. Currently, SparkR only supports the [yarn-client](http://spark.apache.org/docs/latest/running-on-yarn.html) mode. sparkR-submit is installed with the SparkR package. By default, it can be found under the default Library (['library'](https://stat.ethz.ch/R-manual/R-devel/library/base/html/libPaths.html) subdirectory of R_HOME) From 2b6f98036c9d124c9d8430644420dc8a60dbb1ac Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 Mar 2015 12:08:07 -0800 Subject: [PATCH 095/121] shutdown the JVM after R process die --- pkg/R/sparkR.R | 6 ++++- .../cs/amplab/sparkr/SparkRBackend.scala | 26 ++++++++++++++++++- 2 files changed, 30 insertions(+), 2 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index a623d84812d17..2e8797c2d39d1 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -149,11 +149,15 @@ sparkR.init <- function( } f <- file(path, open='rb') sparkRBackendPort <- readInt(f) + monitorPort <- readInt(f) close(f) file.remove(path) - if (length(sparkRBackendPort) == 0) { + if (length(sparkRBackendPort) == 0 || sparkRBackendPort == 0 || + length(monitorPort) == 0 || monitorPort == 0) { stop("JVM failed to launch") } + # never close this socket, JVM is waiting for it + .sparkREnv$monitorF <- socketConnection(port = monitorPort) } .sparkREnv$sparkRBackendPort <- sparkRBackendPort diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala index 89b2e7ada6b4c..a8365a465bf01 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala @@ -1,9 +1,11 @@ package edu.berkeley.cs.amplab.sparkr import java.io.{File, FileOutputStream, DataOutputStream, IOException} -import java.net.{InetSocketAddress, Socket} +import java.net.{ServerSocket, InetSocketAddress, Socket} import java.util.concurrent.TimeUnit +import scala.util.control.NonFatal + import io.netty.bootstrap.ServerBootstrap import io.netty.channel.ChannelFuture import io.netty.channel.ChannelInitializer @@ -85,13 +87,35 @@ object SparkRBackend { 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 stdin") { + 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) + val inSocket = serverSocket.accept() + serverSocket.close() + // wait for the end of socket, closed if R process die + inSocket.getInputStream().read(buf) + sparkRBackend.close() + System.exit(0) + } + }.start() + sparkRBackend.run() } catch { case e: IOException => From 2e7b19002918a1e447efe4b0b43af181c6b49844 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Tue, 3 Mar 2015 13:41:25 -0800 Subject: [PATCH 096/121] small update on yarn deploy mode. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 92ee42adf8363..92bc035b87842 100644 --- a/README.md +++ b/README.md @@ -155,7 +155,7 @@ YARN_CONF_DIR=/root/ephemeral-hdfs/conf/ ./sparkR examples/pi.R yarn-client sparkR-submit is a script introduced to facilitate submission of SparkR jobs to a Spark supported cluster (eg. Standalone, Mesos, YARN). It supports the same commandline parameters as [spark-submit](http://spark.apache.org/docs/latest/submitting-applications.html). SPARK_HOME and JAVA_HOME must be defined. -On YARN, YARN_HOME must be defined. Currently, SparkR only supports the [yarn-client](http://spark.apache.org/docs/latest/running-on-yarn.html) mode. +On YARN, YARN_CONF_DIR must be defined. sparkR-submit supports [YARN deploy modes](http://spark.apache.org/docs/latest/running-on-yarn.html): yarn-client and yarn-cluster. sparkR-submit is installed with the SparkR package. By default, it can be found under the default Library (['library'](https://stat.ethz.ch/R-manual/R-devel/library/base/html/libPaths.html) subdirectory of R_HOME) From 8de958d9f295643f3e5b7089ddb9363dad4f7421 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 Mar 2015 22:45:02 -0800 Subject: [PATCH 097/121] Update SparkRBackend.scala --- .../scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala index a8365a465bf01..be64c68204c87 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala @@ -100,7 +100,7 @@ object SparkRBackend { f.renameTo(new File(path)) // wait for the end of stdin, then exit - new Thread("wait for stdin") { + new Thread("wait for socket to close") { setDaemon(true) override def run(): Unit = { // any un-catched exception will also shutdown JVM From a37fd808254f506d252885ee4f9c9829eb6c8847 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 Mar 2015 23:49:48 -0800 Subject: [PATCH 098/121] Update sparkR.R --- pkg/R/sparkR.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 2e8797c2d39d1..eccc9aeec6cb4 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -153,7 +153,7 @@ sparkR.init <- function( close(f) file.remove(path) if (length(sparkRBackendPort) == 0 || sparkRBackendPort == 0 || - length(monitorPort) == 0 || monitorPort == 0) { + length(monitorPort) == 0 || monitorPort == 0) { stop("JVM failed to launch") } # never close this socket, JVM is waiting for it From 3865f394e5e2187bc4ac710d587a48df1177b310 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Wed, 4 Mar 2015 18:26:16 +0800 Subject: [PATCH 099/121] [SPARKR-156] phase 1: implement zipWithUniqueId() of the RDD class. --- pkg/NAMESPACE | 3 ++- pkg/R/RDD.R | 41 ++++++++++++++++++++++++++++++++++++++ pkg/inst/tests/test_rdd.R | 14 +++++++++++++ pkg/man/zipWithUniqueId.Rd | 36 +++++++++++++++++++++++++++++++++ 4 files changed, 93 insertions(+), 1 deletion(-) create mode 100644 pkg/man/zipWithUniqueId.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index 26511b8c0c29d..f2f22ad4d6f1b 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -62,7 +62,8 @@ exportMethods( "unionRDD", "unpersist", "value", - "values" + "values", + "zipWithUniqueId" ) # S3 methods exported diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index d7d92e2aa8d7e..07a51a28e3d76 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1427,6 +1427,47 @@ setMethod("setName", x }) +#' Zip an RDD with generated unique Long IDs. +#' +#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +#' n is the number of partitions. So there may exist gaps, but this +#' method won't trigger a spark job, which is different from +#' zipWithIndex. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @rdname zipWithUniqueId +#' @seealso zipWithIndex +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithUniqueId(rdd)) +#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#'} +setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) + +#' @rdname zipWithUniqueId +#' @aliases zipWithUniqueId,RDD +setMethod("zipWithUniqueId", + signature(x = "RDD"), + function(x) { + n <- numPartitions(x) + + partitionFunc <- function(split, part) { + index <- 0 + result <- vector("list", length(part)) + for (item in part) { + result[[index + 1]] <- list(item, index * n + split) + index <- index + 1 + } + result + } + + lapplyPartitionsWithIndex(x, partitionFunc) + }) + ############ Binary Functions ############# #' Return the union RDD of two RDDs. diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 709bf6716ea30..7541bfe99631b 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -348,6 +348,20 @@ test_that("aggregateRDD() on RDDs", { 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("keys() on RDDs", { keys <- keys(intRdd) actual <- collect(keys) diff --git a/pkg/man/zipWithUniqueId.Rd b/pkg/man/zipWithUniqueId.Rd new file mode 100644 index 0000000000000..a25d91c665e08 --- /dev/null +++ b/pkg/man/zipWithUniqueId.Rd @@ -0,0 +1,36 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{zipWithUniqueId} +\alias{zipWithUniqueId} +\alias{zipWithUniqueId,RDD} +\alias{zipWithUniqueId,RDD-method} +\title{Zip an RDD with generated unique Long IDs.} +\usage{ +zipWithUniqueId(x) + +\S4method{zipWithUniqueId}{RDD}(x) +} +\arguments{ +\item{x}{An RDD to be zipped.} +} +\value{ +An RDD with zipped items. +} +\description{ +Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +n is the number of partitions. So there may exist gaps, but this +method won't trigger a spark job, which is different from +zipWithIndex. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +collect(zipWithUniqueId(rdd)) +# list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +} +} +\seealso{ +zipWithIndex +} + From ac8a85213d9cdc4621c3f521f1d25fbde276ccc8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 4 Mar 2015 21:32:58 -0800 Subject: [PATCH 100/121] close monitor connection in sparkR.stop() --- pkg/R/sparkR.R | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 2e8797c2d39d1..e34404267ff6e 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -47,14 +47,21 @@ sparkR.stop <- function(env = .sparkREnv) { callJStatic("SparkRHandler", "stopBackend") # Also close the connection and remove it from our env - conn <- get(".sparkRCon", env) + conn <- get(".sparkRCon", envir = env) close(conn) rm(".sparkRCon", envir = env) + + if (exists(".monitorConn", envir = env)) { + conn <- get(".monitorConn", envir = env) + close(conn) + rm(".monitorConn", envir = env) + } + # Finally, sleep for 1 sec to let backend finish exiting. # Without this we get port conflicts in RStudio when we try to 'Restart R'. Sys.sleep(1) } - + } #' Initialize a new Spark Context. @@ -156,8 +163,7 @@ sparkR.init <- function( length(monitorPort) == 0 || monitorPort == 0) { stop("JVM failed to launch") } - # never close this socket, JVM is waiting for it - .sparkREnv$monitorF <- socketConnection(port = monitorPort) + assign(".monitorConn", socketConnection(port = monitorPort), envir = .sparkREnv) } .sparkREnv$sparkRBackendPort <- sparkRBackendPort From f06ccec7c89d981eb8ed02313929ca2e43e25109 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 5 Mar 2015 14:25:50 +0800 Subject: [PATCH 101/121] Use mapply() instead of for statement. --- pkg/R/RDD.R | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 07a51a28e3d76..1148ef58d720f 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1456,13 +1456,13 @@ setMethod("zipWithUniqueId", n <- numPartitions(x) partitionFunc <- function(split, part) { - index <- 0 - result <- vector("list", length(part)) - for (item in part) { - result[[index + 1]] <- list(item, index * n + split) - index <- index + 1 - } - result + mapply( + function(item, index) { + list(item, (index - 1) * n + split) + }, + part, + seq_along(part), + SIMPLIFY = FALSE) } lapplyPartitionsWithIndex(x, partitionFunc) From d8c1c09fa55985d563514839ca01c1a9955163aa Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 5 Mar 2015 11:18:28 -0800 Subject: [PATCH 102/121] add test to start and stop context multiple times --- pkg/R/jobj.R | 16 +++++++++------- pkg/R/sparkR.R | 30 ++++++++---------------------- pkg/inst/tests/test_context.R | 10 ++++++++++ 3 files changed, 27 insertions(+), 29 deletions(-) create mode 100644 pkg/inst/tests/test_context.R diff --git a/pkg/R/jobj.R b/pkg/R/jobj.R index de459110c426c..09cb7c472b95d 100644 --- a/pkg/R/jobj.R +++ b/pkg/R/jobj.R @@ -48,13 +48,15 @@ print.jobj <- function(x, ...) { cleanup.jobj <- function(jobj) { objId <- jobj$id - .validJobjs[[objId]] <- .validJobjs[[objId]] - 1 + 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 + 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 + } } } diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 5407d04c6af41..7938f459f5837 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -24,19 +24,6 @@ connExists <- function(env) { # Also terminates the backend this R session is connected to sparkR.stop <- function(env = .sparkREnv) { - if (!connExists(env)) { - # When the workspace is saved in R, the connections are closed - # *before* the finalizer is run. In these cases, we reconnect - # to the backend, so we can shut it down. - tryCatch({ - connectBackend("localhost", .sparkREnv$sparkRBackendPort) - }, error = function(err) { - cat("Error in Connection: Use sparkR.init() to restart SparkR\n") - }, warning = function(war) { - cat("No Connection Found: Use sparkR.init() to restart SparkR\n") - }) - } - if (exists(".sparkRCon", envir = env)) { cat("Stopping SparkR\n") if (exists(".sparkRjsc", envir = env)) { @@ -50,18 +37,17 @@ sparkR.stop <- function(env = .sparkREnv) { conn <- get(".sparkRCon", envir = env) close(conn) rm(".sparkRCon", envir = env) + } - if (exists(".monitorConn", envir = env)) { - conn <- get(".monitorConn", envir = env) - close(conn) - rm(".monitorConn", envir = env) - } - - # Finally, sleep for 1 sec to let backend finish exiting. - # Without this we get port conflicts in RStudio when we try to 'Restart R'. - Sys.sleep(1) + if (exists(".monitorConn", envir = env)) { + conn <- get(".monitorConn", envir = env) + close(conn) + rm(".monitorConn", envir = env) } + # clear all the reference of Java object + rm(list = ls(.validJobjs, all = TRUE), envir = .validJobjs) + rm(list = ls(.toRemoveJobjs, all = TRUE), envir = .toRemoveJobjs) } #' Initialize a new Spark Context. diff --git a/pkg/inst/tests/test_context.R b/pkg/inst/tests/test_context.R new file mode 100644 index 0000000000000..7e1671c0ead8d --- /dev/null +++ b/pkg/inst/tests/test_context.R @@ -0,0 +1,10 @@ +context("functions in sparkR.R") + +test_that("start and stop SparkContext multiple times", { + for (i in 1:3) { + sc <- sparkR.init() + rdd <- parallelize(sc, 1:10) + expect_equal(count(rdd), 10) + sparkR.stop() + } +}) From 428a99a1e11324aa4c9da889ebf9c997d9282b9b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 5 Mar 2015 13:31:32 -0800 Subject: [PATCH 103/121] remove test, catch exception --- pkg/R/jobj.R | 16 +++++++--------- pkg/R/sparkR.R | 11 +---------- pkg/inst/tests/test_context.R | 10 ---------- .../cs/amplab/sparkr/SparkRBackend.scala | 15 +++++++++------ 4 files changed, 17 insertions(+), 35 deletions(-) delete mode 100644 pkg/inst/tests/test_context.R diff --git a/pkg/R/jobj.R b/pkg/R/jobj.R index 09cb7c472b95d..de459110c426c 100644 --- a/pkg/R/jobj.R +++ b/pkg/R/jobj.R @@ -48,15 +48,13 @@ print.jobj <- function(x, ...) { cleanup.jobj <- function(jobj) { objId <- jobj$id - if (exists(objId, envir = .validJobjs)) { - .validJobjs[[objId]] <- .validJobjs[[objId]] - 1 + .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 - } + 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 } } diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index 7938f459f5837..d92b908ad5502 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -44,10 +44,6 @@ sparkR.stop <- function(env = .sparkREnv) { close(conn) rm(".monitorConn", envir = env) } - - # clear all the reference of Java object - rm(list = ls(.validJobjs, all = TRUE), envir = .validJobjs) - rm(list = ls(.toRemoveJobjs, all = TRUE), envir = .toRemoveJobjs) } #' Initialize a new Spark Context. @@ -197,10 +193,5 @@ sparkR.init <- function( envir = .sparkREnv ) - sc <- get(".sparkRjsc", envir = .sparkREnv) - - # Register a finalizer to stop backend on R exit - reg.finalizer(.sparkREnv, sparkR.stop, onexit = TRUE) - - sc + get(".sparkRjsc", envir = .sparkREnv) } diff --git a/pkg/inst/tests/test_context.R b/pkg/inst/tests/test_context.R deleted file mode 100644 index 7e1671c0ead8d..0000000000000 --- a/pkg/inst/tests/test_context.R +++ /dev/null @@ -1,10 +0,0 @@ -context("functions in sparkR.R") - -test_that("start and stop SparkContext multiple times", { - for (i in 1:3) { - sc <- sparkR.init() - rdd <- parallelize(sc, 1:10) - expect_equal(count(rdd), 10) - sparkR.stop() - } -}) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala index be64c68204c87..3cc5813e84c4d 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SparkRBackend.scala @@ -107,12 +107,15 @@ object SparkRBackend { val buf = new Array[Byte](1024) // shutdown JVM if R does not connect back in 10 seconds serverSocket.setSoTimeout(10000) - val inSocket = serverSocket.accept() - serverSocket.close() - // wait for the end of socket, closed if R process die - inSocket.getInputStream().read(buf) - sparkRBackend.close() - System.exit(0) + 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() From f3d99a66a27fc5929ea406e8ba8220218b55b42b Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Fri, 6 Mar 2015 09:23:04 +0800 Subject: [PATCH 104/121] [SPARKR-156] phase 2: implement zipWithIndex() of the RDD class. --- pkg/NAMESPACE | 1 + pkg/R/RDD.R | 57 +++++++++++++++++++++++++++++++++++++++ pkg/inst/tests/test_rdd.R | 14 ++++++++++ pkg/man/zipWithIndex.Rd | 40 +++++++++++++++++++++++++++ 4 files changed, 112 insertions(+) create mode 100644 pkg/man/zipWithIndex.Rd diff --git a/pkg/NAMESPACE b/pkg/NAMESPACE index f2f22ad4d6f1b..512d078e707bb 100644 --- a/pkg/NAMESPACE +++ b/pkg/NAMESPACE @@ -63,6 +63,7 @@ exportMethods( "unpersist", "value", "values", + "zipWithIndex", "zipWithUniqueId" ) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 1148ef58d720f..8313794fa4b38 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1468,6 +1468,63 @@ setMethod("zipWithUniqueId", lapplyPartitionsWithIndex(x, partitionFunc) }) +#' Zip an RDD with its element indices. +#' +#' The ordering is first based on the partition index and then the +#' ordering of items within each partition. So the first item in +#' the first partition gets index 0, and the last item in the last +#' partition receives the largest index. +#' +#' This method needs to trigger a spark job when this RDD contains +#' more than one partitions. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @rdname zipWithIndex +#' @seealso zipWithUniqueId +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithIndex(rdd)) +#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#'} +setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) + +#' @rdname zipWithIndex +#' @aliases zipWithIndex,RDD +setMethod("zipWithIndex", + signature(x = "RDD"), + function(x) { + n <- numPartitions(x) + if (n > 1) { + nums <- collect(lapplyPartition(x, + function(part) { + list(length(part)) + })) + startIndices <- Reduce("+", nums, accumulate = TRUE) + } + + partitionFunc <- function(split, part) { + if (split == 0) { + startIndex <- 0 + } else { + startIndex <- startIndices[[split]] + } + + mapply( + function(item, index) { + list(item, index - 1 + startIndex) + }, + part, + seq_along(part), + SIMPLIFY = FALSE) + } + + lapplyPartitionsWithIndex(x, partitionFunc) + }) + ############ Binary Functions ############# #' Return the union RDD of two RDDs. diff --git a/pkg/inst/tests/test_rdd.R b/pkg/inst/tests/test_rdd.R index 7541bfe99631b..40d03839c4be6 100644 --- a/pkg/inst/tests/test_rdd.R +++ b/pkg/inst/tests/test_rdd.R @@ -362,6 +362,20 @@ test_that("zipWithUniqueId() on RDDs", { 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("keys() on RDDs", { keys <- keys(intRdd) actual <- collect(keys) diff --git a/pkg/man/zipWithIndex.Rd b/pkg/man/zipWithIndex.Rd new file mode 100644 index 0000000000000..615f007f7d1b0 --- /dev/null +++ b/pkg/man/zipWithIndex.Rd @@ -0,0 +1,40 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{zipWithIndex} +\alias{zipWithIndex} +\alias{zipWithIndex,RDD} +\alias{zipWithIndex,RDD-method} +\title{Zip an RDD with its element indices.} +\usage{ +zipWithIndex(x) + +\S4method{zipWithIndex}{RDD}(x) +} +\arguments{ +\item{x}{An RDD to be zipped.} +} +\value{ +An RDD with zipped items. +} +\description{ +The ordering is first based on the partition index and then the +ordering of items within each partition. So the first item in +the first partition gets index 0, and the last item in the last +partition receives the largest index. +} +\details{ +This method needs to trigger a spark job when this RDD contains +more than one partitions. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +collect(zipWithIndex(rdd)) +# list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +} +} +\seealso{ +zipWithUniqueId +} + From 5e3a576ae8f6718ac0640db5c3a6a2ddd9cfaaf1 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Fri, 6 Mar 2015 09:39:26 +0800 Subject: [PATCH 105/121] Fix indentation. --- pkg/R/RDD.R | 27 ++++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 8313794fa4b38..bc3fd3f7c9f98 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1500,19 +1500,19 @@ setMethod("zipWithIndex", n <- numPartitions(x) if (n > 1) { nums <- collect(lapplyPartition(x, - function(part) { - list(length(part)) - })) - startIndices <- Reduce("+", nums, accumulate = TRUE) - } - - partitionFunc <- function(split, part) { - if (split == 0) { - startIndex <- 0 + function(part) { + list(length(part)) + })) + startIndices <- Reduce("+", nums, accumulate = TRUE) + } + + partitionFunc <- function(split, part) { + if (split == 0) { + startIndex <- 0 } else { - startIndex <- startIndices[[split]] - } - + startIndex <- startIndices[[split]] + } + mapply( function(item, index) { list(item, index - 1 + startIndex) @@ -1521,10 +1521,11 @@ setMethod("zipWithIndex", seq_along(part), SIMPLIFY = FALSE) } - + lapplyPartitionsWithIndex(x, partitionFunc) }) + ############ Binary Functions ############# #' Return the union RDD of two RDDs. From 3f7aed6a4b2c35eaf31077bd66104a74c5f176f4 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Fri, 6 Mar 2015 12:38:19 +0800 Subject: [PATCH 106/121] Fix minor typos in the function description. --- pkg/R/RDD.R | 4 ++-- pkg/man/zipWithIndex.Rd | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index bc3fd3f7c9f98..138aeabf91df7 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -1475,8 +1475,8 @@ setMethod("zipWithUniqueId", #' the first partition gets index 0, and the last item in the last #' partition receives the largest index. #' -#' This method needs to trigger a spark job when this RDD contains -#' more than one partitions. +#' This method needs to trigger a Spark job when this RDD contains +#' more than one partition. #' #' @param x An RDD to be zipped. #' @return An RDD with zipped items. diff --git a/pkg/man/zipWithIndex.Rd b/pkg/man/zipWithIndex.Rd index 615f007f7d1b0..cc4d74987f2d2 100644 --- a/pkg/man/zipWithIndex.Rd +++ b/pkg/man/zipWithIndex.Rd @@ -23,8 +23,8 @@ the first partition gets index 0, and the last item in the last partition receives the largest index. } \details{ -This method needs to trigger a spark job when this RDD contains -more than one partitions. +This method needs to trigger a Spark job when this RDD contains +more than one partition. } \examples{ \dontrun{ From b4c0b2e49f7cc5cab62aa003057214fa56e4a17d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 6 Mar 2015 12:41:16 -0800 Subject: [PATCH 107/121] use fork package --- pkg/inst/worker/daemon.R | 34 ++++++++++++++++++++++++---------- 1 file changed, 24 insertions(+), 10 deletions(-) diff --git a/pkg/inst/worker/daemon.R b/pkg/inst/worker/daemon.R index f2fff54aaee37..381b3c42d7041 100644 --- a/pkg/inst/worker/daemon.R +++ b/pkg/inst/worker/daemon.R @@ -1,7 +1,13 @@ # Worker daemon +# try to load `fork`, or install +FORK_URI <- "http://cran.r-project.org/src/contrib/Archive/fork/fork_1.2.4.tar.gz" +tryCatch(library(fork), error = function(err) { + install.packages(FORK_URI, repos = NULL, type = "source") +}) + rLibDir <- Sys.getenv("SPARKR_RLIBDIR") -script <- paste(rLibDir, "SparkR/worker/worker.R", sep="/") +script <- paste(rLibDir, "SparkR/worker/worker.R", sep = "/") # preload SparkR package, speedup worker .libPaths(c(rLibDir, .libPaths())) @@ -11,15 +17,23 @@ port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") while (TRUE) { - inport <- SparkR:::readInt(inputCon) - if (length(inport) != 1) { - quit(save="no") + ready <- socketSelect(list(inputCon), timeout = 1) + if (ready) { + inport <- SparkR:::readInt(inputCon) + if (length(inport) != 1) { + quit(save="no") + } + p <- fork::fork(NULL) + if (p == 0) { + close(inputCon) + Sys.setenv(SPARKR_WORKER_PORT = inport) + source(script) + fork::exit(0) + } } - p <- parallel:::mcfork() - if (inherits(p, "masterProcess")) { - close(inputCon) - Sys.setenv(SPARKR_WORKER_PORT = inport) - source(script) - parallel:::mcexit(0) + # cleanup all the child process + status <- fork::wait(0, nohang = TRUE) + while (status[1] > 0) { + status <- fork::wait(0, nohang = TRUE) } } From dc1291bfe2fc9d922b0718c1e25deb056badf38a Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sun, 8 Mar 2015 19:25:56 -0400 Subject: [PATCH 108/121] Add checks for namespace access operators in cleanClosure. --- pkg/R/RDD.R | 2 -- pkg/R/utils.R | 2 ++ 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 138aeabf91df7..57de2693e6be4 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -105,8 +105,6 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), return(rdd@env$jrdd_val) } - # TODO: This is to handle anonymous functions. Find out a - # better way to do this. computeFunc <- function(split, part) { rdd@func(split, part) } diff --git a/pkg/R/utils.R b/pkg/R/utils.R index 6bad0abfd0e0d..6b323e4fa6970 100644 --- a/pkg/R/utils.R +++ b/pkg/R/utils.R @@ -278,6 +278,8 @@ processClosure <- function(node, oldEnv, argNames, newEnv) { } } else if (nodeChar == "$") { # Skip the field. processClosure(node[[2]], oldEnv, argNames, newEnv) + } else if (nodeChar == "::" || nodeChar == ":::") { + processClosure(node[[3]], oldEnv, argNames, newEnv) } else { for (i in 1:nodeLen) { processClosure(node[[i]], oldEnv, argNames, newEnv) From 97dde1a5dd86a382b16e64a9d2ec90a66b2b6a3b Mon Sep 17 00:00:00 2001 From: hlin09 Date: Sun, 8 Mar 2015 21:46:50 -0400 Subject: [PATCH 109/121] Add a test for access operators. --- pkg/inst/tests/test_utils.R | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/pkg/inst/tests/test_utils.R b/pkg/inst/tests/test_utils.R index c549aa65708d9..0e09ccd8b502f 100644 --- a/pkg/inst/tests/test_utils.R +++ b/pkg/inst/tests/test_utils.R @@ -83,6 +83,20 @@ test_that("cleanClosure on R functions", { env <- environment(newF) expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. + # Test for access operators `$`, `::` and `:::`. + l <- list(a = 1) + a <- 2 + base <- c(1, 2, 3) + f <- function(x) { + z <- base::as.integer(x) + 1 + l$a <- 3 + z + l$a + } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(ls(env), "l") # "base" and "a" should not be included. + expect_equal(get("l", envir = env, inherits = FALSE), l) + # Test for overriding variables in base namespace (Issue: SparkR-196). nums <- as.list(1:10) rdd <- parallelize(sc, nums, 2L) From 89b886da8250856edd4d7a979481bf9628a084cb Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 9 Mar 2015 10:12:28 -0400 Subject: [PATCH 110/121] Move setGeneric() to 00-generics.R. --- pkg/R/00-generics.R | 1160 +++++++++++++++++++++++++++++++++++++++++++ pkg/R/RDD.R | 699 -------------------------- pkg/R/pairRDD.R | 452 ----------------- 3 files changed, 1160 insertions(+), 1151 deletions(-) create mode 100644 pkg/R/00-generics.R diff --git a/pkg/R/00-generics.R b/pkg/R/00-generics.R new file mode 100644 index 0000000000000..90d0d35a2b653 --- /dev/null +++ b/pkg/R/00-generics.R @@ -0,0 +1,1160 @@ +############ RDD Actions and Transformations ############ + +#' Persist an RDD +#' +#' Persist this RDD with the default storage level (MEMORY_ONLY). +#' +#' @param x The RDD to cache +#' @rdname cache-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) +#'} +setGeneric("cache", function(x) { standardGeneric("cache") }) + +#' Persist an RDD +#' +#' Persist this RDD with the specified storage level. For details of the +#' supported storage levels, refer to +#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#' +#' @param x The RDD to persist +#' @param newLevel The new storage level to be assigned +#' @rdname persist +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' persist(rdd, "MEMORY_AND_DISK") +#'} +setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) + +#' Unpersist an RDD +#' +#' Mark the RDD as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param rdd The RDD to unpersist +#' @rdname unpersist-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) # rdd@@env$isCached == TRUE +#' unpersist(rdd) # rdd@@env$isCached == FALSE +#'} +setGeneric("unpersist", function(x) { standardGeneric("unpersist") }) + +#' Checkpoint an RDD +#' +#' Mark this RDD for checkpointing. It will be saved to a file inside the +#' checkpoint directory set with setCheckpointDir() and all references to its +#' parent RDDs will be removed. This function must be called before any job has +#' been executed on this RDD. It is strongly recommended that this RDD is +#' persisted in memory, otherwise saving it on a file will require recomputation. +#' +#' @param rdd The RDD to checkpoint +#' @rdname checkpoint-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "checkpoints") +#' rdd <- parallelize(sc, 1:10, 2L) +#' checkpoint(rdd) +#'} +setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) + +#' Gets the number of partitions of an RDD +#' +#' @param x A RDD. +#' @return the number of partitions of rdd as an integer. +#' @rdname numPartitions +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' numPartitions(rdd) # 2L +#'} +setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) + +#' Collect elements of an RDD +#' +#' @description +#' \code{collect} returns a list that contains all of the elements in this RDD. +#' +#' @param x The RDD to collect +#' @param ... Other optional arguments to collect +#' @param flatten FALSE if the list should not flattened +#' @return a list containing elements in the RDD +#' @rdname collect-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' collect(rdd) # list from 1 to 10 +#' collectPartition(rdd, 0L) # list from 1 to 5 +#'} +setGeneric("collect", function(x, ...) { standardGeneric("collect") }) + +#' @rdname collect-methods +#' @export +#' @description +#' \code{collectPartition} returns a list that contains all of the elements +#' in the specified partition of the RDD. +#' @param partitionId the partition to collect (starts from 0) +setGeneric("collectPartition", + function(x, partitionId) { + standardGeneric("collectPartition") + }) + +#' @rdname collect-methods +#' @export +#' @description +#' \code{collectAsMap} returns a named list as a map that contains all of the elements +#' in a key-value pair RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) +#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#'} +setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) + +#' Return the number of elements in the RDD. +#' +#' @param x The RDD to count +#' @return number of elements in the RDD. +#' @rdname count +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' count(rdd) # 10 +#' length(rdd) # Same as count +#'} +setGeneric("count", function(x) { standardGeneric("count") }) + +#' Return the count of each unique value in this RDD as a list of +#' (value, count) pairs. +#' +#' Same as countByValue in Spark. +#' +#' @param x The RDD to count +#' @return list of (value, count) pairs, where count is number of each unique +#' value in rdd. +#' @rdname countByValue +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,3,2,1)) +#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) +#'} +setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) + +#' @rdname lapply +#' @export +setGeneric("map", function(X, FUN) { + standardGeneric("map") }) + +#' Flatten results after apply a function to all elements +#' +#' This function return a new RDD by first applying a function to all +#' elements of this RDD, and then flattening the results. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @rdname flatMap +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) +#' collect(multiplyByTwo) # 2,20,4,40,6,60... +#'} +setGeneric("flatMap", function(X, FUN) { + standardGeneric("flatMap") }) + +#' Apply a function to each partition of an RDD +#' +#' Return a new RDD by applying a function to each partition of this RDD. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition. +#' @return a new RDD created by the transformation. +#' @rdname lapplyPartition +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) +#' collect(partitionSum) # 15, 40 +#'} +setGeneric("lapplyPartition", function(X, FUN) { + standardGeneric("lapplyPartition") }) + +#' mapPartitions is the same as lapplyPartition. +#' +#' @rdname lapplyPartition +#' @export +setGeneric("mapPartitions", function(X, FUN) { + standardGeneric("mapPartitions") }) + +#' Return a new RDD by applying a function to each partition of this RDD, while +#' tracking the index of the original partition. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition; takes the partition +#' index and a list of elements in the particular partition. +#' @return a new RDD created by the transformation. +#' @rdname lapplyPartitionsWithIndex +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 5L) +#' prod <- lapplyPartitionsWithIndex(rdd, function(split, part) { +#' split * Reduce("+", part) }) +#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#'} +setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { + standardGeneric("lapplyPartitionsWithIndex") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("mapPartitionsWithIndex", function(X, FUN) { + standardGeneric("mapPartitionsWithIndex") }) + +#' This function returns a new RDD containing only the elements that satisfy +#' a predicate (i.e. returning TRUE in a given logical function). +#' The same as `filter()' in Spark. +#' +#' @param x The RDD to be filtered. +#' @param f A unary predicate function. +#' @rdname filterRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#'} +setGeneric("filterRDD", + function(x, f) { standardGeneric("filterRDD") }) + +#' Reduce across elements of an RDD. +#' +#' This function reduces the elements of this RDD using the +#' specified commutative and associative binary operator. +#' +#' @param rdd The RDD to reduce +#' @param func Commutative and associative function to apply on elements +#' of the RDD. +#' @export +#' @rdname reduce +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' reduce(rdd, "+") # 55 +#'} +setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) + +#' Get the maximum element of an RDD. +#' +#' @param x The RDD to get the maximum element from +#' @export +#' @rdname maximum +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' maximum(rdd) # 10 +#'} +setGeneric("maximum", function(x) { standardGeneric("maximum") }) + +#' Get the minimum element of an RDD. +#' +#' @param x The RDD to get the minimum element from +#' @export +#' @rdname minimum +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' minimum(rdd) # 1 +#'} +setGeneric("minimum", function(x) { standardGeneric("minimum") }) + +#' Applies a function to all elements in an RDD, and force evaluation. +#' +#' @param x The RDD to apply the function +#' @param func The function to be applied. +#' @return invisible NULL. +#' @export +#' @rdname foreach +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreach(rdd, function(x) { save(x, file=...) }) +#'} +setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) + +#' Applies a function to each partition in an RDD, and force evaluation. +#' +#' @export +#' @rdname foreach +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) +#'} +setGeneric("foreachPartition", + function(x, func) { standardGeneric("foreachPartition") }) + +#' Take elements from an RDD. +#' +#' This function takes the first NUM elements in the RDD and +#' returns them in a list. +#' +#' @param x The RDD to take elements from +#' @param num Number of elements to take +#' @rdname take +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' take(rdd, 2L) # list(1, 2) +#'} +setGeneric("take", function(x, num) { standardGeneric("take") }) + +#' Removes the duplicates from RDD. +#' +#' This function returns a new RDD containing the distinct elements in the +#' given RDD. The same as `distinct()' in Spark. +#' +#' @param x The RDD to remove duplicates from. +#' @param numPartitions Number of partitions to create. +#' @rdname distinct +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) +#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) +#'} +setGeneric("distinct", + function(x, numPartitions) { standardGeneric("distinct") }) + +#' Return an RDD that is a sampled subset of the given RDD. +#' +#' The same as `sample()' in Spark. (We rename it due to signature +#' inconsistencies with the `sample()' function in R's base package.) +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @rdname sampleRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) # ensure each num is in its own split +#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#'} +setGeneric("sampleRDD", + function(x, withReplacement, fraction, seed) { + standardGeneric("sampleRDD") + }) + +#' Return a list of the elements that are a sampled subset of the given RDD. +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param num Number of elements to return +#' @param seed Randomness seed value +#' @rdname takeSample +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:100) +#' # exactly 5 elements sampled, which may not be distinct +#' takeSample(rdd, TRUE, 5L, 1618L) +#' # exactly 5 distinct elements sampled +#' takeSample(rdd, FALSE, 5L, 16181618L) +#'} +setGeneric("takeSample", + function(x, withReplacement, num, seed) { + standardGeneric("takeSample") + }) + +#' Creates tuples of the elements in this RDD by applying a function. +#' +#' @param x The RDD. +#' @param func The function to be applied. +#' @rdname keyBy +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3)) +#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) +#'} +setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) + +#' Return a new RDD that has exactly numPartitions partitions. +#' Can increase or decrease the level of parallelism in this RDD. Internally, +#' this uses a shuffle to redistribute data. +#' If you are decreasing the number of partitions in this RDD, consider using +#' coalesce, which can avoid performing a shuffle. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @rdname repartition +#' @seealso coalesce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +#' numPartitions(rdd) # 4 +#' numPartitions(repartition(rdd, 2L)) # 2 +#'} +setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) + +#' Return a new RDD that is reduced into numPartitions partitions. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @rdname coalesce +#' @seealso repartition +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +#' numPartitions(rdd) # 3 +#' numPartitions(coalesce(rdd, 1L)) # 1 +#'} +setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) + +#' Save this RDD as a SequenceFile of serialized objects. +#' +#' @param x The RDD to save +#' @param path The directory where the file is saved +#' @rdname saveAsObjectFile +#' @seealso objectFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") +#'} +setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) + +#' Save this RDD as a text file, using string representations of elements. +#' +#' @param x The RDD to save +#' @param path The directory where the splits of the text file are saved +#' @rdname saveAsTextFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") +#'} +setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) + +#' Sort an RDD by the given key function. +#' +#' @param x An RDD to be sorted. +#' @param func A function used to compute the sort key for each element. +#' @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 elements are sorted. +#' @rdname sortBy +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(3, 2, 1)) +#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#'} +setGeneric("sortBy", function(x, + func, + ascending = TRUE, + numPartitions = 1L) { + standardGeneric("sortBy") +}) + +#' Returns the first N elements from an RDD in ascending order. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The first N elements from the RDD in ascending order. +#' @rdname takeOrdered +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#'} +setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) + +#' Returns the top N elements from an RDD. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The top N elements from the RDD. +#' @rdname top +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#'} +setGeneric("top", function(x, num) { standardGeneric("top") }) + +#' Fold an RDD using a given associative function and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using a given associative function and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param op An associative function for the folding operation. +#' @return The folding result. +#' @rdname fold +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +#' fold(rdd, 0, "+") # 15 +#'} +setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) + +#' Aggregate an RDD using the given combine functions and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using given combine functions and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the RDD elements. It may return a different +#' result type from the type of the RDD elements. +#' @param combOp A function to aggregate results of seqOp. +#' @return The aggregation result. +#' @rdname aggregateRDD +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' 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]]) } +#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#'} +setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) + +#' Pipes elements to a forked external process. +#' +#' The same as 'pipe()' in Spark. +#' +#' @param x The RDD whose elements are piped to the forked external process. +#' @param command The command to fork an external process. +#' @param env A named list to set environment variables of the external process. +#' @return A new RDD created by piping all elements to a forked external process. +#' @rdname pipeRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' collect(pipeRDD(rdd, "more") +#' Output: c("1", "2", ..., "10") +#'} +setGeneric("pipeRDD", function(x, command, env = list()) { + standardGeneric("pipeRDD") +}) + +# TODO: Consider caching the name in the RDD's environment +#' Return an RDD's name. +#' +#' @param x The RDD whose name is returned. +#' @rdname name +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' name(rdd) # NULL (if not set before) +#'} +setGeneric("name", function(x) { standardGeneric("name") }) + +#' Set an RDD's name. +#' +#' @param x The RDD whose name is to be set. +#' @param name The RDD name to be set. +#' @return a new RDD renamed. +#' @rdname setName +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' setName(rdd, "myRDD") +#' name(rdd) # "myRDD" +#'} +setGeneric("setName", function(x, name) { standardGeneric("setName") }) + +#' Zip an RDD with generated unique Long IDs. +#' +#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +#' n is the number of partitions. So there may exist gaps, but this +#' method won't trigger a spark job, which is different from +#' zipWithIndex. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @rdname zipWithUniqueId +#' @seealso zipWithIndex +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithUniqueId(rdd)) +#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#'} +setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) + +#' Zip an RDD with its element indices. +#' +#' The ordering is first based on the partition index and then the +#' ordering of items within each partition. So the first item in +#' the first partition gets index 0, and the last item in the last +#' partition receives the largest index. +#' +#' This method needs to trigger a Spark job when this RDD contains +#' more than one partition. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @rdname zipWithIndex +#' @seealso zipWithUniqueId +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithIndex(rdd)) +#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#'} +setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) + + +############ Binary Functions ############# + + +#' Return the union RDD of two RDDs. +#' The same as union() in Spark. +#' +#' @param x An RDD. +#' @param y An RDD. +#' @return a new RDD created by performing the simple union (witout removing +#' duplicates) of two input RDDs. +#' @rdname unionRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 +#'} +setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) + +#' 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 +#' @rdname lookup +#' @export +#' @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) +#'} +setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) + +#' 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. +#' @rdname countByKey +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +#' countByKey(rdd) # ("a", 2L), ("b", 1L) +#'} +setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) + +#' Return an RDD with the keys of each tuple. +#' +#' @param x The RDD from which the keys of each tuple is returned. +#' @rdname keys +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(keys(rdd)) # list(1, 3) +#'} +setGeneric("keys", function(x) { standardGeneric("keys") }) + +#' Return an RDD with the values of each tuple. +#' +#' @param x The RDD from which the values of each tuple is returned. +#' @rdname values +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(values(rdd)) # list(2, 4) +#'} +setGeneric("values", function(x) { standardGeneric("values") }) + +#' 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. +#' @rdname mapValues +#' @export +#' @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), ...) +#'} +setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) + +#' 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. +#' @rdname flatMapValues +#' @export +#' @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)) +#'} +setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) + + +############ 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. +#' @rdname partitionBy +#' @export +#' @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) +#'} +setGeneric("partitionBy", + function(x, numPartitions, ...) { + standardGeneric("partitionBy") + }) + +#' 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 +#' @rdname groupByKey +#' @export +#' @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)) +#'} +setGeneric("groupByKey", + function(x, numPartitions) { + standardGeneric("groupByKey") + }) + +#' 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 +#' @rdname reduceByKey +#' @seealso groupByKey +#' @export +#' @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) +#'} +setGeneric("reduceByKey", + function(x, combineFunc, numPartitions) { + standardGeneric("reduceByKey") + }) + +#' 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 +#' @rdname reduceByKeyLocally +#' @seealso reduceByKey +#' @export +#' @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)) +#'} +setGeneric("reduceByKeyLocally", + function(x, combineFunc) { + standardGeneric("reduceByKeyLocally") + }) + +#' 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 +#' +#' @rdname combineByKey +#' @seealso groupByKey, reduceByKey +#' @export +#' @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) +#'} +setGeneric("combineByKey", + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { + standardGeneric("combineByKey") + }) + +#' 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. +#' @rdname aggregateByKey +#' @seealso foldByKey, combineByKey +#' @export +#' @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))) +#'} +setGeneric("aggregateByKey", + function(x, zeroValue, seqOp, combOp, numPartitions) { + standardGeneric("aggregateByKey") + }) + +#' 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. +#' @rdname foldByKey +#' @seealso aggregateByKey, combineByKey +#' @export +#' @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)) +#'} +setGeneric("foldByKey", + function(x, zeroValue, func, numPartitions) { + standardGeneric("foldByKey") + }) + +#' 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. +#' @rdname join-methods +#' @export +#' @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)) +#'} +setGeneric("join", function(x, y, numPartitions) { standardGeneric("join") }) + +#' 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. +#' @rdname join-methods +#' @export +#' @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))) +#'} +setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) + +#' 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. +#' @rdname join-methods +#' @export +#' @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))) +#'} +setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) + +#' 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. +#' @rdname join-methods +#' @export +#' @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)), +#'} +setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) + +#' 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. +#' @rdname cogroup +#' @export +#' @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())) +#'} +setGeneric("cogroup", + function(..., numPartitions) { standardGeneric("cogroup") }, + signature = "...") + +#' 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. +#' @rdname sortByKey +#' @export +#' @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)) +#'} +setGeneric("sortByKey", function(x, + ascending = TRUE, + numPartitions = 1L) { + standardGeneric("sortByKey") +}) + diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 138aeabf91df7..34eb4ebd8e80a 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -105,8 +105,6 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), return(rdd@env$jrdd_val) } - # TODO: This is to handle anonymous functions. Find out a - # better way to do this. computeFunc <- function(split, part) { rdd@func(split, part) } @@ -163,21 +161,6 @@ setValidity("RDD", ############ Actions and Transformations ############ -#' Persist an RDD -#' -#' Persist this RDD with the default storage level (MEMORY_ONLY). -#' -#' @param x The RDD to cache -#' @rdname cache-methods -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' cache(rdd) -#'} -setGeneric("cache", function(x) { standardGeneric("cache") }) - #' @rdname cache-methods #' @aliases cache,RDD-method setMethod("cache", @@ -188,24 +171,6 @@ setMethod("cache", x }) -#' Persist an RDD -#' -#' Persist this RDD with the specified storage level. For details of the -#' supported storage levels, refer to -#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. -#' -#' @param x The RDD to persist -#' @param newLevel The new storage level to be assigned -#' @rdname persist -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' persist(rdd, "MEMORY_AND_DISK") -#'} -setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) - #' @rdname persist #' @aliases persist,RDD-method setMethod("persist", @@ -240,23 +205,6 @@ setMethod("persist", x }) -#' Unpersist an RDD -#' -#' Mark the RDD as non-persistent, and remove all blocks for it from memory and -#' disk. -#' -#' @param rdd The RDD to unpersist -#' @rdname unpersist-methods -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' cache(rdd) # rdd@@env$isCached == TRUE -#' unpersist(rdd) # rdd@@env$isCached == FALSE -#'} -setGeneric("unpersist", function(x) { standardGeneric("unpersist") }) - #' @rdname unpersist-methods #' @aliases unpersist,RDD-method setMethod("unpersist", @@ -267,27 +215,6 @@ setMethod("unpersist", x }) - -#' Checkpoint an RDD -#' -#' Mark this RDD for checkpointing. It will be saved to a file inside the -#' checkpoint directory set with setCheckpointDir() and all references to its -#' parent RDDs will be removed. This function must be called before any job has -#' been executed on this RDD. It is strongly recommended that this RDD is -#' persisted in memory, otherwise saving it on a file will require recomputation. -#' -#' @param rdd The RDD to checkpoint -#' @rdname checkpoint-methods -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' setCheckpointDir(sc, "checkpoints") -#' rdd <- parallelize(sc, 1:10, 2L) -#' checkpoint(rdd) -#'} -setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) - #' @rdname checkpoint-methods #' @aliases checkpoint,RDD-method setMethod("checkpoint", @@ -299,20 +226,6 @@ setMethod("checkpoint", x }) -#' Gets the number of partitions of an RDD -#' -#' @param x A RDD. -#' @return the number of partitions of rdd as an integer. -#' @rdname numPartitions -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' numPartitions(rdd) # 2L -#'} -setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) - #' @rdname numPartitions #' @aliases numPartitions,RDD-method setMethod("numPartitions", @@ -323,26 +236,6 @@ setMethod("numPartitions", callJMethod(partitions, "size") }) -#' Collect elements of an RDD -#' -#' @description -#' \code{collect} returns a list that contains all of the elements in this RDD. -#' -#' @param x The RDD to collect -#' @param ... Other optional arguments to collect -#' @param flatten FALSE if the list should not flattened -#' @return a list containing elements in the RDD -#' @rdname collect-methods -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' collect(rdd) # list from 1 to 10 -#' collectPartition(rdd, 0L) # list from 1 to 5 -#'} -setGeneric("collect", function(x, ...) { standardGeneric("collect") }) - #' @rdname collect-methods #' @aliases collect,RDD-method setMethod("collect", @@ -353,18 +246,6 @@ setMethod("collect", convertJListToRList(collected, flatten) }) - -#' @rdname collect-methods -#' @export -#' @description -#' \code{collectPartition} returns a list that contains all of the elements -#' in the specified partition of the RDD. -#' @param partitionId the partition to collect (starts from 0) -setGeneric("collectPartition", - function(x, partitionId) { - standardGeneric("collectPartition") - }) - #' @rdname collect-methods #' @aliases collectPartition,integer,RDD-method setMethod("collectPartition", @@ -378,19 +259,6 @@ setMethod("collectPartition", convertJListToRList(jList, flatten = TRUE) }) -#' @rdname collect-methods -#' @export -#' @description -#' \code{collectAsMap} returns a named list as a map that contains all of the elements -#' in a key-value pair RDD. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) -#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) -#'} -setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) - #' @rdname collect-methods #' @aliases collectAsMap,RDD-method setMethod("collectAsMap", @@ -402,21 +270,6 @@ setMethod("collectAsMap", as.list(map) }) -#' Return the number of elements in the RDD. -#' -#' @param x The RDD to count -#' @return number of elements in the RDD. -#' @rdname count -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' count(rdd) # 10 -#' length(rdd) # Same as count -#'} -setGeneric("count", function(x) { standardGeneric("count") }) - #' @rdname count #' @aliases count,RDD-method setMethod("count", @@ -439,24 +292,6 @@ setMethod("length", count(x) }) -#' Return the count of each unique value in this RDD as a list of -#' (value, count) pairs. -#' -#' Same as countByValue in Spark. -#' -#' @param x The RDD to count -#' @return list of (value, count) pairs, where count is number of each unique -#' value in rdd. -#' @rdname countByValue -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, c(1,2,3,2,1)) -#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) -#'} -setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) - #' @rdname countByValue #' @aliases countByValue,RDD-method setMethod("countByValue", @@ -493,11 +328,6 @@ setMethod("lapply", lapplyPartitionsWithIndex(X, func) }) -#' @rdname lapply -#' @export -setGeneric("map", function(X, FUN) { - standardGeneric("map") }) - #' @rdname lapply #' @aliases map,RDD,function-method setMethod("map", @@ -506,26 +336,6 @@ setMethod("map", lapply(X, FUN) }) -#' Flatten results after apply a function to all elements -#' -#' This function return a new RDD by first applying a function to all -#' elements of this RDD, and then flattening the results. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each element -#' @return a new RDD created by the transformation. -#' @rdname flatMap -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) -#' collect(multiplyByTwo) # 2,20,4,40,6,60... -#'} -setGeneric("flatMap", function(X, FUN) { - standardGeneric("flatMap") }) - #' @rdname flatMap #' @aliases flatMap,RDD,function-method setMethod("flatMap", @@ -540,25 +350,6 @@ setMethod("flatMap", lapplyPartition(X, partitionFunc) }) -#' Apply a function to each partition of an RDD -#' -#' Return a new RDD by applying a function to each partition of this RDD. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each partition. -#' @return a new RDD created by the transformation. -#' @rdname lapplyPartition -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) -#' collect(partitionSum) # 15, 40 -#'} -setGeneric("lapplyPartition", function(X, FUN) { - standardGeneric("lapplyPartition") }) - #' @rdname lapplyPartition #' @aliases lapplyPartition,RDD,function-method setMethod("lapplyPartition", @@ -567,13 +358,6 @@ setMethod("lapplyPartition", lapplyPartitionsWithIndex(X, function(s, part) { FUN(part) }) }) -#' mapPartitions is the same as lapplyPartition. -#' -#' @rdname lapplyPartition -#' @export -setGeneric("mapPartitions", function(X, FUN) { - standardGeneric("mapPartitions") }) - #' @rdname lapplyPartition #' @aliases mapPartitions,RDD,function-method setMethod("mapPartitions", @@ -582,26 +366,6 @@ setMethod("mapPartitions", lapplyPartition(X, FUN) }) -#' Return a new RDD by applying a function to each partition of this RDD, while -#' tracking the index of the original partition. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each partition; takes the partition -#' index and a list of elements in the particular partition. -#' @return a new RDD created by the transformation. -#' @rdname lapplyPartitionsWithIndex -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 5L) -#' prod <- lapplyPartitionsWithIndex(rdd, function(split, part) { -#' split * Reduce("+", part) }) -#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 -#'} -setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { - standardGeneric("lapplyPartitionsWithIndex") }) - #' @rdname lapplyPartitionsWithIndex #' @aliases lapplyPartitionsWithIndex,RDD,function-method setMethod("lapplyPartitionsWithIndex", @@ -614,12 +378,6 @@ setMethod("lapplyPartitionsWithIndex", PipelinedRDD(X, closureCapturingFunc) }) - -#' @rdname lapplyPartitionsWithIndex -#' @export -setGeneric("mapPartitionsWithIndex", function(X, FUN) { - standardGeneric("mapPartitionsWithIndex") }) - #' @rdname lapplyPartitionsWithIndex #' @aliases mapPartitionsWithIndex,RDD,function-method setMethod("mapPartitionsWithIndex", @@ -628,23 +386,6 @@ setMethod("mapPartitionsWithIndex", lapplyPartitionsWithIndex(X, FUN) }) -#' This function returns a new RDD containing only the elements that satisfy -#' a predicate (i.e. returning TRUE in a given logical function). -#' The same as `filter()' in Spark. -#' -#' @param x The RDD to be filtered. -#' @param f A unary predicate function. -#' @rdname filterRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) -#'} -setGeneric("filterRDD", - function(x, f) { standardGeneric("filterRDD") }) - #' @rdname filterRDD #' @aliases filterRDD,RDD,function-method setMethod("filterRDD", @@ -665,24 +406,6 @@ setMethod("Filter", filterRDD(x, f) }) -#' Reduce across elements of an RDD. -#' -#' This function reduces the elements of this RDD using the -#' specified commutative and associative binary operator. -#' -#' @param rdd The RDD to reduce -#' @param func Commutative and associative function to apply on elements -#' of the RDD. -#' @export -#' @rdname reduce -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' reduce(rdd, "+") # 55 -#'} -setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) - #' @rdname reduce #' @aliases reduce,RDD,ANY-method setMethod("reduce", @@ -698,19 +421,6 @@ setMethod("reduce", Reduce(func, partitionList) }) -#' Get the maximum element of an RDD. -#' -#' @param x The RDD to get the maximum element from -#' @export -#' @rdname maximum -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' maximum(rdd) # 10 -#'} -setGeneric("maximum", function(x) { standardGeneric("maximum") }) - #' @rdname maximum #' @aliases maximum,RDD setMethod("maximum", @@ -719,19 +429,6 @@ setMethod("maximum", reduce(x, max) }) -#' Get the minimum element of an RDD. -#' -#' @param x The RDD to get the minimum element from -#' @export -#' @rdname minimum -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' minimum(rdd) # 1 -#'} -setGeneric("minimum", function(x) { standardGeneric("minimum") }) - #' @rdname minimum #' @aliases minimum,RDD setMethod("minimum", @@ -740,21 +437,6 @@ setMethod("minimum", reduce(x, min) }) -#' Applies a function to all elements in an RDD, and force evaluation. -#' -#' @param x The RDD to apply the function -#' @param func The function to be applied. -#' @return invisible NULL. -#' @export -#' @rdname foreach -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' foreach(rdd, function(x) { save(x, file=...) }) -#'} -setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) - #' @rdname foreach #' @aliases foreach,RDD,function-method setMethod("foreach", @@ -767,19 +449,6 @@ setMethod("foreach", invisible(collect(mapPartitions(x, partition.func))) }) -#' Applies a function to each partition in an RDD, and force evaluation. -#' -#' @export -#' @rdname foreach -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) -#'} -setGeneric("foreachPartition", - function(x, func) { standardGeneric("foreachPartition") }) - #' @rdname foreach #' @aliases foreachPartition,RDD,function-method setMethod("foreachPartition", @@ -788,23 +457,6 @@ setMethod("foreachPartition", invisible(collect(mapPartitions(x, func))) }) -#' Take elements from an RDD. -#' -#' This function takes the first NUM elements in the RDD and -#' returns them in a list. -#' -#' @param x The RDD to take elements from -#' @param num Number of elements to take -#' @rdname take -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' take(rdd, 2L) # list(1, 2) -#'} -setGeneric("take", function(x, num) { standardGeneric("take") }) - #' @rdname take #' @aliases take,RDD,numeric-method setMethod("take", @@ -839,24 +491,6 @@ setMethod("take", resList }) -#' Removes the duplicates from RDD. -#' -#' This function returns a new RDD containing the distinct elements in the -#' given RDD. The same as `distinct()' in Spark. -#' -#' @param x The RDD to remove duplicates from. -#' @param numPartitions Number of partitions to create. -#' @rdname distinct -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) -#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) -#'} -setGeneric("distinct", - function(x, numPartitions) { standardGeneric("distinct") }) - setClassUnion("missingOrInteger", c("missing", "integer")) #' @rdname distinct #' @aliases distinct,RDD,missingOrInteger-method @@ -874,29 +508,6 @@ setMethod("distinct", resRDD }) -#' Return an RDD that is a sampled subset of the given RDD. -#' -#' The same as `sample()' in Spark. (We rename it due to signature -#' inconsistencies with the `sample()' function in R's base package.) -#' -#' @param x The RDD to sample elements from -#' @param withReplacement Sampling with replacement or not -#' @param fraction The (rough) sample target fraction -#' @param seed Randomness seed value -#' @rdname sampleRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) # ensure each num is in its own split -#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements -#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates -#'} -setGeneric("sampleRDD", - function(x, withReplacement, fraction, seed) { - standardGeneric("sampleRDD") - }) - #' @rdname sampleRDD #' @aliases sampleRDD,RDD setMethod("sampleRDD", @@ -942,28 +553,6 @@ setMethod("sampleRDD", lapplyPartitionsWithIndex(x, samplingFunc) }) - -#' Return a list of the elements that are a sampled subset of the given RDD. -#' -#' @param x The RDD to sample elements from -#' @param withReplacement Sampling with replacement or not -#' @param num Number of elements to return -#' @param seed Randomness seed value -#' @rdname takeSample -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:100) -#' # exactly 5 elements sampled, which may not be distinct -#' takeSample(rdd, TRUE, 5L, 1618L) -#' # exactly 5 distinct elements sampled -#' takeSample(rdd, FALSE, 5L, 16181618L) -#'} -setGeneric("takeSample", - function(x, withReplacement, num, seed) { - standardGeneric("takeSample") - }) #' @rdname takeSample #' @aliases takeSample,RDD setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", @@ -1012,20 +601,6 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", sample(samples)[1:total] }) -#' Creates tuples of the elements in this RDD by applying a function. -#' -#' @param x The RDD. -#' @param func The function to be applied. -#' @rdname keyBy -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3)) -#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) -#'} -setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) - #' @rdname keyBy #' @aliases keyBy,RDD setMethod("keyBy", @@ -1037,26 +612,6 @@ setMethod("keyBy", lapply(x, apply.func) }) -#' Return a new RDD that has exactly numPartitions partitions. -#' Can increase or decrease the level of parallelism in this RDD. Internally, -#' this uses a shuffle to redistribute data. -#' If you are decreasing the number of partitions in this RDD, consider using -#' coalesce, which can avoid performing a shuffle. -#' -#' @param x The RDD. -#' @param numPartitions Number of partitions to create. -#' @rdname repartition -#' @seealso coalesce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) -#' numPartitions(rdd) # 4 -#' numPartitions(repartition(rdd, 2L)) # 2 -#'} -setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) - #' @rdname repartition #' @aliases repartition,RDD setMethod("repartition", @@ -1065,22 +620,6 @@ setMethod("repartition", coalesce(x, numPartitions, TRUE) }) -#' Return a new RDD that is reduced into numPartitions partitions. -#' -#' @param x The RDD. -#' @param numPartitions Number of partitions to create. -#' @rdname coalesce -#' @seealso repartition -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) -#' numPartitions(rdd) # 3 -#' numPartitions(coalesce(rdd, 1L)) # 1 -#'} -setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) - #' @rdname coalesce #' @aliases coalesce,RDD setMethod("coalesce", @@ -1109,21 +648,6 @@ setMethod("coalesce", } }) -#' Save this RDD as a SequenceFile of serialized objects. -#' -#' @param x The RDD to save -#' @param path The directory where the file is saved -#' @rdname saveAsObjectFile -#' @seealso objectFile -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") -#'} -setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) - #' @rdname saveAsObjectFile #' @aliases saveAsObjectFile,RDD setMethod("saveAsObjectFile", @@ -1138,20 +662,6 @@ setMethod("saveAsObjectFile", invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) }) -#' Save this RDD as a text file, using string representations of elements. -#' -#' @param x The RDD to save -#' @param path The directory where the splits of the text file are saved -#' @rdname saveAsTextFile -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") -#'} -setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) - #' @rdname saveAsTextFile #' @aliases saveAsTextFile,RDD setMethod("saveAsTextFile", @@ -1166,28 +676,6 @@ setMethod("saveAsTextFile", callJMethod(getJRDD(stringRdd, dataSerialization = FALSE), "saveAsTextFile", path)) }) -#' Sort an RDD by the given key function. -#' -#' @param x An RDD to be sorted. -#' @param func A function used to compute the sort key for each element. -#' @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 elements are sorted. -#' @rdname sortBy -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(3, 2, 1)) -#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) -#'} -setGeneric("sortBy", function(x, - func, - ascending = TRUE, - numPartitions = 1L) { - standardGeneric("sortBy") - }) - #' @rdname sortBy #' @aliases sortBy,RDD,RDD-method setMethod("sortBy", @@ -1230,21 +718,6 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { reduce(newRdd, reduceFunc) } -#' Returns the first N elements from an RDD in ascending order. -#' -#' @param x An RDD. -#' @param num Number of elements to return. -#' @return The first N elements from the RDD in ascending order. -#' @rdname takeOrdered -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) -#'} -setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) - #' @rdname takeOrdered #' @aliases takeOrdered,RDD,RDD-method setMethod("takeOrdered", @@ -1253,21 +726,6 @@ setMethod("takeOrdered", takeOrderedElem(x, num) }) -#' Returns the top N elements from an RDD. -#' -#' @param x An RDD. -#' @param num Number of elements to return. -#' @return The top N elements from the RDD. -#' @rdname top -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) -#'} -setGeneric("top", function(x, num) { standardGeneric("top") }) - #' @rdname top #' @aliases top,RDD,RDD-method setMethod("top", @@ -1276,26 +734,6 @@ setMethod("top", takeOrderedElem(x, num, FALSE) }) -#' Fold an RDD using a given associative function and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using a given associative function and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param op An associative function for the folding operation. -#' @return The folding result. -#' @rdname fold -#' @seealso reduce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) -#' fold(rdd, 0, "+") # 15 -#'} -setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) - #' @rdname fold #' @aliases fold,RDD,RDD-method setMethod("fold", @@ -1304,31 +742,6 @@ setMethod("fold", aggregateRDD(x, zeroValue, op, op) }) -#' Aggregate an RDD using the given combine functions and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using given combine functions and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the RDD elements. It may return a different -#' result type from the type of the RDD elements. -#' @param combOp A function to aggregate results of seqOp. -#' @return The aggregation result. -#' @rdname aggregateRDD -#' @seealso reduce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' 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]]) } -#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) -#'} -setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) - #' @rdname aggregateRDD #' @aliases aggregateRDD,RDD,RDD-method setMethod("aggregateRDD", @@ -1343,27 +756,6 @@ setMethod("aggregateRDD", Reduce(combOp, partitionList, zeroValue) }) -#' Pipes elements to a forked external process. -#' -#' The same as 'pipe()' in Spark. -#' -#' @param x The RDD whose elements are piped to the forked external process. -#' @param command The command to fork an external process. -#' @param env A named list to set environment variables of the external process. -#' @return A new RDD created by piping all elements to a forked external process. -#' @rdname pipeRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' collect(pipeRDD(rdd, "more") -#' Output: c("1", "2", ..., "10") -#'} -setGeneric("pipeRDD", function(x, command, env = list()) { - standardGeneric("pipeRDD") -}) - #' @rdname pipeRDD #' @aliases pipeRDD,RDD,character-method setMethod("pipeRDD", @@ -1380,20 +772,6 @@ setMethod("pipeRDD", lapplyPartition(x, func) }) -# TODO: Consider caching the name in the RDD's environment -#' Return an RDD's name. -#' -#' @param x The RDD whose name is returned. -#' @rdname name -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' name(rdd) # NULL (if not set before) -#'} -setGeneric("name", function(x) { standardGeneric("name") }) - #' @rdname name #' @aliases name,RDD setMethod("name", @@ -1402,22 +780,6 @@ setMethod("name", callJMethod(getJRDD(x), "name") }) -#' Set an RDD's name. -#' -#' @param x The RDD whose name is to be set. -#' @param name The RDD name to be set. -#' @return a new RDD renamed. -#' @rdname setName -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' setName(rdd, "myRDD") -#' name(rdd) # "myRDD" -#'} -setGeneric("setName", function(x, name) { standardGeneric("setName") }) - #' @rdname setName #' @aliases setName,RDD setMethod("setName", @@ -1427,27 +789,6 @@ setMethod("setName", x }) -#' Zip an RDD with generated unique Long IDs. -#' -#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where -#' n is the number of partitions. So there may exist gaps, but this -#' method won't trigger a spark job, which is different from -#' zipWithIndex. -#' -#' @param x An RDD to be zipped. -#' @return An RDD with zipped items. -#' @rdname zipWithUniqueId -#' @seealso zipWithIndex -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithUniqueId(rdd)) -#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) -#'} -setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) - #' @rdname zipWithUniqueId #' @aliases zipWithUniqueId,RDD setMethod("zipWithUniqueId", @@ -1468,30 +809,6 @@ setMethod("zipWithUniqueId", lapplyPartitionsWithIndex(x, partitionFunc) }) -#' Zip an RDD with its element indices. -#' -#' The ordering is first based on the partition index and then the -#' ordering of items within each partition. So the first item in -#' the first partition gets index 0, and the last item in the last -#' partition receives the largest index. -#' -#' This method needs to trigger a Spark job when this RDD contains -#' more than one partition. -#' -#' @param x An RDD to be zipped. -#' @return An RDD with zipped items. -#' @rdname zipWithIndex -#' @seealso zipWithUniqueId -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithIndex(rdd)) -#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) -#'} -setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) - #' @rdname zipWithIndex #' @aliases zipWithIndex,RDD setMethod("zipWithIndex", @@ -1528,22 +845,6 @@ setMethod("zipWithIndex", ############ Binary Functions ############# -#' Return the union RDD of two RDDs. -#' The same as union() in Spark. -#' -#' @param x An RDD. -#' @param y An RDD. -#' @return a new RDD created by performing the simple union (witout removing -#' duplicates) of two input RDDs. -#' @rdname unionRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 -#'} -setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) #' @rdname unionRDD #' @aliases unionRDD,RDD,RDD-method diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R index 2cd5d83cf383b..665270385b583 100644 --- a/pkg/R/pairRDD.R +++ b/pkg/R/pairRDD.R @@ -2,24 +2,6 @@ ############ 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 -#' @rdname lookup -#' @export -#' @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) -#'} -setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) #' @rdname lookup #' @aliases lookup,RDD-method @@ -34,23 +16,6 @@ setMethod("lookup", 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. -#' @rdname countByKey -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) -#' countByKey(rdd) # ("a", 2L), ("b", 1L) -#'} -setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) - #' @rdname countByKey #' @aliases countByKey,RDD-method setMethod("countByKey", @@ -60,19 +25,6 @@ setMethod("countByKey", countByValue(keys) }) -#' Return an RDD with the keys of each tuple. -#' -#' @param x The RDD from which the keys of each tuple is returned. -#' @rdname keys -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(keys(rdd)) # list(1, 3) -#'} -setGeneric("keys", function(x) { standardGeneric("keys") }) - #' @rdname keys #' @aliases keys,RDD setMethod("keys", @@ -84,19 +36,6 @@ setMethod("keys", 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. -#' @rdname values -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(values(rdd)) # list(2, 4) -#'} -setGeneric("values", function(x) { standardGeneric("values") }) - #' @rdname values #' @aliases values,RDD setMethod("values", @@ -108,25 +47,6 @@ setMethod("values", 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. -#' @rdname mapValues -#' @export -#' @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), ...) -#'} -setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) - #' @rdname mapValues #' @aliases mapValues,RDD,function-method setMethod("mapValues", @@ -138,25 +58,6 @@ setMethod("mapValues", 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. -#' @rdname flatMapValues -#' @export -#' @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)) -#'} -setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) - #' @rdname flatMapValues #' @aliases flatMapValues,RDD,function-method setMethod("flatMapValues", @@ -170,34 +71,6 @@ setMethod("flatMapValues", ############ 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. -#' @rdname partitionBy -#' @export -#' @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) -#'} -setGeneric("partitionBy", - function(x, numPartitions, ...) { - standardGeneric("partitionBy") - }) #' @rdname partitionBy #' @aliases partitionBy,RDD,integer-method @@ -246,32 +119,6 @@ setMethod("partitionBy", RDD(r, serialized = TRUE) }) -#' 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 -#' @rdname groupByKey -#' @export -#' @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)) -#'} -setGeneric("groupByKey", - function(x, numPartitions) { - standardGeneric("groupByKey") - }) - #' @rdname groupByKey #' @aliases groupByKey,RDD,integer-method setMethod("groupByKey", @@ -311,34 +158,6 @@ setMethod("groupByKey", 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 -#' @rdname reduceByKey -#' @seealso groupByKey -#' @export -#' @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) -#'} -setGeneric("reduceByKey", - function(x, combineFunc, numPartitions) { - standardGeneric("reduceByKey") - }) - #' @rdname reduceByKey #' @aliases reduceByKey,RDD,integer-method setMethod("reduceByKey", @@ -360,32 +179,6 @@ setMethod("reduceByKey", 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 -#' @rdname reduceByKeyLocally -#' @seealso reduceByKey -#' @export -#' @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)) -#'} -setGeneric("reduceByKeyLocally", - function(x, combineFunc) { - standardGeneric("reduceByKeyLocally") - }) - #' @rdname reduceByKeyLocally #' @aliases reduceByKeyLocally,RDD,integer-method setMethod("reduceByKeyLocally", @@ -419,46 +212,6 @@ setMethod("reduceByKeyLocally", 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 -#' -#' @rdname combineByKey -#' @seealso groupByKey, reduceByKey -#' @export -#' @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) -#'} -setGeneric("combineByKey", - function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { - standardGeneric("combineByKey") - }) - #' @rdname combineByKey #' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method setMethod("combineByKey", @@ -492,41 +245,6 @@ setMethod("combineByKey", 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. -#' @rdname aggregateByKey -#' @seealso foldByKey, combineByKey -#' @export -#' @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))) -#'} -setGeneric("aggregateByKey", - function(x, zeroValue, seqOp, combOp, numPartitions) { - standardGeneric("aggregateByKey") - }) - #' @rdname aggregateByKey #' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method setMethod("aggregateByKey", @@ -540,31 +258,6 @@ setMethod("aggregateByKey", 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. -#' @rdname foldByKey -#' @seealso aggregateByKey, combineByKey -#' @export -#' @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)) -#'} -setGeneric("foldByKey", - function(x, zeroValue, func, numPartitions) { - standardGeneric("foldByKey") - }) - #' @rdname foldByKey #' @aliases foldByKey,RDD,ANY,ANY,integer-method setMethod("foldByKey", @@ -576,29 +269,6 @@ setMethod("foldByKey", ############ 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. -#' @rdname join-methods -#' @export -#' @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)) -#'} -setGeneric("join", function(x, y, numPartitions) { standardGeneric("join") }) #' @rdname join-methods #' @aliases join,RDD,RDD-method @@ -615,32 +285,6 @@ setMethod("join", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), 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. -#' @rdname join-methods -#' @export -#' @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))) -#'} -setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) - #' @rdname join-methods #' @aliases leftOuterJoin,RDD,RDD-method setMethod("leftOuterJoin", @@ -656,32 +300,6 @@ setMethod("leftOuterJoin", 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. -#' @rdname join-methods -#' @export -#' @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))) -#'} -setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) - #' @rdname join-methods #' @aliases rightOuterJoin,RDD,RDD-method setMethod("rightOuterJoin", @@ -697,35 +315,6 @@ setMethod("rightOuterJoin", 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. -#' @rdname join-methods -#' @export -#' @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)), -#'} -setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) - #' @rdname join-methods #' @aliases fullOuterJoin,RDD,RDD-method @@ -742,27 +331,6 @@ setMethod("fullOuterJoin", 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. -#' @rdname cogroup -#' @export -#' @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())) -#'} -setGeneric("cogroup", - function(..., numPartitions) { standardGeneric("cogroup") }, - signature = "...") - #' @rdname cogroup #' @aliases cogroup,RDD-method setMethod("cogroup", @@ -804,26 +372,6 @@ setMethod("cogroup", 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. -#' @rdname sortByKey -#' @export -#' @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)) -#'} -setGeneric("sortByKey", function(x, - ascending = TRUE, - numPartitions = 1L) { - standardGeneric("sortByKey") - }) - #' @rdname sortByKey #' @aliases sortByKey,RDD,RDD-method setMethod("sortByKey", From 471c794fdcbf4a3b873d087a169cf91e67f4443a Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 9 Mar 2015 10:16:21 -0400 Subject: [PATCH 111/121] Move getJRDD and broadcast's value to 00-generic.R. --- pkg/R/00-generics.R | 15 +++++++++++++++ pkg/R/RDD.R | 1 - pkg/R/broadcast.R | 9 --------- 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/pkg/R/00-generics.R b/pkg/R/00-generics.R index 90d0d35a2b653..6d111f592b2f0 100644 --- a/pkg/R/00-generics.R +++ b/pkg/R/00-generics.R @@ -1,5 +1,8 @@ ############ RDD Actions and Transformations ############ +# The jrdd accessor function. +setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) + #' Persist an RDD #' #' Persist this RDD with the default storage level (MEMORY_ONLY). @@ -1158,3 +1161,15 @@ setGeneric("sortByKey", function(x, standardGeneric("sortByKey") }) + +############ Broadcast Variable Methods ############ + +#' @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 +#' @export +setGeneric("value", function(bcast) { standardGeneric("value") }) + diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 34eb4ebd8e80a..9377feb1533d1 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -97,7 +97,6 @@ PipelinedRDD <- function(prev, func) { # The jrdd accessor function. -setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) setMethod("getJRDD", signature(rdd = "RDD"), function(rdd) rdd@jrdd ) setMethod("getJRDD", signature(rdd = "PipelinedRDD"), function(rdd, dataSerialization = TRUE) { diff --git a/pkg/R/broadcast.R b/pkg/R/broadcast.R index 45881599f4f18..7f9e1c179ac11 100644 --- a/pkg/R/broadcast.R +++ b/pkg/R/broadcast.R @@ -28,15 +28,6 @@ Broadcast <- function(id, value, jBroadcastRef, 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 -#' @export -setGeneric("value", function(bcast) { standardGeneric("value") }) - #' @rdname broadcast #' @aliases value,Broadcast-method setMethod("value", From 6bccbbfc1b72b0eb62ec1eda7e17505ca44185b7 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 9 Mar 2015 16:09:48 -0400 Subject: [PATCH 112/121] Move roxygen doc back to implementation. --- pkg/DESCRIPTION | 14 + pkg/R/00-generics.R | 1175 ------------------------------------------- pkg/R/RDD.R | 545 +++++++++++++++++++- pkg/R/broadcast.R | 6 + pkg/R/generics.R | 331 ++++++++++++ pkg/R/pairRDD.R | 372 +++++++++++++- 6 files changed, 1266 insertions(+), 1177 deletions(-) delete mode 100644 pkg/R/00-generics.R create mode 100644 pkg/R/generics.R diff --git a/pkg/DESCRIPTION b/pkg/DESCRIPTION index 618847c951640..897fe77abc13a 100644 --- a/pkg/DESCRIPTION +++ b/pkg/DESCRIPTION @@ -13,3 +13,17 @@ Suggests: testthat Description: R frontend for Spark License: Apache License (== 2.0) +Collate: + 'generics.R' + 'jobj.R' + 'RDD.R' + 'pairRDD.R' + 'broadcast.R' + 'context.R' + 'deserialize.R' + 'serialize.R' + 'sparkR.R' + 'sparkRBackend.R' + 'sparkRClient.R' + 'utils.R' + 'zzz.R' diff --git a/pkg/R/00-generics.R b/pkg/R/00-generics.R deleted file mode 100644 index 6d111f592b2f0..0000000000000 --- a/pkg/R/00-generics.R +++ /dev/null @@ -1,1175 +0,0 @@ -############ RDD Actions and Transformations ############ - -# The jrdd accessor function. -setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) - -#' Persist an RDD -#' -#' Persist this RDD with the default storage level (MEMORY_ONLY). -#' -#' @param x The RDD to cache -#' @rdname cache-methods -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' cache(rdd) -#'} -setGeneric("cache", function(x) { standardGeneric("cache") }) - -#' Persist an RDD -#' -#' Persist this RDD with the specified storage level. For details of the -#' supported storage levels, refer to -#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. -#' -#' @param x The RDD to persist -#' @param newLevel The new storage level to be assigned -#' @rdname persist -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' persist(rdd, "MEMORY_AND_DISK") -#'} -setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) - -#' Unpersist an RDD -#' -#' Mark the RDD as non-persistent, and remove all blocks for it from memory and -#' disk. -#' -#' @param rdd The RDD to unpersist -#' @rdname unpersist-methods -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' cache(rdd) # rdd@@env$isCached == TRUE -#' unpersist(rdd) # rdd@@env$isCached == FALSE -#'} -setGeneric("unpersist", function(x) { standardGeneric("unpersist") }) - -#' Checkpoint an RDD -#' -#' Mark this RDD for checkpointing. It will be saved to a file inside the -#' checkpoint directory set with setCheckpointDir() and all references to its -#' parent RDDs will be removed. This function must be called before any job has -#' been executed on this RDD. It is strongly recommended that this RDD is -#' persisted in memory, otherwise saving it on a file will require recomputation. -#' -#' @param rdd The RDD to checkpoint -#' @rdname checkpoint-methods -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' setCheckpointDir(sc, "checkpoints") -#' rdd <- parallelize(sc, 1:10, 2L) -#' checkpoint(rdd) -#'} -setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) - -#' Gets the number of partitions of an RDD -#' -#' @param x A RDD. -#' @return the number of partitions of rdd as an integer. -#' @rdname numPartitions -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' numPartitions(rdd) # 2L -#'} -setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) - -#' Collect elements of an RDD -#' -#' @description -#' \code{collect} returns a list that contains all of the elements in this RDD. -#' -#' @param x The RDD to collect -#' @param ... Other optional arguments to collect -#' @param flatten FALSE if the list should not flattened -#' @return a list containing elements in the RDD -#' @rdname collect-methods -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' collect(rdd) # list from 1 to 10 -#' collectPartition(rdd, 0L) # list from 1 to 5 -#'} -setGeneric("collect", function(x, ...) { standardGeneric("collect") }) - -#' @rdname collect-methods -#' @export -#' @description -#' \code{collectPartition} returns a list that contains all of the elements -#' in the specified partition of the RDD. -#' @param partitionId the partition to collect (starts from 0) -setGeneric("collectPartition", - function(x, partitionId) { - standardGeneric("collectPartition") - }) - -#' @rdname collect-methods -#' @export -#' @description -#' \code{collectAsMap} returns a named list as a map that contains all of the elements -#' in a key-value pair RDD. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) -#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) -#'} -setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) - -#' Return the number of elements in the RDD. -#' -#' @param x The RDD to count -#' @return number of elements in the RDD. -#' @rdname count -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' count(rdd) # 10 -#' length(rdd) # Same as count -#'} -setGeneric("count", function(x) { standardGeneric("count") }) - -#' Return the count of each unique value in this RDD as a list of -#' (value, count) pairs. -#' -#' Same as countByValue in Spark. -#' -#' @param x The RDD to count -#' @return list of (value, count) pairs, where count is number of each unique -#' value in rdd. -#' @rdname countByValue -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, c(1,2,3,2,1)) -#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) -#'} -setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) - -#' @rdname lapply -#' @export -setGeneric("map", function(X, FUN) { - standardGeneric("map") }) - -#' Flatten results after apply a function to all elements -#' -#' This function return a new RDD by first applying a function to all -#' elements of this RDD, and then flattening the results. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each element -#' @return a new RDD created by the transformation. -#' @rdname flatMap -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) -#' collect(multiplyByTwo) # 2,20,4,40,6,60... -#'} -setGeneric("flatMap", function(X, FUN) { - standardGeneric("flatMap") }) - -#' Apply a function to each partition of an RDD -#' -#' Return a new RDD by applying a function to each partition of this RDD. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each partition. -#' @return a new RDD created by the transformation. -#' @rdname lapplyPartition -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) -#' collect(partitionSum) # 15, 40 -#'} -setGeneric("lapplyPartition", function(X, FUN) { - standardGeneric("lapplyPartition") }) - -#' mapPartitions is the same as lapplyPartition. -#' -#' @rdname lapplyPartition -#' @export -setGeneric("mapPartitions", function(X, FUN) { - standardGeneric("mapPartitions") }) - -#' Return a new RDD by applying a function to each partition of this RDD, while -#' tracking the index of the original partition. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each partition; takes the partition -#' index and a list of elements in the particular partition. -#' @return a new RDD created by the transformation. -#' @rdname lapplyPartitionsWithIndex -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 5L) -#' prod <- lapplyPartitionsWithIndex(rdd, function(split, part) { -#' split * Reduce("+", part) }) -#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 -#'} -setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { - standardGeneric("lapplyPartitionsWithIndex") }) - -#' @rdname lapplyPartitionsWithIndex -#' @export -setGeneric("mapPartitionsWithIndex", function(X, FUN) { - standardGeneric("mapPartitionsWithIndex") }) - -#' This function returns a new RDD containing only the elements that satisfy -#' a predicate (i.e. returning TRUE in a given logical function). -#' The same as `filter()' in Spark. -#' -#' @param x The RDD to be filtered. -#' @param f A unary predicate function. -#' @rdname filterRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) -#'} -setGeneric("filterRDD", - function(x, f) { standardGeneric("filterRDD") }) - -#' Reduce across elements of an RDD. -#' -#' This function reduces the elements of this RDD using the -#' specified commutative and associative binary operator. -#' -#' @param rdd The RDD to reduce -#' @param func Commutative and associative function to apply on elements -#' of the RDD. -#' @export -#' @rdname reduce -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' reduce(rdd, "+") # 55 -#'} -setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) - -#' Get the maximum element of an RDD. -#' -#' @param x The RDD to get the maximum element from -#' @export -#' @rdname maximum -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' maximum(rdd) # 10 -#'} -setGeneric("maximum", function(x) { standardGeneric("maximum") }) - -#' Get the minimum element of an RDD. -#' -#' @param x The RDD to get the minimum element from -#' @export -#' @rdname minimum -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' minimum(rdd) # 1 -#'} -setGeneric("minimum", function(x) { standardGeneric("minimum") }) - -#' Applies a function to all elements in an RDD, and force evaluation. -#' -#' @param x The RDD to apply the function -#' @param func The function to be applied. -#' @return invisible NULL. -#' @export -#' @rdname foreach -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' foreach(rdd, function(x) { save(x, file=...) }) -#'} -setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) - -#' Applies a function to each partition in an RDD, and force evaluation. -#' -#' @export -#' @rdname foreach -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) -#'} -setGeneric("foreachPartition", - function(x, func) { standardGeneric("foreachPartition") }) - -#' Take elements from an RDD. -#' -#' This function takes the first NUM elements in the RDD and -#' returns them in a list. -#' -#' @param x The RDD to take elements from -#' @param num Number of elements to take -#' @rdname take -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' take(rdd, 2L) # list(1, 2) -#'} -setGeneric("take", function(x, num) { standardGeneric("take") }) - -#' Removes the duplicates from RDD. -#' -#' This function returns a new RDD containing the distinct elements in the -#' given RDD. The same as `distinct()' in Spark. -#' -#' @param x The RDD to remove duplicates from. -#' @param numPartitions Number of partitions to create. -#' @rdname distinct -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) -#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) -#'} -setGeneric("distinct", - function(x, numPartitions) { standardGeneric("distinct") }) - -#' Return an RDD that is a sampled subset of the given RDD. -#' -#' The same as `sample()' in Spark. (We rename it due to signature -#' inconsistencies with the `sample()' function in R's base package.) -#' -#' @param x The RDD to sample elements from -#' @param withReplacement Sampling with replacement or not -#' @param fraction The (rough) sample target fraction -#' @param seed Randomness seed value -#' @rdname sampleRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) # ensure each num is in its own split -#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements -#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates -#'} -setGeneric("sampleRDD", - function(x, withReplacement, fraction, seed) { - standardGeneric("sampleRDD") - }) - -#' Return a list of the elements that are a sampled subset of the given RDD. -#' -#' @param x The RDD to sample elements from -#' @param withReplacement Sampling with replacement or not -#' @param num Number of elements to return -#' @param seed Randomness seed value -#' @rdname takeSample -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:100) -#' # exactly 5 elements sampled, which may not be distinct -#' takeSample(rdd, TRUE, 5L, 1618L) -#' # exactly 5 distinct elements sampled -#' takeSample(rdd, FALSE, 5L, 16181618L) -#'} -setGeneric("takeSample", - function(x, withReplacement, num, seed) { - standardGeneric("takeSample") - }) - -#' Creates tuples of the elements in this RDD by applying a function. -#' -#' @param x The RDD. -#' @param func The function to be applied. -#' @rdname keyBy -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3)) -#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) -#'} -setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) - -#' Return a new RDD that has exactly numPartitions partitions. -#' Can increase or decrease the level of parallelism in this RDD. Internally, -#' this uses a shuffle to redistribute data. -#' If you are decreasing the number of partitions in this RDD, consider using -#' coalesce, which can avoid performing a shuffle. -#' -#' @param x The RDD. -#' @param numPartitions Number of partitions to create. -#' @rdname repartition -#' @seealso coalesce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) -#' numPartitions(rdd) # 4 -#' numPartitions(repartition(rdd, 2L)) # 2 -#'} -setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) - -#' Return a new RDD that is reduced into numPartitions partitions. -#' -#' @param x The RDD. -#' @param numPartitions Number of partitions to create. -#' @rdname coalesce -#' @seealso repartition -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) -#' numPartitions(rdd) # 3 -#' numPartitions(coalesce(rdd, 1L)) # 1 -#'} -setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) - -#' Save this RDD as a SequenceFile of serialized objects. -#' -#' @param x The RDD to save -#' @param path The directory where the file is saved -#' @rdname saveAsObjectFile -#' @seealso objectFile -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") -#'} -setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) - -#' Save this RDD as a text file, using string representations of elements. -#' -#' @param x The RDD to save -#' @param path The directory where the splits of the text file are saved -#' @rdname saveAsTextFile -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") -#'} -setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) - -#' Sort an RDD by the given key function. -#' -#' @param x An RDD to be sorted. -#' @param func A function used to compute the sort key for each element. -#' @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 elements are sorted. -#' @rdname sortBy -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(3, 2, 1)) -#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) -#'} -setGeneric("sortBy", function(x, - func, - ascending = TRUE, - numPartitions = 1L) { - standardGeneric("sortBy") -}) - -#' Returns the first N elements from an RDD in ascending order. -#' -#' @param x An RDD. -#' @param num Number of elements to return. -#' @return The first N elements from the RDD in ascending order. -#' @rdname takeOrdered -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) -#'} -setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) - -#' Returns the top N elements from an RDD. -#' -#' @param x An RDD. -#' @param num Number of elements to return. -#' @return The top N elements from the RDD. -#' @rdname top -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) -#'} -setGeneric("top", function(x, num) { standardGeneric("top") }) - -#' Fold an RDD using a given associative function and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using a given associative function and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param op An associative function for the folding operation. -#' @return The folding result. -#' @rdname fold -#' @seealso reduce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) -#' fold(rdd, 0, "+") # 15 -#'} -setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) - -#' Aggregate an RDD using the given combine functions and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using given combine functions and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the RDD elements. It may return a different -#' result type from the type of the RDD elements. -#' @param combOp A function to aggregate results of seqOp. -#' @return The aggregation result. -#' @rdname aggregateRDD -#' @seealso reduce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' 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]]) } -#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) -#'} -setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) - -#' Pipes elements to a forked external process. -#' -#' The same as 'pipe()' in Spark. -#' -#' @param x The RDD whose elements are piped to the forked external process. -#' @param command The command to fork an external process. -#' @param env A named list to set environment variables of the external process. -#' @return A new RDD created by piping all elements to a forked external process. -#' @rdname pipeRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' collect(pipeRDD(rdd, "more") -#' Output: c("1", "2", ..., "10") -#'} -setGeneric("pipeRDD", function(x, command, env = list()) { - standardGeneric("pipeRDD") -}) - -# TODO: Consider caching the name in the RDD's environment -#' Return an RDD's name. -#' -#' @param x The RDD whose name is returned. -#' @rdname name -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' name(rdd) # NULL (if not set before) -#'} -setGeneric("name", function(x) { standardGeneric("name") }) - -#' Set an RDD's name. -#' -#' @param x The RDD whose name is to be set. -#' @param name The RDD name to be set. -#' @return a new RDD renamed. -#' @rdname setName -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' setName(rdd, "myRDD") -#' name(rdd) # "myRDD" -#'} -setGeneric("setName", function(x, name) { standardGeneric("setName") }) - -#' Zip an RDD with generated unique Long IDs. -#' -#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where -#' n is the number of partitions. So there may exist gaps, but this -#' method won't trigger a spark job, which is different from -#' zipWithIndex. -#' -#' @param x An RDD to be zipped. -#' @return An RDD with zipped items. -#' @rdname zipWithUniqueId -#' @seealso zipWithIndex -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithUniqueId(rdd)) -#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) -#'} -setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) - -#' Zip an RDD with its element indices. -#' -#' The ordering is first based on the partition index and then the -#' ordering of items within each partition. So the first item in -#' the first partition gets index 0, and the last item in the last -#' partition receives the largest index. -#' -#' This method needs to trigger a Spark job when this RDD contains -#' more than one partition. -#' -#' @param x An RDD to be zipped. -#' @return An RDD with zipped items. -#' @rdname zipWithIndex -#' @seealso zipWithUniqueId -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithIndex(rdd)) -#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) -#'} -setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) - - -############ Binary Functions ############# - - -#' Return the union RDD of two RDDs. -#' The same as union() in Spark. -#' -#' @param x An RDD. -#' @param y An RDD. -#' @return a new RDD created by performing the simple union (witout removing -#' duplicates) of two input RDDs. -#' @rdname unionRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 -#'} -setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) - -#' 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 -#' @rdname lookup -#' @export -#' @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) -#'} -setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) - -#' 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. -#' @rdname countByKey -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) -#' countByKey(rdd) # ("a", 2L), ("b", 1L) -#'} -setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) - -#' Return an RDD with the keys of each tuple. -#' -#' @param x The RDD from which the keys of each tuple is returned. -#' @rdname keys -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(keys(rdd)) # list(1, 3) -#'} -setGeneric("keys", function(x) { standardGeneric("keys") }) - -#' Return an RDD with the values of each tuple. -#' -#' @param x The RDD from which the values of each tuple is returned. -#' @rdname values -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(values(rdd)) # list(2, 4) -#'} -setGeneric("values", function(x) { standardGeneric("values") }) - -#' 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. -#' @rdname mapValues -#' @export -#' @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), ...) -#'} -setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) - -#' 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. -#' @rdname flatMapValues -#' @export -#' @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)) -#'} -setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) - - -############ 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. -#' @rdname partitionBy -#' @export -#' @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) -#'} -setGeneric("partitionBy", - function(x, numPartitions, ...) { - standardGeneric("partitionBy") - }) - -#' 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 -#' @rdname groupByKey -#' @export -#' @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)) -#'} -setGeneric("groupByKey", - function(x, numPartitions) { - standardGeneric("groupByKey") - }) - -#' 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 -#' @rdname reduceByKey -#' @seealso groupByKey -#' @export -#' @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) -#'} -setGeneric("reduceByKey", - function(x, combineFunc, numPartitions) { - standardGeneric("reduceByKey") - }) - -#' 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 -#' @rdname reduceByKeyLocally -#' @seealso reduceByKey -#' @export -#' @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)) -#'} -setGeneric("reduceByKeyLocally", - function(x, combineFunc) { - standardGeneric("reduceByKeyLocally") - }) - -#' 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 -#' -#' @rdname combineByKey -#' @seealso groupByKey, reduceByKey -#' @export -#' @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) -#'} -setGeneric("combineByKey", - function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { - standardGeneric("combineByKey") - }) - -#' 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. -#' @rdname aggregateByKey -#' @seealso foldByKey, combineByKey -#' @export -#' @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))) -#'} -setGeneric("aggregateByKey", - function(x, zeroValue, seqOp, combOp, numPartitions) { - standardGeneric("aggregateByKey") - }) - -#' 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. -#' @rdname foldByKey -#' @seealso aggregateByKey, combineByKey -#' @export -#' @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)) -#'} -setGeneric("foldByKey", - function(x, zeroValue, func, numPartitions) { - standardGeneric("foldByKey") - }) - -#' 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. -#' @rdname join-methods -#' @export -#' @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)) -#'} -setGeneric("join", function(x, y, numPartitions) { standardGeneric("join") }) - -#' 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. -#' @rdname join-methods -#' @export -#' @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))) -#'} -setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) - -#' 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. -#' @rdname join-methods -#' @export -#' @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))) -#'} -setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) - -#' 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. -#' @rdname join-methods -#' @export -#' @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)), -#'} -setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) - -#' 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. -#' @rdname cogroup -#' @export -#' @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())) -#'} -setGeneric("cogroup", - function(..., numPartitions) { standardGeneric("cogroup") }, - signature = "...") - -#' 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. -#' @rdname sortByKey -#' @export -#' @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)) -#'} -setGeneric("sortByKey", function(x, - ascending = TRUE, - numPartitions = 1L) { - standardGeneric("sortByKey") -}) - - -############ Broadcast Variable Methods ############ - -#' @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 -#' @export -setGeneric("value", function(bcast) { standardGeneric("value") }) - diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 9377feb1533d1..8a98d408709ab 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -159,8 +159,19 @@ setValidity("RDD", ############ Actions and Transformations ############ - +#' Persist an RDD +#' +#' Persist this RDD with the default storage level (MEMORY_ONLY). +#' +#' @param x The RDD to cache #' @rdname cache-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) +#'} #' @aliases cache,RDD-method setMethod("cache", signature(x = "RDD"), @@ -170,7 +181,22 @@ setMethod("cache", x }) +#' Persist an RDD +#' +#' Persist this RDD with the specified storage level. For details of the +#' supported storage levels, refer to +#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#' +#' @param x The RDD to persist +#' @param newLevel The new storage level to be assigned #' @rdname persist +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' persist(rdd, "MEMORY_AND_DISK") +#'} #' @aliases persist,RDD-method setMethod("persist", signature(x = "RDD", newLevel = "character"), @@ -204,7 +230,21 @@ setMethod("persist", x }) +#' Unpersist an RDD +#' +#' Mark the RDD as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param rdd The RDD to unpersist #' @rdname unpersist-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) # rdd@@env$isCached == TRUE +#' unpersist(rdd) # rdd@@env$isCached == FALSE +#'} #' @aliases unpersist,RDD-method setMethod("unpersist", signature(x = "RDD"), @@ -214,7 +254,24 @@ setMethod("unpersist", x }) +#' Checkpoint an RDD +#' +#' Mark this RDD for checkpointing. It will be saved to a file inside the +#' checkpoint directory set with setCheckpointDir() and all references to its +#' parent RDDs will be removed. This function must be called before any job has +#' been executed on this RDD. It is strongly recommended that this RDD is +#' persisted in memory, otherwise saving it on a file will require recomputation. +#' +#' @param rdd The RDD to checkpoint #' @rdname checkpoint-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "checkpoints") +#' rdd <- parallelize(sc, 1:10, 2L) +#' checkpoint(rdd) +#'} #' @aliases checkpoint,RDD-method setMethod("checkpoint", signature(x = "RDD"), @@ -225,7 +282,18 @@ setMethod("checkpoint", x }) +#' Gets the number of partitions of an RDD +#' +#' @param x A RDD. +#' @return the number of partitions of rdd as an integer. #' @rdname numPartitions +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' numPartitions(rdd) # 2L +#'} #' @aliases numPartitions,RDD-method setMethod("numPartitions", signature(x = "RDD"), @@ -235,7 +303,24 @@ setMethod("numPartitions", callJMethod(partitions, "size") }) +#' Collect elements of an RDD +#' +#' @description +#' \code{collect} returns a list that contains all of the elements in this RDD. +#' +#' @param x The RDD to collect +#' @param ... Other optional arguments to collect +#' @param flatten FALSE if the list should not flattened +#' @return a list containing elements in the RDD #' @rdname collect-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' collect(rdd) # list from 1 to 10 +#' collectPartition(rdd, 0L) # list from 1 to 5 +#'} #' @aliases collect,RDD-method setMethod("collect", signature(x = "RDD"), @@ -246,6 +331,11 @@ setMethod("collect", }) #' @rdname collect-methods +#' @export +#' @description +#' \code{collectPartition} returns a list that contains all of the elements +#' in the specified partition of the RDD. +#' @param partitionId the partition to collect (starts from 0) #' @aliases collectPartition,integer,RDD-method setMethod("collectPartition", signature(x = "RDD", partitionId = "integer"), @@ -259,6 +349,16 @@ setMethod("collectPartition", }) #' @rdname collect-methods +#' @export +#' @description +#' \code{collectAsMap} returns a named list as a map that contains all of the elements +#' in a key-value pair RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) +#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#'} #' @aliases collectAsMap,RDD-method setMethod("collectAsMap", signature(x = "RDD"), @@ -269,7 +369,19 @@ setMethod("collectAsMap", as.list(map) }) +#' Return the number of elements in the RDD. +#' +#' @param x The RDD to count +#' @return number of elements in the RDD. #' @rdname count +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' count(rdd) # 10 +#' length(rdd) # Same as count +#'} #' @aliases count,RDD-method setMethod("count", signature(x = "RDD"), @@ -291,7 +403,22 @@ setMethod("length", count(x) }) +#' Return the count of each unique value in this RDD as a list of +#' (value, count) pairs. +#' +#' Same as countByValue in Spark. +#' +#' @param x The RDD to count +#' @return list of (value, count) pairs, where count is number of each unique +#' value in rdd. #' @rdname countByValue +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,3,2,1)) +#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) +#'} #' @aliases countByValue,RDD-method setMethod("countByValue", signature(x = "RDD"), @@ -328,6 +455,7 @@ setMethod("lapply", }) #' @rdname lapply +#' @export #' @aliases map,RDD,function-method setMethod("map", signature(X = "RDD", FUN = "function"), @@ -335,7 +463,23 @@ setMethod("map", lapply(X, FUN) }) +#' Flatten results after apply a function to all elements +#' +#' This function return a new RDD by first applying a function to all +#' elements of this RDD, and then flattening the results. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. #' @rdname flatMap +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) +#' collect(multiplyByTwo) # 2,20,4,40,6,60... +#'} #' @aliases flatMap,RDD,function-method setMethod("flatMap", signature(X = "RDD", FUN = "function"), @@ -349,7 +493,22 @@ setMethod("flatMap", lapplyPartition(X, partitionFunc) }) +#' Apply a function to each partition of an RDD +#' +#' Return a new RDD by applying a function to each partition of this RDD. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition. +#' @return a new RDD created by the transformation. #' @rdname lapplyPartition +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) +#' collect(partitionSum) # 15, 40 +#'} #' @aliases lapplyPartition,RDD,function-method setMethod("lapplyPartition", signature(X = "RDD", FUN = "function"), @@ -357,7 +516,10 @@ setMethod("lapplyPartition", lapplyPartitionsWithIndex(X, function(s, part) { FUN(part) }) }) +#' mapPartitions is the same as lapplyPartition. +#' #' @rdname lapplyPartition +#' @export #' @aliases mapPartitions,RDD,function-method setMethod("mapPartitions", signature(X = "RDD", FUN = "function"), @@ -365,7 +527,23 @@ setMethod("mapPartitions", lapplyPartition(X, FUN) }) +#' Return a new RDD by applying a function to each partition of this RDD, while +#' tracking the index of the original partition. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition; takes the partition +#' index and a list of elements in the particular partition. +#' @return a new RDD created by the transformation. #' @rdname lapplyPartitionsWithIndex +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 5L) +#' prod <- lapplyPartitionsWithIndex(rdd, function(split, part) { +#' split * Reduce("+", part) }) +#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#'} #' @aliases lapplyPartitionsWithIndex,RDD,function-method setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), @@ -378,6 +556,7 @@ setMethod("lapplyPartitionsWithIndex", }) #' @rdname lapplyPartitionsWithIndex +#' @export #' @aliases mapPartitionsWithIndex,RDD,function-method setMethod("mapPartitionsWithIndex", signature(X = "RDD", FUN = "function"), @@ -385,7 +564,20 @@ setMethod("mapPartitionsWithIndex", lapplyPartitionsWithIndex(X, FUN) }) +#' This function returns a new RDD containing only the elements that satisfy +#' a predicate (i.e. returning TRUE in a given logical function). +#' The same as `filter()' in Spark. +#' +#' @param x The RDD to be filtered. +#' @param f A unary predicate function. #' @rdname filterRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#'} #' @aliases filterRDD,RDD,function-method setMethod("filterRDD", signature(x = "RDD", f = "function"), @@ -405,7 +597,22 @@ setMethod("Filter", filterRDD(x, f) }) +#' Reduce across elements of an RDD. +#' +#' This function reduces the elements of this RDD using the +#' specified commutative and associative binary operator. +#' +#' @param rdd The RDD to reduce +#' @param func Commutative and associative function to apply on elements +#' of the RDD. +#' @export #' @rdname reduce +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' reduce(rdd, "+") # 55 +#'} #' @aliases reduce,RDD,ANY-method setMethod("reduce", signature(x = "RDD", func = "ANY"), @@ -420,7 +627,17 @@ setMethod("reduce", Reduce(func, partitionList) }) +#' Get the maximum element of an RDD. +#' +#' @param x The RDD to get the maximum element from +#' @export #' @rdname maximum +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' maximum(rdd) # 10 +#'} #' @aliases maximum,RDD setMethod("maximum", signature(x = "RDD"), @@ -428,7 +645,17 @@ setMethod("maximum", reduce(x, max) }) +#' Get the minimum element of an RDD. +#' +#' @param x The RDD to get the minimum element from +#' @export #' @rdname minimum +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' minimum(rdd) # 1 +#'} #' @aliases minimum,RDD setMethod("minimum", signature(x = "RDD"), @@ -436,7 +663,19 @@ setMethod("minimum", reduce(x, min) }) +#' Applies a function to all elements in an RDD, and force evaluation. +#' +#' @param x The RDD to apply the function +#' @param func The function to be applied. +#' @return invisible NULL. +#' @export #' @rdname foreach +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreach(rdd, function(x) { save(x, file=...) }) +#'} #' @aliases foreach,RDD,function-method setMethod("foreach", signature(x = "RDD", func = "function"), @@ -448,7 +687,16 @@ setMethod("foreach", invisible(collect(mapPartitions(x, partition.func))) }) +#' Applies a function to each partition in an RDD, and force evaluation. +#' +#' @export #' @rdname foreach +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) +#'} #' @aliases foreachPartition,RDD,function-method setMethod("foreachPartition", signature(x = "RDD", func = "function"), @@ -456,7 +704,21 @@ setMethod("foreachPartition", invisible(collect(mapPartitions(x, func))) }) +#' Take elements from an RDD. +#' +#' This function takes the first NUM elements in the RDD and +#' returns them in a list. +#' +#' @param x The RDD to take elements from +#' @param num Number of elements to take #' @rdname take +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' take(rdd, 2L) # list(1, 2) +#'} #' @aliases take,RDD,numeric-method setMethod("take", signature(x = "RDD", num = "numeric"), @@ -491,7 +753,21 @@ setMethod("take", }) setClassUnion("missingOrInteger", c("missing", "integer")) +#' Removes the duplicates from RDD. +#' +#' This function returns a new RDD containing the distinct elements in the +#' given RDD. The same as `distinct()' in Spark. +#' +#' @param x The RDD to remove duplicates from. +#' @param numPartitions Number of partitions to create. #' @rdname distinct +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) +#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) +#'} #' @aliases distinct,RDD,missingOrInteger-method setMethod("distinct", signature(x = "RDD", numPartitions = "missingOrInteger"), @@ -507,7 +783,24 @@ setMethod("distinct", resRDD }) +#' Return an RDD that is a sampled subset of the given RDD. +#' +#' The same as `sample()' in Spark. (We rename it due to signature +#' inconsistencies with the `sample()' function in R's base package.) +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value #' @rdname sampleRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) # ensure each num is in its own split +#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#'} #' @aliases sampleRDD,RDD setMethod("sampleRDD", signature(x = "RDD", withReplacement = "logical", @@ -552,7 +845,23 @@ setMethod("sampleRDD", lapplyPartitionsWithIndex(x, samplingFunc) }) +#' Return a list of the elements that are a sampled subset of the given RDD. +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param num Number of elements to return +#' @param seed Randomness seed value #' @rdname takeSample +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:100) +#' # exactly 5 elements sampled, which may not be distinct +#' takeSample(rdd, TRUE, 5L, 1618L) +#' # exactly 5 distinct elements sampled +#' takeSample(rdd, FALSE, 5L, 16181618L) +#'} #' @aliases takeSample,RDD setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", num = "integer", seed = "integer"), @@ -600,7 +909,18 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", sample(samples)[1:total] }) +#' Creates tuples of the elements in this RDD by applying a function. +#' +#' @param x The RDD. +#' @param func The function to be applied. #' @rdname keyBy +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3)) +#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) +#'} #' @aliases keyBy,RDD setMethod("keyBy", signature(x = "RDD", func = "function"), @@ -611,7 +931,24 @@ setMethod("keyBy", lapply(x, apply.func) }) +#' Return a new RDD that has exactly numPartitions partitions. +#' Can increase or decrease the level of parallelism in this RDD. Internally, +#' this uses a shuffle to redistribute data. +#' If you are decreasing the number of partitions in this RDD, consider using +#' coalesce, which can avoid performing a shuffle. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. #' @rdname repartition +#' @seealso coalesce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +#' numPartitions(rdd) # 4 +#' numPartitions(repartition(rdd, 2L)) # 2 +#'} #' @aliases repartition,RDD setMethod("repartition", signature(x = "RDD", numPartitions = "numeric"), @@ -619,7 +956,20 @@ setMethod("repartition", coalesce(x, numPartitions, TRUE) }) +#' Return a new RDD that is reduced into numPartitions partitions. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. #' @rdname coalesce +#' @seealso repartition +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +#' numPartitions(rdd) # 3 +#' numPartitions(coalesce(rdd, 1L)) # 1 +#'} #' @aliases coalesce,RDD setMethod("coalesce", signature(x = "RDD", numPartitions = "numeric"), @@ -647,7 +997,19 @@ setMethod("coalesce", } }) +#' Save this RDD as a SequenceFile of serialized objects. +#' +#' @param x The RDD to save +#' @param path The directory where the file is saved #' @rdname saveAsObjectFile +#' @seealso objectFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") +#'} #' @aliases saveAsObjectFile,RDD setMethod("saveAsObjectFile", signature(x = "RDD", path = "character"), @@ -661,7 +1023,18 @@ setMethod("saveAsObjectFile", invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) }) +#' Save this RDD as a text file, using string representations of elements. +#' +#' @param x The RDD to save +#' @param path The directory where the splits of the text file are saved #' @rdname saveAsTextFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") +#'} #' @aliases saveAsTextFile,RDD setMethod("saveAsTextFile", signature(x = "RDD", path = "character"), @@ -675,7 +1048,21 @@ setMethod("saveAsTextFile", callJMethod(getJRDD(stringRdd, dataSerialization = FALSE), "saveAsTextFile", path)) }) +#' Sort an RDD by the given key function. +#' +#' @param x An RDD to be sorted. +#' @param func A function used to compute the sort key for each element. +#' @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 elements are sorted. #' @rdname sortBy +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(3, 2, 1)) +#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#'} #' @aliases sortBy,RDD,RDD-method setMethod("sortBy", signature(x = "RDD", func = "function"), @@ -717,7 +1104,19 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { reduce(newRdd, reduceFunc) } +#' Returns the first N elements from an RDD in ascending order. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The first N elements from the RDD in ascending order. #' @rdname takeOrdered +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#'} #' @aliases takeOrdered,RDD,RDD-method setMethod("takeOrdered", signature(x = "RDD", num = "integer"), @@ -725,7 +1124,19 @@ setMethod("takeOrdered", takeOrderedElem(x, num) }) +#' Returns the top N elements from an RDD. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The top N elements from the RDD. #' @rdname top +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#'} #' @aliases top,RDD,RDD-method setMethod("top", signature(x = "RDD", num = "integer"), @@ -733,7 +1144,24 @@ setMethod("top", takeOrderedElem(x, num, FALSE) }) +#' Fold an RDD using a given associative function and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using a given associative function and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param op An associative function for the folding operation. +#' @return The folding result. #' @rdname fold +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +#' fold(rdd, 0, "+") # 15 +#'} #' @aliases fold,RDD,RDD-method setMethod("fold", signature(x = "RDD", zeroValue = "ANY", op = "ANY"), @@ -741,7 +1169,29 @@ setMethod("fold", aggregateRDD(x, zeroValue, op, op) }) +#' Aggregate an RDD using the given combine functions and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using given combine functions and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the RDD elements. It may return a different +#' result type from the type of the RDD elements. +#' @param combOp A function to aggregate results of seqOp. +#' @return The aggregation result. #' @rdname aggregateRDD +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' 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]]) } +#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#'} #' @aliases aggregateRDD,RDD,RDD-method setMethod("aggregateRDD", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), @@ -755,7 +1205,23 @@ setMethod("aggregateRDD", Reduce(combOp, partitionList, zeroValue) }) +#' Pipes elements to a forked external process. +#' +#' The same as 'pipe()' in Spark. +#' +#' @param x The RDD whose elements are piped to the forked external process. +#' @param command The command to fork an external process. +#' @param env A named list to set environment variables of the external process. +#' @return A new RDD created by piping all elements to a forked external process. #' @rdname pipeRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' collect(pipeRDD(rdd, "more") +#' Output: c("1", "2", ..., "10") +#'} #' @aliases pipeRDD,RDD,character-method setMethod("pipeRDD", signature(x = "RDD", command = "character"), @@ -771,7 +1237,18 @@ setMethod("pipeRDD", lapplyPartition(x, func) }) +# TODO: Consider caching the name in the RDD's environment +#' Return an RDD's name. +#' +#' @param x The RDD whose name is returned. #' @rdname name +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' name(rdd) # NULL (if not set before) +#'} #' @aliases name,RDD setMethod("name", signature(x = "RDD"), @@ -779,7 +1256,20 @@ setMethod("name", callJMethod(getJRDD(x), "name") }) +#' Set an RDD's name. +#' +#' @param x The RDD whose name is to be set. +#' @param name The RDD name to be set. +#' @return a new RDD renamed. #' @rdname setName +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' setName(rdd, "myRDD") +#' name(rdd) # "myRDD" +#'} #' @aliases setName,RDD setMethod("setName", signature(x = "RDD", name = "character"), @@ -788,7 +1278,25 @@ setMethod("setName", x }) +#' Zip an RDD with generated unique Long IDs. +#' +#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +#' n is the number of partitions. So there may exist gaps, but this +#' method won't trigger a spark job, which is different from +#' zipWithIndex. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. #' @rdname zipWithUniqueId +#' @seealso zipWithIndex +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithUniqueId(rdd)) +#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#'} #' @aliases zipWithUniqueId,RDD setMethod("zipWithUniqueId", signature(x = "RDD"), @@ -808,7 +1316,28 @@ setMethod("zipWithUniqueId", lapplyPartitionsWithIndex(x, partitionFunc) }) +#' Zip an RDD with its element indices. +#' +#' The ordering is first based on the partition index and then the +#' ordering of items within each partition. So the first item in +#' the first partition gets index 0, and the last item in the last +#' partition receives the largest index. +#' +#' This method needs to trigger a Spark job when this RDD contains +#' more than one partition. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. #' @rdname zipWithIndex +#' @seealso zipWithUniqueId +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithIndex(rdd)) +#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#'} #' @aliases zipWithIndex,RDD setMethod("zipWithIndex", signature(x = "RDD"), @@ -845,7 +1374,21 @@ setMethod("zipWithIndex", ############ Binary Functions ############# +#' Return the union RDD of two RDDs. +#' The same as union() in Spark. +#' +#' @param x An RDD. +#' @param y An RDD. +#' @return a new RDD created by performing the simple union (witout removing +#' duplicates) of two input RDDs. #' @rdname unionRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 +#'} #' @aliases unionRDD,RDD,RDD-method setMethod("unionRDD", signature(x = "RDD", y = "RDD"), diff --git a/pkg/R/broadcast.R b/pkg/R/broadcast.R index 7f9e1c179ac11..e589dfb668b06 100644 --- a/pkg/R/broadcast.R +++ b/pkg/R/broadcast.R @@ -28,7 +28,13 @@ Broadcast <- function(id, value, jBroadcastRef, 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 +#' @export #' @aliases value,Broadcast-method setMethod("value", signature(bcast = "Broadcast"), diff --git a/pkg/R/generics.R b/pkg/R/generics.R new file mode 100644 index 0000000000000..85bdd088d2c4e --- /dev/null +++ b/pkg/R/generics.R @@ -0,0 +1,331 @@ +############ RDD Actions and Transformations ############ + +# The jrdd accessor function. +setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) + +#' @rdname cache-methods +#' @export +setGeneric("cache", function(x) { standardGeneric("cache") }) + +#' @rdname persist +#' @export +setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) + +#' @rdname unpersist-methods +#' @export +setGeneric("unpersist", function(x) { standardGeneric("unpersist") }) + +#' @rdname checkpoint-methods +#' @export +setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) + +#' @rdname numPartitions +#' @export +setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) + +#' @rdname collect-methods +#' @export +setGeneric("collect", function(x, ...) { standardGeneric("collect") }) + +#' @rdname collect-methods +#' @export +setGeneric("collectPartition", + function(x, partitionId) { + standardGeneric("collectPartition") + }) + +#' @rdname collect-methods +#' @export +setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) + +#' @rdname count +#' @export +setGeneric("count", function(x) { standardGeneric("count") }) + +#' @rdname countByValue +#' @export +setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) + +#' @rdname lapply +#' @export +setGeneric("map", function(X, FUN) { + standardGeneric("map") }) + + +#' @rdname flatMap +#' @export +setGeneric("flatMap", function(X, FUN) { + standardGeneric("flatMap") }) + +#' @rdname lapplyPartition +#' @export +setGeneric("lapplyPartition", function(X, FUN) { + standardGeneric("lapplyPartition") }) + +#' @rdname lapplyPartition +#' @export +setGeneric("mapPartitions", function(X, FUN) { + standardGeneric("mapPartitions") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { + standardGeneric("lapplyPartitionsWithIndex") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("mapPartitionsWithIndex", function(X, FUN) { + standardGeneric("mapPartitionsWithIndex") }) + +#' @rdname filterRDD +#' @export +setGeneric("filterRDD", + function(x, f) { standardGeneric("filterRDD") }) + +#' @rdname reduce +#' @export +setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) + +#' @rdname maximum +#' @export +setGeneric("maximum", function(x) { standardGeneric("maximum") }) + +#' @rdname minimum +#' @export +setGeneric("minimum", function(x) { standardGeneric("minimum") }) + +#' @rdname foreach +#' @export +setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) + +#' @rdname foreach +#' @export +setGeneric("foreachPartition", + function(x, func) { standardGeneric("foreachPartition") }) + +#' @rdname take +#' @export +setGeneric("take", function(x, num) { standardGeneric("take") }) + +#' @rdname distinct +#' @export +setGeneric("distinct", + function(x, numPartitions) { standardGeneric("distinct") }) + +#' @rdname sampleRDD +#' @export +setGeneric("sampleRDD", + function(x, withReplacement, fraction, seed) { + standardGeneric("sampleRDD") + }) + +#' @rdname takeSample +#' @export +setGeneric("takeSample", + function(x, withReplacement, num, seed) { + standardGeneric("takeSample") + }) + +#' @rdname keyBy +#' @export +setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) + +#' @rdname repartition +#' @seealso coalesce +#' @export +setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) + +#' @rdname coalesce +#' @seealso repartition +#' @export +setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) + +#' @rdname saveAsObjectFile +#' @seealso objectFile +#' @export +setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) + +#' @rdname saveAsTextFile +#' @export +setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) + +#' @rdname sortBy +#' @export +setGeneric("sortBy", function(x, + func, + ascending = TRUE, + numPartitions = 1L) { + standardGeneric("sortBy") +}) + +#' @rdname takeOrdered +#' @export +setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) + +#' @rdname top +#' @export +setGeneric("top", function(x, num) { standardGeneric("top") }) + +#' @rdname fold +#' @seealso reduce +#' @export +setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) + +#' @rdname aggregateRDD +#' @seealso reduce +#' @export +setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) + +#' @rdname pipeRDD +#' @export +setGeneric("pipeRDD", function(x, command, env = list()) { + standardGeneric("pipeRDD") +}) + +#' @rdname name +#' @export +setGeneric("name", function(x) { standardGeneric("name") }) + +#' @rdname setName +#' @export +setGeneric("setName", function(x, name) { standardGeneric("setName") }) + +#' @rdname zipWithUniqueId +#' @seealso zipWithIndex +#' @export +setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) + +#' @rdname zipWithIndex +#' @seealso zipWithUniqueId +#' @export +setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) + + +############ Binary Functions ############# + + +#' @rdname unionRDD +#' @export +setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) + +#' @rdname lookup +#' @export +setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) + +#' @rdname countByKey +#' @export +setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) + +#' @rdname keys +#' @export +setGeneric("keys", function(x) { standardGeneric("keys") }) + +#' @rdname values +#' @export +setGeneric("values", function(x) { standardGeneric("values") }) + +#' @rdname mapValues +#' @export +setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) + +#' @rdname flatMapValues +#' @export +setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) + + +############ Shuffle Functions ############ + + +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", + function(x, numPartitions, ...) { + standardGeneric("partitionBy") + }) + +#' @rdname groupByKey +#' @seealso reduceByKey +#' @export +setGeneric("groupByKey", + function(x, numPartitions) { + standardGeneric("groupByKey") + }) + +#' @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 combineByKey +#' @seealso groupByKey, reduceByKey +#' @export +setGeneric("combineByKey", + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { + standardGeneric("combineByKey") + }) + +#' @rdname aggregateByKey +#' @seealso foldByKey, combineByKey +#' @export +setGeneric("aggregateByKey", + function(x, zeroValue, seqOp, combOp, numPartitions) { + standardGeneric("aggregateByKey") + }) + +#' @rdname foldByKey +#' @seealso aggregateByKey, combineByKey +#' @export +setGeneric("foldByKey", + function(x, zeroValue, func, numPartitions) { + standardGeneric("foldByKey") + }) + +#' @rdname join-methods +#' @export +setGeneric("join", function(x, y, numPartitions) { standardGeneric("join") }) + +#' @rdname join-methods +#' @export +setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) + +#' @rdname join-methods +#' @export +setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) + +#' @rdname join-methods +#' @export +setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) + +#' @rdname cogroup +#' @export +setGeneric("cogroup", + function(..., numPartitions) { standardGeneric("cogroup") }, + signature = "...") + +#' @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") }) + diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R index 665270385b583..dd33003aa191b 100644 --- a/pkg/R/pairRDD.R +++ b/pkg/R/pairRDD.R @@ -3,7 +3,23 @@ ############ 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 #' @rdname lookup +#' @export +#' @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) +#'} #' @aliases lookup,RDD-method setMethod("lookup", signature(x = "RDD", key = "ANY"), @@ -16,7 +32,21 @@ setMethod("lookup", 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. #' @rdname countByKey +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +#' countByKey(rdd) # ("a", 2L), ("b", 1L) +#'} #' @aliases countByKey,RDD-method setMethod("countByKey", signature(x = "RDD"), @@ -25,7 +55,17 @@ setMethod("countByKey", countByValue(keys) }) +#' Return an RDD with the keys of each tuple. +#' +#' @param x The RDD from which the keys of each tuple is returned. #' @rdname keys +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(keys(rdd)) # list(1, 3) +#'} #' @aliases keys,RDD setMethod("keys", signature(x = "RDD"), @@ -36,7 +76,17 @@ setMethod("keys", 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. #' @rdname values +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(values(rdd)) # list(2, 4) +#'} #' @aliases values,RDD setMethod("values", signature(x = "RDD"), @@ -47,7 +97,23 @@ setMethod("values", 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. #' @rdname mapValues +#' @export +#' @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), ...) +#'} #' @aliases mapValues,RDD,function-method setMethod("mapValues", signature(X = "RDD", FUN = "function"), @@ -58,7 +124,23 @@ setMethod("mapValues", 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. #' @rdname flatMapValues +#' @export +#' @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)) +#'} #' @aliases flatMapValues,RDD,function-method setMethod("flatMapValues", signature(X = "RDD", FUN = "function"), @@ -72,7 +154,30 @@ setMethod("flatMapValues", ############ 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. #' @rdname partitionBy +#' @export +#' @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) +#'} #' @aliases partitionBy,RDD,integer-method setMethod("partitionBy", signature(x = "RDD", numPartitions = "integer"), @@ -119,7 +224,27 @@ setMethod("partitionBy", RDD(r, serialized = TRUE) }) +#' 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 #' @rdname groupByKey +#' @export +#' @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)) +#'} #' @aliases groupByKey,RDD,integer-method setMethod("groupByKey", signature(x = "RDD", numPartitions = "integer"), @@ -158,7 +283,29 @@ setMethod("groupByKey", 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 #' @rdname reduceByKey +#' @seealso groupByKey +#' @export +#' @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) +#'} #' @aliases reduceByKey,RDD,integer-method setMethod("reduceByKey", signature(x = "RDD", combineFunc = "ANY", numPartitions = "integer"), @@ -179,7 +326,27 @@ setMethod("reduceByKey", 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 #' @rdname reduceByKeyLocally +#' @seealso reduceByKey +#' @export +#' @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)) +#'} #' @aliases reduceByKeyLocally,RDD,integer-method setMethod("reduceByKeyLocally", signature(x = "RDD", combineFunc = "ANY"), @@ -212,7 +379,41 @@ setMethod("reduceByKeyLocally", 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 +#' #' @rdname combineByKey +#' @seealso groupByKey, reduceByKey +#' @export +#' @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) +#'} #' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method setMethod("combineByKey", signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY", @@ -245,7 +446,36 @@ setMethod("combineByKey", 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. #' @rdname aggregateByKey +#' @seealso foldByKey, combineByKey +#' @export +#' @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))) +#'} #' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method setMethod("aggregateByKey", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", @@ -258,7 +488,26 @@ setMethod("aggregateByKey", 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. #' @rdname foldByKey +#' @seealso aggregateByKey, combineByKey +#' @export +#' @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)) +#'} #' @aliases foldByKey,RDD,ANY,ANY,integer-method setMethod("foldByKey", signature(x = "RDD", zeroValue = "ANY", @@ -270,7 +519,28 @@ setMethod("foldByKey", ############ 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. #' @rdname join-methods +#' @export +#' @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)) +#'} #' @aliases join,RDD,RDD-method setMethod("join", signature(x = "RDD", y = "RDD", numPartitions = "integer"), @@ -285,7 +555,30 @@ setMethod("join", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), 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. #' @rdname join-methods +#' @export +#' @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))) +#'} #' @aliases leftOuterJoin,RDD,RDD-method setMethod("leftOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "integer"), @@ -300,7 +593,30 @@ setMethod("leftOuterJoin", 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. #' @rdname join-methods +#' @export +#' @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))) +#'} #' @aliases rightOuterJoin,RDD,RDD-method setMethod("rightOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "integer"), @@ -315,9 +631,34 @@ setMethod("rightOuterJoin", 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. #' @rdname join-methods +#' @export +#' @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)), +#'} #' @aliases fullOuterJoin,RDD,RDD-method - setMethod("fullOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "integer"), function(x, y, numPartitions) { @@ -331,7 +672,23 @@ setMethod("fullOuterJoin", 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. #' @rdname cogroup +#' @export +#' @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())) +#'} #' @aliases cogroup,RDD-method setMethod("cogroup", "RDD", @@ -372,7 +729,20 @@ setMethod("cogroup", 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. #' @rdname sortByKey +#' @export +#' @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)) +#'} #' @aliases sortByKey,RDD,RDD-method setMethod("sortByKey", signature(x = "RDD"), From 8f8813faee633c7b51f0cf89d845ba2ae248df8b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 9 Mar 2015 13:57:43 -0700 Subject: [PATCH 113/121] switch back to use parallel --- pkg/inst/worker/daemon.R | 66 +++++++++++++++++++++++++--------------- 1 file changed, 42 insertions(+), 24 deletions(-) diff --git a/pkg/inst/worker/daemon.R b/pkg/inst/worker/daemon.R index 381b3c42d7041..5af55aa4b255c 100644 --- a/pkg/inst/worker/daemon.R +++ b/pkg/inst/worker/daemon.R @@ -1,11 +1,5 @@ # Worker daemon -# try to load `fork`, or install -FORK_URI <- "http://cran.r-project.org/src/contrib/Archive/fork/fork_1.2.4.tar.gz" -tryCatch(library(fork), error = function(err) { - install.packages(FORK_URI, repos = NULL, type = "source") -}) - rLibDir <- Sys.getenv("SPARKR_RLIBDIR") script <- paste(rLibDir, "SparkR/worker/worker.R", sep = "/") @@ -14,26 +8,50 @@ script <- paste(rLibDir, "SparkR/worker/worker.R", sep = "/") suppressPackageStartupMessages(library(SparkR)) port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) -inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") +inputCon <- socketConnection(port = port, open = "rb", blocking = TRUE, timeout = 3600) + +# Read from connection, retrying to read exactly 'size' bytes +readBinFull <- function(con, size) { + # readBin() could be interruptted by signal, we have no way to tell + # if it's interruptted or the socket is closed, so we retry at most + # 20 times, to avoid the deadloop if the socket is closed + c <- 1 + data <- readBin(con, raw(), size) + while (length(data) < size && c < 20) { + extra <- readBin(con, raw(), size - length(data)) + data <- c(data, extra) + c <- c + 1 + } + data +} + +# Utility function to read the port with retry +# Returns -1 if the socket was closed +readPort <- function(con) { + data <- readBinFull(con, 4L) + if (length(data) != 4) { + -1 + } else { + rc <- rawConnection(data) + ret <- SparkR:::readInt(rc) + close(rc) + ret + } +} while (TRUE) { - ready <- socketSelect(list(inputCon), timeout = 1) - if (ready) { - inport <- SparkR:::readInt(inputCon) - if (length(inport) != 1) { - quit(save="no") - } - p <- fork::fork(NULL) - if (p == 0) { - close(inputCon) - Sys.setenv(SPARKR_WORKER_PORT = inport) - source(script) - fork::exit(0) - } + port <- readPort(inputCon) + if (port < 0) { + cat("quitting daemon\n") + quit(save = "no") } - # cleanup all the child process - status <- fork::wait(0, nohang = TRUE) - while (status[1] > 0) { - status <- fork::wait(0, nohang = TRUE) + 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) } } From 411b751ebda76b43dab7c2f80cdd4cb6c27fc8db Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 9 Mar 2015 15:13:50 -0700 Subject: [PATCH 114/121] make RStudio happy --- pkg/R/sparkR.R | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pkg/R/sparkR.R b/pkg/R/sparkR.R index d92b908ad5502..44ee3dbec3836 100644 --- a/pkg/R/sparkR.R +++ b/pkg/R/sparkR.R @@ -193,5 +193,10 @@ sparkR.init <- function( envir = .sparkREnv ) - get(".sparkRjsc", 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 } From ecdfda10f79f92fb14937a17797d7ef6aed4a41f Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 9 Mar 2015 22:22:02 -0400 Subject: [PATCH 115/121] Remove duplication. --- pkg/R/RDD.R | 122 +++++++++++++++------------------------------- pkg/R/broadcast.R | 1 - pkg/R/pairRDD.R | 57 ++++++++-------------- 3 files changed, 58 insertions(+), 122 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 8a98d408709ab..9134258eca2fd 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -164,14 +164,13 @@ setValidity("RDD", #' Persist this RDD with the default storage level (MEMORY_ONLY). #' #' @param x The RDD to cache -#' @rdname cache-methods -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10, 2L) #' cache(rdd) #'} +#' @rdname cache-methods #' @aliases cache,RDD-method setMethod("cache", signature(x = "RDD"), @@ -189,14 +188,13 @@ setMethod("cache", #' #' @param x The RDD to persist #' @param newLevel The new storage level to be assigned -#' @rdname persist -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10, 2L) #' persist(rdd, "MEMORY_AND_DISK") #'} +#' @rdname persist #' @aliases persist,RDD-method setMethod("persist", signature(x = "RDD", newLevel = "character"), @@ -236,8 +234,6 @@ setMethod("persist", #' disk. #' #' @param rdd The RDD to unpersist -#' @rdname unpersist-methods -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -245,6 +241,7 @@ setMethod("persist", #' cache(rdd) # rdd@@env$isCached == TRUE #' unpersist(rdd) # rdd@@env$isCached == FALSE #'} +#' @rdname unpersist-methods #' @aliases unpersist,RDD-method setMethod("unpersist", signature(x = "RDD"), @@ -263,8 +260,6 @@ setMethod("unpersist", #' persisted in memory, otherwise saving it on a file will require recomputation. #' #' @param rdd The RDD to checkpoint -#' @rdname checkpoint-methods -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -272,6 +267,7 @@ setMethod("unpersist", #' rdd <- parallelize(sc, 1:10, 2L) #' checkpoint(rdd) #'} +#' @rdname checkpoint-methods #' @aliases checkpoint,RDD-method setMethod("checkpoint", signature(x = "RDD"), @@ -286,14 +282,13 @@ setMethod("checkpoint", #' #' @param x A RDD. #' @return the number of partitions of rdd as an integer. -#' @rdname numPartitions -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10, 2L) #' numPartitions(rdd) # 2L #'} +#' @rdname numPartitions #' @aliases numPartitions,RDD-method setMethod("numPartitions", signature(x = "RDD"), @@ -312,8 +307,6 @@ setMethod("numPartitions", #' @param ... Other optional arguments to collect #' @param flatten FALSE if the list should not flattened #' @return a list containing elements in the RDD -#' @rdname collect-methods -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -321,6 +314,7 @@ setMethod("numPartitions", #' collect(rdd) # list from 1 to 10 #' collectPartition(rdd, 0L) # list from 1 to 5 #'} +#' @rdname collect-methods #' @aliases collect,RDD-method setMethod("collect", signature(x = "RDD"), @@ -330,12 +324,11 @@ setMethod("collect", convertJListToRList(collected, flatten) }) -#' @rdname collect-methods -#' @export #' @description #' \code{collectPartition} returns a list that contains all of the elements #' in the specified partition of the RDD. #' @param partitionId the partition to collect (starts from 0) +#' @rdname collect-methods #' @aliases collectPartition,integer,RDD-method setMethod("collectPartition", signature(x = "RDD", partitionId = "integer"), @@ -348,8 +341,6 @@ setMethod("collectPartition", convertJListToRList(jList, flatten = TRUE) }) -#' @rdname collect-methods -#' @export #' @description #' \code{collectAsMap} returns a named list as a map that contains all of the elements #' in a key-value pair RDD. @@ -359,6 +350,7 @@ setMethod("collectPartition", #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) #' collectAsMap(rdd) # list(`1` = 2, `3` = 4) #'} +#' @rdname collect-methods #' @aliases collectAsMap,RDD-method setMethod("collectAsMap", signature(x = "RDD"), @@ -373,8 +365,6 @@ setMethod("collectAsMap", #' #' @param x The RDD to count #' @return number of elements in the RDD. -#' @rdname count -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -382,6 +372,7 @@ setMethod("collectAsMap", #' count(rdd) # 10 #' length(rdd) # Same as count #'} +#' @rdname count #' @aliases count,RDD-method setMethod("count", signature(x = "RDD"), @@ -411,14 +402,13 @@ setMethod("length", #' @param x The RDD to count #' @return list of (value, count) pairs, where count is number of each unique #' value in rdd. -#' @rdname countByValue -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, c(1,2,3,2,1)) #' countByValue(rdd) # (1,2L), (2,2L), (3,1L) #'} +#' @rdname countByValue #' @aliases countByValue,RDD-method setMethod("countByValue", signature(x = "RDD"), @@ -437,7 +427,6 @@ setMethod("countByValue", #' @return a new RDD created by the transformation. #' @rdname lapply #' @aliases lapply -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -455,7 +444,6 @@ setMethod("lapply", }) #' @rdname lapply -#' @export #' @aliases map,RDD,function-method setMethod("map", signature(X = "RDD", FUN = "function"), @@ -471,8 +459,6 @@ setMethod("map", #' @param X The RDD to apply the transformation. #' @param FUN the transformation to apply on each element #' @return a new RDD created by the transformation. -#' @rdname flatMap -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -480,6 +466,7 @@ setMethod("map", #' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) #' collect(multiplyByTwo) # 2,20,4,40,6,60... #'} +#' @rdname flatMap #' @aliases flatMap,RDD,function-method setMethod("flatMap", signature(X = "RDD", FUN = "function"), @@ -500,8 +487,6 @@ setMethod("flatMap", #' @param X The RDD to apply the transformation. #' @param FUN the transformation to apply on each partition. #' @return a new RDD created by the transformation. -#' @rdname lapplyPartition -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -509,6 +494,7 @@ setMethod("flatMap", #' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) #' collect(partitionSum) # 15, 40 #'} +#' @rdname lapplyPartition #' @aliases lapplyPartition,RDD,function-method setMethod("lapplyPartition", signature(X = "RDD", FUN = "function"), @@ -519,7 +505,6 @@ setMethod("lapplyPartition", #' mapPartitions is the same as lapplyPartition. #' #' @rdname lapplyPartition -#' @export #' @aliases mapPartitions,RDD,function-method setMethod("mapPartitions", signature(X = "RDD", FUN = "function"), @@ -534,8 +519,6 @@ setMethod("mapPartitions", #' @param FUN the transformation to apply on each partition; takes the partition #' index and a list of elements in the particular partition. #' @return a new RDD created by the transformation. -#' @rdname lapplyPartitionsWithIndex -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -544,6 +527,7 @@ setMethod("mapPartitions", #' split * Reduce("+", part) }) #' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 #'} +#' @rdname lapplyPartitionsWithIndex #' @aliases lapplyPartitionsWithIndex,RDD,function-method setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), @@ -556,7 +540,6 @@ setMethod("lapplyPartitionsWithIndex", }) #' @rdname lapplyPartitionsWithIndex -#' @export #' @aliases mapPartitionsWithIndex,RDD,function-method setMethod("mapPartitionsWithIndex", signature(X = "RDD", FUN = "function"), @@ -570,14 +553,13 @@ setMethod("mapPartitionsWithIndex", #' #' @param x The RDD to be filtered. #' @param f A unary predicate function. -#' @rdname filterRDD -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) #'} +#' @rdname filterRDD #' @aliases filterRDD,RDD,function-method setMethod("filterRDD", signature(x = "RDD", f = "function"), @@ -589,7 +571,6 @@ setMethod("filterRDD", }) #' @rdname filterRDD -#' @export #' @aliases Filter setMethod("Filter", signature(f = "function", x = "RDD"), @@ -605,14 +586,13 @@ setMethod("Filter", #' @param rdd The RDD to reduce #' @param func Commutative and associative function to apply on elements #' of the RDD. -#' @export -#' @rdname reduce #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' reduce(rdd, "+") # 55 #'} +#' @rdname reduce #' @aliases reduce,RDD,ANY-method setMethod("reduce", signature(x = "RDD", func = "ANY"), @@ -630,14 +610,13 @@ setMethod("reduce", #' Get the maximum element of an RDD. #' #' @param x The RDD to get the maximum element from -#' @export -#' @rdname maximum #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' maximum(rdd) # 10 #'} +#' @rdname maximum #' @aliases maximum,RDD setMethod("maximum", signature(x = "RDD"), @@ -648,14 +627,13 @@ setMethod("maximum", #' Get the minimum element of an RDD. #' #' @param x The RDD to get the minimum element from -#' @export -#' @rdname minimum #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' minimum(rdd) # 1 #'} +#' @rdname minimum #' @aliases minimum,RDD setMethod("minimum", signature(x = "RDD"), @@ -668,14 +646,13 @@ setMethod("minimum", #' @param x The RDD to apply the function #' @param func The function to be applied. #' @return invisible NULL. -#' @export -#' @rdname foreach #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' foreach(rdd, function(x) { save(x, file=...) }) #'} +#' @rdname foreach #' @aliases foreach,RDD,function-method setMethod("foreach", signature(x = "RDD", func = "function"), @@ -689,14 +666,13 @@ setMethod("foreach", #' Applies a function to each partition in an RDD, and force evaluation. #' -#' @export -#' @rdname foreach #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) #'} +#' @rdname foreach #' @aliases foreachPartition,RDD,function-method setMethod("foreachPartition", signature(x = "RDD", func = "function"), @@ -711,14 +687,13 @@ setMethod("foreachPartition", #' #' @param x The RDD to take elements from #' @param num Number of elements to take -#' @rdname take -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' take(rdd, 2L) # list(1, 2) #'} +#' @rdname take #' @aliases take,RDD,numeric-method setMethod("take", signature(x = "RDD", num = "numeric"), @@ -760,14 +735,13 @@ setClassUnion("missingOrInteger", c("missing", "integer")) #' #' @param x The RDD to remove duplicates from. #' @param numPartitions Number of partitions to create. -#' @rdname distinct -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, c(1,2,2,3,3,3)) #' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) #'} +#' @rdname distinct #' @aliases distinct,RDD,missingOrInteger-method setMethod("distinct", signature(x = "RDD", numPartitions = "missingOrInteger"), @@ -792,8 +766,6 @@ setMethod("distinct", #' @param withReplacement Sampling with replacement or not #' @param fraction The (rough) sample target fraction #' @param seed Randomness seed value -#' @rdname sampleRDD -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -801,6 +773,7 @@ setMethod("distinct", #' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements #' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates #'} +#' @rdname sampleRDD #' @aliases sampleRDD,RDD setMethod("sampleRDD", signature(x = "RDD", withReplacement = "logical", @@ -851,8 +824,6 @@ setMethod("sampleRDD", #' @param withReplacement Sampling with replacement or not #' @param num Number of elements to return #' @param seed Randomness seed value -#' @rdname takeSample -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -862,6 +833,7 @@ setMethod("sampleRDD", #' # exactly 5 distinct elements sampled #' takeSample(rdd, FALSE, 5L, 16181618L) #'} +#' @rdname takeSample #' @aliases takeSample,RDD setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", num = "integer", seed = "integer"), @@ -913,14 +885,13 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", #' #' @param x The RDD. #' @param func The function to be applied. -#' @rdname keyBy -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(1, 2, 3)) #' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) #'} +#' @rdname keyBy #' @aliases keyBy,RDD setMethod("keyBy", signature(x = "RDD", func = "function"), @@ -939,9 +910,7 @@ setMethod("keyBy", #' #' @param x The RDD. #' @param numPartitions Number of partitions to create. -#' @rdname repartition #' @seealso coalesce -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -949,6 +918,7 @@ setMethod("keyBy", #' numPartitions(rdd) # 4 #' numPartitions(repartition(rdd, 2L)) # 2 #'} +#' @rdname repartition #' @aliases repartition,RDD setMethod("repartition", signature(x = "RDD", numPartitions = "numeric"), @@ -960,9 +930,7 @@ setMethod("repartition", #' #' @param x The RDD. #' @param numPartitions Number of partitions to create. -#' @rdname coalesce #' @seealso repartition -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -970,6 +938,7 @@ setMethod("repartition", #' numPartitions(rdd) # 3 #' numPartitions(coalesce(rdd, 1L)) # 1 #'} +#' @rdname coalesce #' @aliases coalesce,RDD setMethod("coalesce", signature(x = "RDD", numPartitions = "numeric"), @@ -1001,15 +970,14 @@ setMethod("coalesce", #' #' @param x The RDD to save #' @param path The directory where the file is saved -#' @rdname saveAsObjectFile #' @seealso objectFile -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:3) #' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") #'} +#' @rdname saveAsObjectFile #' @aliases saveAsObjectFile,RDD setMethod("saveAsObjectFile", signature(x = "RDD", path = "character"), @@ -1027,14 +995,13 @@ setMethod("saveAsObjectFile", #' #' @param x The RDD to save #' @param path The directory where the splits of the text file are saved -#' @rdname saveAsTextFile -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:3) #' saveAsTextFile(rdd, "/tmp/sparkR-tmp") #'} +#' @rdname saveAsTextFile #' @aliases saveAsTextFile,RDD setMethod("saveAsTextFile", signature(x = "RDD", path = "character"), @@ -1055,14 +1022,13 @@ setMethod("saveAsTextFile", #' @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 elements are sorted. -#' @rdname sortBy -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(3, 2, 1)) #' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) #'} +#' @rdname sortBy #' @aliases sortBy,RDD,RDD-method setMethod("sortBy", signature(x = "RDD", func = "function"), @@ -1109,14 +1075,13 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { #' @param x An RDD. #' @param num Number of elements to return. #' @return The first N elements from the RDD in ascending order. -#' @rdname takeOrdered -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) #' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) #'} +#' @rdname takeOrdered #' @aliases takeOrdered,RDD,RDD-method setMethod("takeOrdered", signature(x = "RDD", num = "integer"), @@ -1129,14 +1094,13 @@ setMethod("takeOrdered", #' @param x An RDD. #' @param num Number of elements to return. #' @return The top N elements from the RDD. -#' @rdname top -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) #' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) #'} +#' @rdname top #' @aliases top,RDD,RDD-method setMethod("top", signature(x = "RDD", num = "integer"), @@ -1153,15 +1117,14 @@ setMethod("top", #' @param zeroValue A neutral "zero value". #' @param op An associative function for the folding operation. #' @return The folding result. -#' @rdname fold #' @seealso reduce -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) #' fold(rdd, 0, "+") # 15 #'} +#' @rdname fold #' @aliases fold,RDD,RDD-method setMethod("fold", signature(x = "RDD", zeroValue = "ANY", op = "ANY"), @@ -1180,9 +1143,7 @@ setMethod("fold", #' result type from the type of the RDD elements. #' @param combOp A function to aggregate results of seqOp. #' @return The aggregation result. -#' @rdname aggregateRDD #' @seealso reduce -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -1192,6 +1153,7 @@ setMethod("fold", #' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } #' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) #'} +#' @rdname aggregateRDD #' @aliases aggregateRDD,RDD,RDD-method setMethod("aggregateRDD", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), @@ -1213,8 +1175,6 @@ setMethod("aggregateRDD", #' @param command The command to fork an external process. #' @param env A named list to set environment variables of the external process. #' @return A new RDD created by piping all elements to a forked external process. -#' @rdname pipeRDD -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -1222,6 +1182,7 @@ setMethod("aggregateRDD", #' collect(pipeRDD(rdd, "more") #' Output: c("1", "2", ..., "10") #'} +#' @rdname pipeRDD #' @aliases pipeRDD,RDD,character-method setMethod("pipeRDD", signature(x = "RDD", command = "character"), @@ -1241,14 +1202,13 @@ setMethod("pipeRDD", #' Return an RDD's name. #' #' @param x The RDD whose name is returned. -#' @rdname name -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(1,2,3)) #' name(rdd) # NULL (if not set before) #'} +#' @rdname name #' @aliases name,RDD setMethod("name", signature(x = "RDD"), @@ -1261,8 +1221,6 @@ setMethod("name", #' @param x The RDD whose name is to be set. #' @param name The RDD name to be set. #' @return a new RDD renamed. -#' @rdname setName -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -1270,6 +1228,7 @@ setMethod("name", #' setName(rdd, "myRDD") #' name(rdd) # "myRDD" #'} +#' @rdname setName #' @aliases setName,RDD setMethod("setName", signature(x = "RDD", name = "character"), @@ -1287,9 +1246,7 @@ setMethod("setName", #' #' @param x An RDD to be zipped. #' @return An RDD with zipped items. -#' @rdname zipWithUniqueId #' @seealso zipWithIndex -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -1297,6 +1254,7 @@ setMethod("setName", #' collect(zipWithUniqueId(rdd)) #' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) #'} +#' @rdname zipWithUniqueId #' @aliases zipWithUniqueId,RDD setMethod("zipWithUniqueId", signature(x = "RDD"), @@ -1328,9 +1286,7 @@ setMethod("zipWithUniqueId", #' #' @param x An RDD to be zipped. #' @return An RDD with zipped items. -#' @rdname zipWithIndex #' @seealso zipWithUniqueId -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -1338,6 +1294,7 @@ setMethod("zipWithUniqueId", #' collect(zipWithIndex(rdd)) #' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) #'} +#' @rdname zipWithIndex #' @aliases zipWithIndex,RDD setMethod("zipWithIndex", signature(x = "RDD"), @@ -1381,14 +1338,13 @@ setMethod("zipWithIndex", #' @param y An RDD. #' @return a new RDD created by performing the simple union (witout removing #' duplicates) of two input RDDs. -#' @rdname unionRDD -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:3) #' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 #'} +#' @rdname unionRDD #' @aliases unionRDD,RDD,RDD-method setMethod("unionRDD", signature(x = "RDD", y = "RDD"), diff --git a/pkg/R/broadcast.R b/pkg/R/broadcast.R index e589dfb668b06..e359d81b151d4 100644 --- a/pkg/R/broadcast.R +++ b/pkg/R/broadcast.R @@ -34,7 +34,6 @@ Broadcast <- function(id, value, jBroadcastRef, objName) { #' #' @param bcast The broadcast variable to get #' @rdname broadcast -#' @export #' @aliases value,Broadcast-method setMethod("value", signature(bcast = "Broadcast"), diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R index dd33003aa191b..722da9238f734 100644 --- a/pkg/R/pairRDD.R +++ b/pkg/R/pairRDD.R @@ -11,8 +11,6 @@ #' @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 -#' @rdname lookup -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -20,6 +18,7 @@ #' rdd <- parallelize(sc, pairs) #' lookup(rdd, 1) # list(1, 3) #'} +#' @rdname lookup #' @aliases lookup,RDD-method setMethod("lookup", signature(x = "RDD", key = "ANY"), @@ -39,14 +38,13 @@ setMethod("lookup", #' #' @param x The RDD to count keys. #' @return list of (key, count) pairs, where count is number of each key in rdd. -#' @rdname countByKey -#' @export #' @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"), @@ -58,14 +56,13 @@ setMethod("countByKey", #' Return an RDD with the keys of each tuple. #' #' @param x The RDD from which the keys of each tuple is returned. -#' @rdname keys -#' @export #' @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"), @@ -79,14 +76,13 @@ setMethod("keys", #' Return an RDD with the values of each tuple. #' #' @param x The RDD from which the values of each tuple is returned. -#' @rdname values -#' @export #' @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"), @@ -104,8 +100,6 @@ setMethod("values", #' @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. -#' @rdname mapValues -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -114,6 +108,7 @@ setMethod("values", #' 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"), @@ -132,8 +127,6 @@ setMethod("mapValues", #' @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. -#' @rdname flatMapValues -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -141,6 +134,7 @@ setMethod("mapValues", #' 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"), @@ -168,8 +162,6 @@ setMethod("flatMapValues", #' @param partitionFunc The partition function to use. Uses a default hashCode #' function if not provided #' @return An RDD partitioned using the specified partitioner. -#' @rdname partitionBy -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -178,6 +170,7 @@ setMethod("flatMapValues", #' 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"), @@ -234,8 +227,6 @@ setMethod("partitionBy", #' @param numPartitions Number of partitions to create. #' @return An RDD where each element is list(K, list(V)) #' @seealso reduceByKey -#' @rdname groupByKey -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -245,6 +236,7 @@ setMethod("partitionBy", #' 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"), @@ -294,9 +286,7 @@ setMethod("groupByKey", #' @param numPartitions Number of partitions to create. #' @return An RDD where each element is list(K, V') where V' is the merged #' value -#' @rdname reduceByKey #' @seealso groupByKey -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -306,6 +296,7 @@ setMethod("groupByKey", #' 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"), @@ -336,9 +327,7 @@ setMethod("reduceByKey", #' 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 -#' @rdname reduceByKeyLocally #' @seealso reduceByKey -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -347,6 +336,7 @@ setMethod("reduceByKey", #' 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"), @@ -402,9 +392,7 @@ setMethod("reduceByKeyLocally", #' @param numPartitions Number of partitions to create. #' @return An RDD where each element is list(K, C) where C is the combined type #' -#' @rdname combineByKey #' @seealso groupByKey, reduceByKey -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -414,6 +402,7 @@ setMethod("reduceByKeyLocally", #' 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", @@ -463,9 +452,7 @@ setMethod("combineByKey", #' 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. -#' @rdname aggregateByKey #' @seealso foldByKey, combineByKey -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -476,6 +463,7 @@ setMethod("combineByKey", #' 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", @@ -499,15 +487,14 @@ setMethod("aggregateByKey", #' @param zeroValue A neutral "zero value". #' @param func An associative function for folding values of each key. #' @return An RDD containing the aggregation result. -#' @rdname foldByKey #' @seealso aggregateByKey, combineByKey -#' @export #' @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", @@ -532,8 +519,6 @@ setMethod("foldByKey", #' @param numPartitions Number of partitions to create. #' @return a new RDD containing all pairs of elements with matching keys in #' two input RDDs. -#' @rdname join-methods -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -541,6 +526,7 @@ setMethod("foldByKey", #' 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", numPartitions = "integer"), @@ -569,8 +555,6 @@ setMethod("join", #' @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. -#' @rdname join-methods -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -579,6 +563,7 @@ setMethod("join", #' 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"), @@ -607,8 +592,6 @@ setMethod("leftOuterJoin", #' @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. -#' @rdname join-methods -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -617,6 +600,7 @@ setMethod("leftOuterJoin", #' 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"), @@ -646,8 +630,6 @@ setMethod("rightOuterJoin", #' 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. -#' @rdname join-methods -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -658,6 +640,7 @@ setMethod("rightOuterJoin", #' # 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"), @@ -679,8 +662,6 @@ setMethod("fullOuterJoin", #' @param numPartitions Number of partitions to create. #' @return a new RDD containing all pairs of elements with values in a list #' in all RDDs. -#' @rdname cogroup -#' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() @@ -689,6 +670,7 @@ setMethod("fullOuterJoin", #' 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", @@ -735,14 +717,13 @@ setMethod("cogroup", #' @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. -#' @rdname sortByKey -#' @export #' @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"), From ff948dbe9af8271c41b0b4867709ac6e73bdb0f5 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Tue, 10 Mar 2015 15:37:44 -0400 Subject: [PATCH 116/121] Remove missingOrInteger. --- pkg/R/RDD.R | 10 +++------- pkg/R/generics.R | 2 +- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 9134258eca2fd..88f00f3bbc60e 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -727,7 +727,6 @@ setMethod("take", resList }) -setClassUnion("missingOrInteger", c("missing", "integer")) #' Removes the duplicates from RDD. #' #' This function returns a new RDD containing the distinct elements in the @@ -742,13 +741,10 @@ setClassUnion("missingOrInteger", c("missing", "integer")) #' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) #'} #' @rdname distinct -#' @aliases distinct,RDD,missingOrInteger-method +#' @aliases distinct,RDD-method setMethod("distinct", - signature(x = "RDD", numPartitions = "missingOrInteger"), - function(x, numPartitions) { - if (missing(numPartitions)) { - numPartitions <- SparkR::numPartitions(x) - } + signature(x = "RDD"), + function(x, numPartitions = SparkR::numPartitions(x)) { identical.mapped <- lapply(x, function(x) { list(x, NULL) }) reduced <- reduceByKey(identical.mapped, function(x, y) { x }, diff --git a/pkg/R/generics.R b/pkg/R/generics.R index 85bdd088d2c4e..d1870b613ef73 100644 --- a/pkg/R/generics.R +++ b/pkg/R/generics.R @@ -110,7 +110,7 @@ setGeneric("take", function(x, num) { standardGeneric("take") }) #' @rdname distinct #' @export setGeneric("distinct", - function(x, numPartitions) { standardGeneric("distinct") }) + function(x, numPartitions = 1L) { standardGeneric("distinct") }) #' @rdname sampleRDD #' @export From 01aa5ee2cb848ae5215ae715461e93b743296ff5 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 10 Mar 2015 12:59:38 -0700 Subject: [PATCH 117/121] add config for using daemon, refactor --- pkg/inst/worker/daemon.R | 58 +++++-------------- .../edu/berkeley/cs/amplab/sparkr/RRDD.scala | 8 +-- 2 files changed, 20 insertions(+), 46 deletions(-) diff --git a/pkg/inst/worker/daemon.R b/pkg/inst/worker/daemon.R index 5af55aa4b255c..13132abf647ec 100644 --- a/pkg/inst/worker/daemon.R +++ b/pkg/inst/worker/daemon.R @@ -10,48 +10,22 @@ suppressPackageStartupMessages(library(SparkR)) port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) inputCon <- socketConnection(port = port, open = "rb", blocking = TRUE, timeout = 3600) -# Read from connection, retrying to read exactly 'size' bytes -readBinFull <- function(con, size) { - # readBin() could be interruptted by signal, we have no way to tell - # if it's interruptted or the socket is closed, so we retry at most - # 20 times, to avoid the deadloop if the socket is closed - c <- 1 - data <- readBin(con, raw(), size) - while (length(data) < size && c < 20) { - extra <- readBin(con, raw(), size - length(data)) - data <- c(data, extra) - c <- c + 1 - } - data -} - -# Utility function to read the port with retry -# Returns -1 if the socket was closed -readPort <- function(con) { - data <- readBinFull(con, 4L) - if (length(data) != 4) { - -1 - } else { - rc <- rawConnection(data) - ret <- SparkR:::readInt(rc) - close(rc) - ret - } -} - while (TRUE) { - port <- readPort(inputCon) - if (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) + ready <- socketSelect(list(inputCon)) + if (ready) { + 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) + } } } diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index fe9095eccd398..8e2e0d2d453b3 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -307,13 +307,11 @@ private[sparkr] class BufferedStreamThread( } 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. - // TODO(davies): make it configurable - val useDaemon = !System.getProperty("os.name").startsWith("Windows") + val inWindows = System.getProperty("os.name").startsWith("Windows") private[this] var errThread: BufferedStreamThread = _ private[this] var daemonSocket: Socket = _ private[this] var daemonChannel: DataOutputStream = _ @@ -345,6 +343,7 @@ object RRDD { for ((name, value) <- sparkExecutorEnvMap) { sparkConf.setExecutorEnv(name.asInstanceOf[String], value.asInstanceOf[String]) } + new JavaSparkContext(sparkConf) } @@ -381,7 +380,8 @@ object RRDD { * ProcessBuilder used to launch worker R processes. */ def createRWorker(rLibDir: String, port: Int) = { - if (useDaemon) { + val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) + if (!inWindows && useDaemon) { synchronized { if (daemonSocket == null) { // we expect one connections From 46cea3d75e5390b26a9763d6142e0f04a8004909 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 10 Mar 2015 14:02:09 -0700 Subject: [PATCH 118/121] retry --- pkg/inst/worker/daemon.R | 8 +++-- .../edu/berkeley/cs/amplab/sparkr/RRDD.scala | 30 ++++++++++++------- 2 files changed, 25 insertions(+), 13 deletions(-) diff --git a/pkg/inst/worker/daemon.R b/pkg/inst/worker/daemon.R index 13132abf647ec..d25c1ea2af9ad 100644 --- a/pkg/inst/worker/daemon.R +++ b/pkg/inst/worker/daemon.R @@ -14,9 +14,13 @@ 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) { - cat("quitting daemon", "\n") - quit(save = "no") + port <- SparkR:::readInt(inputCon) + if (length(port) == 0) { + cat("quitting daemon\n") + quit(save = "no") + } } p <- parallel:::mcfork() if (inherits(p, "masterProcess")) { diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 8e2e0d2d453b3..8c5cf6065e3a5 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -1,7 +1,7 @@ package edu.berkeley.cs.amplab.sparkr import java.io._ -import java.net.{Socket, ServerSocket} +import java.net.ServerSocket import java.util.{Map => JMap} import scala.collection.JavaConversions._ @@ -9,11 +9,10 @@ import scala.io.Source import scala.reflect.ClassTag import scala.util.Try -import org.apache.spark.{SparkEnv, Partition, SparkException, TaskContext, SparkConf} -import org.apache.spark.api.java.{JavaSparkContext, JavaRDD, JavaPairRDD} +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.{Partition, SparkConf, SparkEnv, SparkException, TaskContext} private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( parent: RDD[T], @@ -313,7 +312,6 @@ object RRDD { // also fall back to launching workers (worker.R) directly. val inWindows = System.getProperty("os.name").startsWith("Windows") private[this] var errThread: BufferedStreamThread = _ - private[this] var daemonSocket: Socket = _ private[this] var daemonChannel: DataOutputStream = _ def createSparkContext( @@ -383,19 +381,29 @@ object RRDD { val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) if (!inWindows && useDaemon) { synchronized { - if (daemonSocket == null) { + if (daemonChannel == null) { // we expect one connections val serverSocket = new ServerSocket(0, 1) - val daemonPort = serverSocket.getLocalPort() + val daemonPort = serverSocket.getLocalPort errThread = createRProcess(rLibDir, daemonPort, "daemon.R") // the socket used to send out the input of task serverSocket.setSoTimeout(10000) - daemonSocket = serverSocket.accept() - daemonChannel = new DataOutputStream(daemonSocket.getOutputStream) + val sock = serverSocket.accept() + daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) serverSocket.close() } - daemonChannel.writeInt(port) - daemonChannel.flush() + 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 { From 8583968b531db0762b698115ae59c69bc7e812ee Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 10 Mar 2015 15:05:30 -0700 Subject: [PATCH 119/121] readFully() --- .../src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala | 4 ++-- .../src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala | 5 ++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala index 8c5cf6065e3a5..4ee0dc576b163 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/RRDD.scala @@ -189,7 +189,7 @@ private class PairwiseRRDD[T: ClassTag]( val hashedKey = dataStream.readInt() val contentPairsLength = dataStream.readInt() val contentPairs = new Array[Byte](contentPairsLength) - dataStream.read(contentPairs, 0, contentPairsLength) + dataStream.readFully(contentPairs) (hashedKey, contentPairs) case _ => null // End of input } @@ -231,7 +231,7 @@ private class RRDD[T: ClassTag]( length match { case length if length > 0 => val obj = new Array[Byte](length) - dataStream.read(obj, 0, length) + dataStream.readFully(obj, 0, length) obj case _ => null } diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala index 0174be7840a46..afec6f5070e81 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala @@ -50,8 +50,7 @@ object SerDe { def readBytes(in: DataInputStream) = { val len = readInt(in) val out = new Array[Byte](len) - val bytesRead = in.read(out, 0, len) - assert(len == bytesRead) + val bytesRead = in.readFully(out) out } @@ -66,7 +65,7 @@ object SerDe { def readString(in: DataInputStream) = { val len = in.readInt() val asciiBytes = new Array[Byte](len) - in.read(asciiBytes, 0, len) + in.readFully(asciiBytes) assert(asciiBytes(len - 1) == 0) val str = new String(asciiBytes.dropRight(1).map(_.toChar)) str From bc2ff380be08f8571951a66a0e4a6507ed1a6ffe Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 14 Mar 2015 00:15:56 -0700 Subject: [PATCH 120/121] handle NULL --- pkg/R/deserialize.R | 2 ++ pkg/R/serialize.R | 11 ++++++++++- .../scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala | 2 ++ 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/pkg/R/deserialize.R b/pkg/R/deserialize.R index c5f8ac7f06a4e..2500c072bdb50 100644 --- a/pkg/R/deserialize.R +++ b/pkg/R/deserialize.R @@ -9,6 +9,8 @@ # Double -> double # Long -> double # Array[Byte] -> raw +# Date -> Date +# Time -> POSIXct # # Array[T] -> list() # Object -> jobj diff --git a/pkg/R/serialize.R b/pkg/R/serialize.R index cdeeb1604fbd2..22a462fac89b0 100644 --- a/pkg/R/serialize.R +++ b/pkg/R/serialize.R @@ -1,12 +1,15 @@ # 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 @@ -21,6 +24,7 @@ writeObject <- function(con, object, writeType = TRUE) { writeType(con, type) } switch(type, + NULL = writeVoid(con), integer = writeInt(con, object), character = writeString(con, object), logical = writeBoolean(con, object), @@ -36,6 +40,10 @@ writeObject <- function(con, object, writeType = TRUE) { stop(paste("Unsupported type for serialization", type))) } +writeVoid <- function(con) { + # no value for NULL +} + writeString <- function(con, value) { writeInt(con, as.integer(nchar(value) + 1)) writeBin(value, con, endian = "big") @@ -88,6 +96,7 @@ writeRaw <- function(con, batch) { writeType <- function(con, class) { type <- switch(class, + NULL = "n", integer = "i", character = "c", logical = "b", diff --git a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala index 6758aa56e8e22..1de10b2a5053b 100644 --- a/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala +++ b/pkg/src/src/main/scala/edu/berkeley/cs/amplab/sparkr/SerDe.scala @@ -12,6 +12,7 @@ object SerDe { // Type mapping from R to Java // + // NULL -> void // integer -> Int // character -> String // logical -> Boolean @@ -35,6 +36,7 @@ object SerDe { 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)) From 70f620c95fc3a9cdc7a0b47c97e06150c6cc7ac4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 14 Mar 2015 15:26:42 -0700 Subject: [PATCH 121/121] address comments --- pkg/R/RDD.R | 152 +---------------------------------------------- pkg/R/generics.R | 6 +- pkg/R/pairRDD.R | 15 ++--- 3 files changed, 11 insertions(+), 162 deletions(-) diff --git a/pkg/R/RDD.R b/pkg/R/RDD.R index 7c696a6f521d1..30a083364ce43 100644 --- a/pkg/R/RDD.R +++ b/pkg/R/RDD.R @@ -762,15 +762,11 @@ setMethod("first", #' rdd <- parallelize(sc, c(1,2,2,3,3,3)) #' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) #'} -setClassUnion("missingOrInteger", c("missing", "integer")) #' @rdname distinct #' @aliases distinct,RDD-method setMethod("distinct", - signature(x = "RDD", numPartitions = "missingOrInteger"), - function(x, numPartitions) { - if (missing(numPartitions)) { - numPartitions <- SparkR::numPartitions(x) - } + signature(x = "RDD"), + function(x, numPartitions = SparkR::numPartitions(x)) { identical.mapped <- lapply(x, function(x) { list(x, NULL) }) reduced <- reduceByKey(identical.mapped, function(x, y) { x }, @@ -1273,149 +1269,6 @@ setMethod("name", #'} setGeneric("setName", function(x, name) { standardGeneric("setName") }) -#' @rdname setName -#' @aliases setName,RDD -setMethod("setName", - signature(x = "RDD", name = "character"), - function(x, name) { - callJMethod(getJRDD(x), "setName", name) - x - }) - -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) -#'} -#' @rdname top -#' @aliases top,RDD,RDD-method -setMethod("top", - signature(x = "RDD", num = "integer"), - function(x, num) { - takeOrderedElem(x, num, FALSE) - }) - -#' Fold an RDD using a given associative function and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using a given associative function and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param op An associative function for the folding operation. -#' @return The folding result. -#' @seealso reduce -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) -#' fold(rdd, 0, "+") # 15 -#'} -#' @rdname fold -#' @aliases fold,RDD,RDD-method -setMethod("fold", - signature(x = "RDD", zeroValue = "ANY", op = "ANY"), - function(x, zeroValue, op) { - aggregateRDD(x, zeroValue, op, op) - }) - -#' Aggregate an RDD using the given combine functions and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using given combine functions and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the RDD elements. It may return a different -#' result type from the type of the RDD elements. -#' @param combOp A function to aggregate results of seqOp. -#' @return The aggregation result. -#' @seealso reduce -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' 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]]) } -#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) -#'} -#' @rdname aggregateRDD -#' @aliases aggregateRDD,RDD,RDD-method -setMethod("aggregateRDD", - signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), - function(x, zeroValue, seqOp, combOp) { - partitionFunc <- function(part) { - Reduce(seqOp, part, zeroValue) - } - - partitionList <- collect(lapplyPartition(x, partitionFunc), - flatten = FALSE) - Reduce(combOp, partitionList, zeroValue) - }) - -#' Pipes elements to a forked external process. -#' -#' The same as 'pipe()' in Spark. -#' -#' @param x The RDD whose elements are piped to the forked external process. -#' @param command The command to fork an external process. -#' @param env A named list to set environment variables of the external process. -#' @return A new RDD created by piping all elements to a forked external process. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' collect(pipeRDD(rdd, "more") -#' Output: c("1", "2", ..., "10") -#'} -#' @rdname pipeRDD -#' @aliases pipeRDD,RDD,character-method -setMethod("pipeRDD", - signature(x = "RDD", command = "character"), - function(x, command, env = list()) { - func <- function(part) { - trim.trailing.func <- function(x) { - sub("[\r\n]*$", "", toString(x)) - } - input <- unlist(lapply(part, trim.trailing.func)) - res <- system2(command, stdout = TRUE, input = input, env = env) - lapply(res, trim.trailing.func) - } - lapplyPartition(x, func) - }) - -# TODO: Consider caching the name in the RDD's environment -#' Return an RDD's name. -#' -#' @param x The RDD whose name is returned. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' name(rdd) # NULL (if not set before) -#'} -#' @rdname name -#' @aliases name,RDD -setMethod("name", - signature(x = "RDD"), - function(x) { - callJMethod(getJRDD(x), "name") - }) - -#' Set an RDD's name. -#' -#' @param x The RDD whose name is to be set. -#' @param name The RDD name to be set. -#' @return a new RDD renamed. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' setName(rdd, "myRDD") -#' name(rdd) # "myRDD" -#'} #' @rdname setName #' @aliases setName,RDD setMethod("setName", @@ -1518,7 +1371,6 @@ setMethod("zipWithIndex", ############ Binary Functions ############# - #' Return the union RDD of two RDDs. #' The same as union() in Spark. #' diff --git a/pkg/R/generics.R b/pkg/R/generics.R index 05300bb0aca8f..76787099a1590 100644 --- a/pkg/R/generics.R +++ b/pkg/R/generics.R @@ -297,15 +297,15 @@ setGeneric("join", function(x, y, ...) { standardGeneric("join") }) #' @rdname join-methods #' @export -setGeneric("leftOuterJoin", function(x, y, ...) { standardGeneric("leftOuterJoin") }) +setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) #' @rdname join-methods #' @export -setGeneric("rightOuterJoin", function(x, y, ...) { standardGeneric("rightOuterJoin") }) +setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) #' @rdname join-methods #' @export -setGeneric("fullOuterJoin", function(x, y, ...) { standardGeneric("fullOuterJoin") }) +setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) #' @rdname cogroup #' @export diff --git a/pkg/R/pairRDD.R b/pkg/R/pairRDD.R index 319baf9c41ca3..9189410bb2ca6 100644 --- a/pkg/R/pairRDD.R +++ b/pkg/R/pairRDD.R @@ -563,7 +563,7 @@ setMethod("join", #' @rdname join-methods #' @aliases leftOuterJoin,RDD,RDD-method setMethod("leftOuterJoin", - signature(x = "RDD", y = "RDD"), + 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]])) }) @@ -572,8 +572,7 @@ setMethod("leftOuterJoin", joinTaggedList(v, list(FALSE, TRUE)) } - joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)), - doJoin) + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' Right outer join two RDDs @@ -601,7 +600,7 @@ setMethod("leftOuterJoin", #' @rdname join-methods #' @aliases rightOuterJoin,RDD,RDD-method setMethod("rightOuterJoin", - signature(x = "RDD", y = "RDD"), + 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]])) }) @@ -610,8 +609,7 @@ setMethod("rightOuterJoin", joinTaggedList(v, list(TRUE, FALSE)) } - joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)), - doJoin) + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' Full outer join two RDDs @@ -642,7 +640,7 @@ setMethod("rightOuterJoin", #' @rdname join-methods #' @aliases fullOuterJoin,RDD,RDD-method setMethod("fullOuterJoin", - signature(x = "RDD", y = "RDD"), + 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]])) }) @@ -651,8 +649,7 @@ setMethod("fullOuterJoin", joinTaggedList(v, list(TRUE, TRUE)) } - joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)), - doJoin) + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' For each key k in several RDDs, return a resulting RDD that