From 19dcb2dccd5fca4e9014dac6fb57e38f56165530 Mon Sep 17 00:00:00 2001 From: NarineK Date: Sun, 1 May 2016 23:31:15 -0700 Subject: [PATCH 01/39] First commit gapply --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 59 ++++++++++++ R/pkg/R/generics.R | 4 + R/pkg/inst/tests/testthat/test_sparkSQL.R | 37 +++++++ .../sql/catalyst/plans/logical/object.scala | 96 ++++++++++++++----- .../scala/org/apache/spark/sql/Dataset.scala | 28 ++++++ .../spark/sql/KeyValueGroupedDataset.scala | 29 +++++- .../org/apache/spark/sql/api/r/SQLUtils.scala | 46 +++++++++ .../spark/sql/execution/SparkStrategies.scala | 4 + .../execution/r/MapPartitionsRWrapper.scala | 38 +++++++- 10 files changed, 315 insertions(+), 27 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 73f7c595f4437..2921af43fd4d2 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -60,6 +60,7 @@ exportMethods("arrange", "filter", "first", "freqItems", + "gapply", "group_by", "groupBy", "head", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 9e30fa0dbf26a..5526a00cad3fe 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1187,6 +1187,65 @@ setMethod("dapply", dataFrame(sdf) }) +#' gapply +#' +#' Apply a function to each group of a DataFrame. +#' +#' @param x A SparkDataFrame +#' @param func A function to be applied to each group partition specified by grouping +#' column of the SparkDataFrame. +#' The output of func is a data.frame. +#' @param schema The schema of the resulting DataFrame after the function is applied. +#' It must match the output of func. +#' @family SparkDataFrame functions +#' @rdname gapply +#' @name gapply +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame (sqlContext, iris) +#' gdf <- gapply(df, function(x) { x }, schema(df), "Petal_Width") +#' collect(gdf) +#' +#' Compute the square of the first columns and output it +#' +#' df <- createDataFrame ( +#' sqlContext, +#' list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), +#' c("a", "b", "c", "d")) +#' +#' schema <- structType(structField("result", "integer")) +#' df1 <- gapply( +#' df, +#' function(x) { +#' y <- x[1] * x[1] +#' }, +#' schema, "a") +#' collect(df1) +#' +#' result +#' ------ +#' 1 +#' 1 +#' 9 +#' +setMethod("gapply", + signature(x = "SparkDataFrame", func = "function", schema = "structType", + col = "character"), + function(x, func, schema, col) { + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "gapply", x@sdf, + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + schema$jobj, col) + dataFrame(sdf) + }) ############################## RDD Map Functions ################################## # All of the following functions mirror the existing RDD map functions, # # but allow for use with DataFrames by first converting to an RRDD before calling # diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index f936ea6039981..a4c1e71fd9492 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -450,6 +450,10 @@ setGeneric("covar_pop", function(col1, col2) {standardGeneric("covar_pop") }) #' @export setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) +#' @rdname gapply +#' @export +setGeneric("gapply", function(x, func, schema, col) { standardGeneric("gapply") }) + #' @rdname summary #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 5cf9dc405b169..2298492cc46bf 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2083,6 +2083,43 @@ test_that("dapply() on a DataFrame", { expect_identical(expected, result) }) +test_that("gapply() on a DataFrame", { + df <- createDataFrame ( + sqlContext, + list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), + c("a", "b", "c", "d")) + expected <- collect(df) + df1 <- gapply(df, function(x) { x }, schema(df), "a") + actual <- collect(df1) + expect_identical(actual, expected) + + # Add a boolean column + schema <- structType(structField("a", "integer"), structField("b", "double"), + structField("c", "string"), structField("d", "double"), + structField("e", "boolean")) + df2 <- gapply( + df, + function(x) { + y <- cbind(x, x[1] > 1) + }, + schema, "a") + actual <- collect(df2)$e + expected <- c(FALSE, FALSE, TRUE) + expect_identical(actual, expected) + + # remove columns + schema <- structType(structField("a", "integer")) + df3 <- gapply( + df, + function(x) { + y <- x[1] + }, + schema, "a") + actual <- collect(df3) + expected <- collect(select(df, "a")) + expect_identical(actual, expected) +}) + unlink(parquetPath) unlink(jsonPath) unlink(jsonPathNa) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 84339f439a666..fb14bc87028fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -120,40 +120,50 @@ case class MapPartitions( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer -object MapPartitionsInR { +/** Factory for constructing new `MapGroupsR` nodes. */ +object MapGroupsR { def apply( - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Broadcast[Object]], - schema: StructType, - encoder: ExpressionEncoder[Row], - child: LogicalPlan): LogicalPlan = { - val deserialized = CatalystSerde.deserialize(child, encoder) - val mapped = MapPartitionsInR( - func, - packageNames, - broadcastVars, - encoder.schema, - schema, - CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), - deserialized) - CatalystSerde.serialize(mapped, RowEncoder(schema)) + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + schema: StructType, + encoder: Expression, + keyEncoder: Expression, + rowEncoder: ExpressionEncoder[Row], + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + child: LogicalPlan): LogicalPlan = { + + val mapped = MapGroupsR( + func, + packageNames, + broadcastVars, + rowEncoder.schema, + schema, + UnresolvedDeserializer(keyEncoder, groupingAttributes), + UnresolvedDeserializer(encoder, dataAttributes), + groupingAttributes, + dataAttributes, + CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), + child) + CatalystSerde.serialize(mapped, RowEncoder(schema)) } } -/** - * A relation produced by applying a serialized R function `func` to each partition of the `child`. - * - */ -case class MapPartitionsInR( +case class MapGroupsR( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], inputSchema: StructType, outputSchema: StructType, + keyDeserializer: Expression, + valueDeserializer: Expression, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer { - override lazy val schema = outputSchema + child: LogicalPlan) extends UnaryNode with ObjectProducer{ + + override lazy val schema = outputSchema } object MapElements { @@ -256,6 +266,44 @@ case class MapGroups( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectProducer + +object MapPartitionsInR { + def apply( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + schema: StructType, + encoder: ExpressionEncoder[Row], + child: LogicalPlan): LogicalPlan = { + val deserialized = CatalystSerde.deserialize(child, encoder) + val mapped = MapPartitionsInR( + func, + packageNames, + broadcastVars, + encoder.schema, + schema, + CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), + deserialized) + CatalystSerde.serialize(mapped, RowEncoder(schema)) + } +} + +/** + * A relation produced by applying a serialized R function `func` to each partition of the `child`. + * + */ +case class MapPartitionsInR( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + outputSchema: StructType, + outputObjAttr: Attribute, + child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer { + override lazy val schema = outputSchema +} + + /** Factory for constructing new `CoGroup` nodes. */ object CoGroup { def apply[K : Encoder, L : Encoder, R : Encoder, OUT : Encoder]( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 08be94e8d4f12..c165ff38a0bd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1998,6 +1998,34 @@ class Dataset[T] private[sql]( MapPartitionsInR(func, packageNames, broadcastVars, schema, rowEncoder, logicalPlan)) } + /** + * Returns a new [[DataFrame]] which contains the aggregated result of applying + * [[func]] R function to each group + * + * @group func + * @since 2.0.0 + */ + def mapGroupPartitionsInR[K: Encoder]( + gfunc: T => K, + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + outputSchema: StructType): DataFrame = { + + val inputPlan = logicalPlan + val withGroupingKey = AppendColumns(gfunc, inputPlan) + val executed = sqlContext.executePlan(withGroupingKey) + + val kvdg = new KeyValueGroupedDataset( + encoderFor[K], + encoderFor[T], + executed, + inputPlan.output, + withGroupingKey.newColumns) + + kvdg.flatMapRGroups(func, packageNames, broadcastVars, outputSchema) + } + /** * :: Experimental :: * (Scala-specific) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 3a5ea19b8ad14..b9d380c54f325 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -21,10 +21,13 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function._ +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, OuterScopes} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateStruct} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.types.StructType /** * :: Experimental :: @@ -112,6 +115,30 @@ class KeyValueGroupedDataset[K, V] private[sql]( logicalPlan)) } + def flatMapRGroups( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + outputSchema: StructType): DataFrame = { + + val broadcastVarObj = broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]]) + val rowEncoder = vEncoder.asInstanceOf[ExpressionEncoder[Row]] + + Dataset.ofRows( + sparkSession, + MapGroupsR( + func, + packageNames, + broadcastVarObj, + outputSchema, + unresolvedVEncoder.deserializer, + unresolvedKEncoder.deserializer, + rowEncoder, + groupingAttributes, + dataAttributes, + logicalPlan)) + } + /** * Applies the given function to each group of data. For each unique group, the function will * be passed the group key and an iterator that contains all of the elements in the group. The 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 36173a49250b5..a4600dbcfdab2 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 @@ -155,6 +155,52 @@ private[sql] object SQLUtils { df.mapPartitionsInR(func, packageNames, bv, realSchema) } + /** + * The helper function for gapply() on R side. + */ + def gapply( + df: DataFrame, + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + schema: StructType, + col: String): DataFrame = { + + val realSchema = + if (schema == null) { + SERIALIZED_R_DATA_SCHEMA + } else { + schema + } + + val dfSchema = df.select(df(col)).schema + if (dfSchema.length == 0) throw new IllegalArgumentException(s"Invaid column name $col") + val dataType = dfSchema(0).dataType + + val sqlContext = df.sqlContext + import sqlContext.implicits._ + + dataType match { + case ByteType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Byte](col), + func, packageNames, broadcastVars, realSchema) + case IntegerType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Int](col), + func, packageNames, broadcastVars, realSchema) + case FloatType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Float](col), + func, packageNames, broadcastVars, realSchema) + case DoubleType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Double](col), + func, packageNames, broadcastVars, realSchema) + case StringType => df.mapGroupPartitionsInR((r: Row) => r.getAs[String](col), + func, packageNames, broadcastVars, realSchema) + case BinaryType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Array[Byte]](col), + func, packageNames, broadcastVars, realSchema) + case BooleanType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Boolean](col), + func, packageNames, broadcastVars, realSchema) + case TimestampType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Long](col), + func, packageNames, broadcastVars, realSchema) + case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + } + } + def dfToCols(df: DataFrame): Array[Array[Any]] = { val localDF: Array[Row] = df.collect() val numCols = df.columns.length diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 238334e26b45c..5bb521c161847 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -310,6 +310,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.MapPartitionsInR(f, p, b, is, os, objAttr, child) => execution.MapPartitionsExec( execution.r.MapPartitionsRWrapper(f, p, b, is, os), objAttr, planLater(child)) :: Nil + case logical.MapGroupsR(f, p, b, is, os, key, value, grouping, data, objAttr, child) => + execution.MapGroupsExec( + execution.r.MapGroupRWrapper(f, p, b, is, os), key, value, grouping, + data, objAttr, planLater(child)) :: Nil case logical.MapElements(f, objAttr, child) => execution.MapElementsExec(f, objAttr, planLater(child)) :: Nil case logical.AppendColumns(f, in, out, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index dc6f2ef371584..35d425234e3fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -24,6 +24,22 @@ import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.Row import org.apache.spark.sql.types.{BinaryType, StructField, StructType} +/** + * A function wrapper that applies the given R function to each partition of each group. + */ +private[sql] case class MapGroupRWrapper( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + outputSchema: StructType) extends ((Any, Iterator[Any]) => TraversableOnce[Any]) { + + def apply(key: Any, iter: Iterator[Any]): TraversableOnce[Any] = { + PartitionsRHelper.mapPartitionsRHelper(func, + packageNames, broadcastVars, inputSchema, outputSchema, iter) + } +} + /** * A function wrapper that applies the given R function to each partition. */ @@ -33,8 +49,26 @@ private[sql] case class MapPartitionsRWrapper( broadcastVars: Array[Broadcast[Object]], inputSchema: StructType, outputSchema: StructType) extends (Iterator[Any] => Iterator[Any]) { + def apply(iter: Iterator[Any]): Iterator[Any] = { - // If the content of current DataFrame is serialized R data? + PartitionsRHelper.mapPartitionsRHelper(func, + packageNames, broadcastVars, inputSchema, outputSchema, iter) + } +} + + +object PartitionsRHelper { + /** + * A helper function to run R UDFs on partitions + */ + private[sql] def mapPartitionsRHelper( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + outputSchema: StructType, + iter: Iterator[Any]): Iterator[Any] = { + // If the content of current DataFrame is serialized R data? val isSerializedRData = if (inputSchema == SERIALIZED_R_DATA_SCHEMA) true else false @@ -64,5 +98,5 @@ private[sql] case class MapPartitionsRWrapper( } else { outputIter.map { bytes => Row.fromSeq(Seq(bytes)) } } - } + } } From 9c5473f61257e84c3b888b3181d24a0e46646342 Mon Sep 17 00:00:00 2001 From: NarineK Date: Sun, 1 May 2016 23:50:02 -0700 Subject: [PATCH 02/39] Fixed Roxigen issue --- R/pkg/R/DataFrame.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 5526a00cad3fe..18f3f89337f33 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1228,7 +1228,7 @@ setMethod("dapply", #' 1 #' 1 #' 9 -#' +#'} setMethod("gapply", signature(x = "SparkDataFrame", func = "function", schema = "structType", col = "character"), From 66ca64e71a52cfb0f4a64993de8070fc486164d4 Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 00:24:41 -0700 Subject: [PATCH 03/39] Fix test cases --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 2298492cc46bf..4481c35de373b 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2092,7 +2092,7 @@ test_that("gapply() on a DataFrame", { df1 <- gapply(df, function(x) { x }, schema(df), "a") actual <- collect(df1) expect_identical(actual, expected) - + # Add a boolean column schema <- structType(structField("a", "integer"), structField("b", "double"), structField("c", "string"), structField("d", "double"), @@ -2106,7 +2106,7 @@ test_that("gapply() on a DataFrame", { actual <- collect(df2)$e expected <- c(FALSE, FALSE, TRUE) expect_identical(actual, expected) - + # remove columns schema <- structType(structField("a", "integer")) df3 <- gapply( From 6bc882baa7460d42945c9db2afe46928654881ae Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 01:05:19 -0700 Subject: [PATCH 04/39] fixed ordering for MapGroupsR and MapPartitionsInR --- .../sql/catalyst/plans/logical/object.scala | 118 +++++++++--------- 1 file changed, 58 insertions(+), 60 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index fb14bc87028fb..6ab911e2f9555 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -120,50 +120,40 @@ case class MapPartitions( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer -/** Factory for constructing new `MapGroupsR` nodes. */ -object MapGroupsR { +object MapPartitionsInR { def apply( - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Broadcast[Object]], - schema: StructType, - encoder: Expression, - keyEncoder: Expression, - rowEncoder: ExpressionEncoder[Row], - groupingAttributes: Seq[Attribute], - dataAttributes: Seq[Attribute], - child: LogicalPlan): LogicalPlan = { - - val mapped = MapGroupsR( - func, - packageNames, - broadcastVars, - rowEncoder.schema, - schema, - UnresolvedDeserializer(keyEncoder, groupingAttributes), - UnresolvedDeserializer(encoder, dataAttributes), - groupingAttributes, - dataAttributes, - CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), - child) - CatalystSerde.serialize(mapped, RowEncoder(schema)) + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + schema: StructType, + encoder: ExpressionEncoder[Row], + child: LogicalPlan): LogicalPlan = { + val deserialized = CatalystSerde.deserialize(child, encoder) + val mapped = MapPartitionsInR( + func, + packageNames, + broadcastVars, + encoder.schema, + schema, + CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), + deserialized) + CatalystSerde.serialize(mapped, RowEncoder(schema)) } } -case class MapGroupsR( +/** + * A relation produced by applying a serialized R function `func` to each partition of the `child`. + * + */ +case class MapPartitionsInR( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], inputSchema: StructType, outputSchema: StructType, - keyDeserializer: Expression, - valueDeserializer: Expression, - groupingAttributes: Seq[Attribute], - dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectProducer{ - - override lazy val schema = outputSchema + child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer { + override lazy val schema = outputSchema } object MapElements { @@ -266,43 +256,51 @@ case class MapGroups( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectProducer - -object MapPartitionsInR { +/** Factory for constructing new `MapGroupsR` nodes. */ +object MapGroupsR { def apply( - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Broadcast[Object]], - schema: StructType, - encoder: ExpressionEncoder[Row], - child: LogicalPlan): LogicalPlan = { - val deserialized = CatalystSerde.deserialize(child, encoder) - val mapped = MapPartitionsInR( - func, - packageNames, - broadcastVars, - encoder.schema, - schema, - CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), - deserialized) - CatalystSerde.serialize(mapped, RowEncoder(schema)) + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + schema: StructType, + encoder: Expression, + keyEncoder: Expression, + rowEncoder: ExpressionEncoder[Row], + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + child: LogicalPlan): LogicalPlan = { + + val mapped = MapGroupsR( + func, + packageNames, + broadcastVars, + rowEncoder.schema, + schema, + UnresolvedDeserializer(keyEncoder, groupingAttributes), + UnresolvedDeserializer(encoder, dataAttributes), + groupingAttributes, + dataAttributes, + CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), + child) + CatalystSerde.serialize(mapped, RowEncoder(schema)) } } -/** - * A relation produced by applying a serialized R function `func` to each partition of the `child`. - * - */ -case class MapPartitionsInR( +case class MapGroupsR( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], inputSchema: StructType, outputSchema: StructType, + keyDeserializer: Expression, + valueDeserializer: Expression, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer { - override lazy val schema = outputSchema -} + child: LogicalPlan) extends UnaryNode with ObjectProducer{ + override lazy val schema = outputSchema +} /** Factory for constructing new `CoGroup` nodes. */ object CoGroup { From 43f8ec38a136149cf633f6fe8968a4d2240ccb1a Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 12:35:05 -0700 Subject: [PATCH 05/39] Fixed scala style-check issues + added/fixed comments --- R/pkg/R/DataFrame.R | 9 ++--- .../sql/catalyst/plans/logical/object.scala | 6 ++-- .../scala/org/apache/spark/sql/Dataset.scala | 21 +++++------ .../spark/sql/KeyValueGroupedDataset.scala | 26 +++++++++++--- .../org/apache/spark/sql/api/r/SQLUtils.scala | 36 +++++++++---------- .../spark/sql/execution/SparkStrategies.scala | 6 ++-- .../execution/r/MapPartitionsRWrapper.scala | 20 +++++------ 7 files changed, 67 insertions(+), 57 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 18f3f89337f33..f2f9a572cf6f1 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1135,7 +1135,7 @@ setMethod("summarize", #' func should have only one parameter, to which a data.frame corresponds #' to each partition will be passed. #' The output of func should be a data.frame. -#' @param schema The schema of the resulting DataFrame after the function is applied. +#' @param schema The schema of the resulting SparkDataFrame after the function is applied. #' It must match the output of func. #' @family SparkDataFrame functions #' @rdname dapply @@ -1189,13 +1189,14 @@ setMethod("dapply", #' gapply #' -#' Apply a function to each group of a DataFrame. +#' Apply a function to each group of a DataFrame. The group is defined by an input +#' grouping column. #' #' @param x A SparkDataFrame #' @param func A function to be applied to each group partition specified by grouping #' column of the SparkDataFrame. -#' The output of func is a data.frame. -#' @param schema The schema of the resulting DataFrame after the function is applied. +#' The output of func is a local R data.frame. +#' @param schema The schema of the resulting SparkDataFrame after the function is applied. #' It must match the output of func. #' @family SparkDataFrame functions #' @rdname gapply diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 6ab911e2f9555..8dedc31a5e735 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -257,7 +257,7 @@ case class MapGroups( child: LogicalPlan) extends UnaryNode with ObjectProducer /** Factory for constructing new `MapGroupsR` nodes. */ -object MapGroupsR { +object MapGroupsPartitionsInR { def apply( func: Array[Byte], packageNames: Array[Byte], @@ -270,7 +270,7 @@ object MapGroupsR { dataAttributes: Seq[Attribute], child: LogicalPlan): LogicalPlan = { - val mapped = MapGroupsR( + val mapped = MapGroupsPartitionsInR( func, packageNames, broadcastVars, @@ -286,7 +286,7 @@ object MapGroupsR { } } -case class MapGroupsR( +case class MapGroupsPartitionsInR( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index c165ff38a0bd5..f4e11871ffd32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2005,26 +2005,23 @@ class Dataset[T] private[sql]( * @group func * @since 2.0.0 */ - def mapGroupPartitionsInR[K: Encoder]( - gfunc: T => K, - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Object], - outputSchema: StructType): DataFrame = { - + private[sql] def mapGroupPartitionsInR[K: Encoder]( + gfunc: T => K, + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + outputSchema: StructType): DataFrame = { val inputPlan = logicalPlan val withGroupingKey = AppendColumns(gfunc, inputPlan) val executed = sqlContext.executePlan(withGroupingKey) - - val kvdg = new KeyValueGroupedDataset( + val keyValueGroupedData = new KeyValueGroupedDataset( encoderFor[K], encoderFor[T], executed, inputPlan.output, withGroupingKey.newColumns) - - kvdg.flatMapRGroups(func, packageNames, broadcastVars, outputSchema) - } + keyValueGroupedData.flatMapRGroups(func, packageNames, broadcastVars, outputSchema) + } /** * :: Experimental :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index b9d380c54f325..9a3f6cc9c54bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -115,19 +115,35 @@ class KeyValueGroupedDataset[K, V] private[sql]( logicalPlan)) } + /** + * Applies the given R function to each group of data. For each unique group, the function will + * be passed the group key and an iterator that contains all of the elements in the group. The + * function can return an iterator containing elements of an arbitrary type which will be returned + * as a new [[Dataset]]. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [[Dataset]]. If an application intends to perform an aggregation over each + * key, it is best to use the reduce function or an + * [[org.apache.spark.sql.expressions#Aggregator Aggregator]]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the memory + * constraints of their cluster. + * + * @since 2.0.0 + */ def flatMapRGroups( - func: Array[Byte], + f: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Object], outputSchema: StructType): DataFrame = { - val broadcastVarObj = broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]]) val rowEncoder = vEncoder.asInstanceOf[ExpressionEncoder[Row]] - Dataset.ofRows( sparkSession, - MapGroupsR( - func, + MapGroupsPartitionsInR( + f, packageNames, broadcastVarObj, outputSchema, 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 a4600dbcfdab2..a77e5aaee15bc 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 @@ -155,38 +155,36 @@ private[sql] object SQLUtils { df.mapPartitionsInR(func, packageNames, bv, realSchema) } - /** - * The helper function for gapply() on R side. - */ - def gapply( + /** + * The helper function for gapply() on R side. + */ + def gapply( df: DataFrame, func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Object], schema: StructType, col: String): DataFrame = { - - val realSchema = + val realSchema = if (schema == null) { SERIALIZED_R_DATA_SCHEMA } else { schema } + val dfSchema = df.select(df(col)).schema + if (dfSchema.length == 0) throw new IllegalArgumentException(s"Invaid column name $col") + val dataType = dfSchema(0).dataType + val sqlContext = df.sqlContext - val dfSchema = df.select(df(col)).schema - if (dfSchema.length == 0) throw new IllegalArgumentException(s"Invaid column name $col") - val dataType = dfSchema(0).dataType - - val sqlContext = df.sqlContext - import sqlContext.implicits._ + import sqlContext.implicits._ dataType match { case ByteType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Byte](col), - func, packageNames, broadcastVars, realSchema) + func, packageNames, broadcastVars, realSchema) case IntegerType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Int](col), - func, packageNames, broadcastVars, realSchema) + func, packageNames, broadcastVars, realSchema) case FloatType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Float](col), - func, packageNames, broadcastVars, realSchema) + func, packageNames, broadcastVars, realSchema) case DoubleType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Double](col), func, packageNames, broadcastVars, realSchema) case StringType => df.mapGroupPartitionsInR((r: Row) => r.getAs[String](col), @@ -194,12 +192,12 @@ private[sql] object SQLUtils { case BinaryType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Array[Byte]](col), func, packageNames, broadcastVars, realSchema) case BooleanType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Boolean](col), - func, packageNames, broadcastVars, realSchema) + func, packageNames, broadcastVars, realSchema) case TimestampType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Long](col), - func, packageNames, broadcastVars, realSchema) + func, packageNames, broadcastVars, realSchema) case _ => throw new IllegalArgumentException(s"Invaid type $dataType") - } - } + } + } def dfToCols(df: DataFrame): Array[Array[Any]] = { val localDF: Array[Row] = df.collect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5bb521c161847..f703070fd1e9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -310,9 +310,9 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.MapPartitionsInR(f, p, b, is, os, objAttr, child) => execution.MapPartitionsExec( execution.r.MapPartitionsRWrapper(f, p, b, is, os), objAttr, planLater(child)) :: Nil - case logical.MapGroupsR(f, p, b, is, os, key, value, grouping, data, objAttr, child) => - execution.MapGroupsExec( - execution.r.MapGroupRWrapper(f, p, b, is, os), key, value, grouping, + case logical.MapGroupsPartitionsInR(f, p, b, is, os, key, value, grouping, data, objAttr, + child) => execution.MapGroupsExec( + execution.r.MapGroupsPartitionsRWrapper(f, p, b, is, os), key, value, grouping, data, objAttr, planLater(child)) :: Nil case logical.MapElements(f, objAttr, child) => execution.MapElementsExec(f, objAttr, planLater(child)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index 35d425234e3fd..8c2d0f0872cc4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -27,16 +27,15 @@ import org.apache.spark.sql.types.{BinaryType, StructField, StructType} /** * A function wrapper that applies the given R function to each partition of each group. */ -private[sql] case class MapGroupRWrapper( +private[sql] case class MapGroupsPartitionsRWrapper( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], inputSchema: StructType, outputSchema: StructType) extends ((Any, Iterator[Any]) => TraversableOnce[Any]) { - def apply(key: Any, iter: Iterator[Any]): TraversableOnce[Any] = { PartitionsRHelper.mapPartitionsRHelper(func, - packageNames, broadcastVars, inputSchema, outputSchema, iter) + packageNames, broadcastVars, inputSchema, outputSchema, iter) } } @@ -49,18 +48,17 @@ private[sql] case class MapPartitionsRWrapper( broadcastVars: Array[Broadcast[Object]], inputSchema: StructType, outputSchema: StructType) extends (Iterator[Any] => Iterator[Any]) { - def apply(iter: Iterator[Any]): Iterator[Any] = { PartitionsRHelper.mapPartitionsRHelper(func, - packageNames, broadcastVars, inputSchema, outputSchema, iter) + packageNames, broadcastVars, inputSchema, outputSchema, iter) } } - object PartitionsRHelper { - /** - * A helper function to run R UDFs on partitions - */ + + /** + * A helper function to run R UDFs on partitions + */ private[sql] def mapPartitionsRHelper( func: Array[Byte], packageNames: Array[Byte], @@ -68,7 +66,7 @@ object PartitionsRHelper { inputSchema: StructType, outputSchema: StructType, iter: Iterator[Any]): Iterator[Any] = { - // If the content of current DataFrame is serialized R data? + // If the content of current DataFrame is serialized R data? val isSerializedRData = if (inputSchema == SERIALIZED_R_DATA_SCHEMA) true else false @@ -98,5 +96,5 @@ object PartitionsRHelper { } else { outputIter.map { bytes => Row.fromSeq(Seq(bytes)) } } - } + } } From f8caa707af9ffd70d94bca37c13a59db6d1efc45 Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 14:30:47 -0700 Subject: [PATCH 06/39] Add pretty test cases and examples --- R/pkg/R/DataFrame.R | 29 +++++++++++++---------- R/pkg/inst/tests/testthat/test_sparkSQL.R | 12 ++++++---- 2 files changed, 24 insertions(+), 17 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index f2f9a572cf6f1..51d4dbeee9213 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1203,32 +1203,35 @@ setMethod("dapply", #' @name gapply #' @export #' @examples +#' #' \dontrun{ -#' df <- createDataFrame (sqlContext, iris) -#' gdf <- gapply(df, function(x) { x }, schema(df), "Petal_Width") -#' collect(gdf) #' -#' Compute the square of the first columns and output it +#' df <- createDataFrame (sqlContext, iris) +#' gdf <- gapply(df, function(x) { x }, schema(df), "Petal_Width") +#' collect(gdf) +#' +#' Compute the arithmetic mean of the second column by grouping +#' on the first column. Output the groupping value and the average. #' #' df <- createDataFrame ( #' sqlContext, #' list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), #' c("a", "b", "c", "d")) #' -#' schema <- structType(structField("result", "integer")) +#' schema <- structType(structField("a", "integer"), structField("avg", "double")) #' df1 <- gapply( #' df, #' function(x) { -#' y <- x[1] * x[1] -#' }, -#' schema, "a") -#' collect(df1) +#' y <- (data.frame(x$a[1], mean(x$b))) +#' }, +#' schema, "a") +#' collect(df1) #' -#' result +#' Result #' ------ -#' 1 -#' 1 -#' 9 +#' a avg +#' 1 1.5 +#' 3 3.0 #'} setMethod("gapply", signature(x = "SparkDataFrame", func = "function", schema = "structType", diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 4481c35de373b..8a526f1be3ca0 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2107,16 +2107,20 @@ test_that("gapply() on a DataFrame", { expected <- c(FALSE, FALSE, TRUE) expect_identical(actual, expected) - # remove columns - schema <- structType(structField("a", "integer")) + # Compute the arithmetic mean of the second column by grouping + # on the first column. Output the groupping value and the average. + schema <- structType(structField("a", "integer"), structField("avg", "double")) df3 <- gapply( df, function(x) { - y <- x[1] + y <- (data.frame(x$a[1], mean(x$b))) }, schema, "a") actual <- collect(df3) - expected <- collect(select(df, "a")) + expected <- collect(select(df, "a", "b")) + expected <- data.frame(aggregate(expected$b, by = list(expected$a), FUN = mean)) + colnames(expected) <- c("a", "avg") + expect_identical(actual, expected) }) From 1b2f5c112beddc43fc4738478b514b5cccb64723 Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 18:06:58 -0700 Subject: [PATCH 07/39] More scala stylecheck fixes. --- R/pkg/R/DataFrame.R | 10 +++-- .../sql/catalyst/plans/logical/object.scala | 11 +++-- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../spark/sql/KeyValueGroupedDataset.scala | 38 ++++++++--------- .../org/apache/spark/sql/api/r/SQLUtils.scala | 42 +++++++------------ .../spark/sql/execution/SparkStrategies.scala | 6 +-- .../execution/r/MapPartitionsRWrapper.scala | 18 ++++---- 7 files changed, 59 insertions(+), 68 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 51d4dbeee9213..e13b6a7776ac4 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1191,11 +1191,13 @@ setMethod("dapply", #' #' Apply a function to each group of a DataFrame. The group is defined by an input #' grouping column. +#' Currently only one grouping column is allowed. Support for multiple columns will +#' be added later. #' #' @param x A SparkDataFrame #' @param func A function to be applied to each group partition specified by grouping -#' column of the SparkDataFrame. -#' The output of func is a local R data.frame. +#' column of the SparkDataFrame. +#' The output of func is a local R data.frame. #' @param schema The schema of the resulting SparkDataFrame after the function is applied. #' It must match the output of func. #' @family SparkDataFrame functions @@ -1220,8 +1222,8 @@ setMethod("dapply", #' #' schema <- structType(structField("a", "integer"), structField("avg", "double")) #' df1 <- gapply( -#' df, -#' function(x) { +#' df, +#' function(x) { #' y <- (data.frame(x$a[1], mean(x$b))) #' }, #' schema, "a") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 8dedc31a5e735..6b60844e3e637 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -256,8 +256,8 @@ case class MapGroups( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectProducer -/** Factory for constructing new `MapGroupsR` nodes. */ -object MapGroupsPartitionsInR { +/** Factory for constructing new `MapGroupsInR` nodes. */ +object MapGroupsInR { def apply( func: Array[Byte], packageNames: Array[Byte], @@ -269,8 +269,7 @@ object MapGroupsPartitionsInR { groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], child: LogicalPlan): LogicalPlan = { - - val mapped = MapGroupsPartitionsInR( + val mapped = MapGroupsInR( func, packageNames, broadcastVars, @@ -286,7 +285,7 @@ object MapGroupsPartitionsInR { } } -case class MapGroupsPartitionsInR( +case class MapGroupsInR( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], @@ -299,7 +298,7 @@ case class MapGroupsPartitionsInR( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectProducer{ - override lazy val schema = outputSchema + override lazy val schema = outputSchema } /** Factory for constructing new `CoGroup` nodes. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index f4e11871ffd32..ee3c888f0f7ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2005,7 +2005,7 @@ class Dataset[T] private[sql]( * @group func * @since 2.0.0 */ - private[sql] def mapGroupPartitionsInR[K: Encoder]( + private[sql] def mapGroupInR[K: Encoder]( gfunc: T => K, func: Array[Byte], packageNames: Array[Byte], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 9a3f6cc9c54bb..890132adfd712 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -134,25 +134,25 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.0.0 */ def flatMapRGroups( - f: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Object], - outputSchema: StructType): DataFrame = { - val broadcastVarObj = broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]]) - val rowEncoder = vEncoder.asInstanceOf[ExpressionEncoder[Row]] - Dataset.ofRows( - sparkSession, - MapGroupsPartitionsInR( - f, - packageNames, - broadcastVarObj, - outputSchema, - unresolvedVEncoder.deserializer, - unresolvedKEncoder.deserializer, - rowEncoder, - groupingAttributes, - dataAttributes, - logicalPlan)) + f: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + outputSchema: StructType): DataFrame = { + val broadcastVarObj = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) + val rowEncoder = vEncoder.asInstanceOf[ExpressionEncoder[Row]] + Dataset.ofRows( + sparkSession, + MapGroupsInR( + f, + packageNames, + broadcastVarObj, + outputSchema, + unresolvedVEncoder.deserializer, + unresolvedKEncoder.deserializer, + rowEncoder, + groupingAttributes, + dataAttributes, + logicalPlan)) } /** 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 a77e5aaee15bc..7729a328d8a55 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 @@ -146,12 +146,7 @@ private[sql] object SQLUtils { broadcastVars: Array[Object], schema: StructType): DataFrame = { val bv = broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]]) - val realSchema = - if (schema == null) { - SERIALIZED_R_DATA_SCHEMA - } else { - schema - } + val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema df.mapPartitionsInR(func, packageNames, bv, realSchema) } @@ -159,18 +154,13 @@ private[sql] object SQLUtils { * The helper function for gapply() on R side. */ def gapply( - df: DataFrame, - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Object], - schema: StructType, - col: String): DataFrame = { - val realSchema = - if (schema == null) { - SERIALIZED_R_DATA_SCHEMA - } else { - schema - } + df: DataFrame, + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + schema: StructType, + col: String): DataFrame = { + val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema val dfSchema = df.select(df(col)).schema if (dfSchema.length == 0) throw new IllegalArgumentException(s"Invaid column name $col") val dataType = dfSchema(0).dataType @@ -179,21 +169,21 @@ private[sql] object SQLUtils { import sqlContext.implicits._ dataType match { - case ByteType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Byte](col), + case ByteType => df.mapGroupInR((r: Row) => r.getAs[Byte](col), func, packageNames, broadcastVars, realSchema) - case IntegerType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Int](col), + case IntegerType => df.mapGroupInR((r: Row) => r.getAs[Int](col), func, packageNames, broadcastVars, realSchema) - case FloatType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Float](col), + case FloatType => df.mapGroupInR((r: Row) => r.getAs[Float](col), func, packageNames, broadcastVars, realSchema) - case DoubleType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Double](col), + case DoubleType => df.mapGroupInR((r: Row) => r.getAs[Double](col), func, packageNames, broadcastVars, realSchema) - case StringType => df.mapGroupPartitionsInR((r: Row) => r.getAs[String](col), + case StringType => df.mapGroupInR((r: Row) => r.getAs[String](col), func, packageNames, broadcastVars, realSchema) - case BinaryType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Array[Byte]](col), + case BinaryType => df.mapGroupInR((r: Row) => r.getAs[Array[Byte]](col), func, packageNames, broadcastVars, realSchema) - case BooleanType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Boolean](col), + case BooleanType => df.mapGroupInR((r: Row) => r.getAs[Boolean](col), func, packageNames, broadcastVars, realSchema) - case TimestampType => df.mapGroupPartitionsInR((r: Row) => r.getAs[Long](col), + case TimestampType => df.mapGroupInR((r: Row) => r.getAs[Long](col), func, packageNames, broadcastVars, realSchema) case _ => throw new IllegalArgumentException(s"Invaid type $dataType") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f703070fd1e9d..0aaac793efaa9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -310,10 +310,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.MapPartitionsInR(f, p, b, is, os, objAttr, child) => execution.MapPartitionsExec( execution.r.MapPartitionsRWrapper(f, p, b, is, os), objAttr, planLater(child)) :: Nil - case logical.MapGroupsPartitionsInR(f, p, b, is, os, key, value, grouping, data, objAttr, + case logical.MapGroupsInR(f, p, b, is, os, key, value, grouping, data, objAttr, child) => execution.MapGroupsExec( - execution.r.MapGroupsPartitionsRWrapper(f, p, b, is, os), key, value, grouping, - data, objAttr, planLater(child)) :: Nil + execution.r.MapGroupsRWrapper(f, p, b, is, os), key, value, grouping, + data, objAttr, planLater(child)) :: Nil case logical.MapElements(f, objAttr, child) => execution.MapElementsExec(f, objAttr, planLater(child)) :: Nil case logical.AppendColumns(f, in, out, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index 8c2d0f0872cc4..ec8544e2ba375 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.types.{BinaryType, StructField, StructType} /** * A function wrapper that applies the given R function to each partition of each group. */ -private[sql] case class MapGroupsPartitionsRWrapper( +private[sql] case class MapGroupsRWrapper( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], @@ -49,8 +49,8 @@ private[sql] case class MapPartitionsRWrapper( inputSchema: StructType, outputSchema: StructType) extends (Iterator[Any] => Iterator[Any]) { def apply(iter: Iterator[Any]): Iterator[Any] = { - PartitionsRHelper.mapPartitionsRHelper(func, - packageNames, broadcastVars, inputSchema, outputSchema, iter) + PartitionsRHelper.mapPartitionsRHelper( + func, packageNames, broadcastVars, inputSchema, outputSchema, iter) } } @@ -60,12 +60,12 @@ object PartitionsRHelper { * A helper function to run R UDFs on partitions */ private[sql] def mapPartitionsRHelper( - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Broadcast[Object]], - inputSchema: StructType, - outputSchema: StructType, - iter: Iterator[Any]): Iterator[Any] = { + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + outputSchema: StructType, + iter: Iterator[Any]): Iterator[Any] = { // If the content of current DataFrame is serialized R data? val isSerializedRData = if (inputSchema == SERIALIZED_R_DATA_SCHEMA) true else false From caefc7155bc9e0b7fab27fe930c0f1d51b27079a Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 21:04:33 -0700 Subject: [PATCH 08/39] added ml examples --- R/pkg/R/DataFrame.R | 29 +++++++++++++++---- .../org/apache/spark/sql/api/r/SQLUtils.scala | 2 +- 2 files changed, 25 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index e13b6a7776ac4..552c0c18b8b0b 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1208,10 +1208,6 @@ setMethod("dapply", #' #' \dontrun{ #' -#' df <- createDataFrame (sqlContext, iris) -#' gdf <- gapply(df, function(x) { x }, schema(df), "Petal_Width") -#' collect(gdf) -#' #' Compute the arithmetic mean of the second column by grouping #' on the first column. Output the groupping value and the average. #' @@ -1224,7 +1220,7 @@ setMethod("dapply", #' df1 <- gapply( #' df, #' function(x) { -#' y <- (data.frame(x$a[1], mean(x$b))) +#' y <- (data.frame(x$a[1], mean(x$b))) #' }, #' schema, "a") #' collect(df1) @@ -1234,6 +1230,29 @@ setMethod("dapply", #' a avg #' 1 1.5 #' 3 3.0 +#' +#' Fit linear models on iris data set by grouping on the 'Species' column and +#' using Sepal_Length as a target variable, Sepal_Width, Petal_Length and Petal_Width +#' as training features. +#' schema <- structType(structField("(Intercept)", "double"), +#' structField("Sepal_Width", "double"),structField("Petal_Length", "double"), +#' structField("Petal_Width", "double")) +#' df1 <- gapply( +#' df, +#' function(x) { +#' m <- suppressWarnings(lm(Sepal_Length ~ +#' Sepal_Width + Petal_Length + Petal_Width, x)) +#' data.frame(t(coef(m))) +#' }, schema, "Species") +#' collect(df1) +#' +#'Result +#'--------- +#' Model (Intercept) Sepal_Width Petal_Length Petal_Width +#' 1 0.699883 0.3303370 0.9455356 -0.1697527 +#' 2 1.895540 0.3868576 0.9083370 -0.6792238 +#' 3 2.351890 0.6548350 0.2375602 0.2521257 +#' #'} setMethod("gapply", signature(x = "SparkDataFrame", func = "function", schema = "structType", 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 7729a328d8a55..92fad7d5f2d35 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 @@ -145,7 +145,7 @@ private[sql] object SQLUtils { packageNames: Array[Byte], broadcastVars: Array[Object], schema: StructType): DataFrame = { - val bv = broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]]) + val bv = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema df.mapPartitionsInR(func, packageNames, bv, realSchema) } From 84fe17692b0f685e70bceea6decfed8a922fbecd Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 21:57:01 -0700 Subject: [PATCH 09/39] Added support for 'Column' type on R side and made the test cases more meaningful --- R/pkg/R/DataFrame.R | 29 +++++++++++++---------- R/pkg/inst/tests/testthat/test_sparkSQL.R | 12 ++++------ 2 files changed, 22 insertions(+), 19 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 552c0c18b8b0b..7e6a2647ff6d6 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1208,7 +1208,7 @@ setMethod("dapply", #' #' \dontrun{ #' -#' Compute the arithmetic mean of the second column by grouping +#' Computes the arithmetic mean of the second column by grouping #' on the first column. Output the groupping value and the average. #' #' df <- createDataFrame ( @@ -1222,7 +1222,7 @@ setMethod("dapply", #' function(x) { #' y <- (data.frame(x$a[1], mean(x$b))) #' }, -#' schema, "a") +#' schema, df$"a") #' collect(df1) #' #' Result @@ -1231,9 +1231,11 @@ setMethod("dapply", #' 1 1.5 #' 3 3.0 #' -#' Fit linear models on iris data set by grouping on the 'Species' column and -#' using Sepal_Length as a target variable, Sepal_Width, Petal_Length and Petal_Width -#' as training features. +#' Fits linear models on iris dataset by grouping on the 'Species' column and +#' using 'Sepal_Length' as a target variable, 'Sepal_Width', 'Petal_Length' +#' and 'Petal_Width' as training features. +#' +#' df <- createDataFrame (sqlContext, iris) #' schema <- structType(structField("(Intercept)", "double"), #' structField("Sepal_Width", "double"),structField("Petal_Length", "double"), #' structField("Petal_Width", "double")) @@ -1256,19 +1258,22 @@ setMethod("dapply", #'} setMethod("gapply", signature(x = "SparkDataFrame", func = "function", schema = "structType", - col = "character"), + col = "characterOrColumn"), function(x, func, schema, col) { packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) - broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - + if (class(col) == "Column") { + colStr <- callJMethod(col@jc, "toString") + } else { + colStr <- col + } sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "gapply", x@sdf, - serialize(cleanClosure(func), connection = NULL), - packageNamesArr, - broadcastArr, - schema$jobj, col) + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + schema$jobj, colStr) dataFrame(sdf) }) ############################## RDD Map Functions ################################## diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 8a526f1be3ca0..2f408fb213365 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2089,22 +2089,20 @@ test_that("gapply() on a DataFrame", { list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), c("a", "b", "c", "d")) expected <- collect(df) - df1 <- gapply(df, function(x) { x }, schema(df), "a") + df1 <- gapply(df, function(x) { x }, schema(df), df$"a") actual <- collect(df1) expect_identical(actual, expected) - # Add a boolean column - schema <- structType(structField("a", "integer"), structField("b", "double"), - structField("c", "string"), structField("d", "double"), - structField("e", "boolean")) + # Check the sum of second column by grouping on the first column + schema <- structType(structField("a", "integer"), structField("e", "boolean")) df2 <- gapply( df, function(x) { - y <- cbind(x, x[1] > 1) + y <-data.frame(x$a[1], sum(x$b) > 2) }, schema, "a") actual <- collect(df2)$e - expected <- c(FALSE, FALSE, TRUE) + expected <- c(TRUE, TRUE) expect_identical(actual, expected) # Compute the arithmetic mean of the second column by grouping From 4067be73b7106e9943f443495f92067c017f82e5 Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 22:08:29 -0700 Subject: [PATCH 10/39] Fixed R check style --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 2f408fb213365..bbeb5c24d94d7 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2098,7 +2098,7 @@ test_that("gapply() on a DataFrame", { df2 <- gapply( df, function(x) { - y <-data.frame(x$a[1], sum(x$b) > 2) + y <- data.frame(x$a[1], sum(x$b) > 2) }, schema, "a") actual <- collect(df2)$e From f5aab7dd9e2a084376b7a6175713ac95daa49e95 Mon Sep 17 00:00:00 2001 From: NarineK Date: Mon, 2 May 2016 23:14:51 -0700 Subject: [PATCH 11/39] more style beautifications --- R/pkg/R/DataFrame.R | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 7e6a2647ff6d6..0eb89cb87f919 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1209,19 +1209,19 @@ setMethod("dapply", #' \dontrun{ #' #' Computes the arithmetic mean of the second column by grouping -#' on the first column. Output the groupping value and the average. +#' on the first column. Output the grouping value and the average. #' #' df <- createDataFrame ( #' sqlContext, #' list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), -#' c("a", "b", "c", "d")) +#' c("a", "b", "c", "d")) #' #' schema <- structType(structField("a", "integer"), structField("avg", "double")) #' df1 <- gapply( #' df, #' function(x) { #' y <- (data.frame(x$a[1], mean(x$b))) -#' }, +#' }, #' schema, df$"a") #' collect(df1) #' @@ -1237,15 +1237,15 @@ setMethod("dapply", #' #' df <- createDataFrame (sqlContext, iris) #' schema <- structType(structField("(Intercept)", "double"), -#' structField("Sepal_Width", "double"),structField("Petal_Length", "double"), -#' structField("Petal_Width", "double")) +#' structField("Sepal_Width", "double"),structField("Petal_Length", "double"), +#' structField("Petal_Width", "double")) #' df1 <- gapply( #' df, #' function(x) { #' m <- suppressWarnings(lm(Sepal_Length ~ #' Sepal_Width + Petal_Length + Petal_Width, x)) #' data.frame(t(coef(m))) -#' }, schema, "Species") +#' }, schema, "Species") #' collect(df1) #' #'Result From da1bfea34521c903a83500202c3f1f334f3a90e6 Mon Sep 17 00:00:00 2001 From: NarineK Date: Tue, 3 May 2016 13:19:17 -0700 Subject: [PATCH 12/39] Small fixes in the comments --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index bbeb5c24d94d7..b827892e17aa0 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2093,7 +2093,8 @@ test_that("gapply() on a DataFrame", { actual <- collect(df1) expect_identical(actual, expected) - # Check the sum of second column by grouping on the first column + # Computes the sum of second column by grouping on the first column and checks + # if the sum is larger than 2 schema <- structType(structField("a", "integer"), structField("e", "boolean")) df2 <- gapply( df, @@ -2105,7 +2106,7 @@ test_that("gapply() on a DataFrame", { expected <- c(TRUE, TRUE) expect_identical(actual, expected) - # Compute the arithmetic mean of the second column by grouping + # Computes the arithmetic mean of the second column by grouping # on the first column. Output the groupping value and the average. schema <- structType(structField("a", "integer"), structField("avg", "double")) df3 <- gapply( From 8b8ec8ced5cb89d52e93e901bfca811b806f73d7 Mon Sep 17 00:00:00 2001 From: NarineK Date: Thu, 12 May 2016 15:18:23 -0700 Subject: [PATCH 13/39] Supporting multiple groups, gapply support both for SparkDataFrame and GroupedData --- R/pkg/R/DataFrame.R | 49 ++++------- R/pkg/R/generics.R | 2 +- R/pkg/R/group.R | 51 ++++++++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 29 ++++--- .../sql/catalyst/plans/logical/object.scala | 28 +++---- .../scala/org/apache/spark/sql/Dataset.scala | 22 ++--- .../spark/sql/KeyValueGroupedDataset.scala | 41 --------- .../spark/sql/RelationalGroupedDataset.scala | 50 ++++++++++- .../org/apache/spark/sql/api/r/SQLUtils.scala | 41 +-------- .../spark/sql/execution/SparkStrategies.scala | 7 +- .../apache/spark/sql/execution/objects.scala | 83 ++++++++++++++++++- 11 files changed, 243 insertions(+), 160 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index da89ba81ca4c1..ad9dae35cccef 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1216,10 +1216,8 @@ setMethod("dapply", #' gapply #' -#' Apply a function to each group of a DataFrame. The group is defined by an input +#' Apply a R function to each group of a DataFrame. The group is defined by an input #' grouping column. -#' Currently only one grouping column is allowed. Support for multiple columns will -#' be added later. #' #' @param x A SparkDataFrame #' @param func A function to be applied to each group partition specified by grouping @@ -1234,29 +1232,30 @@ setMethod("dapply", #' @examples #' #' \dontrun{ -#' #' Computes the arithmetic mean of the second column by grouping -#' on the first column. Output the grouping value and the average. +#' on the first and third columns. Output the grouping values and the average. #' #' df <- createDataFrame ( #' sqlContext, -#' list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), +#' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), #' c("a", "b", "c", "d")) #' -#' schema <- structType(structField("a", "integer"), structField("avg", "double")) +#' schema <- structType(structField("a", "integer"), structField("c", "string"), +#' structField("avg", "double")) #' df1 <- gapply( #' df, +#' list("a", "c"), #' function(x) { -#' y <- (data.frame(x$a[1], mean(x$b))) +#' y <- data.frame(x$a[1], x$c[1], mean(x$b), stringsAsFactors = FALSE) #' }, -#' schema, df$"a") +#' schema) #' collect(df1) #' #' Result #' ------ -#' a avg -#' 1 1.5 -#' 3 3.0 +#' a c avg +#' 3 3 3.0 +#' 1 1 1.5 #' #' Fits linear models on iris dataset by grouping on the 'Species' column and #' using 'Sepal_Length' as a target variable, 'Sepal_Width', 'Petal_Length' @@ -1268,11 +1267,12 @@ setMethod("dapply", #' structField("Petal_Width", "double")) #' df1 <- gapply( #' df, +#' list(df$"Species"), #' function(x) { #' m <- suppressWarnings(lm(Sepal_Length ~ #' Sepal_Width + Petal_Length + Petal_Width, x)) #' data.frame(t(coef(m))) -#' }, schema, "Species") +#' }, schema) #' collect(df1) #' #'Result @@ -1284,25 +1284,12 @@ setMethod("dapply", #' #'} setMethod("gapply", - signature(x = "SparkDataFrame", func = "function", schema = "structType", - col = "characterOrColumn"), - function(x, func, schema, col) { - packageNamesArr <- serialize(.sparkREnv[[".packages"]], - connection = NULL) - broadcastArr <- lapply(ls(.broadcastNames), - function(name) { get(name, .broadcastNames) }) - if (class(col) == "Column") { - colStr <- callJMethod(col@jc, "toString") - } else { - colStr <- col - } - sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "gapply", x@sdf, - serialize(cleanClosure(func), connection = NULL), - packageNamesArr, - broadcastArr, - schema$jobj, colStr) - dataFrame(sdf) + signature(x = "SparkDataFrame"), + function(x, col, func, schema) { + grouped <- do.call("groupBy", c(x, col)) + gapply(grouped, func, schema) }) + ############################## RDD Map Functions ################################## # All of the following functions mirror the existing RDD map functions, # # but allow for use with DataFrames by first converting to an RRDD before calling # diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 94be9d5f5fab3..ad2b8788bb430 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -452,7 +452,7 @@ setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) #' @rdname gapply #' @export -setGeneric("gapply", function(x, func, schema, col) { standardGeneric("gapply") }) +setGeneric("gapply", function(x, ...) { standardGeneric("gapply") }) #' @rdname summary #' @export diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 08f4a490c883e..8fd6ecc592335 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -142,3 +142,54 @@ createMethods <- function() { } createMethods() + +#' gapply +#' +#' Applies a R function to each group in the input GroupedData +#' +#' @param x a GroupedData +#' @return a SparkDataFrame +#' @rdname gapply +#' @name gapply +#' @family agg_funcs +#' @examples +#' \dontrun{ +#' Computes the arithmetic mean of the second column by grouping +#' on the first and third columns. Output the grouping values and the average. +#' +#' df <- createDataFrame ( +#' sqlContext, +#' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), +#' c("a", "b", "c", "d")) +#' +#' schema <- structType(structField("a", "integer"), structField("c", "string"), +#' structField("avg", "double")) +#' df1 <- gapply( +#' df, +#' list("a", "c"), +#' function(x) { +#' y <- data.frame(x$a[1], x$c[1], mean(x$b), stringsAsFactors = FALSE) +#' }, +#' schema) +#' collect(df1) +#' +#' Result +#' ------ +#' a c avg +#' 3 3 3.0 +#' 1 1 1.5 +#' } +setMethod("gapply", + signature(x = "GroupedData"), + function(x, func, schema) { + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + sdf <- callJMethod(x@sgd, "flatMapGroupsInR", + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + schema$jobj) + dataFrame(sdf) + }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 3b7f34b11ea77..1c08b8872913f 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2121,39 +2121,42 @@ test_that("repartition by columns on DataFrame", { test_that("gapply() on a DataFrame", { df <- createDataFrame ( sqlContext, - list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), + list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), c("a", "b", "c", "d")) expected <- collect(df) - df1 <- gapply(df, function(x) { x }, schema(df), df$"a") + df1 <- gapply(df, list("a"), function(x) { x }, schema(df)) actual <- collect(df1) expect_identical(actual, expected) - # Computes the sum of second column by grouping on the first column and checks - # if the sum is larger than 2 + # Computes the sum of second column by grouping on the first and third columns + # and checks if the sum is larger than 2 schema <- structType(structField("a", "integer"), structField("e", "boolean")) df2 <- gapply( df, + list(df$"a", df$"c"), function(x) { y <- data.frame(x$a[1], sum(x$b) > 2) }, - schema, "a") + schema) actual <- collect(df2)$e expected <- c(TRUE, TRUE) expect_identical(actual, expected) # Computes the arithmetic mean of the second column by grouping - # on the first column. Output the groupping value and the average. - schema <- structType(structField("a", "integer"), structField("avg", "double")) + # on the first and third columns. Output the groupping value and the average. + schema <- structType(structField("a", "integer"), structField("c", "string"), + structField("avg", "double")) df3 <- gapply( df, + list("a", "c"), function(x) { - y <- (data.frame(x$a[1], mean(x$b))) + y <- (data.frame(x$a[1], x$c[1], mean(x$b), stringsAsFactors = FALSE)) }, - schema, "a") - actual <- collect(df3) - expected <- collect(select(df, "a", "b")) - expected <- data.frame(aggregate(expected$b, by = list(expected$a), FUN = mean)) - colnames(expected) <- c("a", "avg") + schema) + actual <- collect(arrange(df3, "a")) + expected <- collect(select(df, "a", "b", "c")) + expected <- data.frame(aggregate(expected$b, by = list(expected$a, expected$c), FUN = mean)) + colnames(expected) <- c("a", "c", "avg") expect_identical(actual, expected) }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 6b60844e3e637..6e71ef15bd7fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -256,20 +256,20 @@ case class MapGroups( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectProducer -/** Factory for constructing new `MapGroupsInR` nodes. */ -object MapGroupsInR { +/** Factory for constructing new `FlatMapGroupsInR` nodes. */ +object FlatMapGroupsInR { def apply( - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Broadcast[Object]], - schema: StructType, - encoder: Expression, - keyEncoder: Expression, - rowEncoder: ExpressionEncoder[Row], - groupingAttributes: Seq[Attribute], - dataAttributes: Seq[Attribute], - child: LogicalPlan): LogicalPlan = { - val mapped = MapGroupsInR( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + schema: StructType, + encoder: Expression, + keyEncoder: Expression, + rowEncoder: ExpressionEncoder[Row], + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + child: LogicalPlan): LogicalPlan = { + val mapped = FlatMapGroupsInR( func, packageNames, broadcastVars, @@ -285,7 +285,7 @@ object MapGroupsInR { } } -case class MapGroupsInR( +case class FlatMapGroupsInR( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 3eb37076511e8..16305e8adcc05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2000,27 +2000,21 @@ class Dataset[T] private[sql]( /** * Returns a new [[DataFrame]] which contains the aggregated result of applying - * [[func]] R function to each group + * a serialized R function `func` to each group * * @group func * @since 2.0.0 */ - private[sql] def mapGroupInR[K: Encoder]( - gfunc: T => K, + private[sql] def flatMapGroupsInR( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Object], - outputSchema: StructType): DataFrame = { - val inputPlan = logicalPlan - val withGroupingKey = AppendColumns(gfunc, inputPlan) - val executed = sqlContext.executePlan(withGroupingKey) - val keyValueGroupedData = new KeyValueGroupedDataset( - encoderFor[K], - encoderFor[T], - executed, - inputPlan.output, - withGroupingKey.newColumns) - keyValueGroupedData.flatMapRGroups(func, packageNames, broadcastVars, outputSchema) + outputSchema: StructType, + cols: Column*): DataFrame = { + val relationalGroupedDataSet = RelationalGroupedDataset(toDF(), cols.map(_.named), + RelationalGroupedDataset.GroupByType) + relationalGroupedDataSet.flatMapGroupsInR(func, packageNames, broadcastVars, + outputSchema) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 890132adfd712..2a8fa5bce8fac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -21,7 +21,6 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function._ -import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, OuterScopes} import org.apache.spark.sql.catalyst.expressions._ @@ -115,46 +114,6 @@ class KeyValueGroupedDataset[K, V] private[sql]( logicalPlan)) } - /** - * Applies the given R function to each group of data. For each unique group, the function will - * be passed the group key and an iterator that contains all of the elements in the group. The - * function can return an iterator containing elements of an arbitrary type which will be returned - * as a new [[Dataset]]. - * - * This function does not support partial aggregation, and as a result requires shuffling all - * the data in the [[Dataset]]. If an application intends to perform an aggregation over each - * key, it is best to use the reduce function or an - * [[org.apache.spark.sql.expressions#Aggregator Aggregator]]. - * - * Internally, the implementation will spill to disk if any given group is too large to fit into - * memory. However, users must take care to avoid materializing the whole iterator for a group - * (for example, by calling `toList`) unless they are sure that this is possible given the memory - * constraints of their cluster. - * - * @since 2.0.0 - */ - def flatMapRGroups( - f: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Object], - outputSchema: StructType): DataFrame = { - val broadcastVarObj = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) - val rowEncoder = vEncoder.asInstanceOf[ExpressionEncoder[Row]] - Dataset.ofRows( - sparkSession, - MapGroupsInR( - f, - packageNames, - broadcastVarObj, - outputSchema, - unresolvedVEncoder.deserializer, - unresolvedKEncoder.deserializer, - rowEncoder, - groupingAttributes, - dataAttributes, - logicalPlan)) - } - /** * Applies the given function to each group of data. For each unique group, the function will * be passed the group key and an iterator that contains all of the elements in the group. The diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 4f5bf633fab2e..7e674023aa50a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -20,13 +20,17 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import scala.language.implicitConversions +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Pivot} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, FlatMapGroupsInR, Pivot} import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.NumericType +import org.apache.spark.sql.types.StructType /** * A set of methods for aggregations on a [[DataFrame]], created by [[Dataset.groupBy]]. @@ -376,6 +380,50 @@ class RelationalGroupedDataset protected[sql]( def pivot(pivotColumn: String, values: java.util.List[Any]): RelationalGroupedDataset = { pivot(pivotColumn, values.asScala) } + + /** + * Applies the given serialized R function `func` to each group of data. For each unique group, + * the function will be passed the group key and an iterator that contains all of the elements in + * the group. The function can return an iterator containing elements of an arbitrary type which + * will be returned as a new [[DataFrame]]. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [[Dataset]]. If an application intends to perform an aggregation over each + * key, it is best to use the reduce function or an + * [[org.apache.spark.sql.expressions#Aggregator Aggregator]]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the memory + * constraints of their cluster. + * + * @since 2.0.0 + */ + def flatMapGroupsInR( + f: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + outputSchema: StructType): DataFrame = { + val broadcastVarObj = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) + val groupingNamedExpressions = groupingExprs.map(alias) + val groupingCols = groupingNamedExpressions.map(Column(_)) + val groupingDataFrame = df.select(groupingCols : _*) + val groupingAttributes = groupingNamedExpressions.map(_.toAttribute) + val realOutputSchema = if (outputSchema == null) SERIALIZED_R_DATA_SCHEMA else outputSchema + Dataset.ofRows( + df.sparkSession, + FlatMapGroupsInR( + f, + packageNames, + broadcastVarObj, + realOutputSchema, + df.unresolvedTEncoder.deserializer, + groupingDataFrame.unresolvedTEncoder.deserializer, + df.resolvedTEncoder, + groupingAttributes, + df.logicalPlan.output, + df.logicalPlan)) + } } 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 92fad7d5f2d35..6e333e2fc5870 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 @@ -25,7 +25,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLContext} +import org.apache.spark.sql.{Column, DataFrame, Row, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.Encoder @@ -150,45 +150,6 @@ private[sql] object SQLUtils { df.mapPartitionsInR(func, packageNames, bv, realSchema) } - /** - * The helper function for gapply() on R side. - */ - def gapply( - df: DataFrame, - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Object], - schema: StructType, - col: String): DataFrame = { - val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema - val dfSchema = df.select(df(col)).schema - if (dfSchema.length == 0) throw new IllegalArgumentException(s"Invaid column name $col") - val dataType = dfSchema(0).dataType - val sqlContext = df.sqlContext - - import sqlContext.implicits._ - - dataType match { - case ByteType => df.mapGroupInR((r: Row) => r.getAs[Byte](col), - func, packageNames, broadcastVars, realSchema) - case IntegerType => df.mapGroupInR((r: Row) => r.getAs[Int](col), - func, packageNames, broadcastVars, realSchema) - case FloatType => df.mapGroupInR((r: Row) => r.getAs[Float](col), - func, packageNames, broadcastVars, realSchema) - case DoubleType => df.mapGroupInR((r: Row) => r.getAs[Double](col), - func, packageNames, broadcastVars, realSchema) - case StringType => df.mapGroupInR((r: Row) => r.getAs[String](col), - func, packageNames, broadcastVars, realSchema) - case BinaryType => df.mapGroupInR((r: Row) => r.getAs[Array[Byte]](col), - func, packageNames, broadcastVars, realSchema) - case BooleanType => df.mapGroupInR((r: Row) => r.getAs[Boolean](col), - func, packageNames, broadcastVars, realSchema) - case TimestampType => df.mapGroupInR((r: Row) => r.getAs[Long](col), - func, packageNames, broadcastVars, realSchema) - case _ => throw new IllegalArgumentException(s"Invaid type $dataType") - } - } - def dfToCols(df: DataFrame): Array[Array[Any]] = { val localDF: Array[Row] = df.collect() val numCols = df.columns.length diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index eaef55c630a78..ccdc3a70e7a3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -311,10 +311,9 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.MapPartitionsInR(f, p, b, is, os, objAttr, child) => execution.MapPartitionsExec( execution.r.MapPartitionsRWrapper(f, p, b, is, os), objAttr, planLater(child)) :: Nil - case logical.MapGroupsInR(f, p, b, is, os, key, value, grouping, data, objAttr, - child) => execution.MapGroupsExec( - execution.r.MapGroupsRWrapper(f, p, b, is, os), key, value, grouping, - data, objAttr, planLater(child)) :: Nil + case logical.FlatMapGroupsInR(f, p, b, is, os, key, value, grouping, data, objAttr, child) => + execution.FlatMapGroupsInRExec(f, p, b, is, os, key, value, grouping, + data, objAttr, planLater(child)) :: Nil case logical.MapElements(f, objAttr, child) => execution.MapElementsExec(f, objAttr, planLater(child)) :: Nil case logical.AppendColumns(f, in, out, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index bafbbdf65724d..7f82b0d7977ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -20,12 +20,17 @@ package org.apache.spark.sql.execution import scala.language.existentials import org.apache.spark.api.java.function.MapFunction +import org.apache.spark.api.r.RRunner +import org.apache.spark.api.r.SerializationFormats +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.types.{DataType, ObjectType} +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataType, ObjectType, StructType} /** * Takes the input row from child and turns it into object using the given deserializer expression. @@ -308,6 +313,82 @@ case class MapGroupsExec( } } +/** + * Groups the input rows together and calls the R function with each group and an iterator + * containing all elements in the group. + * The result of this function is flattened before being output. + */ +case class FlatMapGroupsInRExec( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + outputSchema: StructType, + keyDeserializer: Expression, + valueDeserializer: Expression, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + outputObjAttr: Attribute, + child: SparkPlan) extends UnaryExecNode with ObjectOperator { + + override def output: Seq[Attribute] = outputObjAttr :: Nil + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(groupingAttributes) :: Nil + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + Seq(groupingAttributes.map(SortOrder(_, Ascending))) + + override protected def doExecute(): RDD[InternalRow] = { + val isSerializedRData = + if (inputSchema == SERIALIZED_R_DATA_SCHEMA) true else false + val isDeserializedRData = + if (outputSchema == SERIALIZED_R_DATA_SCHEMA) true else false + val serializerForR = if (!isDeserializedRData) { + SerializationFormats.ROW + } else { + SerializationFormats.BYTE + } + val (deserializerForR, colNames) = + if (!isSerializedRData) { + (SerializationFormats.ROW, inputSchema.fieldNames) + } else { + (SerializationFormats.BYTE, null) + } + + child.execute().mapPartitionsInternal { iter => + val grouped = GroupedIterator(iter, groupingAttributes, child.output) + val getKey = deserializeRowToObject(keyDeserializer, groupingAttributes) + val getValue = deserializeRowToObject(valueDeserializer, dataAttributes) + val outputObject = wrapObjectToRow(outputObjAttr.dataType) + + val runner = new RRunner[Array[Byte]]( + func, deserializerForR, serializerForR, packageNames, broadcastVars, + isDataFrame = true, colNames = colNames) + + grouped.flatMap { case (key, rowIter) => + val iter = rowIter.map(getValue) + val newIter = + if (!isSerializedRData) { + (iter.asInstanceOf[Iterator[Row]].map {row => rowToRBytes(row)}) + } else { + (iter.asInstanceOf[Iterator[Row]].map { row => row(0) }) + } + + val outputIter = runner.compute(newIter, -1) + if (!isDeserializedRData) { + val result = outputIter.map { bytes => bytesToRow(bytes, outputSchema) } + result.map(outputObject) + } else { + val result = outputIter.map { bytes => Row.fromSeq(Seq(bytes)) } + result.map(outputObject) + } + } + } + } +} + /** * Co-groups the data from left and right children, and calls the function with each group and 2 * iterators containing all elements in the group from left and right side. From 0b1b2558189a72903500500b96e22084120b96c0 Mon Sep 17 00:00:00 2001 From: NarineK Date: Sat, 14 May 2016 23:01:06 -0700 Subject: [PATCH 14/39] Added: A worker support for mutiple groups (with minimal modifications) --- R/pkg/R/deserialize.R | 25 ++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- R/pkg/inst/worker/worker.R | 85 ++++++++++--------- .../scala/org/apache/spark/api/r/RRDD.scala | 2 +- .../org/apache/spark/api/r/RRunner.scala | 20 ++++- .../apache/spark/sql/execution/objects.scala | 28 +++--- .../execution/r/MapPartitionsRWrapper.scala | 2 +- 7 files changed, 102 insertions(+), 62 deletions(-) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index ce071b1a848bb..46f3f6919a4a1 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -197,6 +197,31 @@ readMultipleObjects <- function(inputCon) { data # this is a list of named lists now } +readMultipleObjectsWithKeys <- function(inputCon) { + # readMultipleObjects will read multiple continuous objects from + # a DataOutputStream. There is no preceding field telling the count + # of the objects, so the number of objects varies, we try to read + # all objects in a loop until the end of the stream. + data <- list() + subData <- list() + while (TRUE) { + # If reaching the end of the stream, type returned should be "". + type <- readType(inputCon) + print(type) + if (type == "") { + break + } else if (type == "r") { + # key boundary detected + readTypedObject(inputCon, type) + data[[length(data) + 1L]] <- subData + subData <- list() + } else { + subData[[length(subData) + 1L]] <- readTypedObject(inputCon, type) + } + } + data # this is a list of named lists now +} + readRowList <- function(obj) { # readRowList is meant for use inside an lapply. As a result, it is # necessary to open a standalone connection for the row and consume diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index ff6de2b8165a0..9f7095ae63adb 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2172,7 +2172,7 @@ test_that("gapply() on a DataFrame", { y <- (data.frame(x$a[1], x$c[1], mean(x$b), stringsAsFactors = FALSE)) }, schema) - actual <- collect(arrange(df3, "a")) + actual <- collect(arrange(df3, "a", decreasing = FALSE)) expected <- collect(select(df, "a", "b", "c")) expected <- data.frame(aggregate(expected$b, by = list(expected$a, expected$c), FUN = mean)) colnames(expected) <- c("a", "c", "avg") diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 40cda0c5ef9c1..c3109d3f92377 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -93,58 +93,63 @@ if (isDataFrame) { isEmpty <- SparkR:::readInt(inputCon) -if (isEmpty != 0) { +isGapplyMode <- SparkR:::readInt(inputCon) +if (isEmpty != 0) { if (numPartitions == -1) { if (deserializer == "byte") { # Now read as many characters as described in funcLen - data <- SparkR:::readDeserialize(inputCon) + dataList <- list(SparkR:::readDeserialize(inputCon)) } else if (deserializer == "string") { - data <- as.list(readLines(inputCon)) - } else if (deserializer == "row") { - data <- SparkR:::readMultipleObjects(inputCon) + dataList <- list(as.list(readLines(inputCon))) + } else if (deserializer == "row" && isGapplyMode == 1) { + dataList <- SparkR:::readMultipleObjectsWithKeys(inputCon) + } else if (deserializer == "row"){ + dataList <- list(SparkR:::readMultipleObjects(inputCon)) } # Timing reading input data for execution inputElap <- elapsedSecs() - - if (isDataFrame) { - if (deserializer == "row") { - # Transform the list of rows into a data.frame - # Note that the optional argument stringsAsFactors for rbind is - # available since R 3.2.4. So we set the global option here. - oldOpt <- getOption("stringsAsFactors") - options(stringsAsFactors = FALSE) - data <- do.call(rbind.data.frame, data) - options(stringsAsFactors = oldOpt) - - names(data) <- colNames + for (i in 1:length(dataList)) { + data <- dataList[[i]] + if (isDataFrame) { + if (deserializer == "row") { + # Transform the list of rows into a data.frame + # Note that the optional argument stringsAsFactors for rbind is + # available since R 3.2.4. So we set the global option here. + oldOpt <- getOption("stringsAsFactors") + options(stringsAsFactors = FALSE) + data <- do.call(rbind.data.frame, data) + options(stringsAsFactors = oldOpt) + + names(data) <- colNames + } else { + # Check to see if data is a valid data.frame + stopifnot(deserializer == "byte") + stopifnot(class(data) == "data.frame") + } + output <- computeFunc(data) + if (serializer == "row") { + # Transform the result data.frame back to a list of rows + output <- split(output, seq(nrow(output))) + } else { + # Serialize the ouput to a byte array + stopifnot(serializer == "byte") + } } else { - # Check to see if data is a valid data.frame - stopifnot(deserializer == "byte") - stopifnot(class(data) == "data.frame") + output <- computeFunc(partition, data) } - output <- computeFunc(data) - if (serializer == "row") { - # Transform the result data.frame back to a list of rows - output <- split(output, seq(nrow(output))) - } else { - # Serialize the ouput to a byte array - stopifnot(serializer == "byte") - } - } else { - output <- computeFunc(partition, data) - } - # Timing computing - computeElap <- elapsedSecs() + # Timing computing + computeElap <- elapsedSecs() - if (serializer == "byte") { - SparkR:::writeRawSerialize(outputCon, output) - } else if (serializer == "row") { - SparkR:::writeRowSerialize(outputCon, output) - } else { - # write lines one-by-one with flag - lapply(output, function(line) SparkR:::writeString(outputCon, line)) + if (serializer == "byte") { + SparkR:::writeRawSerialize(outputCon, output) + } else if (serializer == "row") { + SparkR:::writeRowSerialize(outputCon, output) + } else { + # write lines one-by-one with flag + lapply(output, function(line) SparkR:::writeString(outputCon, line)) + } } # Timing output outputElap <- elapsedSecs() 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 59c8429c80172..dac0d8c56b20b 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 @@ -46,7 +46,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( // The parent may be also an RRDD, so we should launch it first. val parentIterator = firstParent[T].iterator(partition, context) - runner.compute(parentIterator, partition.index) + runner.compute(parentIterator, partition.index, false) } } diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 24ad689f8321c..3d889850c10b8 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -55,7 +55,8 @@ private[spark] class RRunner[U]( def compute( inputIterator: Iterator[_], - partitionIndex: Int): Iterator[U] = { + partitionIndex: Int, + gapplyMode: Boolean): Iterator[U] = { // Timing start bootTime = System.currentTimeMillis / 1000.0 @@ -74,7 +75,7 @@ private[spark] class RRunner[U]( // the socket used to send out the input of task serverSocket.setSoTimeout(10000) val inSocket = serverSocket.accept() - startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex) + startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex, gapplyMode) // the socket used to receive the output of task val outSocket = serverSocket.accept() @@ -114,7 +115,8 @@ private[spark] class RRunner[U]( private def startStdinThread( output: OutputStream, iter: Iterator[_], - partitionIndex: Int): Unit = { + partitionIndex: Int, + gapplyMode: Boolean): Unit = { val env = SparkEnv.get val taskContext = TaskContext.get() val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt @@ -161,6 +163,12 @@ private[spark] class RRunner[U]( dataOut.writeInt(1) } + if (!gapplyMode) { + dataOut.writeInt(0) + } else { + dataOut.writeInt(1) + } + val printOut = new PrintStream(stream) def writeElem(elem: Any): Unit = { @@ -180,6 +188,12 @@ private[spark] class RRunner[U]( for (elem <- iter) { elem match { + case (key, innerIter: Iterator[_]) => + for (innerElem <- innerIter) { + writeElem(innerElem) + } + // Writes key which can be used as a boundary in group-aggregate + SerDe.writeObject(dataOut, key.asInstanceOf[Array[Byte]]) case (key, value) => writeElem(key) writeElem(value) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 7f82b0d7977ab..38f3635d1b874 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -341,8 +341,6 @@ case class FlatMapGroupsInRExec( Seq(groupingAttributes.map(SortOrder(_, Ascending))) override protected def doExecute(): RDD[InternalRow] = { - val isSerializedRData = - if (inputSchema == SERIALIZED_R_DATA_SCHEMA) true else false val isDeserializedRData = if (outputSchema == SERIALIZED_R_DATA_SCHEMA) true else false val serializerForR = if (!isDeserializedRData) { @@ -351,11 +349,7 @@ case class FlatMapGroupsInRExec( SerializationFormats.BYTE } val (deserializerForR, colNames) = - if (!isSerializedRData) { - (SerializationFormats.ROW, inputSchema.fieldNames) - } else { - (SerializationFormats.BYTE, null) - } + (SerializationFormats.ROW, inputSchema.fieldNames) child.execute().mapPartitionsInternal { iter => val grouped = GroupedIterator(iter, groupingAttributes, child.output) @@ -367,16 +361,16 @@ case class FlatMapGroupsInRExec( func, deserializerForR, serializerForR, packageNames, broadcastVars, isDataFrame = true, colNames = colNames) - grouped.flatMap { case (key, rowIter) => - val iter = rowIter.map(getValue) + val hasGroups = grouped.hasNext + val groupedRBytes = grouped.flatMap { case (key, rowIter) => + val deserializedIter = rowIter.map(getValue) val newIter = - if (!isSerializedRData) { - (iter.asInstanceOf[Iterator[Row]].map {row => rowToRBytes(row)}) - } else { - (iter.asInstanceOf[Iterator[Row]].map { row => row(0) }) - } - - val outputIter = runner.compute(newIter, -1) + deserializedIter.asInstanceOf[Iterator[Row]].map {row => rowToRBytes(row)} + val newKey = rowToRBytes(getKey(key).asInstanceOf[Row]) + Iterator((newKey, newIter)) + } + if (hasGroups) { + val outputIter = runner.compute(groupedRBytes, -1, true) if (!isDeserializedRData) { val result = outputIter.map { bytes => bytesToRow(bytes, outputSchema) } result.map(outputObject) @@ -384,6 +378,8 @@ case class FlatMapGroupsInRExec( val result = outputIter.map { bytes => Row.fromSeq(Seq(bytes)) } result.map(outputObject) } + } else { + Iterator.empty } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index ec8544e2ba375..932db8a5c864d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -89,7 +89,7 @@ object PartitionsRHelper { func, deserializer, serializer, packageNames, broadcastVars, isDataFrame = true, colNames = colNames) // Partition index is ignored. Dataset has no support for mapPartitionsWithIndex. - val outputIter = runner.compute(newIter, -1) + val outputIter = runner.compute(newIter, -1, false) if (serializer == SerializationFormats.ROW) { outputIter.map { bytes => bytesToRow(bytes, outputSchema) } From 7e5847294d685add9a0252f2499b912fd1196810 Mon Sep 17 00:00:00 2001 From: NarineK Date: Wed, 18 May 2016 00:10:49 -0700 Subject: [PATCH 15/39] Removed gapply mode from R side + passing keys(grouping column names) to R worker --- R/pkg/R/deserialize.R | 6 ++--- R/pkg/inst/worker/worker.R | 17 ++++++++------ .../scala/org/apache/spark/api/r/RRDD.scala | 2 +- .../org/apache/spark/api/r/RRunner.scala | 23 +++++++++---------- .../apache/spark/sql/execution/objects.scala | 6 +++-- .../execution/r/MapPartitionsRWrapper.scala | 2 +- 6 files changed, 30 insertions(+), 26 deletions(-) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index 46f3f6919a4a1..84df1d3194e1a 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -198,16 +198,16 @@ readMultipleObjects <- function(inputCon) { } readMultipleObjectsWithKeys <- function(inputCon) { - # readMultipleObjects will read multiple continuous objects from + # readMultipleObjectsWithKeys will read multiple continuous objects from # a DataOutputStream. There is no preceding field telling the count # of the objects, so the number of objects varies, we try to read - # all objects in a loop until the end of the stream. + # all objects in a loop until the end of the stream. The rows in + # the stream are separated by grouping-key boundary data <- list() subData <- list() while (TRUE) { # If reaching the end of the stream, type returned should be "". type <- readType(inputCon) - print(type) if (type == "") { break } else if (type == "r") { diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index c3109d3f92377..9f4775824a096 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -84,17 +84,20 @@ broadcastElap <- elapsedSecs() # as number of partitions to create. numPartitions <- SparkR:::readInt(inputCon) -isDataFrame <- as.logical(SparkR:::readInt(inputCon)) +# 0 - RDD mode, 1 - dapply mode, 2 - gapply mode +mode <- SparkR:::readInt(inputCon) -# If isDataFrame, then read column names -if (isDataFrame) { +# If DataFrame - mode = 1 and mode = 2, then read column names +if (mode > 0) { colNames <- SparkR:::readObject(inputCon) + if (mode == 2) { + # TODO pass the key to R function on gapply mode + key <- SparkR:::readObject(inputCon) + } } isEmpty <- SparkR:::readInt(inputCon) -isGapplyMode <- SparkR:::readInt(inputCon) - if (isEmpty != 0) { if (numPartitions == -1) { if (deserializer == "byte") { @@ -102,7 +105,7 @@ if (isEmpty != 0) { dataList <- list(SparkR:::readDeserialize(inputCon)) } else if (deserializer == "string") { dataList <- list(as.list(readLines(inputCon))) - } else if (deserializer == "row" && isGapplyMode == 1) { + } else if (deserializer == "row" && mode == 2) { dataList <- SparkR:::readMultipleObjectsWithKeys(inputCon) } else if (deserializer == "row"){ dataList <- list(SparkR:::readMultipleObjects(inputCon)) @@ -111,7 +114,7 @@ if (isEmpty != 0) { inputElap <- elapsedSecs() for (i in 1:length(dataList)) { data <- dataList[[i]] - if (isDataFrame) { + if (mode > 0) { if (deserializer == "row") { # Transform the list of rows into a data.frame # Note that the optional argument stringsAsFactors for rbind is 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 dac0d8c56b20b..59c8429c80172 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 @@ -46,7 +46,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( // The parent may be also an RRDD, so we should launch it first. val parentIterator = firstParent[T].iterator(partition, context) - runner.compute(parentIterator, partition.index, false) + runner.compute(parentIterator, partition.index) } } diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 3d889850c10b8..737a15fa62543 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -40,7 +40,8 @@ private[spark] class RRunner[U]( broadcastVars: Array[Broadcast[Object]], numPartitions: Int = -1, isDataFrame: Boolean = false, - colNames: Array[String] = null) + colNames: Array[String] = null, + key: Array[String] = null) extends Logging { private var bootTime: Double = _ private var dataStream: DataInputStream = _ @@ -55,8 +56,7 @@ private[spark] class RRunner[U]( def compute( inputIterator: Iterator[_], - partitionIndex: Int, - gapplyMode: Boolean): Iterator[U] = { + partitionIndex: Int): Iterator[U] = { // Timing start bootTime = System.currentTimeMillis / 1000.0 @@ -75,7 +75,7 @@ private[spark] class RRunner[U]( // the socket used to send out the input of task serverSocket.setSoTimeout(10000) val inSocket = serverSocket.accept() - startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex, gapplyMode) + startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex) // the socket used to receive the output of task val outSocket = serverSocket.accept() @@ -115,8 +115,7 @@ private[spark] class RRunner[U]( private def startStdinThread( output: OutputStream, iter: Iterator[_], - partitionIndex: Int, - gapplyMode: Boolean): Unit = { + partitionIndex: Int): Unit = { val env = SparkEnv.get val taskContext = TaskContext.get() val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt @@ -151,19 +150,18 @@ private[spark] class RRunner[U]( dataOut.writeInt(numPartitions) - dataOut.writeInt(if (isDataFrame) 1 else 0) + dataOut.writeInt(if (isDataFrame && key != null) 2 else + if (isDataFrame) 1 else 0) if (isDataFrame) { SerDe.writeObject(dataOut, colNames) } - if (!iter.hasNext) { - dataOut.writeInt(0) - } else { - dataOut.writeInt(1) + if (key != null) { + SerDe.writeObject(dataOut, key) } - if (!gapplyMode) { + if (!iter.hasNext) { dataOut.writeInt(0) } else { dataOut.writeInt(1) @@ -201,6 +199,7 @@ private[spark] class RRunner[U]( writeElem(elem) } } + stream.flush() } catch { // TODO: We should propagate this error to the task thread diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 38f3635d1b874..683875647c86b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -356,10 +356,11 @@ case class FlatMapGroupsInRExec( val getKey = deserializeRowToObject(keyDeserializer, groupingAttributes) val getValue = deserializeRowToObject(valueDeserializer, dataAttributes) val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val groupNames = groupingAttributes.map(_.name).toArray val runner = new RRunner[Array[Byte]]( func, deserializerForR, serializerForR, packageNames, broadcastVars, - isDataFrame = true, colNames = colNames) + isDataFrame = true, colNames = colNames, key = groupNames) val hasGroups = grouped.hasNext val groupedRBytes = grouped.flatMap { case (key, rowIter) => @@ -369,8 +370,9 @@ case class FlatMapGroupsInRExec( val newKey = rowToRBytes(getKey(key).asInstanceOf[Row]) Iterator((newKey, newIter)) } + if (hasGroups) { - val outputIter = runner.compute(groupedRBytes, -1, true) + val outputIter = runner.compute(groupedRBytes, -1) if (!isDeserializedRData) { val result = outputIter.map { bytes => bytesToRow(bytes, outputSchema) } result.map(outputObject) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index 932db8a5c864d..ec8544e2ba375 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -89,7 +89,7 @@ object PartitionsRHelper { func, deserializer, serializer, packageNames, broadcastVars, isDataFrame = true, colNames = colNames) // Partition index is ignored. Dataset has no support for mapPartitionsWithIndex. - val outputIter = runner.compute(newIter, -1, false) + val outputIter = runner.compute(newIter, -1) if (serializer == SerializationFormats.ROW) { outputIter.map { bytes => bytesToRow(bytes, outputSchema) } From 09287408137f7d6fbe8f899b12810ab16cbb5c3e Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Wed, 25 May 2016 22:57:40 -0700 Subject: [PATCH 16/39] Addressed Shivaram's comments + added the key to R functions --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 10 +++++----- R/pkg/inst/worker/worker.R | 7 +++++-- .../main/scala/org/apache/spark/api/r/RRunner.scala | 10 ++++++++-- .../org/apache/spark/sql/KeyValueGroupedDataset.scala | 4 +--- .../scala/org/apache/spark/sql/execution/objects.scala | 8 ++++---- 5 files changed, 23 insertions(+), 16 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 9f7095ae63adb..6d982ea1b5744 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2143,7 +2143,7 @@ test_that("gapply() on a DataFrame", { list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), c("a", "b", "c", "d")) expected <- collect(df) - df1 <- gapply(df, list("a"), function(x) { x }, schema(df)) + df1 <- gapply(df, list("a"), function(key, x) { x }, schema(df)) actual <- collect(df1) expect_identical(actual, expected) @@ -2153,8 +2153,8 @@ test_that("gapply() on a DataFrame", { df2 <- gapply( df, list(df$"a", df$"c"), - function(x) { - y <- data.frame(x$a[1], sum(x$b) > 2) + function(key, x) { + y <- data.frame(x[, key[[1]]][1], sum(x$b) > 2) }, schema) actual <- collect(df2)$e @@ -2168,8 +2168,8 @@ test_that("gapply() on a DataFrame", { df3 <- gapply( df, list("a", "c"), - function(x) { - y <- (data.frame(x$a[1], x$c[1], mean(x$b), stringsAsFactors = FALSE)) + function(key, x) { + y <- (data.frame(x[, unlist(key)][1, ], mean(x$b), stringsAsFactors = FALSE)) }, schema) actual <- collect(arrange(df3, "a", decreasing = FALSE)) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 9f4775824a096..4eec383e6c2e3 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -91,7 +91,6 @@ mode <- SparkR:::readInt(inputCon) if (mode > 0) { colNames <- SparkR:::readObject(inputCon) if (mode == 2) { - # TODO pass the key to R function on gapply mode key <- SparkR:::readObject(inputCon) } } @@ -130,7 +129,11 @@ if (isEmpty != 0) { stopifnot(deserializer == "byte") stopifnot(class(data) == "data.frame") } - output <- computeFunc(data) + if (mode == 2) { + output <- computeFunc(key, data) + } else { + output <- computeFunc(data) + } if (serializer == "row") { # Transform the result data.frame back to a list of rows output <- split(output, seq(nrow(output))) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 737a15fa62543..b4486eddd1602 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -150,8 +150,14 @@ private[spark] class RRunner[U]( dataOut.writeInt(numPartitions) - dataOut.writeInt(if (isDataFrame && key != null) 2 else - if (isDataFrame) 1 else 0) + val mode = if (isDataFrame && key != null) { + 2 // gapply + } else if (isDataFrame) { + 1 // dapply + } else { + 0 // RDD + } + dataOut.writeInt(mode) if (isDataFrame) { SerDe.writeObject(dataOut, colNames) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 2a8fa5bce8fac..3a5ea19b8ad14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -21,12 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function._ -import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, OuterScopes} -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateStruct} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.types.StructType /** * :: Experimental :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index d4e519b01eae2..2bbae785a66d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -345,7 +345,7 @@ case class FlatMapGroupsInRExec( groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: SparkPlan) extends UnaryExecNode with ObjectOperator { + child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { override def output: Seq[Attribute] = outputObjAttr :: Nil override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) @@ -369,9 +369,9 @@ case class FlatMapGroupsInRExec( child.execute().mapPartitionsInternal { iter => val grouped = GroupedIterator(iter, groupingAttributes, child.output) - val getKey = deserializeRowToObject(keyDeserializer, groupingAttributes) - val getValue = deserializeRowToObject(valueDeserializer, dataAttributes) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) + val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) val groupNames = groupingAttributes.map(_.name).toArray val runner = new RRunner[Array[Byte]]( From f8c994f04375058789f44637c4e84df48c1d5a01 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sun, 29 May 2016 10:55:30 -0700 Subject: [PATCH 17/39] Addressed Shivaram's and some of Sun-Rui's comments --- R/pkg/R/DataFrame.R | 15 ++- R/pkg/R/deserialize.R | 7 +- R/pkg/R/group.R | 9 +- R/pkg/inst/worker/worker.R | 120 +++++++++++------- .../org/apache/spark/api/r/RRunner.scala | 3 +- .../scala/org/apache/spark/sql/Dataset.scala | 19 --- .../spark/sql/RelationalGroupedDataset.scala | 2 +- .../apache/spark/sql/execution/objects.scala | 6 +- 8 files changed, 95 insertions(+), 86 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index bf90761bbd7c9..a0fa4e0eb5252 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1270,10 +1270,11 @@ setMethod("dapplyCollect", #' gapply #' -#' Apply a R function to each group of a DataFrame. The group is defined by an input -#' grouping column. +#' Apply a R function to each group of a DataFrame. The group is defined by input +#' grouping columns. #' #' @param x A SparkDataFrame +#' @param cols Grouping columns #' @param func A function to be applied to each group partition specified by grouping #' column of the SparkDataFrame. #' The output of func is a local R data.frame. @@ -1299,8 +1300,8 @@ setMethod("dapplyCollect", #' df1 <- gapply( #' df, #' list("a", "c"), -#' function(x) { -#' y <- data.frame(x$a[1], x$c[1], mean(x$b), stringsAsFactors = FALSE) +#' function(key, x) { +#' y <- data.frame(x[, unlist(key)][1, ], mean(x$b), stringsAsFactors = FALSE) #' }, #' schema) #' collect(df1) @@ -1322,7 +1323,7 @@ setMethod("dapplyCollect", #' df1 <- gapply( #' df, #' list(df$"Species"), -#' function(x) { +#' function(key, x) { #' m <- suppressWarnings(lm(Sepal_Length ~ #' Sepal_Width + Petal_Length + Petal_Width, x)) #' data.frame(t(coef(m))) @@ -1339,8 +1340,8 @@ setMethod("dapplyCollect", #'} setMethod("gapply", signature(x = "SparkDataFrame"), - function(x, col, func, schema) { - grouped <- do.call("groupBy", c(x, col)) + function(x, cols, func, schema) { + grouped <- do.call("groupBy", c(x, cols)) gapply(grouped, func, schema) }) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index 84df1d3194e1a..9c63da1851efd 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -201,8 +201,9 @@ readMultipleObjectsWithKeys <- function(inputCon) { # readMultipleObjectsWithKeys will read multiple continuous objects from # a DataOutputStream. There is no preceding field telling the count # of the objects, so the number of objects varies, we try to read - # all objects in a loop until the end of the stream. The rows in - # the stream are separated by grouping-key boundary + # all objects in a loop until the end of the stream. This function + # is for use by gapply. Each group of rows is followed by the grouping + # key for this group which is then followed by next group. data <- list() subData <- list() while (TRUE) { @@ -211,7 +212,7 @@ readMultipleObjectsWithKeys <- function(inputCon) { if (type == "") { break } else if (type == "r") { - # key boundary detected + # A grouping boundary detected readTypedObject(inputCon, type) data[[length(data) + 1L]] <- subData subData <- list() diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 8fd6ecc592335..36f55ef1ebc4a 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -148,10 +148,13 @@ createMethods() #' Applies a R function to each group in the input GroupedData #' #' @param x a GroupedData +#' @param func A function to be applied to each group partition specified by GroupedData. +#' The output of func is a local R data.frame. +#' @param schema The schema of the resulting SparkDataFrame after the function is applied. +#' It must match the output of func. #' @return a SparkDataFrame #' @rdname gapply #' @name gapply -#' @family agg_funcs #' @examples #' \dontrun{ #' Computes the arithmetic mean of the second column by grouping @@ -167,8 +170,8 @@ createMethods() #' df1 <- gapply( #' df, #' list("a", "c"), -#' function(x) { -#' y <- data.frame(x$a[1], x$c[1], mean(x$b), stringsAsFactors = FALSE) +#' function(key, x) { +#' y <- data.frame(x[, unlist(key)][1, ], mean(x$b), stringsAsFactors = FALSE) #' }, #' schema) #' collect(df1) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 4eec383e6c2e3..ccb8b408904c5 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -27,6 +27,58 @@ elapsedSecs <- function() { proc.time()[3] } +computeHelper <- function(mode, partition, serializer, deserializer, key, + colNames, computeFunc, outputCon, inputData) { + if (mode > 0) { + if (deserializer == "row") { + # Transform the list of rows into a data.frame + # Note that the optional argument stringsAsFactors for rbind is + # available since R 3.2.4. So we set the global option here. + oldOpt <- getOption("stringsAsFactors") + options(stringsAsFactors = FALSE) + inputData <- do.call(rbind.data.frame, inputData) + options(stringsAsFactors = oldOpt) + + names(inputData) <- colNames + } else { + # Check to see if inputData is a valid data.frame + stopifnot(deserializer == "byte") + stopifnot(class(inputData) == "data.frame") + } + + if (mode == 2) { + output <- computeFunc(key, inputData) + } else { + output <- computeFunc(inputData) + } + if (serializer == "row") { + # Transform the result data.frame back to a list of rows + output <- split(output, seq(nrow(output))) + } else { + # Serialize the ouput to a byte array + stopifnot(serializer == "byte") + } + } else { + output <- computeFunc(partition, inputData) + } + + # Timing computing + # TODO calling elapsedSecs() function directly, makes the test cases and stylecheck + # to fail + computeElap <- proc.time()[3] + + if (serializer == "byte") { + SparkR:::writeRawSerialize(outputCon, output) + } else if (serializer == "row") { + SparkR:::writeRowSerialize(outputCon, output) + } else { + # write lines one-by-one with flag + lapply(output, function(line) SparkR:::writeString(outputCon, line)) + } + + return (computeElap) +} + # Constants specialLengths <- list(END_OF_STERAM = 0L, TIMING_DATA = -1L) @@ -87,7 +139,6 @@ numPartitions <- SparkR:::readInt(inputCon) # 0 - RDD mode, 1 - dapply mode, 2 - gapply mode mode <- SparkR:::readInt(inputCon) -# If DataFrame - mode = 1 and mode = 2, then read column names if (mode > 0) { colNames <- SparkR:::readObject(inputCon) if (mode == 2) { @@ -101,62 +152,35 @@ if (isEmpty != 0) { if (numPartitions == -1) { if (deserializer == "byte") { # Now read as many characters as described in funcLen - dataList <- list(SparkR:::readDeserialize(inputCon)) + data <- SparkR:::readDeserialize(inputCon) } else if (deserializer == "string") { - dataList <- list(as.list(readLines(inputCon))) + data <- as.list(readLines(inputCon)) } else if (deserializer == "row" && mode == 2) { - dataList <- SparkR:::readMultipleObjectsWithKeys(inputCon) + data <- SparkR:::readMultipleObjectsWithKeys(inputCon) } else if (deserializer == "row"){ - dataList <- list(SparkR:::readMultipleObjects(inputCon)) + data <- SparkR:::readMultipleObjects(inputCon) } + # Timing reading input data for execution inputElap <- elapsedSecs() - for (i in 1:length(dataList)) { - data <- dataList[[i]] - if (mode > 0) { - if (deserializer == "row") { - # Transform the list of rows into a data.frame - # Note that the optional argument stringsAsFactors for rbind is - # available since R 3.2.4. So we set the global option here. - oldOpt <- getOption("stringsAsFactors") - options(stringsAsFactors = FALSE) - data <- do.call(rbind.data.frame, data) - options(stringsAsFactors = oldOpt) - - names(data) <- colNames - } else { - # Check to see if data is a valid data.frame - stopifnot(deserializer == "byte") - stopifnot(class(data) == "data.frame") - } - if (mode == 2) { - output <- computeFunc(key, data) - } else { - output <- computeFunc(data) - } - if (serializer == "row") { - # Transform the result data.frame back to a list of rows - output <- split(output, seq(nrow(output))) - } else { - # Serialize the ouput to a byte array - stopifnot(serializer == "byte") - } + if (mode > 0) { + if (mode == 1) { + computeElap <- computeHelper(mode, partition, serializer, deserializer, key, + colNames, computeFunc, outputCon, data) } else { - output <- computeFunc(partition, data) - } - - # Timing computing - computeElap <- elapsedSecs() - - if (serializer == "byte") { - SparkR:::writeRawSerialize(outputCon, output) - } else if (serializer == "row") { - SparkR:::writeRowSerialize(outputCon, output) - } else { - # write lines one-by-one with flag - lapply(output, function(line) SparkR:::writeString(outputCon, line)) + # gapply mode + for (i in 1:length(data)) { + # TODO compute the diff between outputElap and computeElap + # Currently, only `computeElap` for the last iteration will be used. + computeElap <- computeHelper(mode, partition, serializer, deserializer, key, + colNames, computeFunc, outputCon, data[[i]]) + } } + } else { + computeElap <- computeHelper(mode, partition, serializer, deserializer, key, + colNames, computeFunc, outputCon, data) } + # Timing output outputElap <- elapsedSecs() } else { diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index b4486eddd1602..b411e5da27e2c 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -163,7 +163,8 @@ private[spark] class RRunner[U]( SerDe.writeObject(dataOut, colNames) } - if (key != null) { + // Write key - the grouping columns for gapply mode + if (mode == 2) { SerDe.writeObject(dataOut, key) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index ab16eb9f7708e..e5140fcf1337e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2010,25 +2010,6 @@ class Dataset[T] private[sql]( MapPartitionsInR(func, packageNames, broadcastVars, schema, rowEncoder, logicalPlan)) } - /** - * Returns a new [[DataFrame]] which contains the aggregated result of applying - * a serialized R function `func` to each group - * - * @group func - * @since 2.0.0 - */ - private[sql] def flatMapGroupsInR( - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Object], - outputSchema: StructType, - cols: Column*): DataFrame = { - val relationalGroupedDataSet = RelationalGroupedDataset(toDF(), cols.map(_.named), - RelationalGroupedDataset.GroupByType) - relationalGroupedDataSet.flatMapGroupsInR(func, packageNames, broadcastVars, - outputSchema) - } - /** * :: Experimental :: * (Scala-specific) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 0b888f55b3a3b..bd128c2a292d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -402,7 +402,7 @@ class RelationalGroupedDataset protected[sql]( * * @since 2.0.0 */ - def flatMapGroupsInR( + private[sql] def flatMapGroupsInR( f: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Object], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 2bbae785a66d8..b08d7c24ece65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -364,8 +364,6 @@ case class FlatMapGroupsInRExec( } else { SerializationFormats.BYTE } - val (deserializerForR, colNames) = - (SerializationFormats.ROW, inputSchema.fieldNames) child.execute().mapPartitionsInternal { iter => val grouped = GroupedIterator(iter, groupingAttributes, child.output) @@ -375,8 +373,8 @@ case class FlatMapGroupsInRExec( val groupNames = groupingAttributes.map(_.name).toArray val runner = new RRunner[Array[Byte]]( - func, deserializerForR, serializerForR, packageNames, broadcastVars, - isDataFrame = true, colNames = colNames, key = groupNames) + func, SerializationFormats.ROW, serializerForR, packageNames, broadcastVars, + isDataFrame = true, colNames = inputSchema.fieldNames, key = groupNames) val hasGroups = grouped.hasNext val groupedRBytes = grouped.flatMap { case (key, rowIter) => From 453210233b55562a85eff75f2ced79f86e2e3255 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sun, 29 May 2016 12:27:33 -0700 Subject: [PATCH 18/39] merge with master + remove sqlContext --- R/pkg/R/DataFrame.R | 3 +-- R/pkg/R/group.R | 1 - R/pkg/inst/tests/testthat/test_sparkSQL.R | 1 - 3 files changed, 1 insertion(+), 4 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index e66c006b0b01b..b77f37300eab6 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1289,7 +1289,6 @@ setMethod("dapplyCollect", #' on the first and third columns. Output the grouping values and the average. #' #' df <- createDataFrame ( -#' sqlContext, #' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), #' c("a", "b", "c", "d")) #' @@ -1314,7 +1313,7 @@ setMethod("dapplyCollect", #' using 'Sepal_Length' as a target variable, 'Sepal_Width', 'Petal_Length' #' and 'Petal_Width' as training features. #' -#' df <- createDataFrame (sqlContext, iris) +#' df <- createDataFrame (iris) #' schema <- structType(structField("(Intercept)", "double"), #' structField("Sepal_Width", "double"),structField("Petal_Length", "double"), #' structField("Petal_Width", "double")) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 36f55ef1ebc4a..08d6e085bbad8 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -161,7 +161,6 @@ createMethods() #' on the first and third columns. Output the grouping values and the average. #' #' df <- createDataFrame ( -#' sqlContext, #' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), #' c("a", "b", "c", "d")) #' diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index e9785ac10e326..1012ba157d0a6 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2140,7 +2140,6 @@ test_that("repartition by columns on DataFrame", { test_that("gapply() on a DataFrame", { df <- createDataFrame ( - sqlContext, list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), c("a", "b", "c", "d")) expected <- collect(df) From 52c9f6da1cfb0c520e248e187b27cd616ed14f1d Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sun, 29 May 2016 21:57:50 -0700 Subject: [PATCH 19/39] Calling elapsedSecs() in computeHelper --- R/pkg/inst/worker/worker.R | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index ccb8b408904c5..fe0f281ed0b5c 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -62,10 +62,8 @@ computeHelper <- function(mode, partition, serializer, deserializer, key, output <- computeFunc(partition, inputData) } - # Timing computing - # TODO calling elapsedSecs() function directly, makes the test cases and stylecheck - # to fail - computeElap <- proc.time()[3] + # Computation time + computeElap <- elapsedSecs() if (serializer == "byte") { SparkR:::writeRawSerialize(outputCon, output) From 6b918583ba34b5b7514f8bd6ee87247b0256b77a Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sun, 29 May 2016 22:09:54 -0700 Subject: [PATCH 20/39] Fixing elapsedSecs() in computeHelper --- 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 fe0f281ed0b5c..2cd22268bf39c 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -63,7 +63,7 @@ computeHelper <- function(mode, partition, serializer, deserializer, key, } # Computation time - computeElap <- elapsedSecs() + computeElap <- SparkR:::elapsedSecs() if (serializer == "byte") { SparkR:::writeRawSerialize(outputCon, output) From aca539575d192056345c166adb5cf8ee0a814c84 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Mon, 30 May 2016 00:05:14 -0700 Subject: [PATCH 21/39] Bringing back the changes for computing the elap --- R/pkg/inst/worker/worker.R | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 2cd22268bf39c..3ceb140231401 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -63,7 +63,9 @@ computeHelper <- function(mode, partition, serializer, deserializer, key, } # Computation time - computeElap <- SparkR:::elapsedSecs() + # TODO calling elapsedSecs() function directly, makes the test cases and stylecheck + # to fail + computeElap <- proc.time()[3] if (serializer == "byte") { SparkR:::writeRawSerialize(outputCon, output) From a0425c17906fcd2ea1d8dd6fb33c0fd8a860d4a7 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Mon, 30 May 2016 20:39:53 -0700 Subject: [PATCH 22/39] Updated comments for - R function --- R/pkg/R/DataFrame.R | 5 +++-- R/pkg/R/group.R | 4 +++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index b77f37300eab6..00b68498cf67b 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1274,8 +1274,9 @@ setMethod("dapplyCollect", #' @param x A SparkDataFrame #' @param cols Grouping columns #' @param func A function to be applied to each group partition specified by grouping -#' column of the SparkDataFrame. -#' The output of func is a local R data.frame. +#' column of the SparkDataFrame. The function `func` takes as argument +#' a key - grouping columns and a data frame - a local R data.frame. +#' The output of `func` is a local R data.frame. #' @param schema The schema of the resulting SparkDataFrame after the function is applied. #' It must match the output of func. #' @family SparkDataFrame functions diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 08d6e085bbad8..631de74caf315 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -149,7 +149,9 @@ createMethods() #' #' @param x a GroupedData #' @param func A function to be applied to each group partition specified by GroupedData. -#' The output of func is a local R data.frame. +#' The function `func` takes as argument a key - grouping columns and +#' a data frame - a local R data.frame. +#' The output of `func` is a local R data.frame. #' @param schema The schema of the resulting SparkDataFrame after the function is applied. #' It must match the output of func. #' @return a SparkDataFrame From 7b5767ad25aaa1f091c4b2d22d7a99cf3d8ec00b Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Mon, 30 May 2016 20:50:26 -0700 Subject: [PATCH 23/39] updated SparkDataFrame's gapply comment --- R/pkg/R/DataFrame.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 00b68498cf67b..67d1638f9fc3e 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1268,8 +1268,8 @@ setMethod("dapplyCollect", #' gapply #' -#' Apply a R function to each group of a DataFrame. The group is defined by input -#' grouping columns. +#' Group the SparkDataFrame using the specified columns and apply the R function to each +#' group. The group is defined by input grouping columns. #' #' @param x A SparkDataFrame #' @param cols Grouping columns From 10f99d118e2aa94f65cb0a12ac74650533b60416 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Tue, 31 May 2016 22:18:32 -0700 Subject: [PATCH 24/39] move groupNames above the execute() --- .../main/scala/org/apache/spark/sql/execution/objects.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index b08d7c24ece65..81e812a7e0c50 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -357,6 +357,7 @@ case class FlatMapGroupsInRExec( Seq(groupingAttributes.map(SortOrder(_, Ascending))) override protected def doExecute(): RDD[InternalRow] = { + val groupNames = groupingAttributes.map(_.name).toArray val isDeserializedRData = if (outputSchema == SERIALIZED_R_DATA_SCHEMA) true else false val serializerForR = if (!isDeserializedRData) { @@ -370,8 +371,6 @@ case class FlatMapGroupsInRExec( val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) - val groupNames = groupingAttributes.map(_.name).toArray - val runner = new RRunner[Array[Byte]]( func, SerializationFormats.ROW, serializerForR, packageNames, broadcastVars, isDataFrame = true, colNames = inputSchema.fieldNames, key = groupNames) From 7e1f7c2077c60d2445565d1eed9a5f1b3ece7d1d Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sat, 4 Jun 2016 14:55:00 -0700 Subject: [PATCH 25/39] Addressed sun-rui's comments + added a new test case --- R/pkg/R/DataFrame.R | 4 +- R/pkg/R/deserialize.R | 10 ++- R/pkg/R/group.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 20 +++++- R/pkg/inst/worker/worker.R | 67 +++++++++++-------- .../org/apache/spark/api/r/RRunner.scala | 18 +---- .../apache/spark/sql/execution/objects.scala | 24 +++---- .../execution/r/MapPartitionsRWrapper.scala | 4 +- 8 files changed, 80 insertions(+), 69 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 67d1638f9fc3e..4b0dd8e1513c4 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1269,7 +1269,7 @@ setMethod("dapplyCollect", #' gapply #' #' Group the SparkDataFrame using the specified columns and apply the R function to each -#' group. The group is defined by input grouping columns. +#' group. #' #' @param x A SparkDataFrame #' @param cols Grouping columns @@ -1299,7 +1299,7 @@ setMethod("dapplyCollect", #' df, #' list("a", "c"), #' function(key, x) { -#' y <- data.frame(x[, unlist(key)][1, ], mean(x$b), stringsAsFactors = FALSE) +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) #' }, #' schema) #' collect(df1) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index 9c63da1851efd..0e99b171cabeb 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -204,6 +204,7 @@ readMultipleObjectsWithKeys <- function(inputCon) { # all objects in a loop until the end of the stream. This function # is for use by gapply. Each group of rows is followed by the grouping # key for this group which is then followed by next group. + keys <- list() data <- list() subData <- list() while (TRUE) { @@ -212,15 +213,18 @@ readMultipleObjectsWithKeys <- function(inputCon) { if (type == "") { break } else if (type == "r") { + type <- readType(inputCon) # A grouping boundary detected - readTypedObject(inputCon, type) - data[[length(data) + 1L]] <- subData + key <- readTypedObject(inputCon, type) + index <- length(data) + 1L + data[[index]] <- subData + keys[[index]] <- key subData <- list() } else { subData[[length(subData) + 1L]] <- readTypedObject(inputCon, type) } } - data # this is a list of named lists now + list(keys = keys, data = data) # this is a list of keys and corresponding data } readRowList <- function(obj) { diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 631de74caf315..b9a7dab51bc28 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -172,7 +172,7 @@ createMethods() #' df, #' list("a", "c"), #' function(key, x) { -#' y <- data.frame(x[, unlist(key)][1, ], mean(x$b), stringsAsFactors = FALSE) +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) #' }, #' schema) #' collect(df1) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 1012ba157d0a6..71054ef31ef95 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2154,7 +2154,7 @@ test_that("gapply() on a DataFrame", { df, list(df$"a", df$"c"), function(key, x) { - y <- data.frame(x[, key[[1]]][1], sum(x$b) > 2) + y <- data.frame(key[1], sum(x$b) > 2) }, schema) actual <- collect(df2)$e @@ -2169,14 +2169,30 @@ test_that("gapply() on a DataFrame", { df, list("a", "c"), function(key, x) { - y <- (data.frame(x[, unlist(key)][1, ], mean(x$b), stringsAsFactors = FALSE)) + y <- (data.frame(key, mean(x$b), stringsAsFactors = FALSE)) }, schema) actual <- collect(arrange(df3, "a", decreasing = FALSE)) expected <- collect(select(df, "a", "b", "c")) expected <- data.frame(aggregate(expected$b, by = list(expected$a, expected$c), FUN = mean)) colnames(expected) <- c("a", "c", "avg") + expect_identical(actual, expected) + df <- suppressWarnings(createDataFrame (sqlContext, iris)) + schema <- structType(structField("Sepal_Length", "double"), structField("Avg", "double")) + # Groups by `Sepal_Length` and computes the average for Sepal_Width + df1 <- gapply( + cols = list("Sepal_Length"), + df, + function(key, x) { + data.frame(key, mean(x$Sepal_Width), stringsAsFactors = FALSE) + }, + schema) + actual <- collect(arrange(df1, "Sepal_Length")) + agg_local_df <- data.frame(aggregate(iris$Sepal.Width, by = list(iris$Sepal.Length), FUN = mean), + stringsAsFactors = FALSE) + colnames(agg_local_df) <- c("Sepal_Length", "Avg") + expected <- agg_local_df[order(agg_local_df$Sepal_Length), ] expect_identical(actual, expected) }) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 3ceb140231401..faabd3756a506 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -27,8 +27,8 @@ elapsedSecs <- function() { proc.time()[3] } -computeHelper <- function(mode, partition, serializer, deserializer, key, - colNames, computeFunc, outputCon, inputData) { +compute <- function(mode, partition, serializer, deserializer, key, + colNames, computeFunc, outputCon, inputData) { if (mode > 0) { if (deserializer == "row") { # Transform the list of rows into a data.frame @@ -61,12 +61,10 @@ computeHelper <- function(mode, partition, serializer, deserializer, key, } else { output <- computeFunc(partition, inputData) } + return (output) +} - # Computation time - # TODO calling elapsedSecs() function directly, makes the test cases and stylecheck - # to fail - computeElap <- proc.time()[3] - +outputResult <- function(serializer, output, outputCon) { if (serializer == "byte") { SparkR:::writeRawSerialize(outputCon, output) } else if (serializer == "row") { @@ -75,8 +73,6 @@ computeHelper <- function(mode, partition, serializer, deserializer, key, # write lines one-by-one with flag lapply(output, function(line) SparkR:::writeString(outputCon, line)) } - - return (computeElap) } # Constants @@ -141,9 +137,6 @@ mode <- SparkR:::readInt(inputCon) if (mode > 0) { colNames <- SparkR:::readObject(inputCon) - if (mode == 2) { - key <- SparkR:::readObject(inputCon) - } } isEmpty <- SparkR:::readInt(inputCon) @@ -156,33 +149,45 @@ if (isEmpty != 0) { } else if (deserializer == "string") { data <- as.list(readLines(inputCon)) } else if (deserializer == "row" && mode == 2) { - data <- SparkR:::readMultipleObjectsWithKeys(inputCon) + dataWithKeys <- SparkR:::readMultipleObjectsWithKeys(inputCon) + keys <- dataWithKeys$keys + data <- dataWithKeys$data } else if (deserializer == "row"){ data <- SparkR:::readMultipleObjects(inputCon) } - - # Timing reading input data for execution + computeInputElapsDiff <- 0 + outputComputeElapsDiff <- 0 inputElap <- elapsedSecs() if (mode > 0) { if (mode == 1) { - computeElap <- computeHelper(mode, partition, serializer, deserializer, key, - colNames, computeFunc, outputCon, data) - } else { + # Timing reading input data for execution + output <- compute(mode, partition, serializer, deserializer, NULL, + colNames, computeFunc, outputCon, data) + } else { # gapply mode for (i in 1:length(data)) { - # TODO compute the diff between outputElap and computeElap - # Currently, only `computeElap` for the last iteration will be used. - computeElap <- computeHelper(mode, partition, serializer, deserializer, key, - colNames, computeFunc, outputCon, data[[i]]) + # Timing reading input data for execution + inputElap <- elapsedSecs() + output <- compute(mode, partition, serializer, deserializer, keys[[i]], + colNames, computeFunc, outputCon, data[[i]]) + computeElap <- elapsedSecs() + outputResult(serializer, output, outputCon) + computeInputElapsDiff <- computeInputElapsDiff + (computeElap - inputElap) + outputComputeElapsDiff <- outputComputeElapsDiff + (elapsedSecs() - computeElap) } } } else { - computeElap <- computeHelper(mode, partition, serializer, deserializer, key, - colNames, computeFunc, outputCon, data) + # Timing reading input data for execution + output <- compute(mode, partition, serializer, deserializer, NULL, + colNames, computeFunc, outputCon, data) + } + if (mode != 2) { + # Not a gapply mode + computeElap <- elapsedSecs() + outputResult(serializer, output, outputCon) + computeInputElapsDiff <- computeElap - inputElap + outputComputeElapsDiff <- elapsedSecs() - computeElap } - - # Timing output - outputElap <- elapsedSecs() } else { if (deserializer == "byte") { # Now read as many characters as described in funcLen @@ -224,11 +229,15 @@ if (isEmpty != 0) { } # Timing output outputElap <- elapsedSecs() + computeInputElapsDiff <- computeElap - inputElap + outputComputeElapsDiff <- outputElap - computeElap } } else { inputElap <- broadcastElap computeElap <- broadcastElap outputElap <- broadcastElap + computeInputElapsDiff <- computeElap - inputElap + outputComputeElapsDiff <- outputElap - computeElap } # Report timing @@ -237,8 +246,8 @@ 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 +SparkR:::writeDouble(outputCon, computeInputElapsDiff) # compute +SparkR:::writeDouble(outputCon, outputComputeElapsDiff) # output # End of output SparkR:::writeInt(outputCon, specialLengths$END_OF_STERAM) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index b411e5da27e2c..244897a174845 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -41,7 +41,7 @@ private[spark] class RRunner[U]( numPartitions: Int = -1, isDataFrame: Boolean = false, colNames: Array[String] = null, - key: Array[String] = null) + mode: Int = 0) extends Logging { private var bootTime: Double = _ private var dataStream: DataInputStream = _ @@ -149,25 +149,12 @@ private[spark] class RRunner[U]( } dataOut.writeInt(numPartitions) - - val mode = if (isDataFrame && key != null) { - 2 // gapply - } else if (isDataFrame) { - 1 // dapply - } else { - 0 // RDD - } dataOut.writeInt(mode) if (isDataFrame) { SerDe.writeObject(dataOut, colNames) } - // Write key - the grouping columns for gapply mode - if (mode == 2) { - SerDe.writeObject(dataOut, key) - } - if (!iter.hasNext) { dataOut.writeInt(0) } else { @@ -198,7 +185,8 @@ private[spark] class RRunner[U]( writeElem(innerElem) } // Writes key which can be used as a boundary in group-aggregate - SerDe.writeObject(dataOut, key.asInstanceOf[Array[Byte]]) + dataOut.writeByte('r') + writeElem(key) case (key, value) => writeElem(key) writeElem(value) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 81e812a7e0c50..320b02ab6201d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -357,10 +357,9 @@ case class FlatMapGroupsInRExec( Seq(groupingAttributes.map(SortOrder(_, Ascending))) override protected def doExecute(): RDD[InternalRow] = { - val groupNames = groupingAttributes.map(_.name).toArray - val isDeserializedRData = + val isSerializedRData = if (outputSchema == SERIALIZED_R_DATA_SCHEMA) true else false - val serializerForR = if (!isDeserializedRData) { + val serializerForR = if (!isSerializedRData) { SerializationFormats.ROW } else { SerializationFormats.BYTE @@ -373,9 +372,8 @@ case class FlatMapGroupsInRExec( val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) val runner = new RRunner[Array[Byte]]( func, SerializationFormats.ROW, serializerForR, packageNames, broadcastVars, - isDataFrame = true, colNames = inputSchema.fieldNames, key = groupNames) + isDataFrame = true, colNames = inputSchema.fieldNames, mode = 2) - val hasGroups = grouped.hasNext val groupedRBytes = grouped.flatMap { case (key, rowIter) => val deserializedIter = rowIter.map(getValue) val newIter = @@ -384,17 +382,13 @@ case class FlatMapGroupsInRExec( Iterator((newKey, newIter)) } - if (hasGroups) { - val outputIter = runner.compute(groupedRBytes, -1) - if (!isDeserializedRData) { - val result = outputIter.map { bytes => bytesToRow(bytes, outputSchema) } - result.map(outputObject) - } else { - val result = outputIter.map { bytes => Row.fromSeq(Seq(bytes)) } - result.map(outputObject) - } + val outputIter = runner.compute(groupedRBytes, -1) + if (!isSerializedRData) { + val result = outputIter.map { bytes => bytesToRow(bytes, outputSchema) } + result.map(outputObject) } else { - Iterator.empty + val result = outputIter.map { bytes => Row.fromSeq(Seq(bytes)) } + result.map(outputObject) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index ec8544e2ba375..dbfd2dc9e7c79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.types.{BinaryType, StructField, StructType} /** - * A function wrapper that applies the given R function to each partition of each group. + * A function wrapper that applies the given R function to each partition. */ private[sql] case class MapGroupsRWrapper( func: Array[Byte], @@ -87,7 +87,7 @@ object PartitionsRHelper { val runner = new RRunner[Array[Byte]]( func, deserializer, serializer, packageNames, broadcastVars, - isDataFrame = true, colNames = colNames) + isDataFrame = true, colNames = colNames, mode = 1) // Partition index is ignored. Dataset has no support for mapPartitionsWithIndex. val outputIter = runner.compute(newIter, -1) From cbde29a0752719ee7bd9fd0cce45525b1339e723 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sat, 4 Jun 2016 15:12:30 -0700 Subject: [PATCH 26/39] Optimized worker --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- R/pkg/inst/worker/worker.R | 13 +++++-------- .../spark/sql/catalyst/plans/logical/object.scala | 4 ++-- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 71054ef31ef95..2bca51c4c2214 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2180,7 +2180,7 @@ test_that("gapply() on a DataFrame", { df <- suppressWarnings(createDataFrame (sqlContext, iris)) schema <- structType(structField("Sepal_Length", "double"), structField("Avg", "double")) - # Groups by `Sepal_Length` and computes the average for Sepal_Width + # Groups by `Sepal_Length` and computes the average for `Sepal_Width` df1 <- gapply( cols = list("Sepal_Length"), df, diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index faabd3756a506..a6606ec57ad51 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -127,6 +127,8 @@ if (numBroadcastVars > 0) { # Timing broadcast broadcastElap <- elapsedSecs() +# Initial input timing +inputElap <- broadcastElap # If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int # as number of partitions to create. @@ -140,6 +142,8 @@ if (mode > 0) { } isEmpty <- SparkR:::readInt(inputCon) +computeInputElapsDiff <- 0 +outputComputeElapsDiff <- 0 if (isEmpty != 0) { if (numPartitions == -1) { @@ -155,8 +159,7 @@ if (isEmpty != 0) { } else if (deserializer == "row"){ data <- SparkR:::readMultipleObjects(inputCon) } - computeInputElapsDiff <- 0 - outputComputeElapsDiff <- 0 + inputElap <- elapsedSecs() if (mode > 0) { if (mode == 1) { @@ -232,12 +235,6 @@ if (isEmpty != 0) { computeInputElapsDiff <- computeElap - inputElap outputComputeElapsDiff <- outputElap - computeElap } -} else { - inputElap <- broadcastElap - computeElap <- broadcastElap - outputElap <- broadcastElap - computeInputElapsDiff <- computeElap - inputElap - outputComputeElapsDiff <- outputElap - computeElap } # Report timing diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 3792c7e2b0632..5ad60d319f5c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -135,7 +135,7 @@ object MapPartitionsInR { broadcastVars, encoder.schema, schema, - CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), + CatalystSerde.generateObjAttr(RowEncoder(schema)), deserialized) CatalystSerde.serialize(mapped, RowEncoder(schema)) } @@ -279,7 +279,7 @@ object FlatMapGroupsInR { UnresolvedDeserializer(encoder, dataAttributes), groupingAttributes, dataAttributes, - CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), + CatalystSerde.generateObjAttr(RowEncoder(schema)), child) CatalystSerde.serialize(mapped, RowEncoder(schema)) } From 46df2ee029aebe2433de8b1295e81d10375db29b Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sat, 4 Jun 2016 16:04:21 -0700 Subject: [PATCH 27/39] pass correct encoder and deserializer after merge --- .../apache/spark/sql/catalyst/plans/logical/object.scala | 2 +- .../org/apache/spark/sql/RelationalGroupedDataset.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index bd66bfd556745..fdb1164fd801a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -268,7 +268,7 @@ object FlatMapGroupsInR { dataAttributes, CatalystSerde.generateObjAttr(RowEncoder(schema)), child) - CatalystSerde.serialize(mapped, RowEncoder(schema)) + CatalystSerde.serialize(mapped)(RowEncoder(schema)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index c93b07e9c5408..cf685d2e8e6b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -422,9 +422,9 @@ class RelationalGroupedDataset protected[sql]( packageNames, broadcastVarObj, realOutputSchema, - df.unresolvedTEncoder.deserializer, - groupingDataFrame.unresolvedTEncoder.deserializer, - df.resolvedTEncoder, + df.exprEnc.deserializer, + groupingDataFrame.exprEnc.deserializer, + df.exprEnc, groupingAttributes, df.logicalPlan.output, df.logicalPlan)) From afa7e4e96b5590d0e122664aa58709f438ef4637 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sat, 4 Jun 2016 17:58:17 -0700 Subject: [PATCH 28/39] move output elaps to a variable --- R/pkg/inst/worker/worker.R | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index a6606ec57ad51..bb981dec8f79d 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -175,8 +175,9 @@ if (isEmpty != 0) { colNames, computeFunc, outputCon, data[[i]]) computeElap <- elapsedSecs() outputResult(serializer, output, outputCon) + outputElap <- elapsedSecs() computeInputElapsDiff <- computeInputElapsDiff + (computeElap - inputElap) - outputComputeElapsDiff <- outputComputeElapsDiff + (elapsedSecs() - computeElap) + outputComputeElapsDiff <- outputComputeElapsDiff + (outputElap - computeElap) } } } else { @@ -188,8 +189,9 @@ if (isEmpty != 0) { # Not a gapply mode computeElap <- elapsedSecs() outputResult(serializer, output, outputCon) + outputElap <- elapsedSecs() computeInputElapsDiff <- computeElap - inputElap - outputComputeElapsDiff <- elapsedSecs() - computeElap + outputComputeElapsDiff <- outputElap - computeElap } } else { if (deserializer == "byte") { From 0a2204219ae02004ef80695bb7357fc5f42c8b7e Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sat, 4 Jun 2016 19:46:54 -0700 Subject: [PATCH 29/39] revert changes in MapPartitionsRWrapper --- .../execution/r/MapPartitionsRWrapper.scala | 32 ------------------- 1 file changed, 32 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index dbfd2dc9e7c79..37f360f8a6248 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -24,21 +24,6 @@ import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.Row import org.apache.spark.sql.types.{BinaryType, StructField, StructType} -/** - * A function wrapper that applies the given R function to each partition. - */ -private[sql] case class MapGroupsRWrapper( - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Broadcast[Object]], - inputSchema: StructType, - outputSchema: StructType) extends ((Any, Iterator[Any]) => TraversableOnce[Any]) { - def apply(key: Any, iter: Iterator[Any]): TraversableOnce[Any] = { - PartitionsRHelper.mapPartitionsRHelper(func, - packageNames, broadcastVars, inputSchema, outputSchema, iter) - } -} - /** * A function wrapper that applies the given R function to each partition. */ @@ -49,23 +34,6 @@ private[sql] case class MapPartitionsRWrapper( inputSchema: StructType, outputSchema: StructType) extends (Iterator[Any] => Iterator[Any]) { def apply(iter: Iterator[Any]): Iterator[Any] = { - PartitionsRHelper.mapPartitionsRHelper( - func, packageNames, broadcastVars, inputSchema, outputSchema, iter) - } -} - -object PartitionsRHelper { - - /** - * A helper function to run R UDFs on partitions - */ - private[sql] def mapPartitionsRHelper( - func: Array[Byte], - packageNames: Array[Byte], - broadcastVars: Array[Broadcast[Object]], - inputSchema: StructType, - outputSchema: StructType, - iter: Iterator[Any]): Iterator[Any] = { // If the content of current DataFrame is serialized R data? val isSerializedRData = if (inputSchema == SERIALIZED_R_DATA_SCHEMA) true else false From e4fa8e66896be19430ae4cfabef2669b5ecc4dd7 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sun, 5 Jun 2016 20:13:08 -0700 Subject: [PATCH 30/39] Fixing test cases - sorting the dataframe after collecting it --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 2bca51c4c2214..30d012e096498 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2172,10 +2172,14 @@ test_that("gapply() on a DataFrame", { y <- (data.frame(key, mean(x$b), stringsAsFactors = FALSE)) }, schema) - actual <- collect(arrange(df3, "a", decreasing = FALSE)) + actual <- collect(df3) + actual <- actual[order(actual$a), ] + rownames(actual) <- NULL expected <- collect(select(df, "a", "b", "c")) expected <- data.frame(aggregate(expected$b, by = list(expected$a, expected$c), FUN = mean)) colnames(expected) <- c("a", "c", "avg") + expected <- expected[order(expected$a), ] + rownames(expected) <- NULL expect_identical(actual, expected) df <- suppressWarnings(createDataFrame (sqlContext, iris)) @@ -2188,11 +2192,14 @@ test_that("gapply() on a DataFrame", { data.frame(key, mean(x$Sepal_Width), stringsAsFactors = FALSE) }, schema) - actual <- collect(arrange(df1, "Sepal_Length")) + actual <- collect(df1) + actual <- actual[order(actual$Sepal_Length), ] + rownames(actual) <- NULL agg_local_df <- data.frame(aggregate(iris$Sepal.Width, by = list(iris$Sepal.Length), FUN = mean), stringsAsFactors = FALSE) colnames(agg_local_df) <- c("Sepal_Length", "Avg") expected <- agg_local_df[order(agg_local_df$Sepal_Length), ] + rownames(expected) <- NULL expect_identical(actual, expected) }) From 00a091ef74c85d4a5830245aa65b0bf557a144aa Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Mon, 6 Jun 2016 00:07:23 -0700 Subject: [PATCH 31/39] test case with iris - removing sqlContext argument + other small changes --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 30d012e096498..a823b81065311 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2164,12 +2164,12 @@ test_that("gapply() on a DataFrame", { # Computes the arithmetic mean of the second column by grouping # on the first and third columns. Output the groupping value and the average. schema <- structType(structField("a", "integer"), structField("c", "string"), - structField("avg", "double")) + structField("avg", "double")) df3 <- gapply( df, list("a", "c"), function(key, x) { - y <- (data.frame(key, mean(x$b), stringsAsFactors = FALSE)) + y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, schema) actual <- collect(df3) @@ -2182,17 +2182,17 @@ test_that("gapply() on a DataFrame", { rownames(expected) <- NULL expect_identical(actual, expected) - df <- suppressWarnings(createDataFrame (sqlContext, iris)) + irisDF <- suppressWarnings(createDataFrame (iris)) schema <- structType(structField("Sepal_Length", "double"), structField("Avg", "double")) # Groups by `Sepal_Length` and computes the average for `Sepal_Width` - df1 <- gapply( + df4 <- gapply( cols = list("Sepal_Length"), - df, + irisDF, function(key, x) { - data.frame(key, mean(x$Sepal_Width), stringsAsFactors = FALSE) + y <- data.frame(key, mean(x$Sepal_Width), stringsAsFactors = FALSE) }, schema) - actual <- collect(df1) + actual <- collect(df4) actual <- actual[order(actual$Sepal_Length), ] rownames(actual) <- NULL agg_local_df <- data.frame(aggregate(iris$Sepal.Width, by = list(iris$Sepal.Length), FUN = mean), From afa385dde3dbbebb2e7f605566e8827187fb434e Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Wed, 8 Jun 2016 22:24:48 -0700 Subject: [PATCH 32/39] fixes white spaces in test cases --- R/pkg/R/group.R | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index b9a7dab51bc28..1bbb38d6fa854 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -187,13 +187,13 @@ setMethod("gapply", signature(x = "GroupedData"), function(x, func, schema) { packageNamesArr <- serialize(.sparkREnv[[".packages"]], - connection = NULL) + connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), - function(name) { get(name, .broadcastNames) }) + function(name) { get(name, .broadcastNames) }) sdf <- callJMethod(x@sgd, "flatMapGroupsInR", - serialize(cleanClosure(func), connection = NULL), - packageNamesArr, - broadcastArr, - schema$jobj) + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + schema$jobj) dataFrame(sdf) }) From 20a1c37f348ca90a03100814e6c3d5327396a6ce Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Thu, 9 Jun 2016 22:49:35 -0700 Subject: [PATCH 33/39] overriding stringArgs in FlatMapGroupsInR --- .../apache/spark/sql/catalyst/plans/logical/object.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index fdb1164fd801a..8fbe174fb6ad8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -140,7 +140,11 @@ case class MapPartitionsInR( outputSchema: StructType, outputObjAttr: Attribute, child: LogicalPlan) extends ObjectConsumer with ObjectProducer { + override lazy val schema = outputSchema + + override protected def stringArgs: Iterator[Any] = Iterator(inputSchema, outputSchema, + outputObjAttr, child) } object MapElements { @@ -286,6 +290,9 @@ case class FlatMapGroupsInR( child: LogicalPlan) extends UnaryNode with ObjectProducer{ override lazy val schema = outputSchema + + override protected def stringArgs: Iterator[Any] = Iterator(inputSchema, outputSchema, + outputObjAttr, child) } /** Factory for constructing new `CoGroup` nodes. */ From 0ca74fddc91a94d8b5c69c2e510afeee8531c0e2 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Fri, 10 Jun 2016 00:12:08 -0700 Subject: [PATCH 34/39] overriding stringArgs in FlatMapGroupsInR: changing input arguments --- .../apache/spark/sql/catalyst/plans/logical/object.scala | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 8fbe174fb6ad8..6ed5eb349b724 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -140,11 +140,7 @@ case class MapPartitionsInR( outputSchema: StructType, outputObjAttr: Attribute, child: LogicalPlan) extends ObjectConsumer with ObjectProducer { - override lazy val schema = outputSchema - - override protected def stringArgs: Iterator[Any] = Iterator(inputSchema, outputSchema, - outputObjAttr, child) } object MapElements { @@ -292,7 +288,8 @@ case class FlatMapGroupsInR( override lazy val schema = outputSchema override protected def stringArgs: Iterator[Any] = Iterator(inputSchema, outputSchema, - outputObjAttr, child) + keyDeserializer, valueDeserializer, groupingAttributes, dataAttributes, outputObjAttr, + child) } /** Factory for constructing new `CoGroup` nodes. */ From d51441f704e2abad7f7a3cc829664cd201b0fcd2 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sat, 11 Jun 2016 15:12:27 -0700 Subject: [PATCH 35/39] Addressed Cheng Lian's comments --- R/pkg/R/group.R | 2 +- .../sql/catalyst/plans/logical/object.scala | 32 +++++++++---------- .../spark/sql/RelationalGroupedDataset.scala | 4 +-- .../apache/spark/sql/execution/objects.scala | 2 +- 4 files changed, 20 insertions(+), 20 deletions(-) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 1bbb38d6fa854..e17e05b56eec9 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -194,6 +194,6 @@ setMethod("gapply", serialize(cleanClosure(func), connection = NULL), packageNamesArr, broadcastArr, - schema$jobj) + if (is.null(schema)) { schema } else { schema$jobj }) dataFrame(sdf) }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 6ed5eb349b724..c9be95ac41270 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -250,25 +250,25 @@ object FlatMapGroupsInR { packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], schema: StructType, - encoder: Expression, - keyEncoder: Expression, - rowEncoder: ExpressionEncoder[Row], + keyDeserializer: Expression, + valueDeserializer: Expression, + inputSchema: StructType, groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], child: LogicalPlan): LogicalPlan = { - val mapped = FlatMapGroupsInR( - func, - packageNames, - broadcastVars, - rowEncoder.schema, - schema, - UnresolvedDeserializer(keyEncoder, groupingAttributes), - UnresolvedDeserializer(encoder, dataAttributes), - groupingAttributes, - dataAttributes, - CatalystSerde.generateObjAttr(RowEncoder(schema)), - child) - CatalystSerde.serialize(mapped)(RowEncoder(schema)) + val mapped = FlatMapGroupsInR( + func, + packageNames, + broadcastVars, + inputSchema, + schema, + UnresolvedDeserializer(keyDeserializer, groupingAttributes), + UnresolvedDeserializer(valueDeserializer, dataAttributes), + groupingAttributes, + dataAttributes, + CatalystSerde.generateObjAttr(RowEncoder(schema)), + child) + CatalystSerde.serialize(mapped)(RowEncoder(schema)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index cf685d2e8e6b1..b9bc29c2fad76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -422,9 +422,9 @@ class RelationalGroupedDataset protected[sql]( packageNames, broadcastVarObj, realOutputSchema, - df.exprEnc.deserializer, groupingDataFrame.exprEnc.deserializer, - df.exprEnc, + df.exprEnc.deserializer, + df.exprEnc.schema, groupingAttributes, df.logicalPlan.output, df.logicalPlan)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 320b02ab6201d..87960ec2c5eca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -377,7 +377,7 @@ case class FlatMapGroupsInRExec( val groupedRBytes = grouped.flatMap { case (key, rowIter) => val deserializedIter = rowIter.map(getValue) val newIter = - deserializedIter.asInstanceOf[Iterator[Row]].map {row => rowToRBytes(row)} + deserializedIter.asInstanceOf[Iterator[Row]].map { row => rowToRBytes(row) } val newKey = rowToRBytes(getKey(key).asInstanceOf[Row]) Iterator((newKey, newIter)) } From 1aa368d73708eda2d55f128c88420104a3e62870 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Mon, 13 Jun 2016 00:32:04 -0700 Subject: [PATCH 36/39] Addressed sun-rui's comments --- R/pkg/R/group.R | 8 ++++++-- R/pkg/inst/worker/worker.R | 12 ++++++------ .../scala/org/apache/spark/api/r/RRunner.scala | 8 +++++++- .../spark/sql/RelationalGroupedDataset.scala | 2 +- .../org/apache/spark/sql/api/r/SQLUtils.scala | 17 ++++++++++++++++- .../apache/spark/sql/execution/objects.scala | 10 +++++----- .../sql/execution/r/MapPartitionsRWrapper.scala | 5 ++--- 7 files changed, 43 insertions(+), 19 deletions(-) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index e17e05b56eec9..2676e204eff6a 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -186,14 +186,18 @@ createMethods() setMethod("gapply", signature(x = "GroupedData"), function(x, func, schema) { + try(if (is.null(schema)) stop("schema cannot be NULL")) packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - sdf <- callJMethod(x@sgd, "flatMapGroupsInR", + sdf <- callJStatic( + "org.apache.spark.sql.api.r.SQLUtils", + "gapply", + x@sgd, serialize(cleanClosure(func), connection = NULL), packageNamesArr, broadcastArr, - if (is.null(schema)) { schema } else { schema$jobj }) + schema$jobj) dataFrame(sdf) }) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index bb981dec8f79d..1936c33709383 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -28,7 +28,7 @@ elapsedSecs <- function() { } compute <- function(mode, partition, serializer, deserializer, key, - colNames, computeFunc, outputCon, inputData) { + colNames, computeFunc, inputData) { if (mode > 0) { if (deserializer == "row") { # Transform the list of rows into a data.frame @@ -156,23 +156,23 @@ if (isEmpty != 0) { dataWithKeys <- SparkR:::readMultipleObjectsWithKeys(inputCon) keys <- dataWithKeys$keys data <- dataWithKeys$data - } else if (deserializer == "row"){ + } else if (deserializer == "row") { data <- SparkR:::readMultipleObjects(inputCon) } + # Timing reading input data for execution inputElap <- elapsedSecs() if (mode > 0) { if (mode == 1) { - # Timing reading input data for execution output <- compute(mode, partition, serializer, deserializer, NULL, - colNames, computeFunc, outputCon, data) + colNames, computeFunc, data) } else { # gapply mode for (i in 1:length(data)) { # Timing reading input data for execution inputElap <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], - colNames, computeFunc, outputCon, data[[i]]) + colNames, computeFunc, data[[i]]) computeElap <- elapsedSecs() outputResult(serializer, output, outputCon) outputElap <- elapsedSecs() @@ -183,7 +183,7 @@ if (isEmpty != 0) { } else { # Timing reading input data for execution output <- compute(mode, partition, serializer, deserializer, NULL, - colNames, computeFunc, outputCon, data) + colNames, computeFunc, data) } if (mode != 2) { # Not a gapply mode diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 244897a174845..496fdf851f7db 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -41,7 +41,7 @@ private[spark] class RRunner[U]( numPartitions: Int = -1, isDataFrame: Boolean = false, colNames: Array[String] = null, - mode: Int = 0) + mode: Int = RRunnerModes.RDD) extends Logging { private var bootTime: Double = _ private var dataStream: DataInputStream = _ @@ -276,6 +276,12 @@ private object SpecialLengths { val TIMING_DATA = -1 } +private[spark] object RRunnerModes { + val RDD = 0 + val DATAFRAME_DAPPLY = 1 + val DATAFRAME_GAPPLY = 2 +} + private[r] class BufferedStreamThread( in: InputStream, name: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index b9bc29c2fad76..71cf4b84e86a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -407,7 +407,7 @@ class RelationalGroupedDataset protected[sql]( private[sql] def flatMapGroupsInR( f: Array[Byte], packageNames: Array[Byte], - broadcastVars: Array[Object], + broadcastVars: Array[Broadcast[Object]], outputSchema: StructType): DataFrame = { val broadcastVarObj = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) val groupingNamedExpressions = groupingExprs.map(alias) 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 ea4e5b4489c7d..fe426fa3c7e8a 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 @@ -26,7 +26,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLContext} +import org.apache.spark.sql.{DataFrame, RelationalGroupedDataset, Row, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.types._ @@ -151,6 +151,21 @@ private[sql] object SQLUtils { df.mapPartitionsInR(func, packageNames, bv, realSchema) } + /** + * The helper function for gapply() on R side. + */ + def gapply( + gd: RelationalGroupedDataset, + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + schema: StructType): DataFrame = { + val bv = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) + val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema + gd.flatMapGroupsInR(func, packageNames, bv, realSchema) + } + + def dfToCols(df: DataFrame): Array[Array[Any]] = { val localDF: Array[Row] = df.collect() val numCols = df.columns.length diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 87960ec2c5eca..c7e267152b5cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.execution import scala.language.existentials import org.apache.spark.api.java.function.MapFunction -import org.apache.spark.api.r.RRunner -import org.apache.spark.api.r.SerializationFormats +import org.apache.spark.api.r._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.api.r.SQLUtils._ @@ -372,14 +371,15 @@ case class FlatMapGroupsInRExec( val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) val runner = new RRunner[Array[Byte]]( func, SerializationFormats.ROW, serializerForR, packageNames, broadcastVars, - isDataFrame = true, colNames = inputSchema.fieldNames, mode = 2) + isDataFrame = true, colNames = inputSchema.fieldNames, + mode = RRunnerModes.DATAFRAME_GAPPLY) - val groupedRBytes = grouped.flatMap { case (key, rowIter) => + val groupedRBytes = grouped.map { case (key, rowIter) => val deserializedIter = rowIter.map(getValue) val newIter = deserializedIter.asInstanceOf[Iterator[Row]].map { row => rowToRBytes(row) } val newKey = rowToRBytes(getKey(key).asInstanceOf[Row]) - Iterator((newKey, newIter)) + (newKey, newIter) } val outputIter = runner.compute(groupedRBytes, -1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index 87fad7082b33b..70539da348b0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.execution.r -import org.apache.spark.api.r.RRunner -import org.apache.spark.api.r.SerializationFormats +import org.apache.spark.api.r._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.Row @@ -55,7 +54,7 @@ private[sql] case class MapPartitionsRWrapper( val runner = new RRunner[Array[Byte]]( func, deserializer, serializer, packageNames, broadcastVars, - isDataFrame = true, colNames = colNames, mode = 1) + isDataFrame = true, colNames = colNames, mode = RRunnerModes.DATAFRAME_DAPPLY) // Partition index is ignored. Dataset has no support for mapPartitionsWithIndex. val outputIter = runner.compute(newIter, -1) From 91e1944ffa4927daeedaf5c80b4f3804858d3f0a Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Mon, 13 Jun 2016 00:46:08 -0700 Subject: [PATCH 37/39] RelationalGroupedData.flatMapGroupsInR use passed arguments --- .../org/apache/spark/sql/RelationalGroupedDataset.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 71cf4b84e86a1..1aa5767038d53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -409,19 +409,17 @@ class RelationalGroupedDataset protected[sql]( packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], outputSchema: StructType): DataFrame = { - val broadcastVarObj = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) val groupingNamedExpressions = groupingExprs.map(alias) val groupingCols = groupingNamedExpressions.map(Column(_)) val groupingDataFrame = df.select(groupingCols : _*) val groupingAttributes = groupingNamedExpressions.map(_.toAttribute) - val realOutputSchema = if (outputSchema == null) SERIALIZED_R_DATA_SCHEMA else outputSchema Dataset.ofRows( df.sparkSession, FlatMapGroupsInR( f, packageNames, - broadcastVarObj, - realOutputSchema, + broadcastVars, + outputSchema, groupingDataFrame.exprEnc.deserializer, df.exprEnc.deserializer, df.exprEnc.schema, From 4d1cc6b0fd3dd2839a6a879f43118c5828916733 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Wed, 15 Jun 2016 09:29:54 -0700 Subject: [PATCH 38/39] removed unnecessary comment --- R/pkg/inst/worker/worker.R | 1 - 1 file changed, 1 deletion(-) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 1936c33709383..debf0180183a4 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -181,7 +181,6 @@ if (isEmpty != 0) { } } } else { - # Timing reading input data for execution output <- compute(mode, partition, serializer, deserializer, NULL, colNames, computeFunc, data) } From fe36d24139ca0fe22b80836b85c7ec67503c1104 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Wed, 15 Jun 2016 19:26:04 -0700 Subject: [PATCH 39/39] Updated examples' doc --- R/pkg/R/DataFrame.R | 5 ++++- R/pkg/R/group.R | 5 ++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 4b0dd8e1513c4..c87019d000946 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1278,7 +1278,8 @@ setMethod("dapplyCollect", #' a key - grouping columns and a data frame - a local R data.frame. #' The output of `func` is a local R data.frame. #' @param schema The schema of the resulting SparkDataFrame after the function is applied. -#' It must match the output of func. +#' The schema must match to output of `func`. It has to be defined for each +#' output column with preferred output column name and corresponding data type. #' @family SparkDataFrame functions #' @rdname gapply #' @name gapply @@ -1293,6 +1294,8 @@ setMethod("dapplyCollect", #' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), #' c("a", "b", "c", "d")) #' +#' Here our output contains three columns, the key which is a combination of two +#' columns with data types integer and string and the mean which is a double. #' schema <- structType(structField("a", "integer"), structField("c", "string"), #' structField("avg", "double")) #' df1 <- gapply( diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 2676e204eff6a..b7047769175a3 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -153,7 +153,8 @@ createMethods() #' a data frame - a local R data.frame. #' The output of `func` is a local R data.frame. #' @param schema The schema of the resulting SparkDataFrame after the function is applied. -#' It must match the output of func. +#' The schema must match to output of `func`. It has to be defined for each +#' output column with preferred output column name and corresponding data type. #' @return a SparkDataFrame #' @rdname gapply #' @name gapply @@ -166,6 +167,8 @@ createMethods() #' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), #' c("a", "b", "c", "d")) #' +#' Here our output contains three columns, the key which is a combination of two +#' columns with data types integer and string and the mean which is a double. #' schema <- structType(structField("a", "integer"), structField("c", "string"), #' structField("avg", "double")) #' df1 <- gapply(