You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2016/04/23 09:20:31 UTC

[2/2] spark git commit: [SPARK-12148][SPARKR] SparkR: rename DataFrame to SparkDataFrame

[SPARK-12148][SPARKR] SparkR: rename DataFrame to SparkDataFrame

## What changes were proposed in this pull request?

Changed class name defined in R from "DataFrame" to "SparkDataFrame". A popular package, S4Vector already defines "DataFrame" - this change is to avoid conflict.

Aside from class name and API/roxygen2 references, SparkR APIs like `createDataFrame`, `as.DataFrame` are not changed (S4Vector does not define a "as.DataFrame").

Since in R, one would rarely reference type/class, this change should have minimal/almost-no impact to a SparkR user in terms of back compat.

## How was this patch tested?

SparkR tests, manually loading S4Vector then SparkR package

Author: felixcheung <fe...@hotmail.com>

Closes #12621 from felixcheung/rdataframe.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a55fbe2a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a55fbe2a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a55fbe2a

Branch: refs/heads/master
Commit: a55fbe2a16aa0866ff8aca25bf9f772e6eb516a1
Parents: 86ca8fe
Author: felixcheung <fe...@hotmail.com>
Authored: Sat Apr 23 00:20:27 2016 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Sat Apr 23 00:20:27 2016 -0700

----------------------------------------------------------------------
 R/pkg/NAMESPACE                           |   2 +-
 R/pkg/R/DataFrame.R                       | 653 +++++++++++++------------
 R/pkg/R/RDD.R                             |   4 +-
 R/pkg/R/SQLContext.R                      |  70 +--
 R/pkg/R/column.R                          |   6 +-
 R/pkg/R/deserialize.R                     |   2 +-
 R/pkg/R/functions.R                       |   2 +-
 R/pkg/R/generics.R                        |   2 +-
 R/pkg/R/group.R                           |  14 +-
 R/pkg/R/mllib.R                           |  38 +-
 R/pkg/R/schema.R                          |   6 +-
 R/pkg/R/stats.R                           |  38 +-
 R/pkg/R/utils.R                           |   2 +-
 R/pkg/inst/tests/testthat/test_sparkSQL.R | 102 ++--
 14 files changed, 473 insertions(+), 468 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/NAMESPACE
