Repository: spark Updated Branches: refs/heads/master 19ada15d1 -> ddd1b1e8a
[SPARK-24572][SPARKR] "eager execution" for R shell, IDE ## What changes were proposed in this pull request? Check the `spark.sql.repl.eagerEval.enabled` configuration property in SparkDataFrame `show()` method. If the `SparkSession` has eager execution enabled, the data will be returned to the R client when the data frame is created. So instead of seeing this ``` > df <- createDataFrame(faithful) > df SparkDataFrame[eruptions:double, waiting:double] ``` you will see ``` > df <- createDataFrame(faithful) > df +---------+-------+ |eruptions|waiting| +---------+-------+ | 3.6| 79.0| | 1.8| 54.0| | 3.333| 74.0| | 2.283| 62.0| | 4.533| 85.0| | 2.883| 55.0| | 4.7| 88.0| | 3.6| 85.0| | 1.95| 51.0| | 4.35| 85.0| | 1.833| 54.0| | 3.917| 84.0| | 4.2| 78.0| | 1.75| 47.0| | 4.7| 83.0| | 2.167| 52.0| | 1.75| 62.0| | 4.8| 84.0| | 1.6| 52.0| | 4.25| 79.0| +---------+-------+ only showing top 20 rows ``` ## How was this patch tested? Manual tests as well as unit tests (one new test case is added). Author: adrian555 <v2ave10p> Closes #22455 from adrian555/eager_execution. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ddd1b1e8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ddd1b1e8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ddd1b1e8 Branch: refs/heads/master Commit: ddd1b1e8aec023e61b186c494ccbc182db2eb3ca Parents: 19ada15 Author: adrian555 <v2ave10p> Authored: Wed Oct 24 23:42:06 2018 -0700 Committer: Felix Cheung <[email protected]> Committed: Wed Oct 24 23:42:06 2018 -0700 ---------------------------------------------------------------------- R/pkg/R/DataFrame.R | 36 ++++++++-- R/pkg/tests/fulltests/test_sparkSQL_eager.R | 72 ++++++++++++++++++++ docs/sparkr.md | 42 ++++++++++++ .../org/apache/spark/sql/internal/SQLConf.scala | 7 +- 4 files changed, 148 insertions(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/ddd1b1e8/R/pkg/R/DataFrame.R ---------------------------------------------------------------------- diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 3469188..bf82d0c 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -226,7 +226,9 @@ setMethod("showDF", #' show #' -#' Print class and type information of a Spark object. +#' If eager evaluation is enabled and the Spark object is a SparkDataFrame, evaluate the +#' SparkDataFrame and print top rows of the SparkDataFrame, otherwise, print the class +#' and type information of the Spark object. #' #' @param object a Spark object. Can be a SparkDataFrame, Column, GroupedData, WindowSpec. #' @@ -244,11 +246,33 @@ setMethod("showDF", #' @note show(SparkDataFrame) since 1.4.0 setMethod("show", "SparkDataFrame", function(object) { - cols <- lapply(dtypes(object), function(l) { - paste(l, collapse = ":") - }) - s <- paste(cols, collapse = ", ") - cat(paste(class(object), "[", s, "]\n", sep = "")) + allConf <- sparkR.conf() + prop <- allConf[["spark.sql.repl.eagerEval.enabled"]] + if (!is.null(prop) && identical(prop, "true")) { + argsList <- list() + argsList$x <- object + prop <- allConf[["spark.sql.repl.eagerEval.maxNumRows"]] + if (!is.null(prop)) { + numRows <- as.integer(prop) + if (numRows > 0) { + argsList$numRows <- numRows + } + } + prop <- allConf[["spark.sql.repl.eagerEval.truncate"]] + if (!is.null(prop)) { + truncate <- as.integer(prop) + if (truncate > 0) { + argsList$truncate <- truncate + } + } + do.call(showDF, argsList) + } else { + cols <- lapply(dtypes(object), function(l) { + paste(l, collapse = ":") + }) + s <- paste(cols, collapse = ", ") + cat(paste(class(object), "[", s, "]\n", sep = "")) + } }) #' DataTypes http://git-wip-us.apache.org/repos/asf/spark/blob/ddd1b1e8/R/pkg/tests/fulltests/test_sparkSQL_eager.R ---------------------------------------------------------------------- diff --git a/R/pkg/tests/fulltests/test_sparkSQL_eager.R b/R/pkg/tests/fulltests/test_sparkSQL_eager.R new file mode 100644 index 0000000..df7354f --- /dev/null +++ b/R/pkg/tests/fulltests/test_sparkSQL_eager.R @@ -0,0 +1,72 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("test show SparkDataFrame when eager execution is enabled.") + +test_that("eager execution is not enabled", { + # Start Spark session without eager execution enabled + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + + df <- createDataFrame(faithful) + expect_is(df, "SparkDataFrame") + expected <- "eruptions:double, waiting:double" + expect_output(show(df), expected) + + # Stop Spark session + sparkR.session.stop() +}) + +test_that("eager execution is enabled", { + # Start Spark session with eager execution enabled + sparkConfig <- list(spark.sql.repl.eagerEval.enabled = "true") + + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE, sparkConfig = sparkConfig) + + df <- createDataFrame(faithful) + expect_is(df, "SparkDataFrame") + expected <- paste0("(+---------+-------+\n", + "|eruptions|waiting|\n", + "+---------+-------+\n)*", + "(only showing top 20 rows)") + expect_output(show(df), expected) + + # Stop Spark session + sparkR.session.stop() +}) + +test_that("eager execution is enabled with maxNumRows and truncate set", { + # Start Spark session with eager execution enabled + sparkConfig <- list(spark.sql.repl.eagerEval.enabled = "true", + spark.sql.repl.eagerEval.maxNumRows = as.integer(5), + spark.sql.repl.eagerEval.truncate = as.integer(2)) + + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE, sparkConfig = sparkConfig) + + df <- arrange(createDataFrame(faithful), "waiting") + expect_is(df, "SparkDataFrame") + expected <- paste0("(+---------+-------+\n", + "|eruptions|waiting|\n", + "+---------+-------+\n", + "| 1.| 43|\n)*", + "(only showing top 5 rows)") + expect_output(show(df), expected) + + # Stop Spark session + sparkR.session.stop() +}) http://git-wip-us.apache.org/repos/asf/spark/blob/ddd1b1e8/docs/sparkr.md ---------------------------------------------------------------------- diff --git a/docs/sparkr.md b/docs/sparkr.md index ba4cca8..79f8ab8 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -450,6 +450,48 @@ print(model.summaries) {% endhighlight %} </div> +### Eager execution + +If eager execution is enabled, the data will be returned to R client immediately when the `SparkDataFrame` is created. By default, eager execution is not enabled and can be enabled by setting the configuration property `spark.sql.repl.eagerEval.enabled` to `true` when the `SparkSession` is started up. + +Maximum number of rows and maximum number of characters per column of data to display can be controlled by `spark.sql.repl.eagerEval.maxNumRows` and `spark.sql.repl.eagerEval.truncate` configuration properties, respectively. These properties are only effective when eager execution is enabled. If these properties are not set explicitly, by default, data up to 20 rows and up to 20 characters per column will be showed. + +<div data-lang="r" markdown="1"> +{% highlight r %} + +# Start up spark session with eager execution enabled +sparkR.session(master = "local[*]", + sparkConfig = list(spark.sql.repl.eagerEval.enabled = "true", + spark.sql.repl.eagerEval.maxNumRows = as.integer(10))) + +# Create a grouped and sorted SparkDataFrame +df <- createDataFrame(faithful) +df2 <- arrange(summarize(groupBy(df, df$waiting), count = n(df$waiting)), "waiting") + +# Similar to R data.frame, displays the data returned, instead of SparkDataFrame class string +df2 + +##+-------+-----+ +##|waiting|count| +##+-------+-----+ +##| 43.0| 1| +##| 45.0| 3| +##| 46.0| 5| +##| 47.0| 4| +##| 48.0| 3| +##| 49.0| 5| +##| 50.0| 5| +##| 51.0| 6| +##| 52.0| 5| +##| 53.0| 7| +##+-------+-----+ +##only showing top 10 rows + +{% endhighlight %} +</div> + +Note that to enable eager execution in `sparkR` shell, add `spark.sql.repl.eagerEval.enabled=true` configuration property to the `--conf` option. + ## Running SQL Queries from SparkR A SparkDataFrame can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data. The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. http://git-wip-us.apache.org/repos/asf/spark/blob/ddd1b1e8/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index da70d7d..e852955 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1482,9 +1482,10 @@ object SQLConf { val REPL_EAGER_EVAL_ENABLED = buildConf("spark.sql.repl.eagerEval.enabled") .doc("Enables eager evaluation or not. When true, the top K rows of Dataset will be " + "displayed if and only if the REPL supports the eager evaluation. Currently, the " + - "eager evaluation is only supported in PySpark. For the notebooks like Jupyter, " + - "the HTML table (generated by _repr_html_) will be returned. For plain Python REPL, " + - "the returned outputs are formatted like dataframe.show().") + "eager evaluation is supported in PySpark and SparkR. In PySpark, for the notebooks like " + + "Jupyter, the HTML table (generated by _repr_html_) will be returned. For plain Python " + + "REPL, the returned outputs are formatted like dataframe.show(). In SparkR, the returned " + + "outputs are showed similar to R data.frame would.") .booleanConf .createWithDefault(false) --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
