From 0e2a94f0c4815c254493eec4a1721396c36656a1 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Sun, 29 Mar 2015 19:50:21 -0500 Subject: [PATCH 01/33] Define functions for schema and fields Instead of using a list[list[list[]]], use specific constructors for schema and field objects. --- R/pkg/NAMESPACE | 6 ++- R/pkg/R/SQLContext.R | 74 +++++++++++++++++++++++--------- R/pkg/inst/tests/test_sparkSQL.R | 27 ++++++++---- 3 files changed, 76 insertions(+), 31 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index a354cdce74afa..450aacfb51718 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -171,6 +171,8 @@ export("cacheTable", "jsonRDD", "loadDF", "parquetFile", + "buildSchema", + "field", "sql", "table", "tableNames", @@ -179,4 +181,6 @@ export("cacheTable", "uncacheTable") export("print.structType", - "print.structField") + "print.structField", + "print.struct", + "print.field") diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 930ada22f4c38..65057cc45a2f6 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -54,9 +54,9 @@ infer_type <- function(x) { # StructType types <- lapply(x, infer_type) fields <- lapply(1:length(x), function(i) { - list(name = names[[i]], type = types[[i]], nullable = TRUE) + field(names[[i]], types[[i]], TRUE) }) - list(type = "struct", fields = fields) + do.call(buildSchema, fields) } } else if (length(x) > 1) { list(type = "array", elementType = type, containsNull = TRUE) @@ -67,19 +67,19 @@ infer_type <- function(x) { #' dump the schema into JSON string tojson <- function(x) { - if (is.list(x)) { + if (inherits(x, "struct")) { + # schema object + l <- paste(lapply(x, tojson), collapse = ", ") + paste('{\"type\":\"struct\", \"fields\":','[', l, ']}', sep = '') + } else if (inherits(x, "field")) { + # field object names <- names(x) - if (!is.null(names)) { - items <- lapply(names, function(n) { - safe_n <- gsub('"', '\\"', n) - paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') - }) - d <- paste(items, collapse = ', ') - paste('{', d, '}', sep = '') - } else { - l <- paste(lapply(x, tojson), collapse = ', ') - paste('[', l, ']', sep = '') - } + items <- lapply(names, function(n) { + safe_n <- gsub('"', '\\"', n) + paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') + }) + d <- paste(items, collapse = ", ") + paste('{', d, '}', sep = '') } else if (is.character(x)) { paste('"', x, '"', sep = '') } else if (is.logical(x)) { @@ -134,7 +134,7 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { stop(paste("unexpected type:", class(data))) } - if (is.null(schema) || is.null(names(schema))) { + if (is.null(schema) || (!inherits(schema, "struct") && is.null(names(schema)))) { row <- first(rdd) names <- if (is.null(schema)) { names(row) @@ -143,7 +143,7 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { } if (is.null(names)) { names <- lapply(1:length(row), function(x) { - paste("_", as.character(x), sep = "") + paste("_", as.character(x), sep = "") }) } @@ -159,14 +159,12 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { types <- lapply(row, infer_type) fields <- lapply(1:length(row), function(i) { - list(name = names[[i]], type = types[[i]], nullable = TRUE) + field(names[[i]], types[[i]], TRUE) }) - schema <- list(type = "struct", fields = fields) + schema <- do.call(buildSchema, fields) } - stopifnot(class(schema) == "list") - stopifnot(schema$type == "struct") - stopifnot(class(schema$fields) == "list") + stopifnot(class(schema) == "struct") schemaString <- tojson(schema) jrdd <- getJRDD(lapply(rdd, function(x) x), "row") @@ -518,3 +516,37 @@ createExternalTable <- function(sqlCtx, tableName, path = NULL, source = NULL, . sdf <- callJMethod(sqlCtx, "createExternalTable", tableName, source, options) dataFrame(sdf) } + +buildSchema <- function(field, ...) { + fields <- list(field, ...) + if (!all(sapply(fields, inherits, "field"))) { + stop("All arguments must be Field objects.") + } + + structure(fields, class = "struct") +} + +print.struct <- function(x, ...) { + cat(sapply(x, function(field) { paste("|-", "name = \"", field$name, + "\", type = \"", field$type, + "\", nullable = ", field$nullable, "\n", + sep = "") }) + , sep = "") +} + +field <- function(name, type, nullable = TRUE) { + if (class(name) != "character") { + stop("Field name must be a string.") + } + if (class(type) != "character") { + stop("Field type must be a string.") + } + if (class(nullable) != "logical") { + stop("nullable must be either TRUE or FALSE") + } + structure(list("name" = name, "type" = type, "nullable" = nullable), class = "field") +} + +print.field <- function(x, ...) { + cat("name = \"", x$name, "\", type = \"", x$type, "\", nullable = ", x$nullable, sep = "") +} diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index cf5cf6d1692af..bf2101e276df6 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -44,9 +44,8 @@ test_that("infer types", { expect_equal(infer_type(list(1L, 2L)), list(type = 'array', elementType = "integer", containsNull = TRUE)) expect_equal(infer_type(list(a = 1L, b = "2")), - list(type = "struct", - fields = list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)))) + buildSchema(field(name = "a", type = "integer", nullable = TRUE), + field(name = "b", type = "string", nullable = TRUE))) e <- new.env() assign("a", 1L, envir = e) expect_equal(infer_type(e), @@ -54,6 +53,18 @@ test_that("infer types", { valueContainsNull = TRUE)) }) +test_that("buildSchema and field", { + testField <- field("a", "string") + expect_true(inherits(testField, "field")) + expect_true(testField$name == "a") + expect_true(testField$nullable) + + testSchema <- buildSchema(testField, field("b", "integer")) + expect_true(inherits(testSchema, "struct")) + expect_true(inherits(testSchema[[2]], "field")) + expect_true(testSchema[[1]]$type == "string") +}) + test_that("create DataFrame from RDD", { rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) df <- createDataFrame(sqlCtx, rdd, list("a", "b")) @@ -66,9 +77,8 @@ test_that("create DataFrame from RDD", { expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("_1", "_2")) - fields <- list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)) - schema <- list(type = "struct", fields = fields) + schema <- buildSchema(field(name = "a", type = "integer", nullable = TRUE), + field(name = "b", type = "string", nullable = TRUE)) df <- createDataFrame(sqlCtx, rdd, schema) expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("a", "b")) @@ -94,9 +104,8 @@ test_that("toDF", { expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("_1", "_2")) - fields <- list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)) - schema <- list(type = "struct", fields = fields) + schema <- buildSchema(field(name = "a", type = "integer", nullable = TRUE), + field(name = "b", type = "string", nullable = TRUE)) df <- toDF(rdd, schema) expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("a", "b")) From 7dd81b7399e42e92c27f485d48f1b6a33451438d Mon Sep 17 00:00:00 2001 From: cafreeman Date: Mon, 30 Mar 2015 10:04:21 -0500 Subject: [PATCH 02/33] Documentation --- R/pkg/R/SQLContext.R | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 65057cc45a2f6..18ad27cd16190 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -517,6 +517,23 @@ createExternalTable <- function(sqlCtx, tableName, path = NULL, source = NULL, . dataFrame(sdf) } +#' Create a Schema object +#' +#' Create an object of type "struct" that contains the metadata for a DataFrame. Intended for +#' use with createDataFrame and toDF. +#' +#' @param field a Field object (created with the field() function) +#' @param ... additional Field objects +#' @return a Schema object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' schema <- buildSchema(field("a", "integer"), field("b", "string")) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } buildSchema <- function(field, ...) { fields <- list(field, ...) if (!all(sapply(fields, inherits, "field"))) { @@ -526,6 +543,7 @@ buildSchema <- function(field, ...) { structure(fields, class = "struct") } +# print method for "struct" object print.struct <- function(x, ...) { cat(sapply(x, function(field) { paste("|-", "name = \"", field$name, "\", type = \"", field$type, @@ -534,6 +552,25 @@ print.struct <- function(x, ...) { , sep = "") } +#' Create a Field object +#' +#' Create a Field object that contains the metadata for a single field in a schema. +#' +#' @param name The name of the field +#' @param type The data type of the field +#' @param nullable A logical vector indicating whether or not the field is nullable +#' @return a Field object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' field1 <- field("a", "integer", TRUE) +#' field2 <- field("b", "string", TRUE) +#' schema <- buildSchema(field1, field2) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } field <- function(name, type, nullable = TRUE) { if (class(name) != "character") { stop("Field name must be a string.") @@ -547,6 +584,7 @@ field <- function(name, type, nullable = TRUE) { structure(list("name" = name, "type" = type, "nullable" = nullable), class = "field") } +# print method for Field objects print.field <- function(x, ...) { cat("name = \"", x$name, "\", type = \"", x$type, "\", nullable = ", x$nullable, sep = "") } From 8b76e810bfd1a152a1d6be1091c2e2ee58a4ca03 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 30 Mar 2015 09:44:56 -0700 Subject: [PATCH 03/33] Merge pull request #233 from redbaron/fail-early-on-missing-dep Fail worker early if dependency is missing --- R/pkg/inst/worker/worker.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index c6542928e8ddd..f11870c915766 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -37,7 +37,7 @@ serializer <- SparkR:::readString(inputCon) # Include packages as required packageNames <- unserialize(SparkR:::readRaw(inputCon)) for (pkg in packageNames) { - suppressPackageStartupMessages(require(as.character(pkg), character.only=TRUE)) + suppressPackageStartupMessages(library(as.character(pkg), character.only=TRUE)) } # read function dependencies From cd66603aa6a12c727a414513ad1c2517c373b242 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Mon, 30 Mar 2015 13:34:55 -0500 Subject: [PATCH 04/33] new line at EOF --- R/pkg/R/SQLContext.R | 1 + 1 file changed, 1 insertion(+) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 18ad27cd16190..089a9e4e970e7 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -588,3 +588,4 @@ field <- function(name, type, nullable = TRUE) { print.field <- function(x, ...) { cat("name = \"", x$name, "\", type = \"", x$type, "\", nullable = ", x$nullable, sep = "") } + From 136a07ef5ba741dd144612576f7028fe42779162 Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Fri, 3 Apr 2015 15:54:44 -0700 Subject: [PATCH 05/33] Merge pull request #242 from hqzizania/stats [SPARKR-92] Phase 2: implement sum(rdd) --- R/pkg/R/generics.R | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 5fb1ccaa84ee2..b7d03944431e8 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -134,6 +134,10 @@ setGeneric("minimum", function(x) { standardGeneric("minimum") }) #' @export setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) +#' @rdname foreach +#' @export +setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) + #' @rdname name #' @export setGeneric("name", function(x) { standardGeneric("name") }) From b317aa7a30e1f33fd15600f37c3488ea2b24bb6e Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Sat, 4 Apr 2015 10:10:17 -0700 Subject: [PATCH 06/33] Merge pull request #243 from hqzizania/master [SPARKR-199] Change takeOrdered, top to fetch one partition at a time --- R/pkg/R/RDD.R | 47 +++++++++++++++++++++++++++++++++++------------ 1 file changed, 35 insertions(+), 12 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 604ad03c407b9..eb970a09c0f17 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -738,6 +738,7 @@ setMethod("take", index <- -1 jrdd <- getJRDD(x) numPartitions <- numPartitions(x) + serializedModeRDD <- getSerializedMode(x) # TODO(shivaram): Collect more than one partition based on size # estimates similar to the scala version of `take`. @@ -756,13 +757,14 @@ setMethod("take", elems <- convertJListToRList(partition, flatten = TRUE, logicalUpperBound = size, - serializedMode = getSerializedMode(x)) - # TODO: Check if this append is O(n^2)? + serializedMode = serializedModeRDD) + resList <- append(resList, elems) } resList }) + #' First #' #' Return the first element of an RDD @@ -1100,21 +1102,42 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { if (num < length(part)) { # R limitation: order works only on primitive types! ord <- order(unlist(part, recursive = FALSE), decreasing = !ascending) - list(part[ord[1:num]]) + part[ord[1:num]] } else { - list(part) + 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(x, partitionFunc) - reduce(newRdd, reduceFunc) + + resList <- list() + index <- -1 + jrdd <- getJRDD(newRdd) + numPartitions <- numPartitions(newRdd) + serializedModeRDD <- getSerializedMode(newRdd) + + while (TRUE) { + index <- index + 1 + + if (index >= numPartitions) { + ord <- order(unlist(resList, recursive = FALSE), decreasing = !ascending) + resList <- resList[ord[1:num]] + break + } + + # a JList of byte arrays + partitionArr <- callJMethod(jrdd, "collectPartitions", as.list(as.integer(index))) + partition <- partitionArr[[1]] + + # elems is capped to have at most `num` elements + elems <- convertJListToRList(partition, + flatten = TRUE, + logicalUpperBound = num, + serializedMode = serializedModeRDD) + + resList <- append(resList, elems) + } + resList } #' Returns the first N elements from an RDD in ascending order. From c9497a34ca456df8ce0adcba2fffb07c27922619 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 6 Apr 2015 21:57:20 -0700 Subject: [PATCH 07/33] Merge pull request #208 from lythesia/master [SPARKR-188] Add profiling of R execution on worker side Conflicts: pkg/inst/worker/worker.R --- R/pkg/R/serialize.R | 9 +- R/pkg/inst/worker/worker.R | 57 +++++++- .../scala/org/apache/spark/api/r/RRDD.scala | 131 +++++++++--------- .../scala/org/apache/spark/api/r/SerDe.scala | 14 +- 4 files changed, 132 insertions(+), 79 deletions(-) diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 8a9c0c652ce24..c53d0a961016f 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -69,8 +69,9 @@ writeJobj <- function(con, value) { } writeString <- function(con, value) { - writeInt(con, as.integer(nchar(value) + 1)) - writeBin(value, con, endian = "big") + utfVal <- enc2utf8(value) + writeInt(con, as.integer(nchar(utfVal, type = "bytes") + 1)) + writeBin(utfVal, con, endian = "big") } writeInt <- function(con, value) { @@ -189,7 +190,3 @@ writeArgs <- function(con, args) { } } } - -writeStrings <- function(con, stringList) { - writeLines(unlist(stringList), con) -} diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index f11870c915766..014bf7bd7b3fe 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -17,6 +17,23 @@ # Worker class +# Get current system time +currentTimeSecs <- function() { + as.numeric(Sys.time()) +} + +# Get elapsed time +elapsedSecs <- function() { + proc.time()[3] +} + +# Constants +specialLengths <- list(END_OF_STERAM = 0L, TIMING_DATA = -1L) + +# Timing R process boot +bootTime <- currentTimeSecs() +bootElap <- elapsedSecs() + 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 @@ -46,6 +63,9 @@ computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen)) env <- environment(computeFunc) parent.env(env) <- .GlobalEnv # Attach under global environment. +# Timing init envs for computing +initElap <- elapsedSecs() + # Read and set broadcast variables numBroadcastVars <- SparkR:::readInt(inputCon) if (numBroadcastVars > 0) { @@ -56,6 +76,9 @@ if (numBroadcastVars > 0) { } } +# Timing broadcast +broadcastElap <- elapsedSecs() + # If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int # as number of partitions to create. numPartitions <- SparkR:::readInt(inputCon) @@ -73,14 +96,23 @@ if (isEmpty != 0) { } else if (deserializer == "row") { data <- SparkR:::readDeserializeRows(inputCon) } + # Timing reading input data for execution + inputElap <- elapsedSecs() + output <- computeFunc(partition, data) + # Timing computing + computeElap <- elapsedSecs() + if (serializer == "byte") { SparkR:::writeRawSerialize(outputCon, output) } else if (serializer == "row") { SparkR:::writeRowSerialize(outputCon, output) } else { - SparkR:::writeStrings(outputCon, output) + # write lines one-by-one with flag + lapply(output, function(line) SparkR:::writeString(outputCon, line)) } + # Timing output + outputElap <- elapsedSecs() } else { if (deserializer == "byte") { # Now read as many characters as described in funcLen @@ -90,6 +122,8 @@ if (isEmpty != 0) { } else if (deserializer == "row") { data <- SparkR:::readDeserializeRows(inputCon) } + # Timing reading input data for execution + inputElap <- elapsedSecs() res <- new.env() @@ -107,6 +141,8 @@ if (isEmpty != 0) { res[[bucket]] <- acc } invisible(lapply(data, hashTupleToEnvir)) + # Timing computing + computeElap <- elapsedSecs() # Step 2: write out all of the environment as key-value pairs. for (name in ls(res)) { @@ -116,13 +152,26 @@ if (isEmpty != 0) { length(res[[name]]$data) <- res[[name]]$counter SparkR:::writeRawSerialize(outputCon, res[[name]]$data) } + # Timing output + outputElap <- elapsedSecs() } +} else { + inputElap <- broadcastElap + computeElap <- broadcastElap + outputElap <- broadcastElap } +# Report timing +SparkR:::writeInt(outputCon, specialLengths$TIMING_DATA) +SparkR:::writeDouble(outputCon, bootTime) +SparkR:::writeDouble(outputCon, initElap - bootElap) # init +SparkR:::writeDouble(outputCon, broadcastElap - initElap) # broadcast +SparkR:::writeDouble(outputCon, inputElap - broadcastElap) # input +SparkR:::writeDouble(outputCon, computeElap - inputElap) # compute +SparkR:::writeDouble(outputCon, outputElap - computeElap) # output + # End of output -if (serializer %in% c("byte", "row")) { - SparkR:::writeInt(outputCon, 0L) -} +SparkR:::writeInt(outputCon, specialLengths$END_OF_STERAM) close(outputCon) close(inputCon) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 5fa4d483b8342..6fea5e1144f2f 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -42,10 +42,15 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( rLibDir: String, broadcastVars: Array[Broadcast[Object]]) extends RDD[U](parent) with Logging { + protected var dataStream: DataInputStream = _ + private var bootTime: Double = _ override def getPartitions: Array[Partition] = parent.partitions override def compute(partition: Partition, context: TaskContext): Iterator[U] = { + // Timing start + bootTime = System.currentTimeMillis / 1000.0 + // The parent may be also an RRDD, so we should launch it first. val parentIterator = firstParent[T].iterator(partition, context) @@ -69,7 +74,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( // the socket used to receive the output of task val outSocket = serverSocket.accept() val inputStream = new BufferedInputStream(outSocket.getInputStream) - val dataStream = openDataStream(inputStream) + dataStream = new DataInputStream(inputStream) serverSocket.close() try { @@ -155,6 +160,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( } else if (deserializer == SerializationFormats.ROW) { dataOut.write(elem.asInstanceOf[Array[Byte]]) } else if (deserializer == SerializationFormats.STRING) { + // write string(for StringRRDD) printOut.println(elem) } } @@ -180,9 +186,41 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( }.start() } - protected def openDataStream(input: InputStream): Closeable + protected def readData(length: Int): U - protected def read(): U + protected def read(): U = { + try { + val length = dataStream.readInt() + + length match { + case SpecialLengths.TIMING_DATA => + // Timing data from R worker + val boot = dataStream.readDouble - bootTime + val init = dataStream.readDouble + val broadcast = dataStream.readDouble + val input = dataStream.readDouble + val compute = dataStream.readDouble + val output = dataStream.readDouble + logInfo( + ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + + "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + + "total = %.3f s").format( + boot, + init, + broadcast, + input, + compute, + output, + boot + init + broadcast + input + compute + output)) + read() + case length if length >= 0 => + readData(length) + } + } catch { + case eof: EOFException => + throw new SparkException("R worker exited unexpectedly (cranshed)", eof) + } + } } /** @@ -202,31 +240,16 @@ private class PairwiseRRDD[T: ClassTag]( SerializationFormats.BYTE, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: DataInputStream = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new DataInputStream(input) - dataStream - } - - override protected def read(): (Int, Array[Byte]) = { - try { - val length = dataStream.readInt() - - length match { - case length if length == 2 => - val hashedKey = dataStream.readInt() - val contentPairsLength = dataStream.readInt() - val contentPairs = new Array[Byte](contentPairsLength) - dataStream.readFully(contentPairs) - (hashedKey, contentPairs) - case _ => null // End of input - } - } catch { - case eof: EOFException => { - throw new SparkException("R worker exited unexpectedly (crashed)", eof) - } - } + override protected def readData(length: Int): (Int, Array[Byte]) = { + length match { + case length if length == 2 => + val hashedKey = dataStream.readInt() + val contentPairsLength = dataStream.readInt() + val contentPairs = new Array[Byte](contentPairsLength) + dataStream.readFully(contentPairs) + (hashedKey, contentPairs) + case _ => null + } } lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this) @@ -247,28 +270,13 @@ private class RRDD[T: ClassTag]( parent, -1, func, deserializer, serializer, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: DataInputStream = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new DataInputStream(input) - dataStream - } - - override protected def read(): Array[Byte] = { - try { - val length = dataStream.readInt() - - length match { - case length if length > 0 => - val obj = new Array[Byte](length) - dataStream.readFully(obj, 0, length) - obj - case _ => null - } - } catch { - case eof: EOFException => { - throw new SparkException("R worker exited unexpectedly (crashed)", eof) - } + override protected def readData(length: Int): Array[Byte] = { + length match { + case length if length > 0 => + val obj = new Array[Byte](length) + dataStream.readFully(obj) + obj + case _ => null } } @@ -289,26 +297,21 @@ private class StringRRDD[T: ClassTag]( parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: BufferedReader = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new BufferedReader(new InputStreamReader(input)) - dataStream - } - - override protected def read(): String = { - try { - dataStream.readLine() - } catch { - case e: IOException => { - throw new SparkException("R worker exited unexpectedly (crashed)", e) - } + override protected def readData(length: Int): String = { + length match { + case length if length > 0 => + SerDe.readStringBytes(dataStream, length) + case _ => null } } lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) } +private object SpecialLengths { + val TIMING_DATA = -1 +} + private[r] class BufferedStreamThread( in: InputStream, name: String, diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index ccb2a371f4e48..371dfe454d1a2 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -85,13 +85,17 @@ private[spark] object SerDe { in.readDouble() } + def readStringBytes(in: DataInputStream, len: Int): String = { + val bytes = new Array[Byte](len) + in.readFully(bytes) + assert(bytes(len - 1) == 0) + val str = new String(bytes.dropRight(1), "UTF-8") + str + } + def readString(in: DataInputStream): String = { val len = in.readInt() - val asciiBytes = new Array[Byte](len) - in.readFully(asciiBytes) - assert(asciiBytes(len - 1) == 0) - val str = new String(asciiBytes.dropRight(1).map(_.toChar)) - str + readStringBytes(in, len) } def readBoolean(in: DataInputStream): Boolean = { From ba54e3484ae3cf5c98b6b35ba5e68855be116a4d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 6 Apr 2015 22:13:23 -0700 Subject: [PATCH 08/33] Merge pull request #238 from sun-rui/SPARKR-154_4 [SPARKR-154] Phase 3: implement intersection(). --- R/pkg/NAMESPACE | 1 + R/pkg/R/RDD.R | 37 ++++++++++++++++++++++++++++++++++++ R/pkg/R/generics.R | 6 +++++- R/pkg/inst/tests/test_rdd.R | 16 ++++++++++++++++ pkg/man/intersection.Rd | 38 +++++++++++++++++++++++++++++++++++++ 5 files changed, 97 insertions(+), 1 deletion(-) create mode 100644 pkg/man/intersection.Rd diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 450aacfb51718..920e1b7f598de 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -28,6 +28,7 @@ exportMethods( "fullOuterJoin", "glom", "groupByKey", + "intersection", "join", "keyBy", "keys", diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index eb970a09c0f17..125ea0110001a 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -1560,3 +1560,40 @@ setMethod("zipRDD", PipelinedRDD(zippedRDD, partitionFunc) }) + +#' Intersection of this RDD and another one. +#' +#' Return the intersection of this RDD and another one. +#' The output will not contain any duplicate elements, +#' even if the input RDDs did. Performs a hash partition +#' across the cluster. +#' Note that this method performs a shuffle internally. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions The number of partitions in the result RDD. +#' @return An RDD which is the intersection of these two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) +#' rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) +#' collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) +#' # list(1, 2, 3) +#'} +#' @rdname intersection +#' @aliases intersection,RDD +setMethod("intersection", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + rdd1 <- map(x, function(v) { list(v, NA) }) + rdd2 <- map(other, function(v) { list(v, NA) }) + + filterFunction <- function(elem) { + iters <- elem[[2]] + all(as.vector( + lapply(iters, function(iter) { length(iter) > 0 }), mode = "logical")) + } + + keys(filterRDD(cogroup(rdd1, rdd2, numPartitions = numPartitions), filterFunction)) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index b7d03944431e8..99e377cda4817 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -233,7 +233,6 @@ setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") } ############ Binary Functions ############# - #' @rdname countByKey #' @export setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) @@ -242,6 +241,11 @@ setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) #' @export setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) +#' @rdname intersection +#' @export +setGeneric("intersection", function(x, other, numPartitions = 1L) { + standardGeneric("intersection") }) + #' @rdname keys #' @export setGeneric("keys", function(x) { standardGeneric("keys") }) diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index f75e0817b9406..fc0607ae1de60 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -468,6 +468,22 @@ test_that("zipRDD() on RDDs", { unlink(fileName) }) +test_that("intersection() on RDDs", { + # intersection with self + actual <- collect(intersection(rdd, rdd)) + expect_equal(sort(as.integer(actual)), nums) + + # intersection with an empty RDD + emptyRdd <- parallelize(sc, list()) + actual <- collect(intersection(rdd, emptyRdd)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) + rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) + actual <- collect(intersection(rdd1, rdd2)) + expect_equal(sort(as.integer(actual)), 1:3) +}) + 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/intersection.Rd b/pkg/man/intersection.Rd new file mode 100644 index 0000000000000..7ec6debb3dc08 --- /dev/null +++ b/pkg/man/intersection.Rd @@ -0,0 +1,38 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{intersection,RDD,RDD-method} +\alias{intersection} +\alias{intersection,RDD} +\alias{intersection,RDD,RDD-method} +\title{Intersection of this RDD and another one.} +\usage{ +\S4method{intersection}{RDD,RDD}(x, other, + numPartitions = SparkR::numPartitions(x)) + +intersection(x, other, numPartitions = 1L) +} +\arguments{ +\item{x}{An RDD.} + +\item{other}{An RDD.} +} +\value{ +An RDD which is the intersection of these two RDDs. +} +\description{ +Return the intersection of this RDD and another one. +The output will not contain any duplicate elements, +even if the input RDDs did. Performs a hash partition +across the cluster. +Note that this method performs a shuffle internally. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) +rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) +collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) +# list(1, 2, 3) +} +} + From 20b97a60f970eadefe21456553668957f697f40a Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Tue, 7 Apr 2015 21:26:52 -0700 Subject: [PATCH 09/33] Merge pull request #234 from hqzizania/assist [SPARKR-163] Support sampleByKey() Conflicts: pkg/R/pairRDD.R --- R/pkg/NAMESPACE | 1 + R/pkg/R/generics.R | 6 ++ R/pkg/R/pairRDD.R | 153 ++++++++++++++++++++++++++++-------- R/pkg/inst/tests/test_rdd.R | 78 +++++++++++------- 4 files changed, 177 insertions(+), 61 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 920e1b7f598de..cac2fa2efb6ca 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -54,6 +54,7 @@ exportMethods( "repartition", "rightOuterJoin", "sampleRDD", + "sampleByKey", "saveAsTextFile", "saveAsObjectFile", "sortBy", diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 99e377cda4817..b07581c8ca922 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -262,6 +262,12 @@ setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) #' @export setGeneric("values", function(x) { standardGeneric("values") }) +#' @rdname sampleByKey +#' @export +setGeneric("sampleByKey", + function(x, withReplacement, fractions, seed) { + standardGeneric("sampleByKey") + }) ############ Shuffle Functions ############ diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index c2396c32a7548..08406b90f907e 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -450,19 +450,19 @@ setMethod("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 +#' 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 +#' @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. @@ -474,7 +474,7 @@ setMethod("combineByKey", #' 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) +#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) #' # list(list(1, list(3, 2)), list(2, list(7, 2))) #'} #' @rdname aggregateByKey @@ -491,12 +491,12 @@ setMethod("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 +#' 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. @@ -546,11 +546,11 @@ setMethod("join", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(FALSE, FALSE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)), doJoin) }) @@ -566,8 +566,8 @@ setMethod("join", #' @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)) +#' @return For each element (k, v) in x, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) #' if no elements in rdd2 have key k. #' @examples #'\dontrun{ @@ -584,11 +584,11 @@ setMethod("leftOuterJoin", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(FALSE, TRUE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) @@ -621,18 +621,18 @@ setMethod("rightOuterJoin", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(TRUE, FALSE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' Full outer join two RDDs #' #' @description -#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). +#' \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 @@ -642,7 +642,7 @@ setMethod("rightOuterJoin", #' @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 +#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements #' in x/y have key k. #' @examples #'\dontrun{ @@ -681,7 +681,7 @@ setMethod("fullOuterJoin", #' 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) +#' cogroup(rdd1, rdd2, numPartitions = 2L) #' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) #'} #' @rdname cogroup @@ -692,7 +692,7 @@ setMethod("cogroup", rdds <- list(...) rddsLen <- length(rdds) for (i in 1:rddsLen) { - rdds[[i]] <- lapply(rdds[[i]], + 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. @@ -721,7 +721,7 @@ setMethod("cogroup", } }) } - cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), + cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), group.func) }) @@ -743,18 +743,18 @@ setMethod("sortByKey", signature(x = "RDD"), function(x, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { rangeBounds <- list() - + if (numPartitions > 1) { rddSize <- count(x) # constant from Spark's RangePartitioner maxSampleSize <- numPartitions * 20 fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) - + samples <- collect(keys(sampleRDD(x, FALSE, fraction, 1L))) - + # Note: the built-in R sort() function only works on atomic vectors samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending) - + if (length(samples) > 0) { rangeBounds <- lapply(seq_len(numPartitions - 1), function(i) { @@ -766,24 +766,109 @@ 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 } - + if (ascending) { partition } else { numPartitions - partition - 1 } } - + partitionFunc <- function(part) { sortKeyValueList(part, decreasing = !ascending) } - + newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) lapplyPartition(newRDD, partitionFunc) }) - + +#' @description +#' \code{sampleByKey} return a subset RDD of the given RDD sampled by key +#' +#' @param x The RDD to sample elements by key, where each element is +#' list(K, V) or c(K, V). +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3000) +#' pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) +#' else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) +#' 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE +#' 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE +#' 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE +#' lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE +#' lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE +#' lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE +#' lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE +#' lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE +#' lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored +#' fractions <- list(a = 0.2, b = 0.1) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" +#'} +#' @rdname sampleByKey +#' @aliases sampleByKey,RDD-method +setMethod("sampleByKey", + signature(x = "RDD", withReplacement = "logical", + fractions = "vector", seed = "integer"), + function(x, withReplacement, fractions, seed) { + + for (elem in fractions) { + if (elem < 0.0) { + stop(paste("Negative fraction value ", fractions[which(fractions == elem)])) + } + } + + # The sampler: takes a partition and returns its sampled version. + samplingFunc <- function(split, part) { + set.seed(bitwXor(seed, split)) + res <- vector("list", length(part)) + len <- 0 + + # mixing because the initial seeds are close to each other + runif(10) + + for (elem in part) { + if (elem[[1]] %in% names(fractions)) { + frac <- as.numeric(fractions[which(elem[[1]] == names(fractions))]) + if (withReplacement) { + count <- rpois(1, frac) + if (count > 0) { + res[(len + 1):(len + count)] <- rep(list(elem), count) + len <- len + count + } + } else { + if (runif(1) < frac) { + len <- len + 1 + res[[len]] <- elem + } + } + } else { + stop("KeyError: \"", elem[[1]], "\"") + } + } + + # TODO(zongheng): look into the performance of the current + # implementation. Look into some iterator package? Note that + # Scala avoids many calls to creating an empty list and PySpark + # similarly achieves this using `yield'. (duplicated from sampleRDD) + if (len > 0) { + res[1:len] + } else { + list() + } + } + + lapplyPartitionsWithIndex(x, samplingFunc) + }) diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index fc0607ae1de60..c2acf15514f54 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -35,7 +35,7 @@ test_that("get number of partitions in RDD", { test_that("first on RDD", { expect_true(first(rdd) == 1) newrdd <- lapply(rdd, function(x) x + 1) - expect_true(first(newrdd) == 2) + expect_true(first(newrdd) == 2) }) test_that("count and length on RDD", { @@ -48,7 +48,7 @@ test_that("count by values and keys", { actual <- countByValue(mods) expected <- list(list(0, 3L), list(1, 4L), list(2, 3L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - + actual <- countByKey(intRdd) expected <- list(list(2L, 2L), list(1L, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -82,11 +82,11 @@ test_that("filterRDD on RDD", { filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 }) actual <- collect(filtered.rdd) expect_equal(actual, list(2, 4, 6, 8, 10)) - + filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) actual <- collect(filtered.rdd) expect_equal(actual, list(list(1L, -1))) - + # Filter out all elements. filtered.rdd <- filterRDD(rdd, function(x) { x > 10 }) actual <- collect(filtered.rdd) @@ -96,7 +96,7 @@ test_that("filterRDD on RDD", { test_that("lookup on RDD", { vals <- lookup(intRdd, 1L) expect_equal(vals, list(-1, 200)) - + vals <- lookup(intRdd, 3L) expect_equal(vals, list()) }) @@ -110,7 +110,7 @@ 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, + expected <- list(24, 24, 24, 24, 24, 168, 170, 172, 174, 176) expect_equal(actual, expected) }) @@ -247,10 +247,10 @@ test_that("flatMapValues() on pairwise RDDs", { l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) actual <- collect(flatMapValues(l, function(x) { x })) expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4))) - + # Generate x to x+1 for every value actual <- collect(flatMapValues(intRdd, function(x) { x:(x + 1) })) - expect_equal(actual, + expect_equal(actual, list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101), list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201))) }) @@ -347,7 +347,7 @@ test_that("top() on RDDs", { 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) @@ -357,7 +357,7 @@ test_that("top() on RDDs", { 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) @@ -370,7 +370,7 @@ test_that("aggregateRDD() on RDDs", { 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)) @@ -379,13 +379,13 @@ test_that("aggregateRDD() on RDDs", { 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), + 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), + expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) }) @@ -393,13 +393,13 @@ test_that("zipWithUniqueId() on RDDs", { 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), + 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), + expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) }) @@ -426,12 +426,12 @@ test_that("pipeRDD() on RDDs", { actual <- collect(pipeRDD(rdd, "more")) expected <- as.list(as.character(1:10)) expect_equal(actual, expected) - + trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n")) actual <- collect(pipeRDD(trailed.rdd, "sort")) expected <- list("", "1", "2", "3") expect_equal(actual, expected) - + rev.nums <- 9:0 rev.rdd <- parallelize(sc, rev.nums, 2L) actual <- collect(pipeRDD(rev.rdd, "sort")) @@ -445,11 +445,11 @@ test_that("zipRDD() on RDDs", { actual <- collect(zipRDD(rdd1, rdd2)) expect_equal(actual, list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) - + mockFile = c("Spark is pretty.", "Spark is awesome.") fileName <- tempfile(pattern="spark-test", fileext=".tmp") writeLines(mockFile, fileName) - + rdd <- textFile(sc, fileName, 1) actual <- collect(zipRDD(rdd, rdd)) expected <- lapply(mockFile, function(x) { list(x ,x) }) @@ -464,7 +464,7 @@ test_that("zipRDD() on RDDs", { actual <- collect(zipRDD(rdd, rdd1)) expected <- lapply(mockFile, function(x) { list(x, x) }) expect_equal(actual, expected) - + unlink(fileName) }) @@ -611,9 +611,9 @@ test_that("sortByKey() on pairwise RDDs", { 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) @@ -638,7 +638,7 @@ test_that("sortByKey() on pairwise RDDs", { rdd7 <- parallelize(sc, l3, 2L) sortedRdd7 <- sortByKey(rdd7) actual <- collect(sortedRdd7) - expect_equal(actual, l3) + expect_equal(actual, l3) }) test_that("collectAsMap() on a pairwise RDD", { @@ -649,12 +649,36 @@ test_that("collectAsMap() on a pairwise RDD", { 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")) }) + +test_that("sampleByKey() on pairwise RDDs", { + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list("a", x) else list("b", x) }) + fractions <- list(a = 0.2, b = 0.1) + sample <- sampleByKey(pairsRDD, FALSE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")), TRUE) + expect_equal(50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")), TRUE) + expect_equal(lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0, TRUE) + expect_equal(lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000, TRUE) + expect_equal(lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0, TRUE) + expect_equal(lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000, TRUE) + + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list(2, x) else list(3, x) }) + fractions <- list(`2` = 0.2, `3` = 0.1) + sample <- sampleByKey(pairsRDD, TRUE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, 2)) && 300 > length(lookup(sample, 2)), TRUE) + expect_equal(50 < length(lookup(sample, 3)) && 150 > length(lookup(sample, 3)), TRUE) + expect_equal(lookup(sample, 2)[which.min(lookup(sample, 2))] >= 0, TRUE) + expect_equal(lookup(sample, 2)[which.max(lookup(sample, 2))] <= 2000, TRUE) + expect_equal(lookup(sample, 3)[which.min(lookup(sample, 3))] >= 0, TRUE) + expect_equal(lookup(sample, 3)[which.max(lookup(sample, 3))] <= 2000, TRUE) +}) From 40338a47b6ec2fc46940e5001ff0671decba3e2e Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Tue, 7 Apr 2015 23:14:41 -0700 Subject: [PATCH 10/33] Merge pull request #244 from sun-rui/SPARKR-154_5 [SPARKR-154] Phase 4: implement subtract() and subtractByKey(). --- R/pkg/NAMESPACE | 2 ++ R/pkg/R/RDD.R | 28 +++++++++++++++++++ R/pkg/R/generics.R | 20 +++++++++++-- R/pkg/R/pairRDD.R | 34 ++++++++++++++++++++++ R/pkg/inst/tests/test_rdd.R | 56 +++++++++++++++++++++++++++++++++++++ pkg/man/subtract.Rd | 36 ++++++++++++++++++++++++ pkg/man/subtractByKey.Rd | 37 ++++++++++++++++++++++++ 7 files changed, 210 insertions(+), 3 deletions(-) create mode 100644 pkg/man/subtract.Rd create mode 100644 pkg/man/subtractByKey.Rd diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index cac2fa2efb6ca..3ede0abe4467f 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -59,6 +59,8 @@ exportMethods( "saveAsObjectFile", "sortBy", "sortByKey", + "subtract", + "subtractByKey", "sumRDD", "take", "takeOrdered", diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 125ea0110001a..d3d7d6478536b 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -1561,6 +1561,34 @@ setMethod("zipRDD", PipelinedRDD(zippedRDD, partitionFunc) }) +#' Subtract an RDD with another RDD. +#' +#' Return an RDD with the elements from this that are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the elements from this that are not in other. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) +#' rdd2 <- parallelize(sc, list(2, 4)) +#' collect(subtract(rdd1, rdd2)) +#' # list(1, 1, 3) +#'} +#' @rdname subtract +#' @aliases subtract,RDD +setMethod("subtract", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + mapFunction <- function(e) { list(e, NA) } + rdd1 <- map(x, mapFunction) + rdd2 <- map(other, mapFunction) + + keys(subtractByKey(rdd1, rdd2, numPartitions)) + }) + #' Intersection of this RDD and another one. #' #' Return the intersection of this RDD and another one. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index b07581c8ca922..ec613d488dbb3 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -344,10 +344,24 @@ setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("ri #' @rdname sortByKey #' @export -setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1L) { - standardGeneric("sortByKey") -}) +setGeneric("sortByKey", + function(x, ascending = TRUE, numPartitions = 1L) { + standardGeneric("sortByKey") + }) + +#' @rdname subtractByKey +#' @export +setGeneric("subtractByKey", + function(x, other, numPartitions = 1L) { + standardGeneric("subtractByKey") + }) +#' @rdname subtract +#' @export +setGeneric("subtract", + function(x, other, numPartitions = 1L) { + standardGeneric("subtract") + }) ################### Broadcast Variable Methods ################# diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 08406b90f907e..ca604707df67c 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -786,6 +786,40 @@ setMethod("sortByKey", newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) lapplyPartition(newRDD, partitionFunc) }) + +#' Subtract a pair RDD with another pair RDD. +#' +#' Return an RDD with the pairs from x whose keys are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the pairs from x whose keys are not in other. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), +#' list("b", 5), list("a", 2))) +#' rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) +#' collect(subtractByKey(rdd1, rdd2)) +#' # list(list("b", 4), list("b", 5)) +#'} +#' @rdname subtractByKey +#' @aliases subtractByKey,RDD +setMethod("subtractByKey", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + filterFunction <- function(elem) { + iters <- elem[[2]] + (length(iters[[1]]) > 0) && (length(iters[[2]]) == 0) + } + + flatMapValues(filterRDD(cogroup(x, + other, + numPartitions = numPartitions), + filterFunction), + function (v) { v[[1]] }) + }) #' @description #' \code{sampleByKey} return a subset RDD of the given RDD sampled by key diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index c2acf15514f54..cd1cb931c4e21 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -468,6 +468,62 @@ test_that("zipRDD() on RDDs", { unlink(fileName) }) +test_that("subtract() on RDDs", { + l <- list(1, 1, 2, 2, 3, 4) + rdd1 <- parallelize(sc, l) + + # subtract by itself + actual <- collect(subtract(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtract by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + l) + + rdd2 <- parallelize(sc, list(2, 4)) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + list(1, 1, 3)) + + l <- list("a", "a", "b", "b", "c", "d") + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list("b", "d")) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="character"))), + list("a", "a", "c")) +}) + +test_that("subtractByKey() on pairwise RDDs", { + l <- list(list("a", 1), list("b", 4), + list("b", 5), list("a", 2)) + rdd1 <- parallelize(sc, l) + + # subtractByKey by itself + actual <- collect(subtractByKey(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtractByKey by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(l)) + + rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list("b", 4), list("b", 5))) + + l <- list(list(1, 1), list(2, 4), + list(2, 5), list(1, 2)) + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list(list(1, 3), list(3, 1))) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list(2, 4), list(2, 5))) +}) + test_that("intersection() on RDDs", { # intersection with self actual <- collect(intersection(rdd, rdd)) diff --git a/pkg/man/subtract.Rd b/pkg/man/subtract.Rd new file mode 100644 index 0000000000000..3c289d57b29e3 --- /dev/null +++ b/pkg/man/subtract.Rd @@ -0,0 +1,36 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{subtract,RDD,RDD-method} +\alias{subtract} +\alias{subtract,RDD} +\alias{subtract,RDD,RDD-method} +\title{Subtract an RDD with another RDD.} +\usage{ +\S4method{subtract}{RDD,RDD}(x, other, + numPartitions = SparkR::numPartitions(x)) + +subtract(x, other, numPartitions = 1L) +} +\arguments{ +\item{x}{An RDD.} + +\item{other}{An RDD.} + +\item{numPartitions}{Number of the partitions in the result RDD.} +} +\value{ +An RDD with the elements from this that are not in other. +} +\description{ +Return an RDD with the elements from this that are not in other. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) +rdd2 <- parallelize(sc, list(2, 4)) +collect(subtract(rdd1, rdd2)) +# list(1, 1, 3) +} +} + diff --git a/pkg/man/subtractByKey.Rd b/pkg/man/subtractByKey.Rd new file mode 100644 index 0000000000000..4dd90514b4784 --- /dev/null +++ b/pkg/man/subtractByKey.Rd @@ -0,0 +1,37 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{subtractByKey} +\alias{subtractByKey} +\alias{subtractByKey,RDD} +\alias{subtractByKey,RDD,RDD-method} +\title{Subtract a pair RDD with another pair RDD.} +\usage{ +subtractByKey(x, other, numPartitions = 1L) + +\S4method{subtractByKey}{RDD,RDD}(x, other, + numPartitions = SparkR::numPartitions(x)) +} +\arguments{ +\item{x}{An RDD.} + +\item{other}{An RDD.} + +\item{numPartitions}{Number of the partitions in the result RDD.} +} +\value{ +An RDD with the pairs from x whose keys are not in other. +} +\description{ +Return an RDD with the pairs from x whose keys are not in other. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), + list("b", 5), list("a", 2))) +rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) +collect(subtractByKey(rdd1, rdd2)) +# list(list("b", 4), list("b", 5)) +} +} + From be5d5c1f6b875f7d59c14a55fdf9a907bdcdfc81 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Wed, 8 Apr 2015 14:09:53 -0500 Subject: [PATCH 11/33] refactor schema functions Refactored `structType` and `structField` so that they can be used to create schemas from R for use with `createDataFrame`. Moved everything to `schema.R` Added new methods to `SQLUtils.scala` for handling `StructType` and `StructField` on the JVM side --- R/pkg/R/SQLContext.R | 97 ---------- pkg/R/schema.R | 169 ++++++++++++++++++ .../org/apache/spark/sql/api/r/SQLUtils.scala | 31 +++- 3 files changed, 197 insertions(+), 100 deletions(-) create mode 100644 pkg/R/schema.R diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 089a9e4e970e7..30fb0f8b5a64a 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -65,30 +65,6 @@ infer_type <- function(x) { } } -#' dump the schema into JSON string -tojson <- function(x) { - if (inherits(x, "struct")) { - # schema object - l <- paste(lapply(x, tojson), collapse = ", ") - paste('{\"type\":\"struct\", \"fields\":','[', l, ']}', sep = '') - } else if (inherits(x, "field")) { - # field object - names <- names(x) - items <- lapply(names, function(n) { - safe_n <- gsub('"', '\\"', n) - paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') - }) - d <- paste(items, collapse = ", ") - paste('{', d, '}', sep = '') - } else if (is.character(x)) { - paste('"', x, '"', sep = '') - } else if (is.logical(x)) { - if (x) "true" else "false" - } else { - stop(paste("unexpected type:", class(x))) - } -} - #' Create a DataFrame from an RDD #' #' Converts an RDD to a DataFrame by infer the types. @@ -516,76 +492,3 @@ createExternalTable <- function(sqlCtx, tableName, path = NULL, source = NULL, . sdf <- callJMethod(sqlCtx, "createExternalTable", tableName, source, options) dataFrame(sdf) } - -#' Create a Schema object -#' -#' Create an object of type "struct" that contains the metadata for a DataFrame. Intended for -#' use with createDataFrame and toDF. -#' -#' @param field a Field object (created with the field() function) -#' @param ... additional Field objects -#' @return a Schema object -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) -#' schema <- buildSchema(field("a", "integer"), field("b", "string")) -#' df <- createDataFrame(sqlCtx, rdd, schema) -#' } -buildSchema <- function(field, ...) { - fields <- list(field, ...) - if (!all(sapply(fields, inherits, "field"))) { - stop("All arguments must be Field objects.") - } - - structure(fields, class = "struct") -} - -# print method for "struct" object -print.struct <- function(x, ...) { - cat(sapply(x, function(field) { paste("|-", "name = \"", field$name, - "\", type = \"", field$type, - "\", nullable = ", field$nullable, "\n", - sep = "") }) - , sep = "") -} - -#' Create a Field object -#' -#' Create a Field object that contains the metadata for a single field in a schema. -#' -#' @param name The name of the field -#' @param type The data type of the field -#' @param nullable A logical vector indicating whether or not the field is nullable -#' @return a Field object -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) -#' field1 <- field("a", "integer", TRUE) -#' field2 <- field("b", "string", TRUE) -#' schema <- buildSchema(field1, field2) -#' df <- createDataFrame(sqlCtx, rdd, schema) -#' } -field <- function(name, type, nullable = TRUE) { - if (class(name) != "character") { - stop("Field name must be a string.") - } - if (class(type) != "character") { - stop("Field type must be a string.") - } - if (class(nullable) != "logical") { - stop("nullable must be either TRUE or FALSE") - } - structure(list("name" = name, "type" = type, "nullable" = nullable), class = "field") -} - -# print method for Field objects -print.field <- function(x, ...) { - cat("name = \"", x$name, "\", type = \"", x$type, "\", nullable = ", x$nullable, sep = "") -} - diff --git a/pkg/R/schema.R b/pkg/R/schema.R new file mode 100644 index 0000000000000..2f6221d9a2ffb --- /dev/null +++ b/pkg/R/schema.R @@ -0,0 +1,169 @@ +#' structType +#' +#' Create a structType object that contains the metadata for a DataFrame. Intended for +#' use with createDataFrame and toDF. +#' +#' @param x a Field object (created with the field() function) +#' @param ... additional Field objects +#' @return a structType object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' schema <- buildSchema(field("a", "integer"), field("b", "string")) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } +structType <- function(x, ...) { + UseMethod("structType", x) +} + +structType.jobj <- function(x) { + obj <- structure(list(), class = "structType") + obj$jobj <- x + obj$fields <- function() { lapply(callJMethod(x, "fields"), structField) } + obj +} + +structType.structField <- function(x, ...) { + fields <- list(x, ...) + if (!all(sapply(fields, inherits, "structField"))) { + stop("All arguments must be structField objects.") + } + sfObjList <- lapply(fields, function(field) { + field$jobj + }) + stObj <- callJStatic("edu.berkeley.cs.amplab.sparkr.SQLUtils", + "createStructType", + listToSeq(sfObjList)) + structType(stObj) +} + +#' Print a Spark StructType. +#' +#' This function prints the contents of a StructType returned from the +#' SparkR JVM backend. +#' +#' @param x A StructType object +#' @param ... further arguments passed to or from other methods +print.structType <- function(x, ...) { + cat("StructType\n", + sapply(x$fields(), function(field) { paste("|-", "name = \"", field$name(), + "\", type = \"", field$dataType.toString(), + "\", nullable = ", field$nullable(), "\n", + sep = "") }) + , sep = "") +} + +#' structField +#' +#' Create a structField object that contains the metadata for a single field in a schema. +#' +#' @param x The name of the field +#' @param type The data type of the field +#' @param nullable A logical vector indicating whether or not the field is nullable +#' @return a Field object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' field1 <- field("a", "integer", TRUE) +#' field2 <- field("b", "string", TRUE) +#' schema <- buildSchema(field1, field2) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } + +structField <- function(x, ...) { + UseMethod("structField", x) +} + +structField.jobj <- function(x) { + obj <- structure(list(), class = "structField") + obj$jobj <- x + obj$name <- function() { callJMethod(x, "name") } + obj$dataType <- function() { callJMethod(x, "dataType") } + obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } + obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } + obj$nullable <- function() { callJMethod(x, "nullable") } + obj +} + +structField.character <- function(x, type, nullable = TRUE) { + if (class(x) != "character") { + stop("Field name must be a string.") + } + if (class(type) != "character") { + stop("Field type must be a string.") + } + if (class(nullable) != "logical") { + stop("nullable must be either TRUE or FALSE") + } + options <- c("byte", + "integer", + "double", + "numeric", + "character", + "string", + "binary", + "raw", + "logical", + "boolean", + "timestamp", + "date") + dataType <- if (type %in% options) { + type + } else { + stop(paste("Unsupported type for Dataframe:", type)) + } + sfObj <- callJStatic("edu.berkeley.cs.amplab.sparkr.SQLUtils", + "createStructField", + x, + dataType, + nullable) + structField(sfObj) +} + +#' Print a Spark StructField. +#' +#' This function prints the contents of a StructField returned from the +#' SparkR JVM backend. +#' +#' @param x A StructField object +#' @param ... further arguments passed to or from other methods +print.structField <- function(x, ...) { + cat("StructField(name = \"", x$name(), + "\", type = \"", x$dataType.toString(), + "\", nullable = ", x$nullable(), + ")", + sep = "") +} + +# cfreeman: Don't think we need this function since we can create +# structType in R and pass to createDataFrame +# +# #' dump the schema into JSON string +# tojson <- function(x) { +# if (inherits(x, "struct")) { +# # schema object +# l <- paste(lapply(x, tojson), collapse = ", ") +# paste('{\"type\":\"struct\", \"fields\":','[', l, ']}', sep = '') +# } else if (inherits(x, "field")) { +# # field object +# names <- names(x) +# items <- lapply(names, function(n) { +# safe_n <- gsub('"', '\\"', n) +# paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') +# }) +# d <- paste(items, collapse = ", ") +# paste('{', d, '}', sep = '') +# } else if (is.character(x)) { +# paste('"', x, '"', sep = '') +# } else if (is.logical(x)) { +# if (x) "true" else "false" +# } else { +# stop(paste("unexpected type:", class(x))) +# } +# } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index d1ea7cc3e9162..bd5080d5de283 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -23,7 +23,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame, GroupedData, Row, SQLContext, SaveMode} private[r] object SQLUtils { @@ -39,8 +39,33 @@ private[r] object SQLUtils { arr.toSeq } - def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = { - val schema = DataType.fromJson(schemaString).asInstanceOf[StructType] + def createStructType(fields : Seq[StructField]) : StructType = { + StructType(fields) + } + + def DataTypeObject(dataType: String): DataType = { + dataType match { + case "byte" => org.apache.spark.sql.types.ByteType + case "integer" => org.apache.spark.sql.types.IntegerType + case "double" => org.apache.spark.sql.types.DoubleType + case "numeric" => org.apache.spark.sql.types.DoubleType + case "character" => org.apache.spark.sql.types.StringType + case "string" => org.apache.spark.sql.types.StringType + case "binary" => org.apache.spark.sql.types.BinaryType + case "raw" => org.apache.spark.sql.types.BinaryType + case "logical" => org.apache.spark.sql.types.BooleanType + case "boolean" => org.apache.spark.sql.types.BooleanType + case "timestamp" => org.apache.spark.sql.types.TimestampType + case "date" => org.apache.spark.sql.types.DateType + case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + } + } + + def createStructField(name: String, dataType: String, nullable: Boolean): StructField = { + val dtObj = DataTypeObject(dataType) + StructField(name, dtObj, nullable) + } + val num = schema.fields.size val rowRDD = rdd.map(bytesToRow) sqlContext.createDataFrame(rowRDD, schema) From 836c4bf80cc20f32db403f6e5d82fd82c94bf055 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Wed, 8 Apr 2015 14:11:59 -0500 Subject: [PATCH 12/33] Update `createDataFrame` and `toDF` Refactored to use the new `structType` and `structField` functions. --- R/pkg/R/SQLContext.R | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 30fb0f8b5a64a..4f05ba524a01a 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -54,9 +54,9 @@ infer_type <- function(x) { # StructType types <- lapply(x, infer_type) fields <- lapply(1:length(x), function(i) { - field(names[[i]], types[[i]], TRUE) + structField(names[[i]], types[[i]], TRUE) }) - do.call(buildSchema, fields) + do.call(structType, fields) } } else if (length(x) > 1) { list(type = "array", elementType = type, containsNull = TRUE) @@ -110,7 +110,7 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { stop(paste("unexpected type:", class(data))) } - if (is.null(schema) || (!inherits(schema, "struct") && is.null(names(schema)))) { + if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { row <- first(rdd) names <- if (is.null(schema)) { names(row) @@ -135,18 +135,18 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { types <- lapply(row, infer_type) fields <- lapply(1:length(row), function(i) { - field(names[[i]], types[[i]], TRUE) + structField(names[[i]], types[[i]], TRUE) }) - schema <- do.call(buildSchema, fields) + schema <- do.call(structType, fields) } - stopifnot(class(schema) == "struct") - schemaString <- tojson(schema) + stopifnot(class(schema) == "structType") + # schemaString <- tojson(schema) jrdd <- getJRDD(lapply(rdd, function(x) x), "row") srdd <- callJMethod(jrdd, "rdd") sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", - srdd, schemaString, sqlCtx) + srdd, schema$jobj, sqlCtx) dataFrame(sdf) } From 1a3b63dff2d234946419f8ed09afe9150b3904c0 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Wed, 8 Apr 2015 14:12:50 -0500 Subject: [PATCH 13/33] new version of `CreateDF` New version uses takes a `StructType` from R and creates a DataFrame. Commented out the `tojson` version since we don't currently use it. --- .../main/scala/org/apache/spark/sql/api/r/SQLUtils.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index bd5080d5de283..e09bd25788310 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -66,6 +66,15 @@ private[r] object SQLUtils { StructField(name, dtObj, nullable) } + // cfreeman: I don't think we need this anymore since we can pass structType from R + // def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = { + // val schema = DataType.fromJson(schemaString).asInstanceOf[StructType] + // val num = schema.fields.size + // val rowRDD = rdd.map(bytesToRow) + // sqlContext.createDataFrame(rowRDD, schema) + // } + + def createDF(rdd: RDD[Array[Byte]], schema: StructType, sqlContext: SQLContext): DataFrame = { val num = schema.fields.size val rowRDD = rdd.map(bytesToRow) sqlContext.createDataFrame(rowRDD, schema) From 275deb419ed6fca21f41da91f2e7fa244d010f3f Mon Sep 17 00:00:00 2001 From: cafreeman Date: Wed, 8 Apr 2015 14:14:36 -0500 Subject: [PATCH 14/33] Update `NAMESPACE` and tests Updated `NAMESPACE`, `DESCRIPTION`, and unit tests for new schema functions. Deleted `SQLTypes.R` since everything has been moved to `schema.R`. --- R/pkg/DESCRIPTION | 2 +- R/pkg/NAMESPACE | 15 +++++--- R/pkg/R/SQLTypes.R | 64 -------------------------------- R/pkg/inst/tests/test_sparkSQL.R | 30 +++++++-------- 4 files changed, 26 insertions(+), 85 deletions(-) delete mode 100644 R/pkg/R/SQLTypes.R diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 1842b97d43651..accf2c72761bd 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -17,11 +17,11 @@ License: Apache License (== 2.0) Collate: 'generics.R' 'jobj.R' - 'SQLTypes.R' 'RDD.R' 'pairRDD.R' 'column.R' 'group.R' + 'schema.R' 'DataFrame.R' 'SQLContext.R' 'broadcast.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 3ede0abe4467f..339f71aacb2ba 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -175,8 +175,6 @@ export("cacheTable", "jsonRDD", "loadDF", "parquetFile", - "buildSchema", - "field", "sql", "table", "tableNames", @@ -184,7 +182,14 @@ export("cacheTable", "toDF", "uncacheTable") -export("print.structType", +export("sparkRSQL.init", + "sparkRHive.init") + +export("structField", + "structField.jobj", + "structField.character", "print.structField", - "print.struct", - "print.field") + "structType", + "structType.jobj", + "structType.structField", + "print.structField") diff --git a/R/pkg/R/SQLTypes.R b/R/pkg/R/SQLTypes.R deleted file mode 100644 index 962fba5b3cf03..0000000000000 --- a/R/pkg/R/SQLTypes.R +++ /dev/null @@ -1,64 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Utility functions for handling SparkSQL DataTypes. - -# Handler for StructType -structType <- function(st) { - obj <- structure(new.env(parent = emptyenv()), class = "structType") - obj$jobj <- st - obj$fields <- function() { lapply(callJMethod(st, "fields"), structField) } - obj -} - -#' Print a Spark StructType. -#' -#' This function prints the contents of a StructType returned from the -#' SparkR JVM backend. -#' -#' @param x A StructType object -#' @param ... further arguments passed to or from other methods -print.structType <- function(x, ...) { - fieldsList <- lapply(x$fields(), function(i) { i$print() }) - print(fieldsList) -} - -# Handler for StructField -structField <- function(sf) { - obj <- structure(new.env(parent = emptyenv()), class = "structField") - obj$jobj <- sf - obj$name <- function() { callJMethod(sf, "name") } - obj$dataType <- function() { callJMethod(sf, "dataType") } - obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } - obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } - obj$nullable <- function() { callJMethod(sf, "nullable") } - obj$print <- function() { paste("StructField(", - paste(obj$name(), obj$dataType.toString(), obj$nullable(), sep = ", "), - ")", sep = "") } - obj -} - -#' Print a Spark StructField. -#' -#' This function prints the contents of a StructField returned from the -#' SparkR JVM backend. -#' -#' @param x A StructField object -#' @param ... further arguments passed to or from other methods -print.structField <- function(x, ...) { - cat(x$print()) -} diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index bf2101e276df6..c9ccec876393b 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -44,8 +44,8 @@ test_that("infer types", { expect_equal(infer_type(list(1L, 2L)), list(type = 'array', elementType = "integer", containsNull = TRUE)) expect_equal(infer_type(list(a = 1L, b = "2")), - buildSchema(field(name = "a", type = "integer", nullable = TRUE), - field(name = "b", type = "string", nullable = TRUE))) + structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE))) e <- new.env() assign("a", 1L, envir = e) expect_equal(infer_type(e), @@ -53,16 +53,16 @@ test_that("infer types", { valueContainsNull = TRUE)) }) -test_that("buildSchema and field", { - testField <- field("a", "string") - expect_true(inherits(testField, "field")) - expect_true(testField$name == "a") - expect_true(testField$nullable) +test_that("structType and structField", { + testField <- structField("a", "string") + expect_true(inherits(testField, "structField")) + expect_true(testField$name() == "a") + expect_true(testField$nullable()) - testSchema <- buildSchema(testField, field("b", "integer")) - expect_true(inherits(testSchema, "struct")) - expect_true(inherits(testSchema[[2]], "field")) - expect_true(testSchema[[1]]$type == "string") + testSchema <- structType(testField, structField("b", "integer")) + expect_true(inherits(testSchema, "structType")) + expect_true(inherits(testSchema$fields()[[2]], "structField")) + expect_true(testSchema$fields()[[1]]$dataType.toString() == "StringType") }) test_that("create DataFrame from RDD", { @@ -77,8 +77,8 @@ test_that("create DataFrame from RDD", { expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("_1", "_2")) - schema <- buildSchema(field(name = "a", type = "integer", nullable = TRUE), - field(name = "b", type = "string", nullable = TRUE)) + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) df <- createDataFrame(sqlCtx, rdd, schema) expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("a", "b")) @@ -104,8 +104,8 @@ test_that("toDF", { expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("_1", "_2")) - schema <- buildSchema(field(name = "a", type = "integer", nullable = TRUE), - field(name = "b", type = "string", nullable = TRUE)) + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) df <- toDF(rdd, schema) expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("a", "b")) From f3ba785ec9aeee8782d30a5e71fd7282a5554762 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Wed, 8 Apr 2015 14:21:17 -0500 Subject: [PATCH 15/33] Fixed duplicate export --- R/pkg/NAMESPACE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 339f71aacb2ba..6742fa15016d3 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -192,4 +192,4 @@ export("structField", "structType", "structType.jobj", "structType.structField", - "print.structField") + "print.structType") From ed66c81836ab4e1672cce77081e49804529af46c Mon Sep 17 00:00:00 2001 From: cafreeman Date: Wed, 8 Apr 2015 15:18:57 -0500 Subject: [PATCH 16/33] Update `subtract` to work with `generics.R` --- R/pkg/NAMESPACE | 1 - R/pkg/R/DataFrame.R | 9 ++++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 6742fa15016d3..9d4b7a4be6ca7 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -122,7 +122,6 @@ exportMethods("columns", "show", "showDF", "sortDF", - "subtract", "toJSON", "toRDD", "unionAll", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index feafd56909a67..c906ce9c01a20 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1159,10 +1159,13 @@ setMethod("intersect", #' df2 <- jsonFile(sqlCtx, path2) #' subtractDF <- subtract(df, df2) #' } + +#' @rdname subtract +#' @export setMethod("subtract", - signature(x = "DataFrame", y = "DataFrame"), - function(x, y) { - subtracted <- callJMethod(x@sdf, "except", y@sdf) + signature(x = "DataFrame", other = "DataFrame"), + function(x, other) { + subtracted <- callJMethod(x@sdf, "except", other@sdf) dataFrame(subtracted) }) From 7e8caa344a3095cdfa9e36dde4dae1bd5126516b Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 8 Apr 2015 21:02:35 -0700 Subject: [PATCH 17/33] Merge pull request #246 from hlin09/fixCombineByKey Fixes combineByKey --- R/pkg/R/pairRDD.R | 4 ++-- R/pkg/inst/tests/test_shuffle.R | 12 ++++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index ca604707df67c..0ba5429bcf08b 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -428,7 +428,7 @@ setMethod("combineByKey", pred <- function(item) exists(item$hash, keys) lapply(part, function(item) { - item$hash <- as.character(item[[1]]) + item$hash <- as.character(hashCode(item[[1]])) updateOrCreatePair(item, keys, combiners, pred, mergeValue, createCombiner) }) convertEnvsToList(keys, combiners) @@ -441,7 +441,7 @@ setMethod("combineByKey", pred <- function(item) exists(item$hash, keys) lapply(part, function(item) { - item$hash <- as.character(item[[1]]) + item$hash <- as.character(hashCode(item[[1]])) updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, identity) }) convertEnvsToList(keys, combiners) diff --git a/R/pkg/inst/tests/test_shuffle.R b/R/pkg/inst/tests/test_shuffle.R index d1da8232aea81..b58b6a1a62080 100644 --- a/R/pkg/inst/tests/test_shuffle.R +++ b/R/pkg/inst/tests/test_shuffle.R @@ -87,6 +87,18 @@ test_that("combineByKey for doubles", { expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) }) +test_that("combineByKey for characters", { + stringKeyRDD <- parallelize(sc, + list(list("max", 1L), list("min", 2L), + list("other", 3L), list("max", 4L)), 2L) + reduced <- combineByKey(stringKeyRDD, + function(x) { x }, "+", "+", 2L) + actual <- collect(reduced) + + expected <- list(list("max", 5L), list("min", 2L), list("other", 3L)) + 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))) From 07d0dbc1b52dfee78d07ea6f7e4c630309b32c5a Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 9 Apr 2015 12:44:50 +0800 Subject: [PATCH 18/33] [SPARKR-244] Fix test failure after integration of subtract() and subtractByKey() for RDD. --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 21 +++++++++++---------- R/pkg/inst/tests/test_sparkSQL.R | 8 ++++---- 3 files changed, 16 insertions(+), 14 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 9d4b7a4be6ca7..7a5d7848633d4 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -122,6 +122,7 @@ exportMethods("columns", "show", "showDF", "sortDF", + "except", "toJSON", "toRDD", "unionAll", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index c906ce9c01a20..ebe2e0c97dea0 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1141,15 +1141,15 @@ setMethod("intersect", dataFrame(intersected) }) -#' Subtract +#' except #' #' Return a new DataFrame containing rows in this DataFrame #' but not in another DataFrame. This is equivalent to `EXCEPT` in SQL. #' #' @param x A Spark DataFrame #' @param y A Spark DataFrame -#' @return A DataFrame containing the result of the subtract operation. -#' @rdname subtract +#' @return A DataFrame containing the result of the except operation. +#' @rdname except #' @export #' @examples #'\dontrun{ @@ -1157,16 +1157,17 @@ setMethod("intersect", #' sqlCtx <- sparkRSQL.init(sc) #' df1 <- jsonFile(sqlCtx, path) #' df2 <- jsonFile(sqlCtx, path2) -#' subtractDF <- subtract(df, df2) +#' exceptDF <- except(df, df2) #' } +setGeneric("except", function(x, y) { standardGeneric("except") }) -#' @rdname subtract +#' @rdname except #' @export -setMethod("subtract", - signature(x = "DataFrame", other = "DataFrame"), - function(x, other) { - subtracted <- callJMethod(x@sdf, "except", other@sdf) - dataFrame(subtracted) +setMethod("except", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + excepted <- callJMethod(x@sdf, "except", y@sdf) + dataFrame(excepted) }) #' Save the contents of the DataFrame to a data source diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index c9ccec876393b..25831ae2d9e18 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -644,7 +644,7 @@ test_that("isLocal()", { expect_false(isLocal(df)) }) -test_that("unionAll(), subtract(), and intersect() on a DataFrame", { +test_that("unionAll(), except(), and intersect() on a DataFrame", { df <- jsonFile(sqlCtx, jsonPath) lines <- c("{\"name\":\"Bob\", \"age\":24}", @@ -659,10 +659,10 @@ test_that("unionAll(), subtract(), and intersect() on a DataFrame", { expect_true(count(unioned) == 6) expect_true(first(unioned)$name == "Michael") - subtracted <- sortDF(subtract(df, df2), desc(df$age)) + excepted <- sortDF(except(df, df2), desc(df$age)) expect_true(inherits(unioned, "DataFrame")) - expect_true(count(subtracted) == 2) - expect_true(first(subtracted)$name == "Justin") + expect_true(count(excepted) == 2) + expect_true(first(excepted)$name == "Justin") intersected <- sortDF(intersect(df, df2), df$age) expect_true(inherits(unioned, "DataFrame")) From 40199eb0a50518cd7c935c85c5cd3a95e445ee41 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 8 Apr 2015 23:13:17 -0700 Subject: [PATCH 19/33] Move except into sorted position --- R/pkg/NAMESPACE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 7a5d7848633d4..fdeb5e8ba5489 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -99,6 +99,7 @@ exportClasses("DataFrame") exportMethods("columns", "distinct", "dtypes", + "except", "explain", "filter", "groupBy", @@ -122,7 +123,6 @@ exportMethods("columns", "show", "showDF", "sortDF", - "except", "toJSON", "toRDD", "unionAll", From 938740296c55ba2ad49a7d9724e62c6655e443b0 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 9 Apr 2015 00:07:43 -0700 Subject: [PATCH 20/33] fix style --- R/pkg/NAMESPACE | 2 +- R/pkg/R/RDD.R | 25 ++++++++++++------------- R/pkg/R/generics.R | 24 ++++++++++-------------- R/pkg/R/pairRDD.R | 2 +- R/pkg/inst/tests/test_rdd.R | 16 ++++++++-------- R/pkg/inst/tests/test_shuffle.R | 8 ++++---- 6 files changed, 36 insertions(+), 41 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index fdeb5e8ba5489..891ee7718f5e5 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -53,8 +53,8 @@ exportMethods( "reduceByKeyLocally", "repartition", "rightOuterJoin", - "sampleRDD", "sampleByKey", + "sampleRDD", "saveAsTextFile", "saveAsObjectFile", "sortBy", diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index d3d7d6478536b..5808568e1ff3a 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -1496,15 +1496,15 @@ setMethod("zipRDD", stop("Can only zip RDDs which have the same number of partitions.") } - if (getSerializedMode(x) != getSerializedMode(other) || + if (getSerializedMode(x) != getSerializedMode(other) || getSerializedMode(x) == "byte") { # Append the number of elements in each partition to that partition so that we can later # check if corresponding partitions of both RDDs have the same number of elements. # - # Note that this appending also serves the purpose of reserialization, because even if + # Note that this appending also serves the purpose of reserialization, because even if # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded # as a single byte array. For example, partitions of an RDD generated from partitionBy() - # may be encoded as multiple byte arrays. + # may be encoded as multiple byte arrays. appendLength <- function(part) { part[[length(part) + 1]] <- length(part) + 1 part @@ -1512,13 +1512,13 @@ setMethod("zipRDD", x <- lapplyPartition(x, appendLength) other <- lapplyPartition(other, appendLength) } - + zippedJRDD <- callJMethod(getJRDD(x), "zip", getJRDD(other)) # The zippedRDD's elements are of scala Tuple2 type. The serialized # flag Here is used for the elements inside the tuples. serializerMode <- getSerializedMode(x) zippedRDD <- RDD(zippedJRDD, serializerMode) - + partitionFunc <- function(split, part) { len <- length(part) if (len > 0) { @@ -1526,15 +1526,15 @@ setMethod("zipRDD", lengthOfValues <- part[[len]] lengthOfKeys <- part[[len - lengthOfValues]] stopifnot(len == lengthOfKeys + lengthOfValues) - + # check if corresponding partitions of both RDDs have the same number of elements. if (lengthOfKeys != lengthOfValues) { stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") } - + if (lengthOfKeys > 1) { keys <- part[1 : (lengthOfKeys - 1)] - values <- part[(lengthOfKeys + 1) : (len - 1)] + values <- part[(lengthOfKeys + 1) : (len - 1)] } else { keys <- list() values <- list() @@ -1557,7 +1557,7 @@ setMethod("zipRDD", part } } - + PipelinedRDD(zippedRDD, partitionFunc) }) @@ -1585,17 +1585,16 @@ setMethod("subtract", mapFunction <- function(e) { list(e, NA) } rdd1 <- map(x, mapFunction) rdd2 <- map(other, mapFunction) - keys(subtractByKey(rdd1, rdd2, numPartitions)) }) #' Intersection of this RDD and another one. #' #' Return the intersection of this RDD and another one. -#' The output will not contain any duplicate elements, +#' The output will not contain any duplicate elements, #' even if the input RDDs did. Performs a hash partition #' across the cluster. -#' Note that this method performs a shuffle internally. +#' Note that this method performs a shuffle internally. #' #' @param x An RDD. #' @param other An RDD. @@ -1616,7 +1615,7 @@ setMethod("intersection", function(x, other, numPartitions = SparkR::numPartitions(x)) { rdd1 <- map(x, function(v) { list(v, NA) }) rdd2 <- map(other, function(v) { list(v, NA) }) - + filterFunction <- function(elem) { iters <- elem[[2]] all(as.vector( diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index ec613d488dbb3..0b41c3af9c46c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -134,10 +134,6 @@ setGeneric("minimum", function(x) { standardGeneric("minimum") }) #' @export setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) -#' @rdname foreach -#' @export -setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) - #' @rdname name #' @export setGeneric("name", function(x) { standardGeneric("name") }) @@ -269,6 +265,10 @@ setGeneric("sampleByKey", standardGeneric("sampleByKey") }) +#' @rdname values +#' @export +setGeneric("values", function(x) { standardGeneric("values") }) + ############ Shuffle Functions ############ @@ -349,18 +349,18 @@ setGeneric("sortByKey", standardGeneric("sortByKey") }) -#' @rdname subtractByKey +#' @rdname subtract #' @export -setGeneric("subtractByKey", +setGeneric("subtract", function(x, other, numPartitions = 1L) { - standardGeneric("subtractByKey") + standardGeneric("subtract") }) -#' @rdname subtract +#' @rdname subtractByKey #' @export -setGeneric("subtract", +setGeneric("subtractByKey", function(x, other, numPartitions = 1L) { - standardGeneric("subtract") + standardGeneric("subtractByKey") }) ################### Broadcast Variable Methods ################# @@ -462,10 +462,6 @@ setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) #' @export setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") }) -#' @rdname subtract -#' @export -setGeneric("subtract", function(x, y) { standardGeneric("subtract") }) - #' @rdname tojson #' @export setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 0ba5429bcf08b..14585a96dc3d1 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -813,7 +813,7 @@ setMethod("subtractByKey", iters <- elem[[2]] (length(iters[[1]]) > 0) && (length(iters[[2]]) == 0) } - + flatMapValues(filterRDD(cogroup(x, other, numPartitions = numPartitions), diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index cd1cb931c4e21..5fe73f371f2ea 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -481,12 +481,12 @@ test_that("subtract() on RDDs", { actual <- collect(subtract(rdd1, rdd2)) expect_equal(as.list(sort(as.vector(actual, mode="integer"))), l) - + rdd2 <- parallelize(sc, list(2, 4)) actual <- collect(subtract(rdd1, rdd2)) expect_equal(as.list(sort(as.vector(actual, mode="integer"))), list(1, 1, 3)) - + l <- list("a", "a", "b", "b", "c", "d") rdd1 <- parallelize(sc, l) rdd2 <- parallelize(sc, list("b", "d")) @@ -499,21 +499,21 @@ test_that("subtractByKey() on pairwise RDDs", { l <- list(list("a", 1), list("b", 4), list("b", 5), list("a", 2)) rdd1 <- parallelize(sc, l) - + # subtractByKey by itself actual <- collect(subtractByKey(rdd1, rdd1)) expect_equal(actual, list()) - + # subtractByKey by an empty RDD rdd2 <- parallelize(sc, list()) actual <- collect(subtractByKey(rdd1, rdd2)) expect_equal(sortKeyValueList(actual), sortKeyValueList(l)) - + rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) actual <- collect(subtractByKey(rdd1, rdd2)) expect_equal(actual, - list(list("b", 4), list("b", 5))) + list(list("b", 4), list("b", 5))) l <- list(list(1, 1), list(2, 4), list(2, 5), list(1, 2)) @@ -528,12 +528,12 @@ test_that("intersection() on RDDs", { # intersection with self actual <- collect(intersection(rdd, rdd)) expect_equal(sort(as.integer(actual)), nums) - + # intersection with an empty RDD emptyRdd <- parallelize(sc, list()) actual <- collect(intersection(rdd, emptyRdd)) expect_equal(actual, list()) - + rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) actual <- collect(intersection(rdd1, rdd2)) diff --git a/R/pkg/inst/tests/test_shuffle.R b/R/pkg/inst/tests/test_shuffle.R index b58b6a1a62080..d7dedda553c56 100644 --- a/R/pkg/inst/tests/test_shuffle.R +++ b/R/pkg/inst/tests/test_shuffle.R @@ -88,13 +88,13 @@ test_that("combineByKey for doubles", { }) test_that("combineByKey for characters", { - stringKeyRDD <- parallelize(sc, - list(list("max", 1L), list("min", 2L), + stringKeyRDD <- parallelize(sc, + list(list("max", 1L), list("min", 2L), list("other", 3L), list("max", 4L)), 2L) - reduced <- combineByKey(stringKeyRDD, + reduced <- combineByKey(stringKeyRDD, function(x) { x }, "+", "+", 2L) actual <- collect(reduced) - + expected <- list(list("max", 5L), list("min", 2L), list("other", 3L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) }) From 141efd86a7043bea72642362a9c6ed5ff960ae75 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 9 Apr 2015 14:44:25 -0700 Subject: [PATCH 21/33] Merge pull request #245 from hqzizania/upstream Add Rd files for sampleByKey() of [SPARKR-163] and sumRDD() of [SPARKR-92] --- R/pkg/R/pairRDD.R | 5 +++- pkg/man/sampleByKey.Rd | 52 ++++++++++++++++++++++++++++++++++++++++++ pkg/man/sumRDD.Rd | 27 ++++++++++++++++++++++ 3 files changed, 83 insertions(+), 1 deletion(-) create mode 100644 pkg/man/sampleByKey.Rd create mode 100644 pkg/man/sumRDD.Rd diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 14585a96dc3d1..58c37860e1b31 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -821,8 +821,11 @@ setMethod("subtractByKey", function (v) { v[[1]] }) }) +#' Return a subset of this RDD sampled by key. +#' #' @description -#' \code{sampleByKey} return a subset RDD of the given RDD sampled by key +#' \code{sampleByKey} Create a sample of this RDD using variable sampling rates +#' for different keys as specified by fractions, a key to sampling rate map. #' #' @param x The RDD to sample elements by key, where each element is #' list(K, V) or c(K, V). diff --git a/pkg/man/sampleByKey.Rd b/pkg/man/sampleByKey.Rd new file mode 100644 index 0000000000000..5bda2316810b5 --- /dev/null +++ b/pkg/man/sampleByKey.Rd @@ -0,0 +1,52 @@ +% Generated by roxygen2 (4.1.0): do not edit by hand +% Please edit documentation in R/generics.R, R/pairRDD.R +\docType{methods} +\name{sampleByKey} +\alias{sampleByKey} +\alias{sampleByKey,RDD,logical,vector,integer-method} +\alias{sampleByKey,RDD-method} +\title{Return a subset of this RDD sampled by key.} +\usage{ +sampleByKey(x, withReplacement, fractions, seed) + +\S4method{sampleByKey}{RDD,logical,vector,integer}(x, withReplacement, + fractions, seed) +} +\arguments{ +\item{x}{The RDD to sample elements by key, where each element is +list(K, V) or c(K, V).} + +\item{withReplacement}{Sampling with replacement or not} + +\item{seed}{Randomness seed value} + +\item{fraction}{The (rough) sample target fraction} +} +\description{ +\code{sampleByKey} Create a sample of this RDD using variable sampling rates +for different keys as specified by fractions, a key to sampling rate map. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, 1:3000) +pairs <- lapply(rdd, function(x) { if (x \%\% 3 == 0) list("a", x) + else { if (x \%\% 3 == 1) list("b", x) else list("c", x) }}) +fractions <- list(a = 0.2, b = 0.1, c = 0.3) +sample <- sampleByKey(pairs, FALSE, fractions, 1618L) +100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE +50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE +200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE +lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE +lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE +lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE +lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE +lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE +lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE +fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) +sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored +fractions <- list(a = 0.2, b = 0.1) +sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" +} +} + diff --git a/pkg/man/sumRDD.Rd b/pkg/man/sumRDD.Rd new file mode 100644 index 0000000000000..fafa7d9be7ca8 --- /dev/null +++ b/pkg/man/sumRDD.Rd @@ -0,0 +1,27 @@ +% Generated by roxygen2 (4.1.0): do not edit by hand +% Please edit documentation in R/RDD.R, R/generics.R +\docType{methods} +\name{sumRDD,RDD-method} +\alias{sumRDD} +\alias{sumRDD,RDD} +\alias{sumRDD,RDD-method} +\title{Add up the elements in an RDD.} +\usage{ +\S4method{sumRDD}{RDD}(x) + +sumRDD(x) +} +\arguments{ +\item{x}{The RDD to add up the elements in} +} +\description{ +Add up the elements in an RDD. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, 1:10) +sumRDD(rdd) # 55 +} +} + From 7741d66943d81621898de496b00af3eb3e6968b7 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Fri, 10 Apr 2015 08:45:10 -0500 Subject: [PATCH 22/33] Rename the SQL DataType function --- .../src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index e09bd25788310..4cd0fdffdc926 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -43,7 +43,7 @@ private[r] object SQLUtils { StructType(fields) } - def DataTypeObject(dataType: String): DataType = { + def getSQLDataType(dataType: String): DataType = { dataType match { case "byte" => org.apache.spark.sql.types.ByteType case "integer" => org.apache.spark.sql.types.IntegerType @@ -62,7 +62,7 @@ private[r] object SQLUtils { } def createStructField(name: String, dataType: String, nullable: Boolean): StructField = { - val dtObj = DataTypeObject(dataType) + val dtObj = getSQLDataType(dataType) StructField(name, dtObj, nullable) } From 6ef5f2dd5ef7d408362698a2f01d0ba7b548a12f Mon Sep 17 00:00:00 2001 From: cafreeman Date: Fri, 10 Apr 2015 08:45:20 -0500 Subject: [PATCH 23/33] Fix spacing --- .../src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 4cd0fdffdc926..6532ae690aea4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -39,7 +39,7 @@ private[r] object SQLUtils { arr.toSeq } - def createStructType(fields : Seq[StructField]) : StructType = { + def createStructType(fields : Seq[StructField]): StructType = { StructType(fields) } From 8526d2e71c9cead1af549d2e5982f9ef34498e88 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Fri, 10 Apr 2015 08:45:41 -0500 Subject: [PATCH 24/33] Remove `tojson` functions --- pkg/R/schema.R | 27 ------------------- .../org/apache/spark/sql/api/r/SQLUtils.scala | 8 ------ 2 files changed, 35 deletions(-) diff --git a/pkg/R/schema.R b/pkg/R/schema.R index 2f6221d9a2ffb..7aad74a066c9d 100644 --- a/pkg/R/schema.R +++ b/pkg/R/schema.R @@ -140,30 +140,3 @@ print.structField <- function(x, ...) { ")", sep = "") } - -# cfreeman: Don't think we need this function since we can create -# structType in R and pass to createDataFrame -# -# #' dump the schema into JSON string -# tojson <- function(x) { -# if (inherits(x, "struct")) { -# # schema object -# l <- paste(lapply(x, tojson), collapse = ", ") -# paste('{\"type\":\"struct\", \"fields\":','[', l, ']}', sep = '') -# } else if (inherits(x, "field")) { -# # field object -# names <- names(x) -# items <- lapply(names, function(n) { -# safe_n <- gsub('"', '\\"', n) -# paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') -# }) -# d <- paste(items, collapse = ", ") -# paste('{', d, '}', sep = '') -# } else if (is.character(x)) { -# paste('"', x, '"', sep = '') -# } else if (is.logical(x)) { -# if (x) "true" else "false" -# } else { -# stop(paste("unexpected type:", class(x))) -# } -# } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 6532ae690aea4..ae77f72998a22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -66,14 +66,6 @@ private[r] object SQLUtils { StructField(name, dtObj, nullable) } - // cfreeman: I don't think we need this anymore since we can pass structType from R - // def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = { - // val schema = DataType.fromJson(schemaString).asInstanceOf[StructType] - // val num = schema.fields.size - // val rowRDD = rdd.map(bytesToRow) - // sqlContext.createDataFrame(rowRDD, schema) - // } - def createDF(rdd: RDD[Array[Byte]], schema: StructType, sqlContext: SQLContext): DataFrame = { val num = schema.fields.size val rowRDD = rdd.map(bytesToRow) From 71372d97ada89478eb8fa5018e032d3868be0698 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Fri, 10 Apr 2015 08:46:19 -0500 Subject: [PATCH 25/33] Update docs and examples --- pkg/R/schema.R | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/pkg/R/schema.R b/pkg/R/schema.R index 7aad74a066c9d..02adf0e8abf81 100644 --- a/pkg/R/schema.R +++ b/pkg/R/schema.R @@ -1,10 +1,13 @@ +# A set of S3 classes and methods that support the SparkSQL `StructType` and `StructField +# datatypes. These are used to create and interact with DataFrame schemas. + #' structType #' #' Create a structType object that contains the metadata for a DataFrame. Intended for #' use with createDataFrame and toDF. #' -#' @param x a Field object (created with the field() function) -#' @param ... additional Field objects +#' @param x a structField object (created with the field() function) +#' @param ... additional structField objects #' @return a structType object #' @export #' @examples @@ -12,7 +15,7 @@ #' sc <- sparkR.init() #' sqlCtx <- sparkRSQL.init(sc) #' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) -#' schema <- buildSchema(field("a", "integer"), field("b", "string")) +#' schema <- structType(structField("a", "integer"), structField("b", "string")) #' df <- createDataFrame(sqlCtx, rdd, schema) #' } structType <- function(x, ...) { @@ -63,16 +66,16 @@ print.structType <- function(x, ...) { #' @param x The name of the field #' @param type The data type of the field #' @param nullable A logical vector indicating whether or not the field is nullable -#' @return a Field object +#' @return a structField object #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlCtx <- sparkRSQL.init(sc) #' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) -#' field1 <- field("a", "integer", TRUE) -#' field2 <- field("b", "string", TRUE) -#' schema <- buildSchema(field1, field2) +#' field1 <- structField("a", "integer", TRUE) +#' field2 <- structField("b", "string", TRUE) +#' schema <- structType(field1, field2) #' df <- createDataFrame(sqlCtx, rdd, schema) #' } From 5a553e7cc576184459c0a6cbcfcfae4bb2c5af0b Mon Sep 17 00:00:00 2001 From: cafreeman Date: Fri, 10 Apr 2015 08:46:36 -0500 Subject: [PATCH 26/33] Use object attribute instead of argument --- pkg/R/schema.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/R/schema.R b/pkg/R/schema.R index 02adf0e8abf81..0a758f0cc671c 100644 --- a/pkg/R/schema.R +++ b/pkg/R/schema.R @@ -25,7 +25,7 @@ structType <- function(x, ...) { structType.jobj <- function(x) { obj <- structure(list(), class = "structType") obj$jobj <- x - obj$fields <- function() { lapply(callJMethod(x, "fields"), structField) } + obj$fields <- function() { lapply(callJMethod(obj$jobj, "fields"), structField) } obj } From 1bdcb63b0a3aff91f17f6072b91f3dd8e9709e27 Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Sat, 11 Apr 2015 11:36:02 -0700 Subject: [PATCH 27/33] Updates to README.md. --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c3afc4db9c63c..b4536f17698b2 100644 --- a/README.md +++ b/README.md @@ -9,6 +9,7 @@ and Spark Streaming for stream processing. +*NOTE: As of April 2015, SparkR has been [merged](https://github.com/apache/spark/pull/5096) into Apache Spark and is shipping in an upcoming release (1.4) due early summer 2015. This repo currently targets users using released versions of Spark. __This repo no longer accepts new pull requests, and they should now be submitted to [apache/spark](https://github.com/apache/spark); see [here](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) for some instructions.__* ## Online Documentation From ae78312dd4ad3163a1752c6577eeb1eebc30abb9 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 14 Apr 2015 12:39:54 -0700 Subject: [PATCH 28/33] Merge pull request #237 from sun-rui/SPARKR-154_3 [SPARKR-154] Phase 2: implement cartesian(). --- R/pkg/NAMESPACE | 1 + R/pkg/R/RDD.R | 98 ++++++++++++++----------------------- R/pkg/R/generics.R | 4 ++ R/pkg/R/utils.R | 80 ++++++++++++++++++++++++++++++ R/pkg/inst/tests/test_rdd.R | 43 ++++++++++++++++ pkg/man/cartesian.Rd | 33 +++++++++++++ 6 files changed, 197 insertions(+), 62 deletions(-) create mode 100644 pkg/man/cartesian.Rd diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 891ee7718f5e5..80283643861ac 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -5,6 +5,7 @@ exportMethods( "aggregateByKey", "aggregateRDD", "cache", + "cartesian", "checkpoint", "coalesce", "cogroup", diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 5808568e1ff3a..800657cd4574b 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -1496,69 +1496,43 @@ setMethod("zipRDD", stop("Can only zip RDDs which have the same number of partitions.") } - if (getSerializedMode(x) != getSerializedMode(other) || - getSerializedMode(x) == "byte") { - # Append the number of elements in each partition to that partition so that we can later - # check if corresponding partitions of both RDDs have the same number of elements. - # - # Note that this appending also serves the purpose of reserialization, because even if - # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded - # as a single byte array. For example, partitions of an RDD generated from partitionBy() - # may be encoded as multiple byte arrays. - appendLength <- function(part) { - part[[length(part) + 1]] <- length(part) + 1 - part - } - x <- lapplyPartition(x, appendLength) - other <- lapplyPartition(other, appendLength) - } - - zippedJRDD <- callJMethod(getJRDD(x), "zip", getJRDD(other)) - # The zippedRDD's elements are of scala Tuple2 type. The serialized - # flag Here is used for the elements inside the tuples. - serializerMode <- getSerializedMode(x) - zippedRDD <- RDD(zippedJRDD, serializerMode) - - partitionFunc <- function(split, part) { - len <- length(part) - if (len > 0) { - if (serializerMode == "byte") { - lengthOfValues <- part[[len]] - lengthOfKeys <- part[[len - lengthOfValues]] - stopifnot(len == lengthOfKeys + lengthOfValues) - - # check if corresponding partitions of both RDDs have the same number of elements. - if (lengthOfKeys != lengthOfValues) { - stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") - } - - if (lengthOfKeys > 1) { - keys <- part[1 : (lengthOfKeys - 1)] - values <- part[(lengthOfKeys + 1) : (len - 1)] - } else { - keys <- list() - values <- list() - } - } else { - # Keys, values must have same length here, because this has - # been validated inside the JavaRDD.zip() function. - keys <- part[c(TRUE, FALSE)] - values <- part[c(FALSE, TRUE)] - } - mapply( - function(k, v) { - list(k, v) - }, - keys, - values, - SIMPLIFY = FALSE, - USE.NAMES = FALSE) - } else { - part - } - } + rdds <- appendPartitionLengths(x, other) + jrdd <- callJMethod(getJRDD(rdds[[1]]), "zip", getJRDD(rdds[[2]])) + # The jrdd's elements are of scala Tuple2 type. The serialized + # flag here is used for the elements inside the tuples. + rdd <- RDD(jrdd, getSerializedMode(rdds[[1]])) + + mergePartitions(rdd, TRUE) + }) - PipelinedRDD(zippedRDD, partitionFunc) +#' Cartesian product of this RDD and another one. +#' +#' Return the Cartesian product of this RDD and another one, +#' that is, the RDD of all pairs of elements (a, b) where a +#' is in this and b is in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @return A new RDD which is the Cartesian product of these two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2) +#' sortByKey(cartesian(rdd, rdd)) +#' # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) +#'} +#' @rdname cartesian +#' @aliases cartesian,RDD,RDD-method +setMethod("cartesian", + signature(x = "RDD", other = "RDD"), + function(x, other) { + rdds <- appendPartitionLengths(x, other) + jrdd <- callJMethod(getJRDD(rdds[[1]]), "cartesian", getJRDD(rdds[[2]])) + # The jrdd's elements are of scala Tuple2 type. The serialized + # flag here is used for the elements inside the tuples. + rdd <- RDD(jrdd, getSerializedMode(rdds[[1]])) + + mergePartitions(rdd, FALSE) }) #' Subtract an RDD with another RDD. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 0b41c3af9c46c..7b77577ce0cb0 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -237,6 +237,10 @@ setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) #' @export setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) +#' @rdname cartesian +#' @export +setGeneric("cartesian", function(x, other) { standardGeneric("cartesian") }) + #' @rdname intersection #' @export setGeneric("intersection", function(x, other, numPartitions = 1L) { diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index c337fb0751e72..23305d3c67074 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -465,3 +465,83 @@ cleanClosure <- function(func, checkedFuncs = new.env()) { } func } + +# Append partition lengths to each partition in two input RDDs if needed. +# param +# x An RDD. +# Other An RDD. +# return value +# A list of two result RDDs. +appendPartitionLengths <- function(x, other) { + if (getSerializedMode(x) != getSerializedMode(other) || + getSerializedMode(x) == "byte") { + # Append the number of elements in each partition to that partition so that we can later + # know the boundary of elements from x and other. + # + # Note that this appending also serves the purpose of reserialization, because even if + # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded + # as a single byte array. For example, partitions of an RDD generated from partitionBy() + # may be encoded as multiple byte arrays. + appendLength <- function(part) { + len <- length(part) + part[[len + 1]] <- len + 1 + part + } + x <- lapplyPartition(x, appendLength) + other <- lapplyPartition(other, appendLength) + } + list (x, other) +} + +# Perform zip or cartesian between elements from two RDDs in each partition +# param +# rdd An RDD. +# zip A boolean flag indicating this call is for zip operation or not. +# return value +# A result RDD. +mergePartitions <- function(rdd, zip) { + serializerMode <- getSerializedMode(rdd) + partitionFunc <- function(split, part) { + len <- length(part) + if (len > 0) { + if (serializerMode == "byte") { + lengthOfValues <- part[[len]] + lengthOfKeys <- part[[len - lengthOfValues]] + stopifnot(len == lengthOfKeys + lengthOfValues) + + # For zip operation, check if corresponding partitions of both RDDs have the same number of elements. + if (zip && lengthOfKeys != lengthOfValues) { + stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") + } + + if (lengthOfKeys > 1) { + keys <- part[1 : (lengthOfKeys - 1)] + } else { + keys <- list() + } + if (lengthOfValues > 1) { + values <- part[(lengthOfKeys + 1) : (len - 1)] + } else { + values <- list() + } + + if (!zip) { + return(mergeCompactLists(keys, values)) + } + } else { + keys <- part[c(TRUE, FALSE)] + values <- part[c(FALSE, TRUE)] + } + mapply( + function(k, v) { list(k, v) }, + keys, + values, + SIMPLIFY = FALSE, + USE.NAMES = FALSE) + } else { + part + } + } + + PipelinedRDD(rdd, partitionFunc) +} diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index 5fe73f371f2ea..d84a8f175bcbc 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -468,6 +468,49 @@ test_that("zipRDD() on RDDs", { unlink(fileName) }) +test_that("cartesian() on RDDs", { + rdd <- parallelize(sc, 1:3) + actual <- collect(cartesian(rdd, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(1, 1), list(1, 2), list(1, 3), + list(2, 1), list(2, 2), list(2, 3), + list(3, 1), list(3, 2), list(3, 3))) + + # test case where one RDD is empty + emptyRdd <- parallelize(sc, list()) + actual <- collect(cartesian(rdd, emptyRdd)) + expect_equal(actual, list()) + + mockFile = c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + actual <- collect(cartesian(rdd, rdd)) + expected <- list( + list("Spark is awesome.", "Spark is pretty."), + list("Spark is awesome.", "Spark is awesome."), + list("Spark is pretty.", "Spark is pretty."), + list("Spark is pretty.", "Spark is awesome.")) + expect_equal(sortKeyValueList(actual), expected) + + rdd1 <- parallelize(sc, 0:1) + actual <- collect(cartesian(rdd1, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(0, "Spark is pretty."), + list(0, "Spark is awesome."), + list(1, "Spark is pretty."), + list(1, "Spark is awesome."))) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collect(cartesian(rdd, rdd1)) + expect_equal(sortKeyValueList(actual), expected) + + unlink(fileName) +}) + test_that("subtract() on RDDs", { l <- list(1, 1, 2, 2, 3, 4) rdd1 <- parallelize(sc, l) diff --git a/pkg/man/cartesian.Rd b/pkg/man/cartesian.Rd new file mode 100644 index 0000000000000..41a8590327278 --- /dev/null +++ b/pkg/man/cartesian.Rd @@ -0,0 +1,33 @@ +% Generated by roxygen2 (4.0.2): do not edit by hand +\docType{methods} +\name{cartesian,RDD,RDD-method} +\alias{cartesian} +\alias{cartesian,RDD,RDD-method} +\title{Cartesian product of this RDD and another one.} +\usage{ +\S4method{cartesian}{RDD,RDD}(x, other) + +cartesian(x, other) +} +\arguments{ +\item{x}{An RDD.} + +\item{other}{An RDD.} +} +\value{ +A new RDD which is the Cartesian product of these two RDDs. +} +\description{ +Return the Cartesian product of this RDD and another one, +that is, the RDD of all pairs of elements (a, b) where a +is in this and b is in other. +} +\examples{ +\dontrun{ +sc <- sparkR.init() +rdd <- parallelize(sc, 1:2) +sortByKey(cartesian(rdd, rdd)) +# list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) +} +} + From 41f81840b34febe44121ba130300dbfd0cb15b7a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 14 Apr 2015 13:29:28 -0700 Subject: [PATCH 29/33] rm man --- pkg/man/cartesian.Rd | 33 ------------------------- pkg/man/intersection.Rd | 38 ----------------------------- pkg/man/sampleByKey.Rd | 52 ---------------------------------------- pkg/man/subtract.Rd | 36 ---------------------------- pkg/man/subtractByKey.Rd | 37 ---------------------------- pkg/man/sumRDD.Rd | 27 --------------------- 6 files changed, 223 deletions(-) delete mode 100644 pkg/man/cartesian.Rd delete mode 100644 pkg/man/intersection.Rd delete mode 100644 pkg/man/sampleByKey.Rd delete mode 100644 pkg/man/subtract.Rd delete mode 100644 pkg/man/subtractByKey.Rd delete mode 100644 pkg/man/sumRDD.Rd diff --git a/pkg/man/cartesian.Rd b/pkg/man/cartesian.Rd deleted file mode 100644 index 41a8590327278..0000000000000 --- a/pkg/man/cartesian.Rd +++ /dev/null @@ -1,33 +0,0 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand -\docType{methods} -\name{cartesian,RDD,RDD-method} -\alias{cartesian} -\alias{cartesian,RDD,RDD-method} -\title{Cartesian product of this RDD and another one.} -\usage{ -\S4method{cartesian}{RDD,RDD}(x, other) - -cartesian(x, other) -} -\arguments{ -\item{x}{An RDD.} - -\item{other}{An RDD.} -} -\value{ -A new RDD which is the Cartesian product of these two RDDs. -} -\description{ -Return the Cartesian product of this RDD and another one, -that is, the RDD of all pairs of elements (a, b) where a -is in this and b is in other. -} -\examples{ -\dontrun{ -sc <- sparkR.init() -rdd <- parallelize(sc, 1:2) -sortByKey(cartesian(rdd, rdd)) -# list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) -} -} - diff --git a/pkg/man/intersection.Rd b/pkg/man/intersection.Rd deleted file mode 100644 index 7ec6debb3dc08..0000000000000 --- a/pkg/man/intersection.Rd +++ /dev/null @@ -1,38 +0,0 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand -\docType{methods} -\name{intersection,RDD,RDD-method} -\alias{intersection} -\alias{intersection,RDD} -\alias{intersection,RDD,RDD-method} -\title{Intersection of this RDD and another one.} -\usage{ -\S4method{intersection}{RDD,RDD}(x, other, - numPartitions = SparkR::numPartitions(x)) - -intersection(x, other, numPartitions = 1L) -} -\arguments{ -\item{x}{An RDD.} - -\item{other}{An RDD.} -} -\value{ -An RDD which is the intersection of these two RDDs. -} -\description{ -Return the intersection of this RDD and another one. -The output will not contain any duplicate elements, -even if the input RDDs did. Performs a hash partition -across the cluster. -Note that this method performs a shuffle internally. -} -\examples{ -\dontrun{ -sc <- sparkR.init() -rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) -rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) -collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) -# list(1, 2, 3) -} -} - diff --git a/pkg/man/sampleByKey.Rd b/pkg/man/sampleByKey.Rd deleted file mode 100644 index 5bda2316810b5..0000000000000 --- a/pkg/man/sampleByKey.Rd +++ /dev/null @@ -1,52 +0,0 @@ -% Generated by roxygen2 (4.1.0): do not edit by hand -% Please edit documentation in R/generics.R, R/pairRDD.R -\docType{methods} -\name{sampleByKey} -\alias{sampleByKey} -\alias{sampleByKey,RDD,logical,vector,integer-method} -\alias{sampleByKey,RDD-method} -\title{Return a subset of this RDD sampled by key.} -\usage{ -sampleByKey(x, withReplacement, fractions, seed) - -\S4method{sampleByKey}{RDD,logical,vector,integer}(x, withReplacement, - fractions, seed) -} -\arguments{ -\item{x}{The RDD to sample elements by key, where each element is -list(K, V) or c(K, V).} - -\item{withReplacement}{Sampling with replacement or not} - -\item{seed}{Randomness seed value} - -\item{fraction}{The (rough) sample target fraction} -} -\description{ -\code{sampleByKey} Create a sample of this RDD using variable sampling rates -for different keys as specified by fractions, a key to sampling rate map. -} -\examples{ -\dontrun{ -sc <- sparkR.init() -rdd <- parallelize(sc, 1:3000) -pairs <- lapply(rdd, function(x) { if (x \%\% 3 == 0) list("a", x) - else { if (x \%\% 3 == 1) list("b", x) else list("c", x) }}) -fractions <- list(a = 0.2, b = 0.1, c = 0.3) -sample <- sampleByKey(pairs, FALSE, fractions, 1618L) -100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE -50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE -200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE -lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE -lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE -lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE -lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE -lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE -lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE -fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) -sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored -fractions <- list(a = 0.2, b = 0.1) -sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" -} -} - diff --git a/pkg/man/subtract.Rd b/pkg/man/subtract.Rd deleted file mode 100644 index 3c289d57b29e3..0000000000000 --- a/pkg/man/subtract.Rd +++ /dev/null @@ -1,36 +0,0 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand -\docType{methods} -\name{subtract,RDD,RDD-method} -\alias{subtract} -\alias{subtract,RDD} -\alias{subtract,RDD,RDD-method} -\title{Subtract an RDD with another RDD.} -\usage{ -\S4method{subtract}{RDD,RDD}(x, other, - numPartitions = SparkR::numPartitions(x)) - -subtract(x, other, numPartitions = 1L) -} -\arguments{ -\item{x}{An RDD.} - -\item{other}{An RDD.} - -\item{numPartitions}{Number of the partitions in the result RDD.} -} -\value{ -An RDD with the elements from this that are not in other. -} -\description{ -Return an RDD with the elements from this that are not in other. -} -\examples{ -\dontrun{ -sc <- sparkR.init() -rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) -rdd2 <- parallelize(sc, list(2, 4)) -collect(subtract(rdd1, rdd2)) -# list(1, 1, 3) -} -} - diff --git a/pkg/man/subtractByKey.Rd b/pkg/man/subtractByKey.Rd deleted file mode 100644 index 4dd90514b4784..0000000000000 --- a/pkg/man/subtractByKey.Rd +++ /dev/null @@ -1,37 +0,0 @@ -% Generated by roxygen2 (4.0.2): do not edit by hand -\docType{methods} -\name{subtractByKey} -\alias{subtractByKey} -\alias{subtractByKey,RDD} -\alias{subtractByKey,RDD,RDD-method} -\title{Subtract a pair RDD with another pair RDD.} -\usage{ -subtractByKey(x, other, numPartitions = 1L) - -\S4method{subtractByKey}{RDD,RDD}(x, other, - numPartitions = SparkR::numPartitions(x)) -} -\arguments{ -\item{x}{An RDD.} - -\item{other}{An RDD.} - -\item{numPartitions}{Number of the partitions in the result RDD.} -} -\value{ -An RDD with the pairs from x whose keys are not in other. -} -\description{ -Return an RDD with the pairs from x whose keys are not in other. -} -\examples{ -\dontrun{ -sc <- sparkR.init() -rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), - list("b", 5), list("a", 2))) -rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) -collect(subtractByKey(rdd1, rdd2)) -# list(list("b", 4), list("b", 5)) -} -} - diff --git a/pkg/man/sumRDD.Rd b/pkg/man/sumRDD.Rd deleted file mode 100644 index fafa7d9be7ca8..0000000000000 --- a/pkg/man/sumRDD.Rd +++ /dev/null @@ -1,27 +0,0 @@ -% Generated by roxygen2 (4.1.0): do not edit by hand -% Please edit documentation in R/RDD.R, R/generics.R -\docType{methods} -\name{sumRDD,RDD-method} -\alias{sumRDD} -\alias{sumRDD,RDD} -\alias{sumRDD,RDD-method} -\title{Add up the elements in an RDD.} -\usage{ -\S4method{sumRDD}{RDD}(x) - -sumRDD(x) -} -\arguments{ -\item{x}{The RDD to add up the elements in} -} -\description{ -Add up the elements in an RDD. -} -\examples{ -\dontrun{ -sc <- sparkR.init() -rdd <- parallelize(sc, 1:10) -sumRDD(rdd) # 55 -} -} - From e74c04e0e992f1cd3b3c8bee419de4eb210151a0 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 14 Apr 2015 13:42:51 -0700 Subject: [PATCH 30/33] fix schema.R --- {pkg => R/pkg}/R/schema.R | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename {pkg => R/pkg}/R/schema.R (100%) diff --git a/pkg/R/schema.R b/R/pkg/R/schema.R similarity index 100% rename from pkg/R/schema.R rename to R/pkg/R/schema.R From b1fe460991f48b4a848dec9b45611b5560c26121 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 14 Apr 2015 13:45:11 -0700 Subject: [PATCH 31/33] fix conflict in README.md --- R/pkg/R/schema.R | 21 +++++++++++++++++++-- README.md | 1 - 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R index 0a758f0cc671c..e442119086b17 100644 --- a/R/pkg/R/schema.R +++ b/R/pkg/R/schema.R @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # A set of S3 classes and methods that support the SparkSQL `StructType` and `StructField # datatypes. These are used to create and interact with DataFrame schemas. @@ -37,7 +54,7 @@ structType.structField <- function(x, ...) { sfObjList <- lapply(fields, function(field) { field$jobj }) - stObj <- callJStatic("edu.berkeley.cs.amplab.sparkr.SQLUtils", + stObj <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createStructType", listToSeq(sfObjList)) structType(stObj) @@ -121,7 +138,7 @@ structField.character <- function(x, type, nullable = TRUE) { } else { stop(paste("Unsupported type for Dataframe:", type)) } - sfObj <- callJStatic("edu.berkeley.cs.amplab.sparkr.SQLUtils", + sfObj <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createStructField", x, dataType, diff --git a/README.md b/README.md index b4536f17698b2..c3afc4db9c63c 100644 --- a/README.md +++ b/README.md @@ -9,7 +9,6 @@ and Spark Streaming for stream processing. -*NOTE: As of April 2015, SparkR has been [merged](https://github.com/apache/spark/pull/5096) into Apache Spark and is shipping in an upcoming release (1.4) due early summer 2015. This repo currently targets users using released versions of Spark. __This repo no longer accepts new pull requests, and they should now be submitted to [apache/spark](https://github.com/apache/spark); see [here](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) for some instructions.__* ## Online Documentation From 168b7fea39f67fcc6a0c0d7913e64326655f745f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 14 Apr 2015 13:52:09 -0700 Subject: [PATCH 32/33] sort generics --- R/pkg/R/DataFrame.R | 2 -- R/pkg/R/generics.R | 18 ++++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ebe2e0c97dea0..6ecd35a26d026 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1159,8 +1159,6 @@ setMethod("intersect", #' df2 <- jsonFile(sqlCtx, path2) #' exceptDF <- except(df, df2) #' } -setGeneric("except", function(x, y) { standardGeneric("except") }) - #' @rdname except #' @export setMethod("except", diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 7b77577ce0cb0..6c6233390134c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -229,6 +229,11 @@ setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") } ############ Binary Functions ############# + +#' @rdname cartesian +#' @export +setGeneric("cartesian", function(x, other) { standardGeneric("cartesian") }) + #' @rdname countByKey #' @export setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) @@ -237,10 +242,6 @@ setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) #' @export setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) -#' @rdname cartesian -#' @export -setGeneric("cartesian", function(x, other) { standardGeneric("cartesian") }) - #' @rdname intersection #' @export setGeneric("intersection", function(x, other, numPartitions = 1L) { @@ -258,10 +259,6 @@ setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) #' @export setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) -#' @rdname values -#' @export -setGeneric("values", function(x) { standardGeneric("values") }) - #' @rdname sampleByKey #' @export setGeneric("sampleByKey", @@ -367,6 +364,7 @@ setGeneric("subtractByKey", standardGeneric("subtractByKey") }) + ################### Broadcast Variable Methods ################# #' @rdname broadcast @@ -389,6 +387,10 @@ setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) #' @export setGeneric("explain", function(x, ...) { standardGeneric("explain") }) +#' @rdname except +#' @export +setGeneric("except", function(x, y) { standardGeneric("except") }) + #' @rdname filter #' @export setGeneric("filter", function(x, condition) { standardGeneric("filter") }) From c2b09be4a465a85ad4d362e9def8139e6b16a05f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Apr 2015 13:30:24 -0700 Subject: [PATCH 33/33] SQLTypes -> schema --- R/pkg/DESCRIPTION | 3 +-- R/pkg/R/DataFrame.R | 2 +- R/pkg/R/column.R | 2 +- R/pkg/R/group.R | 2 +- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index aac10df22853b..1c1779a763c7e 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -19,10 +19,9 @@ Collate: 'jobj.R' 'RDD.R' 'pairRDD.R' - 'SQLTypes.R' + 'schema.R' 'column.R' 'group.R' - 'schema.R' 'DataFrame.R' 'SQLContext.R' 'backend.R' diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 21ef8323915fb..861fe1c78b0db 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -17,7 +17,7 @@ # DataFrame.R - DataFrame class and methods implemented in S4 OO classes -#' @include generics.R jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R +#' @include generics.R jobj.R schema.R RDD.R pairRDD.R column.R group.R NULL setOldClass("jobj") diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index b282001d8b6b5..95fb9ff0887b6 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -17,7 +17,7 @@ # Column Class -#' @include generics.R jobj.R SQLTypes.R +#' @include generics.R jobj.R schema.R NULL setOldClass("jobj") diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 855fbdfc7c4ca..02237b3672d6b 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -17,7 +17,7 @@ # group.R - GroupedData class and methods implemented in S4 OO classes -#' @include generics.R jobj.R SQLTypes.R column.R +#' @include generics.R jobj.R schema.R column.R NULL setOldClass("jobj")