----------------------------------------------------------------------
diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE
index b3aff10..0f92b5e 100644
--- a/R/pkg/NAMESPACE
+++ b/R/pkg/NAMESPACE
@@ -27,7 +27,7 @@ export("setJobGroup",
 # Export Utility methods
 export("setLogLevel")
 
-exportClasses("DataFrame")
+exportClasses("SparkDataFrame")
 
 exportMethods("arrange",
               "as.data.frame",

http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/R/DataFrame.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R
index 95e2eb2..69feec7 100644
--- a/R/pkg/R/DataFrame.R
+++ b/R/pkg/R/DataFrame.R
@@ -15,21 +15,21 @@
 # limitations under the License.
 #
 
-# DataFrame.R - DataFrame class and methods implemented in S4 OO classes
+# DataFrame.R - SparkDataFrame class and methods implemented in S4 OO classes
 
 #' @include generics.R jobj.R schema.R RDD.R pairRDD.R column.R group.R
 NULL
 
 setOldClass("jobj")
 
-#' @title S4 class that represents a DataFrame
+#' @title S4 class that represents a SparkDataFrame
 #' @description DataFrames can be created using functions like \link{createDataFrame},
 #'              \link{read.json}, \link{table} etc.
-#' @family DataFrame functions
-#' @rdname DataFrame
+#' @family SparkSparkDataFrame functions
+#' @rdname SparkDataFrame
 #' @docType class
 #'
-#' @slot env An R environment that stores bookkeeping states of the DataFrame
+#' @slot env An R environment that stores bookkeeping states of the SparkDataFrame
 #' @slot sdf A Java object reference to the backing Scala DataFrame
 #' @seealso \link{createDataFrame}, \link{read.json}, \link{table}
 #' @seealso \url{https://spark.apache.org/docs/latest/sparkr.html#sparkr-dataframes}
@@ -40,11 +40,11 @@ setOldClass("jobj")
 #' sqlContext <- sparkRSQL.init(sc)
 #' df <- createDataFrame(sqlContext, faithful)
 #'}
-setClass("DataFrame",
+setClass("SparkDataFrame",
          slots = list(env = "environment",
                       sdf = "jobj"))
 
-setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) {
+setMethod("initialize", "SparkDataFrame", function(.Object, sdf, isCached) {
   .Object@env <- new.env()
   .Object@env$isCached <- isCached
 
@@ -52,23 +52,23 @@ setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) {
   .Object
 })
 
-#' @rdname DataFrame
+#' @rdname SparkDataFrame
 #' @export
 #' @param sdf A Java object reference to the backing Scala DataFrame
-#' @param isCached TRUE if the dataFrame is cached
+#' @param isCached TRUE if the SparkDataFrame is cached
 dataFrame <- function(sdf, isCached = FALSE) {
-  new("DataFrame", sdf, isCached)
+  new("SparkDataFrame", sdf, isCached)
 }
 
-############################ DataFrame Methods ##############################################
+############################ SparkDataFrame Methods ##############################################
 
-#' Print Schema of a DataFrame
+#' Print Schema of a SparkDataFrame
 #'
 #' Prints out the schema in tree format
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkSparkDataFrame functions
 #' @rdname printSchema
 #' @name printSchema
 #' @export
@@ -81,7 +81,7 @@ dataFrame <- function(sdf, isCached = FALSE) {
 #' printSchema(df)
 #'}
 setMethod("printSchema",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             schemaString <- callJMethod(schema(x)$jobj, "treeString")
             cat(schemaString)
@@ -89,11 +89,11 @@ setMethod("printSchema",
 
 #' Get schema object
 #'
-#' Returns the schema of this DataFrame as a structType object.
+#' Returns the schema of this SparkDataFrame as a structType object.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkSparkDataFrame functions
 #' @rdname schema
 #' @name schema
 #' @export
@@ -106,7 +106,7 @@ setMethod("printSchema",
 #' dfSchema <- schema(df)
 #'}
 setMethod("schema",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             structType(callJMethod(x@sdf, "schema"))
           })
@@ -115,9 +115,9 @@ setMethod("schema",
 #'
 #' Print the logical and physical Catalyst plans to the console for debugging.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param extended Logical. If extended is False, explain() only prints the physical plan.
-#' @family DataFrame functions
+#' @family SparkSparkDataFrame functions
 #' @rdname explain
 #' @name explain
 #' @export
@@ -130,7 +130,7 @@ setMethod("schema",
 #' explain(df, TRUE)
 #'}
 setMethod("explain",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, extended = FALSE) {
             queryExec <- callJMethod(x@sdf, "queryExecution")
             if (extended) {
@@ -146,9 +146,9 @@ setMethod("explain",
 #' Returns True if the `collect` and `take` methods can be run locally
 #' (without any Spark executors).
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkSparkDataFrame functions
 #' @rdname isLocal
 #' @name isLocal
 #' @export
@@ -161,19 +161,19 @@ setMethod("explain",
 #' isLocal(df)
 #'}
 setMethod("isLocal",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             callJMethod(x@sdf, "isLocal")
           })
 
 #' showDF
 #'
-#' Print the first numRows rows of a DataFrame
+#' Print the first numRows rows of a SparkDataFrame
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param numRows The number of rows to print. Defaults to 20.
 #'
-#' @family DataFrame functions
+#' @family SparkSparkDataFrame functions
 #' @rdname showDF
 #' @name showDF
 #' @export
@@ -186,7 +186,7 @@ setMethod("isLocal",
 #' showDF(df)
 #'}
 setMethod("showDF",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, numRows = 20, truncate = TRUE) {
             s <- callJMethod(x@sdf, "showString", numToInt(numRows), truncate)
             cat(s)
@@ -194,11 +194,11 @@ setMethod("showDF",
 
 #' show
 #'
-#' Print the DataFrame column names and types
+#' Print the SparkDataFrame column names and types
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkSparkDataFrame functions
 #' @rdname show
 #' @name show
 #' @export
@@ -210,22 +210,22 @@ setMethod("showDF",
 #' df <- read.json(sqlContext, path)
 #' df
 #'}
-setMethod("show", "DataFrame",
+setMethod("show", "SparkDataFrame",
           function(object) {
             cols <- lapply(dtypes(object), function(l) {
               paste(l, collapse = ":")
             })
             s <- paste(cols, collapse = ", ")
-            cat(paste("DataFrame[", s, "]\n", sep = ""))
+            cat(paste(class(object), "[", s, "]\n", sep = ""))
           })
 
 #' DataTypes
 #'
 #' Return all column names and their data types as a list
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkSparkDataFrame functions
 #' @rdname dtypes
 #' @name dtypes
 #' @export
@@ -238,7 +238,7 @@ setMethod("show", "DataFrame",
 #' dtypes(df)
 #'}
 setMethod("dtypes",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             lapply(schema(x)$fields(), function(f) {
               c(f$name(), f$dataType.simpleString())
@@ -249,9 +249,9 @@ setMethod("dtypes",
 #'
 #' Return all column names as a list
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname columns
 #' @name columns
 
@@ -266,7 +266,7 @@ setMethod("dtypes",
 #' colnames(df)
 #'}
 setMethod("columns",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             sapply(schema(x)$fields(), function(f) {
               f$name()
@@ -276,7 +276,7 @@ setMethod("columns",
 #' @rdname columns
 #' @name names
 setMethod("names",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             columns(x)
           })
@@ -284,7 +284,7 @@ setMethod("names",
 #' @rdname columns
 #' @name names<-
 setMethod("names<-",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, value) {
             if (!is.null(value)) {
               sdf <- callJMethod(x@sdf, "toDF", as.list(value))
@@ -295,7 +295,7 @@ setMethod("names<-",
 #' @rdname columns
 #' @name colnames
 setMethod("colnames",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             columns(x)
           })
@@ -303,7 +303,7 @@ setMethod("colnames",
 #' @rdname columns
 #' @name colnames<-
 setMethod("colnames<-",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, value) {
 
             # Check parameter integrity
@@ -331,13 +331,13 @@ setMethod("colnames<-",
 
 #' coltypes
 #'
-#' Get column types of a DataFrame
+#' Get column types of a SparkDataFrame
 #'
-#' @param x A SparkSQL DataFrame
-#' @return value A character vector with the column types of the given DataFrame
+#' @param x A SparkDataFrame
+#' @return value A character vector with the column types of the given SparkDataFrame
 #' @rdname coltypes
 #' @name coltypes
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @export
 #' @examples
 #'\dontrun{
@@ -345,9 +345,9 @@ setMethod("colnames<-",
 #' coltypes(irisDF)
 #'}
 setMethod("coltypes",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
-            # Get the data types of the DataFrame by invoking dtypes() function
+            # Get the data types of the SparkDataFrame by invoking dtypes() function
             types <- sapply(dtypes(x), function(x) {x[[2]]})
 
             # Map Spark data types into R's data types using DATA_TYPES environment
@@ -382,11 +382,11 @@ setMethod("coltypes",
 
 #' coltypes
 #'
-#' Set the column types of a DataFrame.
+#' Set the column types of a SparkDataFrame.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param value A character vector with the target column types for the given
-#'    DataFrame. Column types can be one of integer, numeric/double, character, logical, or NA
+#'    SparkDataFrame. Column types can be one of integer, numeric/double, character, logical, or NA
 #'    to keep that column as-is.
 #' @rdname coltypes
 #' @name coltypes<-
@@ -401,15 +401,15 @@ setMethod("coltypes",
 #' coltypes(df) <- c(NA, "numeric")
 #'}
 setMethod("coltypes<-",
-          signature(x = "DataFrame", value = "character"),
+          signature(x = "SparkDataFrame", value = "character"),
           function(x, value) {
             cols <- columns(x)
             ncols <- length(cols)
             if (length(value) == 0) {
-              stop("Cannot set types of an empty DataFrame with no Column")
+              stop("Cannot set types of an empty SparkDataFrame with no Column")
             }
             if (length(value) != ncols) {
-              stop("Length of type vector should match the number of columns for DataFrame")
+              stop("Length of type vector should match the number of columns for SparkDataFrame")
             }
             newCols <- lapply(seq_len(ncols), function(i) {
               col <- getColumn(x, cols[i])
@@ -429,12 +429,12 @@ setMethod("coltypes<-",
 
 #' Register Temporary Table
 #'
-#' Registers a DataFrame as a Temporary Table in the SQLContext
+#' Registers a SparkDataFrame as a Temporary Table in the SQLContext
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param tableName A character vector containing the name of the table
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname registerTempTable
 #' @name registerTempTable
 #' @export
@@ -448,21 +448,21 @@ setMethod("coltypes<-",
 #' new_df <- sql(sqlContext, "SELECT * FROM json_df")
 #'}
 setMethod("registerTempTable",
-          signature(x = "DataFrame", tableName = "character"),
+          signature(x = "SparkDataFrame", tableName = "character"),
           function(x, tableName) {
               invisible(callJMethod(x@sdf, "registerTempTable", tableName))
           })
 
 #' insertInto
 #'
-#' Insert the contents of a DataFrame into a table registered in the current SQL Context.
+#' Insert the contents of a SparkDataFrame into a table registered in the current SQL Context.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param tableName A character vector containing the name of the table
 #' @param overwrite A logical argument indicating whether or not to overwrite
 #' the existing rows in the table.
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname insertInto
 #' @name insertInto
 #' @export
@@ -476,7 +476,7 @@ setMethod("registerTempTable",
 #' insertInto(df2, "table1", overwrite = TRUE)
 #'}
 setMethod("insertInto",
-          signature(x = "DataFrame", tableName = "character"),
+          signature(x = "SparkDataFrame", tableName = "character"),
           function(x, tableName, overwrite = FALSE) {
             jmode <- convertToJSaveMode(ifelse(overwrite, "overwrite", "append"))
             write <- callJMethod(x@sdf, "write")
@@ -488,9 +488,9 @@ setMethod("insertInto",
 #'
 #' Persist with the default storage level (MEMORY_ONLY).
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname cache
 #' @name cache
 #' @export
@@ -503,7 +503,7 @@ setMethod("insertInto",
 #' cache(df)
 #'}
 setMethod("cache",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             cached <- callJMethod(x@sdf, "cache")
             x@env$isCached <- TRUE
@@ -512,13 +512,13 @@ setMethod("cache",
 
 #' Persist
 #'
-#' Persist this DataFrame with the specified storage level. For details of the
+#' Persist this SparkDataFrame with the specified storage level. For details of the
 #' supported storage levels, refer to
 #' \url{http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence}.
 #'
-#' @param x The DataFrame to persist
+#' @param x The SparkDataFrame to persist
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname persist
 #' @name persist
 #' @export
@@ -531,7 +531,7 @@ setMethod("cache",
 #' persist(df, "MEMORY_AND_DISK")
 #'}
 setMethod("persist",
-          signature(x = "DataFrame", newLevel = "character"),
+          signature(x = "SparkDataFrame", newLevel = "character"),
           function(x, newLevel) {
             callJMethod(x@sdf, "persist", getStorageLevel(newLevel))
             x@env$isCached <- TRUE
@@ -540,13 +540,13 @@ setMethod("persist",
 
 #' Unpersist
 #'
-#' Mark this DataFrame as non-persistent, and remove all blocks for it from memory and
+#' Mark this SparkDataFrame as non-persistent, and remove all blocks for it from memory and
 #' disk.
 #'
-#' @param x The DataFrame to unpersist
+#' @param x The SparkDataFrame to unpersist
 #' @param blocking Whether to block until all blocks are deleted
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname unpersist-methods
 #' @name unpersist
 #' @export
@@ -560,7 +560,7 @@ setMethod("persist",
 #' unpersist(df)
 #'}
 setMethod("unpersist",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, blocking = TRUE) {
             callJMethod(x@sdf, "unpersist", blocking)
             x@env$isCached <- FALSE
@@ -569,12 +569,12 @@ setMethod("unpersist",
 
 #' Repartition
 #'
-#' Return a new DataFrame that has exactly numPartitions partitions.
+#' Return a new SparkDataFrame that has exactly numPartitions partitions.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param numPartitions The number of partitions to use.
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname repartition
 #' @name repartition
 #' @export
@@ -587,7 +587,7 @@ setMethod("unpersist",
 #' newDF <- repartition(df, 2L)
 #'}
 setMethod("repartition",
-          signature(x = "DataFrame", numPartitions = "numeric"),
+          signature(x = "SparkDataFrame", numPartitions = "numeric"),
           function(x, numPartitions) {
             sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions))
             dataFrame(sdf)
@@ -595,12 +595,12 @@ setMethod("repartition",
 
 #' toJSON
 #'
-#' Convert the rows of a DataFrame into JSON objects and return an RDD where
+#' Convert the rows of a SparkDataFrame into JSON objects and return an RDD where
 #' each element contains a JSON string.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @return A StringRRDD of JSON objects
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname tojson
 #' @noRd
 #' @examples
@@ -612,7 +612,7 @@ setMethod("repartition",
 #' newRDD <- toJSON(df)
 #'}
 setMethod("toJSON",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             rdd <- callJMethod(x@sdf, "toJSON")
             jrdd <- callJMethod(rdd, "toJavaRDD")
@@ -621,13 +621,13 @@ setMethod("toJSON",
 
 #' write.json
 #'
-#' Save the contents of a DataFrame as a JSON file (one object per line). Files written out
-#' with this method can be read back in as a DataFrame using read.json().
+#' Save the contents of a SparkDataFrame as a JSON file (one object per line). Files written out
+#' with this method can be read back in as a SparkDataFrame using read.json().
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param path The directory where the file is saved
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname write.json
 #' @name write.json
 #' @export
@@ -640,7 +640,7 @@ setMethod("toJSON",
 #' write.json(df, "/tmp/sparkr-tmp/")
 #'}
 setMethod("write.json",
-          signature(x = "DataFrame", path = "character"),
+          signature(x = "SparkDataFrame", path = "character"),
           function(x, path) {
             write <- callJMethod(x@sdf, "write")
             invisible(callJMethod(write, "json", path))
@@ -648,13 +648,13 @@ setMethod("write.json",
 
 #' write.parquet
 #'
-#' Save the contents of a DataFrame as a Parquet file, preserving the schema. Files written out
-#' with this method can be read back in as a DataFrame using read.parquet().
+#' Save the contents of a SparkDataFrame as a Parquet file, preserving the schema. Files written out
+#' with this method can be read back in as a SparkDataFrame using read.parquet().
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param path The directory where the file is saved
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname write.parquet
 #' @name write.parquet
 #' @export
@@ -668,7 +668,7 @@ setMethod("write.json",
 #' saveAsParquetFile(df, "/tmp/sparkr-tmp2/")
 #'}
 setMethod("write.parquet",
-          signature(x = "DataFrame", path = "character"),
+          signature(x = "SparkDataFrame", path = "character"),
           function(x, path) {
             write <- callJMethod(x@sdf, "write")
             invisible(callJMethod(write, "parquet", path))
@@ -678,7 +678,7 @@ setMethod("write.parquet",
 #' @name saveAsParquetFile
 #' @export
 setMethod("saveAsParquetFile",
-          signature(x = "DataFrame", path = "character"),
+          signature(x = "SparkDataFrame", path = "character"),
           function(x, path) {
             .Deprecated("write.parquet")
             write.parquet(x, path)
@@ -686,14 +686,14 @@ setMethod("saveAsParquetFile",
 
 #' write.text
 #'
-#' Saves the content of the DataFrame in a text file at the specified path.
-#' The DataFrame must have only one column of string type with the name "value".
+#' Saves the content of the SparkDataFrame in a text file at the specified path.
+#' The SparkDataFrame must have only one column of string type with the name "value".
 #' Each row becomes a new line in the output file.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param path The directory where the file is saved
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname write.text
 #' @name write.text
 #' @export
@@ -706,7 +706,7 @@ setMethod("saveAsParquetFile",
 #' write.text(df, "/tmp/sparkr-tmp/")
 #'}
 setMethod("write.text",
-          signature(x = "DataFrame", path = "character"),
+          signature(x = "SparkDataFrame", path = "character"),
           function(x, path) {
             write <- callJMethod(x@sdf, "write")
             invisible(callJMethod(write, "text", path))
@@ -714,11 +714,11 @@ setMethod("write.text",
 
 #' Distinct
 #'
-#' Return a new DataFrame containing the distinct rows in this DataFrame.
+#' Return a new SparkDataFrame containing the distinct rows in this SparkDataFrame.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname distinct
 #' @name distinct
 #' @export
@@ -731,7 +731,7 @@ setMethod("write.text",
 #' distinctDF <- distinct(df)
 #'}
 setMethod("distinct",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             sdf <- callJMethod(x@sdf, "distinct")
             dataFrame(sdf)
@@ -740,21 +740,21 @@ setMethod("distinct",
 #' @rdname distinct
 #' @name unique
 setMethod("unique",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             distinct(x)
           })
 
 #' Sample
 #'
-#' Return a sampled subset of this DataFrame using a random seed.
+#' Return a sampled subset of this SparkDataFrame using a random seed.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param withReplacement Sampling with replacement or not
 #' @param fraction The (rough) sample target fraction
 #' @param seed Randomness seed value
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname sample
 #' @name sample
 #' @export
@@ -768,7 +768,7 @@ setMethod("unique",
 #' collect(sample(df, TRUE, 0.5))
 #'}
 setMethod("sample",
-          signature(x = "DataFrame", withReplacement = "logical",
+          signature(x = "SparkDataFrame", withReplacement = "logical",
                     fraction = "numeric"),
           function(x, withReplacement, fraction, seed) {
             if (fraction < 0.0) stop(cat("Negative fraction value:", fraction))
@@ -785,7 +785,7 @@ setMethod("sample",
 #' @rdname sample
 #' @name sample_frac
 setMethod("sample_frac",
-          signature(x = "DataFrame", withReplacement = "logical",
+          signature(x = "SparkDataFrame", withReplacement = "logical",
                     fraction = "numeric"),
           function(x, withReplacement, fraction, seed) {
             sample(x, withReplacement, fraction, seed)
@@ -793,11 +793,11 @@ setMethod("sample_frac",
 
 #' nrow
 #'
-#' Returns the number of rows in a DataFrame
+#' Returns the number of rows in a SparkDataFrame
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname nrow
 #' @name count
 #' @export
@@ -810,7 +810,7 @@ setMethod("sample_frac",
 #' count(df)
 #' }
 setMethod("count",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             callJMethod(x@sdf, "count")
           })
@@ -818,16 +818,16 @@ setMethod("count",
 #' @name nrow
 #' @rdname nrow
 setMethod("nrow",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             count(x)
           })
 
-#' Returns the number of columns in a DataFrame
+#' Returns the number of columns in a SparkDataFrame
 #'
-#' @param x a SparkSQL DataFrame
+#' @param x a SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname ncol
 #' @name ncol
 #' @export
@@ -840,15 +840,15 @@ setMethod("nrow",
 #' ncol(df)
 #' }
 setMethod("ncol",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             length(columns(x))
           })
 
-#' Returns the dimentions (number of rows and columns) of a DataFrame
-#' @param x a SparkSQL DataFrame
+#' Returns the dimentions (number of rows and columns) of a SparkDataFrame
+#' @param x a SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname dim
 #' @name dim
 #' @export
@@ -861,18 +861,18 @@ setMethod("ncol",
 #' dim(df)
 #' }
 setMethod("dim",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             c(count(x), ncol(x))
           })
 
-#' Collects all the elements of a Spark DataFrame and coerces them into an R data.frame.
+#' Collects all the elements of a SparkDataFrame and coerces them into an R data.frame.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param stringsAsFactors (Optional) A logical indicating whether or not string columns
 #' should be converted to factors. FALSE by default.
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname collect
 #' @name collect
 #' @export
@@ -886,7 +886,7 @@ setMethod("dim",
 #' firstName <- collected[[1]]$name
 #' }
 setMethod("collect",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, stringsAsFactors = FALSE) {
             dtypes <- dtypes(x)
             ncol <- length(dtypes)
@@ -938,13 +938,13 @@ setMethod("collect",
 
 #' Limit
 #'
-#' Limit the resulting DataFrame to the number of rows specified.
+#' Limit the resulting SparkDataFrame to the number of rows specified.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param num The number of rows to return
-#' @return A new DataFrame containing the number of rows specified.
+#' @return A new SparkDataFrame containing the number of rows specified.
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname limit
 #' @name limit
 #' @export
@@ -957,15 +957,15 @@ setMethod("collect",
 #' limitedDF <- limit(df, 10)
 #' }
 setMethod("limit",
-          signature(x = "DataFrame", num = "numeric"),
+          signature(x = "SparkDataFrame", num = "numeric"),
           function(x, num) {
             res <- callJMethod(x@sdf, "limit", as.integer(num))
             dataFrame(res)
           })
 
-#' Take the first NUM rows of a DataFrame and return a the results as a data.frame
+#' Take the first NUM rows of a SparkDataFrame and return a the results as a data.frame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname take
 #' @name take
 #' @export
@@ -978,7 +978,7 @@ setMethod("limit",
 #' take(df, 2)
 #' }
 setMethod("take",
-          signature(x = "DataFrame", num = "numeric"),
+          signature(x = "SparkDataFrame", num = "numeric"),
           function(x, num) {
             limited <- limit(x, num)
             collect(limited)
@@ -986,15 +986,15 @@ setMethod("take",
 
 #' Head
 #'
-#' Return the first NUM rows of a DataFrame as a data.frame. If NUM is NULL,
+#' Return the first NUM rows of a SparkDataFrame as a data.frame. If NUM is NULL,
 #' then head() returns the first 6 rows in keeping with the current data.frame
 #' convention in R.
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param num The number of rows to return. Default is 6.
 #' @return A data.frame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname head
 #' @name head
 #' @export
@@ -1007,17 +1007,17 @@ setMethod("take",
 #' head(df)
 #' }
 setMethod("head",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, num = 6L) {
           # Default num is 6L in keeping with R's data.frame convention
             take(x, num)
           })
 
-#' Return the first row of a DataFrame
+#' Return the first row of a SparkDataFrame
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname first
 #' @name first
 #' @export
@@ -1030,16 +1030,16 @@ setMethod("head",
 #' first(df)
 #' }
 setMethod("first",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             take(x, 1)
           })
 
 #' toRDD
 #'
-#' Converts a Spark DataFrame to an RDD while preserving column names.
+#' Converts a SparkDataFrame to an RDD while preserving column names.
 #'
-#' @param x A Spark DataFrame
+#' @param x A SparkDataFrame
 #'
 #' @noRd
 #' @examples
@@ -1051,7 +1051,7 @@ setMethod("first",
 #' rdd <- toRDD(df)
 #'}
 setMethod("toRDD",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             jrdd <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToRowRDD", x@sdf)
             colNames <- callJMethod(x@sdf, "columns")
@@ -1064,12 +1064,12 @@ setMethod("toRDD",
 
 #' GroupBy
 #'
-#' Groups the DataFrame using the specified columns, so we can run aggregation on them.
+#' Groups the SparkDataFrame using the specified columns, so we can run aggregation on them.
 #'
-#' @param x a DataFrame
+#' @param x a SparkDataFrame
 #' @return a GroupedData
 #' @seealso GroupedData
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname groupBy
 #' @name groupBy
 #' @export
@@ -1082,7 +1082,7 @@ setMethod("toRDD",
 #'   agg(groupBy(df, "department", "gender"), salary="avg", "age" -> "max")
 #' }
 setMethod("groupBy",
-           signature(x = "DataFrame"),
+           signature(x = "SparkDataFrame"),
            function(x, ...) {
              cols <- list(...)
              if (length(cols) >= 1 && class(cols[[1]]) == "character") {
@@ -1097,7 +1097,7 @@ setMethod("groupBy",
 #' @rdname groupBy
 #' @name group_by
 setMethod("group_by",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, ...) {
             groupBy(x, ...)
           })
@@ -1106,13 +1106,13 @@ setMethod("group_by",
 #'
 #' Compute aggregates by specifying a list of columns
 #'
-#' @param x a DataFrame
-#' @family DataFrame functions
+#' @param x a SparkDataFrame
+#' @family SparkDataFrame functions
 #' @rdname agg
 #' @name agg
 #' @export
 setMethod("agg",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, ...) {
             agg(groupBy(x), ...)
           })
@@ -1120,7 +1120,7 @@ setMethod("agg",
 #' @rdname agg
 #' @name summarize
 setMethod("summarize",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, ...) {
             agg(x, ...)
           })
@@ -1135,7 +1135,7 @@ setMethod("summarize",
 #' @rdname lapply
 #' @noRd
 setMethod("lapply",
-          signature(X = "DataFrame", FUN = "function"),
+          signature(X = "SparkDataFrame", FUN = "function"),
           function(X, FUN) {
             rdd <- toRDD(X)
             lapply(rdd, FUN)
@@ -1144,7 +1144,7 @@ setMethod("lapply",
 #' @rdname lapply
 #' @noRd
 setMethod("map",
-          signature(X = "DataFrame", FUN = "function"),
+          signature(X = "SparkDataFrame", FUN = "function"),
           function(X, FUN) {
             lapply(X, FUN)
           })
@@ -1152,7 +1152,7 @@ setMethod("map",
 #' @rdname flatMap
 #' @noRd
 setMethod("flatMap",
-          signature(X = "DataFrame", FUN = "function"),
+          signature(X = "SparkDataFrame", FUN = "function"),
           function(X, FUN) {
             rdd <- toRDD(X)
             flatMap(rdd, FUN)
@@ -1161,7 +1161,7 @@ setMethod("flatMap",
 #' @rdname lapplyPartition
 #' @noRd
 setMethod("lapplyPartition",
-          signature(X = "DataFrame", FUN = "function"),
+          signature(X = "SparkDataFrame", FUN = "function"),
           function(X, FUN) {
             rdd <- toRDD(X)
             lapplyPartition(rdd, FUN)
@@ -1170,7 +1170,7 @@ setMethod("lapplyPartition",
 #' @rdname lapplyPartition
 #' @noRd
 setMethod("mapPartitions",
-          signature(X = "DataFrame", FUN = "function"),
+          signature(X = "SparkDataFrame", FUN = "function"),
           function(X, FUN) {
             lapplyPartition(X, FUN)
           })
@@ -1178,7 +1178,7 @@ setMethod("mapPartitions",
 #' @rdname foreach
 #' @noRd
 setMethod("foreach",
-          signature(x = "DataFrame", func = "function"),
+          signature(x = "SparkDataFrame", func = "function"),
           function(x, func) {
             rdd <- toRDD(x)
             foreach(rdd, func)
@@ -1187,7 +1187,7 @@ setMethod("foreach",
 #' @rdname foreach
 #' @noRd
 setMethod("foreachPartition",
-          signature(x = "DataFrame", func = "function"),
+          signature(x = "SparkDataFrame", func = "function"),
           function(x, func) {
             rdd <- toRDD(x)
             foreachPartition(rdd, func)
@@ -1202,14 +1202,14 @@ getColumn <- function(x, c) {
 
 #' @rdname select
 #' @name $
-setMethod("$", signature(x = "DataFrame"),
+setMethod("$", signature(x = "SparkDataFrame"),
           function(x, name) {
             getColumn(x, name)
           })
 
 #' @rdname select
 #' @name $<-
-setMethod("$<-", signature(x = "DataFrame"),
+setMethod("$<-", signature(x = "SparkDataFrame"),
           function(x, name, value) {
             stopifnot(class(value) == "Column" || is.null(value))
 
@@ -1226,7 +1226,7 @@ setClassUnion("numericOrcharacter", c("numeric", "character"))
 
 #' @rdname subset
 #' @name [[
-setMethod("[[", signature(x = "DataFrame", i = "numericOrcharacter"),
+setMethod("[[", signature(x = "SparkDataFrame", i = "numericOrcharacter"),
           function(x, i) {
             if (is.numeric(i)) {
               cols <- columns(x)
@@ -1237,7 +1237,7 @@ setMethod("[[", signature(x = "DataFrame", i = "numericOrcharacter"),
 
 #' @rdname subset
 #' @name [
-setMethod("[", signature(x = "DataFrame", i = "missing"),
+setMethod("[", signature(x = "SparkDataFrame", i = "missing"),
           function(x, i, j, ...) {
             if (is.numeric(j)) {
               cols <- columns(x)
@@ -1251,7 +1251,7 @@ setMethod("[", signature(x = "DataFrame", i = "missing"),
 
 #' @rdname subset
 #' @name [
-setMethod("[", signature(x = "DataFrame", i = "Column"),
+setMethod("[", signature(x = "SparkDataFrame", i = "Column"),
           function(x, i, j, ...) {
             # It could handle i as "character" but it seems confusing and not required
             # https://stat.ethz.ch/R-manual/R-devel/library/base/html/Extract.data.frame.html
@@ -1265,13 +1265,15 @@ setMethod("[", signature(x = "DataFrame", i = "Column"),
 
 #' Subset
 #'
-#' Return subsets of DataFrame according to given conditions
-#' @param x A DataFrame
+#' Return subsets of SparkDataFrame according to given conditions
+#' @param x A SparkDataFrame
 #' @param subset (Optional) A logical expression to filter on rows
-#' @param select expression for the single Column or a list of columns to select from the DataFrame
-#' @return A new DataFrame containing only the rows that meet the condition with selected columns
+#' @param select expression for the single Column or a list of columns to select from the
+#' SparkDataFrame
+#' @return A new SparkDataFrame containing only the rows that meet the condition with selected
+#' columns
 #' @export
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname subset
 #' @name subset
 #' @family subsetting functions
@@ -1283,14 +1285,14 @@ setMethod("[", signature(x = "DataFrame", i = "Column"),
 #'   df[,c("name", "age")]
 #'   # Or to filter rows
 #'   df[df$age > 20,]
-#'   # DataFrame can be subset on both rows and Columns
+#'   # SparkDataFrame can be subset on both rows and Columns
 #'   df[df$name == "Smith", c(1,2)]
 #'   df[df$age %in% c(19, 30), 1:2]
 #'   subset(df, df$age %in% c(19, 30), 1:2)
 #'   subset(df, df$age %in% c(19), select = c(1,2))
 #'   subset(df, select = c(1,2))
 #' }
-setMethod("subset", signature(x = "DataFrame"),
+setMethod("subset", signature(x = "SparkDataFrame"),
           function(x, subset, select, ...) {
             if (missing(subset)) {
               x[, select, ...]
@@ -1302,11 +1304,11 @@ setMethod("subset", signature(x = "DataFrame"),
 #' Select
 #'
 #' Selects a set of columns with names or Column expressions.
-#' @param x A DataFrame
+#' @param x A SparkDataFrame
 #' @param col A list of columns or single Column or name
-#' @return A new DataFrame with selected columns
+#' @return A new SparkDataFrame with selected columns
 #' @export
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname select
 #' @name select
 #' @family subsetting functions
@@ -1320,7 +1322,7 @@ setMethod("subset", signature(x = "DataFrame"),
 #'   # Similar to R data frames columns can also be selected using `$`
 #'   df[,df$age]
 #' }
-setMethod("select", signature(x = "DataFrame", col = "character"),
+setMethod("select", signature(x = "SparkDataFrame", col = "character"),
           function(x, col, ...) {
             if (length(col) > 1) {
               if (length(list(...)) > 0) {
@@ -1334,10 +1336,10 @@ setMethod("select", signature(x = "DataFrame", col = "character"),
             }
           })
 
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname select
 #' @export
-setMethod("select", signature(x = "DataFrame", col = "Column"),
+setMethod("select", signature(x = "SparkDataFrame", col = "Column"),
           function(x, col, ...) {
             jcols <- lapply(list(col, ...), function(c) {
               c@jc
@@ -1346,11 +1348,11 @@ setMethod("select", signature(x = "DataFrame", col = "Column"),
             dataFrame(sdf)
           })
 
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname select
 #' @export
 setMethod("select",
-          signature(x = "DataFrame", col = "list"),
+          signature(x = "SparkDataFrame", col = "list"),
           function(x, col) {
             cols <- lapply(col, function(c) {
               if (class(c) == "Column") {
@@ -1365,13 +1367,13 @@ setMethod("select",
 
 #' SelectExpr
 #'
-#' Select from a DataFrame using a set of SQL expressions.
+#' Select from a SparkDataFrame using a set of SQL expressions.
 #'
-#' @param x A DataFrame to be selected from.
+#' @param x A SparkDataFrame to be selected from.
 #' @param expr A string containing a SQL expression
 #' @param ... Additional expressions
-#' @return A DataFrame
-#' @family DataFrame functions
+#' @return A SparkDataFrame
+#' @family SparkDataFrame functions
 #' @rdname selectExpr
 #' @name selectExpr
 #' @export
@@ -1384,7 +1386,7 @@ setMethod("select",
 #' selectExpr(df, "col1", "(col2 * 5) as newCol")
 #' }
 setMethod("selectExpr",
-          signature(x = "DataFrame", expr = "character"),
+          signature(x = "SparkDataFrame", expr = "character"),
           function(x, expr, ...) {
             exprList <- list(expr, ...)
             sdf <- callJMethod(x@sdf, "selectExpr", exprList)
@@ -1393,14 +1395,14 @@ setMethod("selectExpr",
 
 #' WithColumn
 #'
-#' Return a new DataFrame by adding a column or replacing the existing column
+#' Return a new SparkDataFrame by adding a column or replacing the existing column
 #' that has the same name.
 #'
-#' @param x A DataFrame
+#' @param x A SparkDataFrame
 #' @param colName A column name.
 #' @param col A Column expression.
-#' @return A DataFrame with the new column added or the existing column replaced.
-#' @family DataFrame functions
+#' @return A SparkDataFrame with the new column added or the existing column replaced.
+#' @family SparkDataFrame functions
 #' @rdname withColumn
 #' @name withColumn
 #' @seealso \link{rename} \link{mutate}
@@ -1416,7 +1418,7 @@ setMethod("selectExpr",
 #' newDF2 <- withColumn(newDF, "newCol", newDF$col1)
 #' }
 setMethod("withColumn",
-          signature(x = "DataFrame", colName = "character", col = "Column"),
+          signature(x = "SparkDataFrame", colName = "character", col = "Column"),
           function(x, colName, col) {
             sdf <- callJMethod(x@sdf, "withColumn", colName, col@jc)
             dataFrame(sdf)
@@ -1424,12 +1426,12 @@ setMethod("withColumn",
 
 #' Mutate
 #'
-#' Return a new DataFrame with the specified columns added.
+#' Return a new SparkDataFrame with the specified columns added.
 #'
-#' @param .data A DataFrame
+#' @param .data A SparkDataFrame
 #' @param col a named argument of the form name = col
-#' @return A new DataFrame with the new columns added.
-#' @family DataFrame functions
+#' @return A new SparkDataFrame with the new columns added.
+#' @family SparkDataFrame functions
 #' @rdname mutate
 #' @name mutate
 #' @seealso \link{rename} \link{withColumn}
@@ -1445,7 +1447,7 @@ setMethod("withColumn",
 #' newDF2 <- transform(df, newCol = df$col1 / 5, newCol2 = df$col1 * 2)
 #' }
 setMethod("mutate",
-          signature(.data = "DataFrame"),
+          signature(.data = "SparkDataFrame"),
           function(.data, ...) {
             x <- .data
             cols <- list(...)
@@ -1466,20 +1468,20 @@ setMethod("mutate",
 #' @rdname mutate
 #' @name transform
 setMethod("transform",
-          signature(`_data` = "DataFrame"),
+          signature(`_data` = "SparkDataFrame"),
           function(`_data`, ...) {
             mutate(`_data`, ...)
           })
 
 #' rename
 #'
-#' Rename an existing column in a DataFrame.
+#' Rename an existing column in a SparkDataFrame.
 #'
-#' @param x A DataFrame
+#' @param x A SparkDataFrame
 #' @param existingCol The name of the column you want to change.
 #' @param newCol The new column name.
-#' @return A DataFrame with the column name changed.
-#' @family DataFrame functions
+#' @return A SparkDataFrame with the column name changed.
+#' @family SparkDataFrame functions
 #' @rdname rename
 #' @name withColumnRenamed
 #' @seealso \link{mutate}
@@ -1493,7 +1495,7 @@ setMethod("transform",
 #' newDF <- withColumnRenamed(df, "col1", "newCol1")
 #' }
 setMethod("withColumnRenamed",
-          signature(x = "DataFrame", existingCol = "character", newCol = "character"),
+          signature(x = "SparkDataFrame", existingCol = "character", newCol = "character"),
           function(x, existingCol, newCol) {
             cols <- lapply(columns(x), function(c) {
               if (c == existingCol) {
@@ -1518,7 +1520,7 @@ setMethod("withColumnRenamed",
 #' newDF <- rename(df, col1 = df$newCol1)
 #' }
 setMethod("rename",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, ...) {
             renameCols <- list(...)
             stopifnot(length(renameCols) > 0)
@@ -1541,15 +1543,15 @@ setClassUnion("characterOrColumn", c("character", "Column"))
 
 #' Arrange
 #'
-#' Sort a DataFrame by the specified column(s).
+#' Sort a SparkDataFrame by the specified column(s).
 #'
-#' @param x A DataFrame to be sorted.
+#' @param x A SparkDataFrame to be sorted.
 #' @param col A character or Column object vector indicating the fields to sort on
 #' @param ... Additional sorting fields
 #' @param decreasing A logical argument indicating sorting order for columns when
 #'                   a character vector is specified for col
-#' @return A DataFrame where all elements are sorted.
-#' @family DataFrame functions
+#' @return A SparkDataFrame where all elements are sorted.
+#' @family SparkDataFrame functions
 #' @rdname arrange
 #' @name arrange
 #' @export
@@ -1565,7 +1567,7 @@ setClassUnion("characterOrColumn", c("character", "Column"))
 #' arrange(df, "col1", "col2", decreasing = c(TRUE, FALSE))
 #' }
 setMethod("arrange",
-          signature(x = "DataFrame", col = "Column"),
+          signature(x = "SparkDataFrame", col = "Column"),
           function(x, col, ...) {
               jcols <- lapply(list(col, ...), function(c) {
                 c@jc
@@ -1579,7 +1581,7 @@ setMethod("arrange",
 #' @name arrange
 #' @export
 setMethod("arrange",
-          signature(x = "DataFrame", col = "character"),
+          signature(x = "SparkDataFrame", col = "character"),
           function(x, col, ..., decreasing = FALSE) {
 
             # all sorting columns
@@ -1611,20 +1613,20 @@ setMethod("arrange",
 #' @name orderBy
 #' @export
 setMethod("orderBy",
-          signature(x = "DataFrame", col = "characterOrColumn"),
+          signature(x = "SparkDataFrame", col = "characterOrColumn"),
           function(x, col) {
             arrange(x, col)
           })
 
 #' Filter
 #'
-#' Filter the rows of a DataFrame according to a given condition.
+#' Filter the rows of a SparkDataFrame according to a given condition.
 #'
-#' @param x A DataFrame to be sorted.
+#' @param x A SparkDataFrame to be sorted.
 #' @param condition The condition to filter on. This may either be a Column expression
 #' or a string containing a SQL statement
-#' @return A DataFrame containing only the rows that meet the condition.
-#' @family DataFrame functions
+#' @return A SparkDataFrame containing only the rows that meet the condition.
+#' @family SparkDataFrame functions
 #' @rdname filter
 #' @name filter
 #' @family subsetting functions
@@ -1639,7 +1641,7 @@ setMethod("orderBy",
 #' filter(df, df$col2 != "abcdefg")
 #' }
 setMethod("filter",
-          signature(x = "DataFrame", condition = "characterOrColumn"),
+          signature(x = "SparkDataFrame", condition = "characterOrColumn"),
           function(x, condition) {
             if (class(condition) == "Column") {
               condition <- condition@jc
@@ -1648,24 +1650,24 @@ setMethod("filter",
             dataFrame(sdf)
           })
 
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname filter
 #' @name where
 setMethod("where",
-          signature(x = "DataFrame", condition = "characterOrColumn"),
+          signature(x = "SparkDataFrame", condition = "characterOrColumn"),
           function(x, condition) {
             filter(x, condition)
           })
 
 #' dropDuplicates
 #'
-#' Returns a new DataFrame with duplicate rows removed, considering only
+#' Returns a new SparkDataFrame with duplicate rows removed, considering only
 #' the subset of columns.
 #'
-#' @param x A DataFrame.
+#' @param x A SparkDataFrame.
 #' @param colnames A character vector of column names.
-#' @return A DataFrame with duplicate rows removed.
-#' @family DataFrame functions
+#' @return A SparkDataFrame with duplicate rows removed.
+#' @family SparkDataFrame functions
 #' @rdname dropduplicates
 #' @name dropDuplicates
 #' @export
@@ -1679,7 +1681,7 @@ setMethod("where",
 #' dropDuplicates(df, c("col1", "col2"))
 #' }
 setMethod("dropDuplicates",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, colNames = columns(x)) {
             stopifnot(class(colNames) == "character")
 
@@ -1689,17 +1691,17 @@ setMethod("dropDuplicates",
 
 #' Join
 #'
-#' Join two DataFrames based on the given join expression.
+#' Join two SparkDataFrames based on the given join expression.
 #'
-#' @param x A Spark DataFrame
-#' @param y A Spark DataFrame
+#' @param x A SparkDataFrame
+#' @param y A SparkDataFrame
 #' @param joinExpr (Optional) The expression used to perform the join. joinExpr must be a
 #' Column expression. If joinExpr is omitted, join() will perform a Cartesian join
 #' @param joinType The type of join to perform. The following join types are available:
 #' 'inner', 'outer', 'full', 'fullouter', leftouter', 'left_outer', 'left',
 #' 'right_outer', 'rightouter', 'right', and 'leftsemi'. The default joinType is "inner".
-#' @return A DataFrame containing the result of the join operation.
-#' @family DataFrame functions
+#' @return A SparkDataFrame containing the result of the join operation.
+#' @family SparkDataFrame functions
 #' @rdname join
 #' @name join
 #' @seealso \link{merge}
@@ -1715,7 +1717,7 @@ setMethod("dropDuplicates",
 #' join(df1, df2, df1$col1 == df2$col2, "right_outer")
 #' }
 setMethod("join",
-          signature(x = "DataFrame", y = "DataFrame"),
+          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
           function(x, y, joinExpr = NULL, joinType = NULL) {
             if (is.null(joinExpr)) {
               sdf <- callJMethod(x@sdf, "join", y@sdf)
@@ -1757,7 +1759,7 @@ setMethod("join",
 #'   be returned. If all.x is set to FALSE and all.y is set to TRUE, a right
 #'   outer join will be returned. If all.x and all.y are set to TRUE, a full
 #'   outer join will be returned.
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname merge
 #' @seealso \link{join}
 #' @export
@@ -1776,7 +1778,7 @@ setMethod("join",
 #' merge(df1, df2, by = "col1", all = TRUE, suffixes = c("-X", "-Y"))
 #' }
 setMethod("merge",
-          signature(x = "DataFrame", y = "DataFrame"),
+          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
           function(x, y, by = intersect(names(x), names(y)), by.x = by, by.y = by,
                    all = FALSE, all.x = all, all.y = all,
                    sort = TRUE, suffixes = c("_x", "_y"), ... ) {
@@ -1858,7 +1860,7 @@ setMethod("merge",
 #' Creates a list of columns by replacing the intersected ones with aliases.
 #' The name of the alias column is formed by concatanating the original column name and a suffix.
 #'
-#' @param x a DataFrame on which the
+#' @param x a SparkDataFrame on which the
 #' @param intersectedColNames a list of intersected column names
 #' @param suffix a suffix for the column name
 #' @return list of columns
@@ -1883,14 +1885,14 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) {
 
 #' rbind
 #'
-#' Return a new DataFrame containing the union of rows in this DataFrame
-#' and another DataFrame. This is equivalent to `UNION ALL` in SQL.
-#' Note that this does not remove duplicate rows across the two DataFrames.
+#' Return a new SparkDataFrame containing the union of rows in this SparkDataFrame
+#' and another SparkDataFrame. This is equivalent to `UNION ALL` in SQL.
+#' Note that this does not remove duplicate rows across the two SparkDataFrames.
 #'
-#' @param x A Spark DataFrame
-#' @param y A Spark DataFrame
-#' @return A DataFrame containing the result of the union.
-#' @family DataFrame functions
+#' @param x A SparkDataFrame
+#' @param y A SparkDataFrame
+#' @return A SparkDataFrame containing the result of the union.
+#' @family SparkDataFrame functions
 #' @rdname rbind
 #' @name unionAll
 #' @export
@@ -1903,20 +1905,20 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) {
 #' unioned <- unionAll(df, df2)
 #' }
 setMethod("unionAll",
-          signature(x = "DataFrame", y = "DataFrame"),
+          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
           function(x, y) {
             unioned <- callJMethod(x@sdf, "unionAll", y@sdf)
             dataFrame(unioned)
           })
 
-#' @title Union two or more DataFrames
-#' @description Returns a new DataFrame containing rows of all parameters.
+#' @title Union two or more SparkDataFrames
+#' @description Returns a new SparkDataFrame containing rows of all parameters.
 #'
 #' @rdname rbind
 #' @name rbind
 #' @export
 setMethod("rbind",
-          signature(... = "DataFrame"),
+          signature(... = "SparkDataFrame"),
           function(x, ..., deparse.level = 1) {
             if (nargs() == 3) {
               unionAll(x, ...)
@@ -1927,13 +1929,13 @@ setMethod("rbind",
 
 #' Intersect
 #'
-#' Return a new DataFrame containing rows only in both this DataFrame
-#' and another DataFrame. This is equivalent to `INTERSECT` in SQL.
+#' Return a new SparkDataFrame containing rows only in both this SparkDataFrame
+#' and another SparkDataFrame. This is equivalent to `INTERSECT` in SQL.
 #'
-#' @param x A Spark DataFrame
-#' @param y A Spark DataFrame
-#' @return A DataFrame containing the result of the intersect.
-#' @family DataFrame functions
+#' @param x A SparkDataFrame
+#' @param y A SparkDataFrame
+#' @return A SparkDataFrame containing the result of the intersect.
+#' @family SparkDataFrame functions
 #' @rdname intersect
 #' @name intersect
 #' @export
@@ -1946,7 +1948,7 @@ setMethod("rbind",
 #' intersectDF <- intersect(df, df2)
 #' }
 setMethod("intersect",
-          signature(x = "DataFrame", y = "DataFrame"),
+          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
           function(x, y) {
             intersected <- callJMethod(x@sdf, "intersect", y@sdf)
             dataFrame(intersected)
@@ -1954,13 +1956,13 @@ setMethod("intersect",
 
 #' except
 #'
-#' Return a new DataFrame containing rows in this DataFrame
-#' but not in another DataFrame. This is equivalent to `EXCEPT` in SQL.
+#' Return a new SparkDataFrame containing rows in this SparkDataFrame
+#' but not in another SparkDataFrame. This is equivalent to `EXCEPT` in SQL.
 #'
-#' @param x A Spark DataFrame
-#' @param y A Spark DataFrame
-#' @return A DataFrame containing the result of the except operation.
-#' @family DataFrame functions
+#' @param x A SparkDataFrame
+#' @param y A SparkDataFrame
+#' @return A SparkDataFrame containing the result of the except operation.
+#' @family SparkDataFrame functions
 #' @rdname except
 #' @name except
 #' @export
@@ -1975,13 +1977,13 @@ setMethod("intersect",
 #' @rdname except
 #' @export
 setMethod("except",
-          signature(x = "DataFrame", y = "DataFrame"),
+          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
           function(x, y) {
             excepted <- callJMethod(x@sdf, "except", y@sdf)
             dataFrame(excepted)
           })
 
-#' Save the contents of the DataFrame to a data source
+#' Save the contents of the SparkDataFrame to a data source
 #'
 #' The data source is specified by the `source` and a set of options (...).
 #' If `source` is not specified, the default data source configured by
@@ -1989,18 +1991,19 @@ setMethod("except",
 #'
 #' Additionally, mode is used to specify the behavior of the save operation when
 #' data already exists in the data source. There are four modes: \cr
-#'  append: Contents of this DataFrame are expected to be appended to existing data. \cr
-#'  overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. \cr
+#'  append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr
+#'  overwrite: Existing data is expected to be overwritten by the contents of this
+#'     SparkDataFrame. \cr
 #'  error: An exception is expected to be thrown. \cr
-#'  ignore: The save operation is expected to not save the contents of the DataFrame
+#'  ignore: The save operation is expected to not save the contents of the SparkDataFrame
 #'     and to not change the existing data. \cr
 #'
-#' @param df A SparkSQL DataFrame
+#' @param df A SparkDataFrame
 #' @param path A name for the table
 #' @param source A name for external data source
 #' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default)
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname write.df
 #' @name write.df
 #' @export
@@ -2014,7 +2017,7 @@ setMethod("except",
 #' saveDF(df, parquetPath2, "parquet", mode = saveMode, mergeSchema = mergeSchema)
 #' }
 setMethod("write.df",
-          signature(df = "DataFrame", path = "character"),
+          signature(df = "SparkDataFrame", path = "character"),
           function(df, path, source = NULL, mode = "error", ...){
             if (is.null(source)) {
               if (exists(".sparkRSQLsc", envir = .sparkREnv)) {
@@ -2042,14 +2045,14 @@ setMethod("write.df",
 #' @name saveDF
 #' @export
 setMethod("saveDF",
-          signature(df = "DataFrame", path = "character"),
+          signature(df = "SparkDataFrame", path = "character"),
           function(df, path, source = NULL, mode = "error", ...){
             write.df(df, path, source, mode, ...)
           })
 
 #' saveAsTable
 #'
-#' Save the contents of the DataFrame to a data source as a table
+#' Save the contents of the SparkDataFrame to a data source as a table
 #'
 #' The data source is specified by the `source` and a set of options (...).
 #' If `source` is not specified, the default data source configured by
@@ -2057,18 +2060,19 @@ setMethod("saveDF",
 #'
 #' Additionally, mode is used to specify the behavior of the save operation when
 #' data already exists in the data source. There are four modes: \cr
-#'  append: Contents of this DataFrame are expected to be appended to existing data. \cr
-#'  overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. \cr
+#'  append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr
+#'  overwrite: Existing data is expected to be overwritten by the contents of this
+#'     SparkDataFrame. \cr
 #'  error: An exception is expected to be thrown. \cr
-#'  ignore: The save operation is expected to not save the contents of the DataFrame
+#'  ignore: The save operation is expected to not save the contents of the SparkDataFrame
 #'     and to not change the existing data. \cr
 #'
-#' @param df A SparkSQL DataFrame
+#' @param df A SparkDataFrame
 #' @param tableName A name for the table
 #' @param source A name for external data source
 #' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default)
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname saveAsTable
 #' @name saveAsTable
 #' @export
@@ -2081,7 +2085,7 @@ setMethod("saveDF",
 #' saveAsTable(df, "myfile")
 #' }
 setMethod("saveAsTable",
-          signature(df = "DataFrame", tableName = "character"),
+          signature(df = "SparkDataFrame", tableName = "character"),
           function(df, tableName, source = NULL, mode="error", ...){
             if (is.null(source)) {
               if (exists(".sparkRSQLsc", envir = .sparkREnv)) {
@@ -2109,11 +2113,11 @@ setMethod("saveAsTable",
 #' Computes statistics for numeric columns.
 #' If no columns are given, this function computes statistics for all numerical columns.
 #'
-#' @param x A DataFrame to be computed.
+#' @param x A SparkDataFrame to be computed.
 #' @param col A string of name
 #' @param ... Additional expressions
-#' @return A DataFrame
-#' @family DataFrame functions
+#' @return A SparkDataFrame
+#' @family SparkDataFrame functions
 #' @rdname summary
 #' @name describe
 #' @export
@@ -2128,7 +2132,7 @@ setMethod("saveAsTable",
 #' describe(df, "col1", "col2")
 #' }
 setMethod("describe",
-          signature(x = "DataFrame", col = "character"),
+          signature(x = "SparkDataFrame", col = "character"),
           function(x, col, ...) {
             colList <- list(col, ...)
             sdf <- callJMethod(x@sdf, "describe", colList)
@@ -2138,7 +2142,7 @@ setMethod("describe",
 #' @rdname summary
 #' @name describe
 setMethod("describe",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x) {
             colList <- as.list(c(columns(x)))
             sdf <- callJMethod(x@sdf, "describe", colList)
@@ -2148,7 +2152,7 @@ setMethod("describe",
 #' @rdname summary
 #' @name summary
 setMethod("summary",
-          signature(object = "DataFrame"),
+          signature(object = "SparkDataFrame"),
           function(object, ...) {
             describe(object)
           })
@@ -2156,9 +2160,9 @@ setMethod("summary",
 
 #' dropna
 #'
-#' Returns a new DataFrame omitting rows with null values.
+#' Returns a new SparkDataFrame omitting rows with null values.
 #'
-#' @param x A SparkSQL DataFrame.
+#' @param x A SparkDataFrame.
 #' @param how "any" or "all".
 #'            if "any", drop a row if it contains any nulls.
 #'            if "all", drop a row only if all its values are null.
@@ -2167,9 +2171,9 @@ setMethod("summary",
 #'                    minNonNulls non-null values.
 #'                    This overwrites the how parameter.
 #' @param cols Optional list of column names to consider.
-#' @return A DataFrame
+#' @return A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname nafunctions
 #' @name dropna
 #' @export
@@ -2182,7 +2186,7 @@ setMethod("summary",
 #' dropna(df)
 #' }
 setMethod("dropna",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, how = c("any", "all"), minNonNulls = NULL, cols = NULL) {
             how <- match.arg(how)
             if (is.null(cols)) {
@@ -2202,7 +2206,7 @@ setMethod("dropna",
 #' @name na.omit
 #' @export
 setMethod("na.omit",
-          signature(object = "DataFrame"),
+          signature(object = "SparkDataFrame"),
           function(object, how = c("any", "all"), minNonNulls = NULL, cols = NULL) {
             dropna(object, how, minNonNulls, cols)
           })
@@ -2211,7 +2215,7 @@ setMethod("na.omit",
 #'
 #' Replace null values.
 #'
-#' @param x A SparkSQL DataFrame.
+#' @param x A SparkDataFrame.
 #' @param value Value to replace null values with.
 #'              Should be an integer, numeric, character or named list.
 #'              If the value is a named list, then cols is ignored and
@@ -2237,7 +2241,7 @@ setMethod("na.omit",
 #' fillna(df, list("age" = 20, "name" = "unknown"))
 #' }
 setMethod("fillna",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, value, cols = NULL) {
             if (!(class(value) %in% c("integer", "numeric", "character", "list"))) {
               stop("value should be an integer, numeric, charactor or named list.")
@@ -2280,14 +2284,14 @@ setMethod("fillna",
             dataFrame(sdf)
           })
 
-#' This function downloads the contents of a DataFrame into an R's data.frame.
+#' This function downloads the contents of a SparkDataFrame into an R's data.frame.
 #' Since data.frames are held in memory, ensure that you have enough memory
 #' in your system to accommodate the contents.
 #'
-#' @title Download data from a DataFrame into a data.frame
-#' @param x a DataFrame
+#' @title Download data from a SparkDataFrame into a data.frame
+#' @param x a SparkDataFrame
 #' @return a data.frame
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname as.data.frame
 #' @examples \dontrun{
 #'
@@ -2295,24 +2299,24 @@ setMethod("fillna",
 #' df <- as.data.frame(irisDF[irisDF$Species == "setosa", ])
 #' }
 setMethod("as.data.frame",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, row.names = NULL, optional = FALSE, ...) {
             as.data.frame(collect(x), row.names, optional, ...)
           })
 
-#' The specified DataFrame is attached to the R search path. This means that
-#' the DataFrame is searched by R when evaluating a variable, so columns in
-#' the DataFrame can be accessed by simply giving their names.
+#' The specified SparkDataFrame is attached to the R search path. This means that
+#' the SparkDataFrame is searched by R when evaluating a variable, so columns in
+#' the SparkDataFrame can be accessed by simply giving their names.
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname attach
-#' @title Attach DataFrame to R search path
-#' @param what (DataFrame) The DataFrame to attach
+#' @title Attach SparkDataFrame to R search path
+#' @param what (SparkDataFrame) The SparkDataFrame to attach
 #' @param pos (integer) Specify position in search() where to attach.
-#' @param name (character) Name to use for the attached DataFrame. Names
+#' @param name (character) Name to use for the attached SparkDataFrame. Names
 #'   starting with package: are reserved for library.
 #' @param warn.conflicts (logical) If TRUE, warnings are printed about conflicts
-#' from attaching the database, unless that DataFrame contains an object
+#' from attaching the database, unless that SparkDataFrame contains an object
 #' @examples
 #' \dontrun{
 #' attach(irisDf)
@@ -2320,21 +2324,21 @@ setMethod("as.data.frame",
 #' }
 #' @seealso \link{detach}
 setMethod("attach",
-          signature(what = "DataFrame"),
+          signature(what = "SparkDataFrame"),
           function(what, pos = 2, name = deparse(substitute(what)), warn.conflicts = TRUE) {
             newEnv <- assignNewEnv(what)
             attach(newEnv, pos = pos, name = name, warn.conflicts = warn.conflicts)
           })
 
-#' Evaluate a R expression in an environment constructed from a DataFrame
-#' with() allows access to columns of a DataFrame by simply referring to
-#' their name. It appends every column of a DataFrame into a new
+#' Evaluate a R expression in an environment constructed from a SparkDataFrame
+#' with() allows access to columns of a SparkDataFrame by simply referring to
+#' their name. It appends every column of a SparkDataFrame into a new
 #' environment. Then, the given expression is evaluated in this new
 #' environment.
 #'
 #' @rdname with
-#' @title Evaluate a R expression in an environment constructed from a DataFrame
-#' @param data (DataFrame) DataFrame to use for constructing an environment.
+#' @title Evaluate a R expression in an environment constructed from a SparkDataFrame
+#' @param data (SparkDataFrame) SparkDataFrame to use for constructing an environment.
 #' @param expr (expression) Expression to evaluate.
 #' @param ... arguments to be passed to future methods.
 #' @examples
@@ -2343,28 +2347,28 @@ setMethod("attach",
 #' }
 #' @seealso \link{attach}
 setMethod("with",
-          signature(data = "DataFrame"),
+          signature(data = "SparkDataFrame"),
           function(data, expr, ...) {
             newEnv <- assignNewEnv(data)
             eval(substitute(expr), envir = newEnv, enclos = newEnv)
           })
 
-#' Display the structure of a DataFrame, including column names, column types, as well as a
+#' Display the structure of a SparkDataFrame, including column names, column types, as well as a
 #' a small sample of rows.
 #' @name str
 #' @title Compactly display the structure of a dataset
 #' @rdname str
-#' @family DataFrame functions
-#' @param object a DataFrame
+#' @family SparkDataFrame functions
+#' @param object a SparkDataFrame
 #' @examples \dontrun{
-#' # Create a DataFrame from the Iris dataset
+#' # Create a SparkDataFrame from the Iris dataset
 #' irisDF <- createDataFrame(sqlContext, iris)
 #'
-#' # Show the structure of the DataFrame
+#' # Show the structure of the SparkDataFrame
 #' str(irisDF)
 #' }
 setMethod("str",
-          signature(object = "DataFrame"),
+          signature(object = "SparkDataFrame"),
           function(object) {
 
             # TODO: These could be made global parameters, though in R it's not the case
@@ -2424,14 +2428,14 @@ setMethod("str",
 
 #' drop
 #'
-#' Returns a new DataFrame with columns dropped.
+#' Returns a new SparkDataFrame with columns dropped.
 #' This is a no-op if schema doesn't contain column name(s).
-#' 
-#' @param x A SparkSQL DataFrame.
+#'
+#' @param x A SparkDataFrame.
 #' @param cols A character vector of column names or a Column.
-#' @return A DataFrame
+#' @return A SparkDataFrame
 #'
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname drop
 #' @name drop
 #' @export
@@ -2446,7 +2450,7 @@ setMethod("str",
 #' drop(df, df$col1)
 #' }
 setMethod("drop",
-          signature(x = "DataFrame"),
+          signature(x = "SparkDataFrame"),
           function(x, col) {
             stopifnot(class(col) == "character" || class(col) == "Column")
 
@@ -2465,23 +2469,24 @@ setMethod("drop",
             base::drop(x)
           })
 
-#' Saves the content of the DataFrame to an external database table via JDBC
+#' Saves the content of the SparkDataFrame to an external database table via JDBC
 #'
 #' Additional JDBC database connection properties can be set (...)
 #'
 #' Also, mode is used to specify the behavior of the save operation when
 #' data already exists in the data source. There are four modes: \cr
-#'  append: Contents of this DataFrame are expected to be appended to existing data. \cr
-#'  overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. \cr
+#'  append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr
+#'  overwrite: Existing data is expected to be overwritten by the contents of this
+#'     SparkDataFrame. \cr
 #'  error: An exception is expected to be thrown. \cr
-#'  ignore: The save operation is expected to not save the contents of the DataFrame
+#'  ignore: The save operation is expected to not save the contents of the SparkDataFrame
 #'     and to not change the existing data. \cr
 #'
-#' @param x A SparkSQL DataFrame
+#' @param x A SparkDataFrame
 #' @param url JDBC database url of the form `jdbc:subprotocol:subname`
 #' @param tableName The name of the table in the external database
 #' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default)
-#' @family DataFrame functions
+#' @family SparkDataFrame functions
 #' @rdname write.jdbc
 #' @name write.jdbc
 #' @export
@@ -2493,7 +2498,7 @@ setMethod("drop",
 #' write.jdbc(df, jdbcUrl, "table", user = "username", password = "password")
 #' }
 setMethod("write.jdbc",
-          signature(x = "DataFrame", url = "character", tableName = "character"),
+          signature(x = "SparkDataFrame", url = "character", tableName = "character"),
           function(x, url, tableName, mode = "error", ...){
             jmode <- convertToJSaveMode(mode)
             jprops <- varargsToJProperties(...)

http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/R/RDD.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R
index 35c4e6f..34d29dd 100644
--- a/R/pkg/R/RDD.R
+++ b/R/pkg/R/RDD.R
@@ -46,7 +46,7 @@ setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode,
   # RDD has three serialization types:
   # byte: The RDD stores data serialized in R.
   # string: The RDD stores data as strings.
-  # row: The RDD stores the serialized rows of a DataFrame.
+  # row: The RDD stores the serialized rows of a SparkDataFrame.
 
   # We use an environment to store mutable states inside an RDD object.
   # Note that R's call-by-value semantics makes modifying slots inside an
@@ -114,7 +114,7 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val)
 #' @noRd
 #' @param jrdd Java object reference to the backing JavaRDD
 #' @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD
-#' stores strings, and "row" if the RDD stores the rows of a DataFrame
+#' stores strings, and "row" if the RDD stores the rows of a SparkDataFrame
 #' @param isCached TRUE if the RDD is cached
 #' @param isCheckpointed TRUE if the RDD has been checkpointed
 RDD <- function(jrdd, serializedMode = "byte", isCached = FALSE,

http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/R/SQLContext.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R
index b726c1e..3824e0a 100644
--- a/R/pkg/R/SQLContext.R
+++ b/R/pkg/R/SQLContext.R
@@ -34,7 +34,7 @@ getInternalType <- function(x) {
          Date = "date",
          POSIXlt = "timestamp",
          POSIXct = "timestamp",
-         stop(paste("Unsupported type for DataFrame:", class(x))))
+         stop(paste("Unsupported type for SparkDataFrame:", class(x))))
 }
 
 #' infer the SQL type
@@ -70,14 +70,14 @@ infer_type <- function(x) {
   }
 }
 
-#' Create a DataFrame
+#' Create a SparkDataFrame
 #'
-#' Converts R data.frame or list into DataFrame.
+#' Converts R data.frame or list into SparkDataFrame.
 #'
 #' @param sqlContext A SQLContext
 #' @param data An RDD or list or data.frame
 #' @param schema a list of column names or named list (StructType), optional
-#' @return an DataFrame
+#' @return a SparkDataFrame
 #' @rdname createDataFrame
 #' @export
 #' @examples
@@ -173,11 +173,11 @@ as.DataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0) {
 
 #' toDF
 #'
-#' Converts an RDD to a DataFrame by infer the types.
+#' Converts an RDD to a SparkDataFrame by infer the types.
 #'
 #' @param x An RDD
 #'
-#' @rdname DataFrame
+#' @rdname SparkDataFrame
 #' @noRd
 #' @examples
 #'\dontrun{
@@ -200,14 +200,14 @@ setMethod("toDF", signature(x = "RDD"),
             createDataFrame(sqlContext, x, ...)
           })
 
-#' Create a DataFrame from a JSON file.
+#' Create a SparkDataFrame from a JSON file.
 #'
-#' Loads a JSON file (one object per line), returning the result as a DataFrame
+#' Loads a JSON file (one object per line), returning the result as a SparkDataFrame
 #' It goes through the entire dataset once to determine the schema.
 #'
 #' @param sqlContext SQLContext to use
 #' @param path Path of file to read. A vector of multiple paths is allowed.
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @rdname read.json
 #' @name read.json
 #' @export
@@ -238,13 +238,13 @@ jsonFile <- function(sqlContext, path) {
 
 #' JSON RDD
 #'
-#' Loads an RDD storing one JSON object per string as a DataFrame.
+#' Loads an RDD storing one JSON object per string as a SparkDataFrame.
 #'
 #' @param sqlContext SQLContext to use
 #' @param rdd An RDD of JSON string
 #' @param schema A StructType object to use as schema
 #' @param samplingRatio The ratio of simpling used to infer the schema
-#' @return A DataFrame
+#' @return A SparkDataFrame
 #' @noRd
 #' @examples
 #'\dontrun{
@@ -268,13 +268,13 @@ jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) {
   }
 }
 
-#' Create a DataFrame from a Parquet file.
+#' Create a SparkDataFrame from a Parquet file.
 #'
-#' Loads a Parquet file, returning the result as a DataFrame.
+#' Loads a Parquet file, returning the result as a SparkDataFrame.
 #'
 #' @param sqlContext SQLContext to use
 #' @param path Path of file to read. A vector of multiple paths is allowed.
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @rdname read.parquet
 #' @name read.parquet
 #' @export
@@ -295,14 +295,14 @@ parquetFile <- function(sqlContext, ...) {
   read.parquet(sqlContext, unlist(list(...)))
 }
 
-#' Create a DataFrame from a text file.
+#' Create a SparkDataFrame from a text file.
 #'
-#' Loads a text file and returns a DataFrame with a single string column named "value".
-#' Each line in the text file is a new row in the resulting DataFrame.
+#' Loads a text file and returns a SparkDataFrame with a single string column named "value".
+#' Each line in the text file is a new row in the resulting SparkDataFrame.
 #'
 #' @param sqlContext SQLContext to use
 #' @param path Path of file to read. A vector of multiple paths is allowed.
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @rdname read.text
 #' @name read.text
 #' @export
@@ -323,11 +323,11 @@ read.text <- function(sqlContext, path) {
 
 #' SQL Query
 #'
-#' Executes a SQL query using Spark, returning the result as a DataFrame.
+#' Executes a SQL query using Spark, returning the result as a SparkDataFrame.
 #'
 #' @param sqlContext SQLContext to use
 #' @param sqlQuery A character vector containing the SQL query
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @export
 #' @examples
 #'\dontrun{
@@ -344,14 +344,14 @@ sql <- function(sqlContext, sqlQuery) {
  dataFrame(sdf)
 }
 
-#' Create a DataFrame from a SparkSQL Table
+#' Create a SparkDataFrame from a SparkSQL Table
 #'
-#' Returns the specified Table as a DataFrame.  The Table must have already been registered
+#' Returns the specified Table as a SparkDataFrame.  The Table must have already been registered
 #' in the SQLContext.
 #'
 #' @param sqlContext SQLContext to use
-#' @param tableName The SparkSQL Table to convert to a DataFrame.
-#' @return DataFrame
+#' @param tableName The SparkSQL Table to convert to a SparkDataFrame.
+#' @return SparkDataFrame
 #' @rdname tableToDF
 #' @name tableToDF
 #' @export
@@ -372,11 +372,11 @@ tableToDF <- function(sqlContext, tableName) {
 
 #' Tables
 #'
-#' Returns a DataFrame containing names of tables in the given database.
+#' Returns a SparkDataFrame containing names of tables in the given database.
 #'
 #' @param sqlContext SQLContext to use
 #' @param databaseName name of the database
-#' @return a DataFrame
+#' @return a SparkDataFrame
 #' @export
 #' @examples
 #'\dontrun{
@@ -425,7 +425,7 @@ tableNames <- function(sqlContext, databaseName = NULL) {
 #'
 #' @param sqlContext SQLContext to use
 #' @param tableName The name of the table being cached
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @export
 #' @examples
 #'\dontrun{
@@ -447,7 +447,7 @@ cacheTable <- function(sqlContext, tableName) {
 #'
 #' @param sqlContext SQLContext to use
 #' @param tableName The name of the table being uncached
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @export
 #' @examples
 #'\dontrun{
@@ -500,9 +500,9 @@ dropTempTable <- function(sqlContext, tableName) {
   callJMethod(sqlContext, "dropTempTable", tableName)
 }
 
-#' Load an DataFrame
+#' Load a SparkDataFrame
 #'
-#' Returns the dataset in a data source as a DataFrame
+#' Returns the dataset in a data source as a SparkDataFrame
 #'
 #' The data source is specified by the `source` and a set of options(...).
 #' If `source` is not specified, the default data source configured by
@@ -512,7 +512,7 @@ dropTempTable <- function(sqlContext, tableName) {
 #' @param path The path of files to load
 #' @param source The name of external data source
 #' @param schema The data schema defined in structType
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @rdname read.df
 #' @name read.df
 #' @export
@@ -556,7 +556,7 @@ loadDF <- function(sqlContext, path = NULL, source = NULL, schema = NULL, ...) {
 #' Create an external table
 #'
 #' Creates an external table based on the dataset in a data source,
-#' Returns the DataFrame associated with the external table.
+#' Returns a SparkDataFrame associated with the external table.
 #'
 #' The data source is specified by the `source` and a set of options(...).
 #' If `source` is not specified, the default data source configured by
@@ -566,7 +566,7 @@ loadDF <- function(sqlContext, path = NULL, source = NULL, schema = NULL, ...) {
 #' @param tableName A name of the table
 #' @param path The path of files to load
 #' @param source the name of external data source
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @export
 #' @examples
 #'\dontrun{
@@ -584,7 +584,7 @@ createExternalTable <- function(sqlContext, tableName, path = NULL, source = NUL
   dataFrame(sdf)
 }
 
-#' Create a DataFrame representing the database table accessible via JDBC URL
+#' Create a SparkDataFrame representing the database table accessible via JDBC URL
 #'
 #' Additional JDBC database connection properties can be set (...)
 #'
@@ -605,7 +605,7 @@ createExternalTable <- function(sqlContext, tableName, path = NULL, source = NUL
 #'                      clause expressions used to split the column `partitionColumn` evenly.
 #'                      This defaults to SparkContext.defaultParallelism when unset.
 #' @param predicates a list of conditions in the where clause; each one defines one partition
-#' @return DataFrame
+#' @return SparkDataFrame
 #' @rdname read.jdbc
 #' @name read.jdbc
 #' @export

http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/R/column.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R
index 3ffd9a9..a3e0937 100644
--- a/R/pkg/R/column.R
+++ b/R/pkg/R/column.R
@@ -22,11 +22,11 @@ NULL
 
 setOldClass("jobj")
 
-#' @title S4 class that represents a DataFrame column
-#' @description The column class supports unary, binary operations on DataFrame columns
+#' @title S4 class that represents a SparkDataFrame column
+#' @description The column class supports unary, binary operations on SparkDataFrame columns
 #' @rdname column
 #'
-#' @slot jc reference to JVM DataFrame column
+#' @slot jc reference to JVM SparkDataFrame column
 #' @export
 setClass("Column",
          slots = list(jc = "jobj"))

http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/R/deserialize.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R
index eefdf17..ce071b1 100644
--- a/R/pkg/R/deserialize.R
+++ b/R/pkg/R/deserialize.R
@@ -139,7 +139,7 @@ readEnv <- function(con) {
   env
 }
 
-# Read a field of StructType from DataFrame
+# Read a field of StructType from SparkDataFrame
 # into a named list in R whose class is "struct"
 readStruct <- function(con) {
   names <- readObject(con)

http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/R/functions.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R
index 54234b0..4a0bdf3 100644
--- a/R/pkg/R/functions.R
+++ b/R/pkg/R/functions.R
@@ -2097,7 +2097,7 @@ setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeri
 #' expr
 #'
 #' Parses the expression string into the column that it represents, similar to
-#' DataFrame.selectExpr
+#' SparkDataFrame.selectExpr
 #'
 #' @family normal_funcs
 #' @rdname expr

http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/R/generics.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R
index 6b67258..04274a1 100644
--- a/R/pkg/R/generics.R
+++ b/R/pkg/R/generics.R
@@ -385,7 +385,7 @@ setGeneric("subtractByKey",
 setGeneric("value", function(bcast) { standardGeneric("value") })
 
 
-####################  DataFrame Methods ########################
+####################  SparkDataFrame Methods ########################
 
 #' @rdname agg
 #' @export

http://git-wip-us.apache.org/repos/asf/spark/blob/a55fbe2a/R/pkg/R/group.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R
index 23b49ae..08f4a49 100644
--- a/R/pkg/R/group.R
+++ b/R/pkg/R/group.R
@@ -23,7 +23,7 @@ NULL
 setOldClass("jobj")
 
 #' @title S4 class that represents a GroupedData
-#' @description GroupedDatas can be created using groupBy() on a DataFrame
+#' @description GroupedDatas can be created using groupBy() on a SparkDataFrame
 #' @rdname GroupedData
 #' @seealso groupBy
 #'
@@ -37,7 +37,7 @@ setMethod("initialize", "GroupedData", function(.Object, sgd) {
   .Object
 })
 
-#' @rdname DataFrame
+#' @rdname GroupedData
 groupedData <- function(sgd) {
   new("GroupedData", sgd)
 }
@@ -52,10 +52,10 @@ setMethod("show", "GroupedData",
 #' Count
 #'
 #' Count the number of rows for each group.
-#' The resulting DataFrame will also contain the grouping columns.
+#' The resulting SparkDataFrame will also contain the grouping columns.
 #'
 #' @param x a GroupedData
-#' @return a DataFrame
+#' @return a SparkDataFrame
 #' @rdname agg
 #' @export
 #' @examples
@@ -70,14 +70,14 @@ setMethod("count",
 
 #' summarize
 #'
-#' Aggregates on the entire DataFrame without groups.
-#' The resulting DataFrame will also contain the grouping columns.
+#' Aggregates on the entire SparkDataFrame without groups.
+#' The resulting SparkDataFrame will also contain the grouping columns.
 #'
 #' df2 <- agg(df, <column> = <aggFunction>)
 #' df2 <- agg(df, newColName = aggFunction(column))
 #'
 #' @param x a GroupedData
-#' @return a DataFrame
+#' @return a SparkDataFrame
 #' @rdname summarize
 #' @name agg
 #' @family agg_funcs


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org