Repository: spark
Updated Branches:
  refs/heads/master d5573c578 -> 41e1416f4


[SPARK-16693][SPARKR] Remove methods deprecated

## What changes were proposed in this pull request?

Remove deprecated functions which includes:
SQLContext/HiveContext stuff
sparkR.init
jsonFile
parquetFile
registerTempTable
saveAsParquetFile
unionAll
createExternalTable
dropTempTable

## How was this patch tested?

jenkins

Author: Felix Cheung <[email protected]>

Closes #22843 from felixcheung/rrddapi.


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

Branch: refs/heads/master
Commit: 41e1416f4d441415212bb5705898509ce5344ec4
Parents: d5573c5
Author: Felix Cheung <[email protected]>
Authored: Sat Oct 27 15:11:29 2018 -0700
Committer: Felix Cheung <[email protected]>
Committed: Sat Oct 27 15:11:29 2018 -0700

----------------------------------------------------------------------
 R/pkg/NAMESPACE                       |  11 +--
 R/pkg/R/DataFrame.R                   |  52 +----------
 R/pkg/R/SQLContext.R                  |  95 ++-----------------
 R/pkg/R/catalog.R                     |  99 +-------------------
 R/pkg/R/generics.R                    |   9 --
 R/pkg/R/sparkR.R                      | 142 +----------------------------
 R/pkg/tests/fulltests/test_context.R  |  12 ---
 R/pkg/tests/fulltests/test_sparkSQL.R |  60 ++----------
 docs/sparkr.md                        |   5 +-
 9 files changed, 32 insertions(+), 453 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/R/pkg/NAMESPACE
----------------------------------------------------------------------
diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE
index 5a5dc20..f9f556e 100644
--- a/R/pkg/NAMESPACE
+++ b/R/pkg/NAMESPACE
@@ -28,9 +28,8 @@ importFrom("utils", "download.file", "object.size", 
"packageVersion", "tail", "u
 
 # S3 methods exported
 export("sparkR.session")
-export("sparkR.init")
-export("sparkR.stop")
 export("sparkR.session.stop")
+export("sparkR.stop")
 export("sparkR.conf")
 export("sparkR.version")
 export("sparkR.uiWebUrl")
@@ -42,9 +41,6 @@ export("sparkR.callJStatic")
 
 export("install.spark")
 
-export("sparkRSQL.init",
-       "sparkRHive.init")
-
 # MLlib integration
 exportMethods("glm",
               "spark.glm",
@@ -151,7 +147,6 @@ exportMethods("arrange",
               "printSchema",
               "randomSplit",
               "rbind",
-              "registerTempTable",
               "rename",
               "repartition",
               "repartitionByRange",
@@ -159,7 +154,6 @@ exportMethods("arrange",
               "sample",
               "sample_frac",
               "sampleBy",
-              "saveAsParquetFile",
               "saveAsTable",
               "saveDF",
               "schema",
@@ -175,7 +169,6 @@ exportMethods("arrange",
               "toJSON",
               "transform",
               "union",
-              "unionAll",
               "unionByName",
               "unique",
               "unpersist",
@@ -415,10 +408,8 @@ export("as.DataFrame",
        "cacheTable",
        "clearCache",
        "createDataFrame",
-       "createExternalTable",
        "createTable",
        "currentDatabase",
-       "dropTempTable",
        "dropTempView",
        "listColumns",
        "listDatabases",

http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/R/pkg/R/DataFrame.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R
index bf82d0c..c99ad76 100644
--- a/R/pkg/R/DataFrame.R
+++ b/R/pkg/R/DataFrame.R
@@ -227,7 +227,7 @@ setMethod("showDF",
 #' show
 #'
 #' 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 
+#' 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.
@@ -521,32 +521,6 @@ setMethod("createOrReplaceTempView",
               invisible(callJMethod(x@sdf, "createOrReplaceTempView", 
viewName))
           })
 
-#' (Deprecated) Register Temporary Table
-#'
-#' Registers a SparkDataFrame as a Temporary Table in the SparkSession
-#' @param x A SparkDataFrame
-#' @param tableName A character vector containing the name of the table
-#'
-#' @seealso \link{createOrReplaceTempView}
-#' @rdname registerTempTable-deprecated
-#' @name registerTempTable
-#' @aliases registerTempTable,SparkDataFrame,character-method
-#' @examples
-#'\dontrun{
-#' sparkR.session()
-#' path <- "path/to/file.json"
-#' df <- read.json(path)
-#' registerTempTable(df, "json_df")
-#' new_df <- sql("SELECT * FROM json_df")
-#'}
-#' @note registerTempTable since 1.4.0
-setMethod("registerTempTable",
-          signature(x = "SparkDataFrame", tableName = "character"),
-          function(x, tableName) {
-              .Deprecated("createOrReplaceTempView")
-              invisible(callJMethod(x@sdf, "createOrReplaceTempView", 
tableName))
-          })
-
 #' insertInto
 #'
 #' Insert the contents of a SparkDataFrame into a table registered in the 
current SparkSession.
@@ -956,7 +930,6 @@ setMethod("write.orc",
 #' path <- "path/to/file.json"
 #' df <- read.json(path)
 #' write.parquet(df, "/tmp/sparkr-tmp1/")
-#' saveAsParquetFile(df, "/tmp/sparkr-tmp2/")
 #'}
 #' @note write.parquet since 1.6.0
 setMethod("write.parquet",
@@ -967,17 +940,6 @@ setMethod("write.parquet",
             invisible(handledCallJMethod(write, "parquet", path))
           })
 
-#' @rdname write.parquet
-#' @name saveAsParquetFile
-#' @aliases saveAsParquetFile,SparkDataFrame,character-method
-#' @note saveAsParquetFile since 1.4.0
-setMethod("saveAsParquetFile",
-          signature(x = "SparkDataFrame", path = "character"),
-          function(x, path) {
-            .Deprecated("write.parquet")
-            write.parquet(x, path)
-          })
-
 #' Save the content of SparkDataFrame in a text file at the specified path.
 #'
 #' Save the content of the SparkDataFrame in a text file at the specified path.
@@ -2762,18 +2724,6 @@ setMethod("union",
             dataFrame(unioned)
           })
 
-#' unionAll is deprecated - use union instead
-#' @rdname union
-#' @name unionAll
-#' @aliases unionAll,SparkDataFrame,SparkDataFrame-method
-#' @note unionAll since 1.4.0
-setMethod("unionAll",
-          signature(x = "SparkDataFrame", y = "SparkDataFrame"),
-          function(x, y) {
-            .Deprecated("union")
-            union(x, y)
-          })
-
 #' Return a new SparkDataFrame containing the union of rows, matched by column 
names
 #'
 #' Return a new SparkDataFrame containing the union of rows in this 
SparkDataFrame

http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/R/pkg/R/SQLContext.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R
index 3f89ee9..afcdd6f 100644
--- a/R/pkg/R/SQLContext.R
+++ b/R/pkg/R/SQLContext.R
@@ -37,37 +37,6 @@ getInternalType <- function(x) {
          stop(paste("Unsupported type for SparkDataFrame:", class(x))))
 }
 
-#' Temporary function to reroute old S3 Method call to new
-#' This function is specifically implemented to remove SQLContext from the 
parameter list.
-#' It determines the target to route the call by checking the parent of this 
callsite (say 'func').
-#' The target should be called 'func.default'.
-#' We need to check the class of x to ensure it is SQLContext/HiveContext 
before dispatching.
-#' @param newFuncSig name of the function the user should call instead in the 
deprecation message
-#' @param x the first parameter of the original call
-#' @param ... the rest of parameter to pass along
-#' @return whatever the target returns
-#' @noRd
-dispatchFunc <- function(newFuncSig, x, ...) {
-  # When called with SparkR::createDataFrame, sys.call()[[1]] returns c(::, 
SparkR, createDataFrame)
-  callsite <- as.character(sys.call(sys.parent())[[1]])
-  funcName <- callsite[[length(callsite)]]
-  f <- get(paste0(funcName, ".default"))
-  # Strip sqlContext from list of parameters and then pass the rest along.
-  contextNames <- c("org.apache.spark.sql.SQLContext",
-                    "org.apache.spark.sql.hive.HiveContext",
-                    "org.apache.spark.sql.hive.test.TestHiveContext",
-                    "org.apache.spark.sql.SparkSession")
-  if (missing(x) && length(list(...)) == 0) {
-    f()
-  } else if (class(x) == "jobj" &&
-            any(grepl(paste(contextNames, collapse = "|"), 
getClassName.jobj(x)))) {
-    .Deprecated(newFuncSig, old = paste0(funcName, "(sqlContext...)"))
-    f(...)
-  } else {
-    f(x, ...)
-  }
-}
-
 #' return the SparkSession
 #' @noRd
 getSparkSession <- function() {
@@ -198,11 +167,10 @@ getDefaultSqlSource <- function() {
 #' df4 <- createDataFrame(cars, numPartitions = 2)
 #' }
 #' @name createDataFrame
-#' @method createDataFrame default
 #' @note createDataFrame since 1.4.0
 # TODO(davies): support sampling and infer type from NA
-createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0,
-                                    numPartitions = NULL) {
+createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0,
+                            numPartitions = NULL) {
   sparkSession <- getSparkSession()
 
   if (is.data.frame(data)) {
@@ -285,31 +253,18 @@ createDataFrame.default <- function(data, schema = NULL, 
samplingRatio = 1.0,
   dataFrame(sdf)
 }
 
-createDataFrame <- function(x, ...) {
-  dispatchFunc("createDataFrame(data, schema = NULL)", x, ...)
-}
-
 #' @rdname createDataFrame
 #' @aliases createDataFrame
-#' @method as.DataFrame default
 #' @note as.DataFrame since 1.6.0
-as.DataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0, 
numPartitions = NULL) {
+as.DataFrame <- function(data, schema = NULL, samplingRatio = 1.0, 
numPartitions = NULL) {
   createDataFrame(data, schema, samplingRatio, numPartitions)
 }
 
-#' @param ... additional argument(s).
-#' @rdname createDataFrame
-#' @aliases as.DataFrame
-as.DataFrame <- function(data, ...) {
-  dispatchFunc("as.DataFrame(data, schema = NULL)", data, ...)
-}
-
 #' toDF
 #'
 #' Converts an RDD to a SparkDataFrame by infer the types.
 #'
 #' @param x An RDD
-#'
 #' @rdname SparkDataFrame
 #' @noRd
 #' @examples
@@ -345,9 +300,8 @@ setMethod("toDF", signature(x = "RDD"),
 #' df <- read.json(path, multiLine = TRUE)
 #' }
 #' @name read.json
-#' @method read.json default
 #' @note read.json since 1.6.0
-read.json.default <- function(path, ...) {
+read.json <- function(path, ...) {
   sparkSession <- getSparkSession()
   options <- varargsToStrEnv(...)
   # Allow the user to have a more flexible definition of the text file path
@@ -358,10 +312,6 @@ read.json.default <- function(path, ...) {
   dataFrame(sdf)
 }
 
-read.json <- function(x, ...) {
-  dispatchFunc("read.json(path)", x, ...)
-}
-
 #' Create a SparkDataFrame from an ORC file.
 #'
 #' Loads an ORC file, returning the result as a SparkDataFrame.
@@ -388,13 +338,12 @@ read.orc <- function(path, ...) {
 #' Loads a Parquet file, returning the result as a SparkDataFrame.
 #'
 #' @param path path of file to read. A vector of multiple paths is allowed.
-#' @param ... additional external data source specific named properties.
+#' @param ... additional data source specific named properties.
 #' @return SparkDataFrame
 #' @rdname read.parquet
 #' @name read.parquet
-#' @method read.parquet default
 #' @note read.parquet since 1.6.0
-read.parquet.default <- function(path, ...) {
+read.parquet <- function(path, ...) {
   sparkSession <- getSparkSession()
   options <- varargsToStrEnv(...)
   # Allow the user to have a more flexible definition of the Parquet file path
@@ -405,10 +354,6 @@ read.parquet.default <- function(path, ...) {
   dataFrame(sdf)
 }
 
-read.parquet <- function(x, ...) {
-  dispatchFunc("read.parquet(...)", x, ...)
-}
-
 #' Create a SparkDataFrame from a text file.
 #'
 #' Loads text files and returns a SparkDataFrame whose schema starts with
@@ -428,9 +373,8 @@ read.parquet <- function(x, ...) {
 #' df <- read.text(path)
 #' }
 #' @name read.text
-#' @method read.text default
 #' @note read.text since 1.6.1
-read.text.default <- function(path, ...) {
+read.text <- function(path, ...) {
   sparkSession <- getSparkSession()
   options <- varargsToStrEnv(...)
   # Allow the user to have a more flexible definition of the text file path
@@ -441,10 +385,6 @@ read.text.default <- function(path, ...) {
   dataFrame(sdf)
 }
 
-read.text <- function(x, ...) {
-  dispatchFunc("read.text(path)", x, ...)
-}
-
 #' SQL Query
 #'
 #' Executes a SQL query using Spark, returning the result as a SparkDataFrame.
@@ -461,18 +401,13 @@ read.text <- function(x, ...) {
 #' new_df <- sql("SELECT * FROM table")
 #' }
 #' @name sql
-#' @method sql default
 #' @note sql since 1.4.0
-sql.default <- function(sqlQuery) {
+sql <- function(sqlQuery) {
   sparkSession <- getSparkSession()
   sdf <- callJMethod(sparkSession, "sql", sqlQuery)
   dataFrame(sdf)
 }
 
-sql <- function(x, ...) {
-  dispatchFunc("sql(sqlQuery)", x, ...)
-}
-
 #' Create a SparkDataFrame from a SparkSQL table or view
 #'
 #' Returns the specified table or view as a SparkDataFrame. The table or view 
must already exist or
@@ -531,9 +466,8 @@ tableToDF <- function(tableName) {
 #' df4 <- read.df(mapTypeJsonPath, "json", stringSchema, multiLine = TRUE)
 #' }
 #' @name read.df
-#' @method read.df default
 #' @note read.df since 1.4.0
-read.df.default <- function(path = NULL, source = NULL, schema = NULL, 
na.strings = "NA", ...) {
+read.df <- function(path = NULL, source = NULL, schema = NULL, na.strings = 
"NA", ...) {
   if (!is.null(path) && !is.character(path)) {
     stop("path should be character, NULL or omitted.")
   }
@@ -568,22 +502,13 @@ read.df.default <- function(path = NULL, source = NULL, 
schema = NULL, na.string
   dataFrame(sdf)
 }
 
-read.df <- function(x = NULL, ...) {
-  dispatchFunc("read.df(path = NULL, source = NULL, schema = NULL, ...)", x, 
...)
-}
-
 #' @rdname read.df
 #' @name loadDF
-#' @method loadDF default
 #' @note loadDF since 1.6.0
-loadDF.default <- function(path = NULL, source = NULL, schema = NULL, ...) {
+loadDF <- function(path = NULL, source = NULL, schema = NULL, ...) {
   read.df(path, source, schema, ...)
 }
 
-loadDF <- function(x = NULL, ...) {
-  dispatchFunc("loadDF(path = NULL, source = NULL, schema = NULL, ...)", x, 
...)
-}
-
 #' Create a SparkDataFrame representing the database table accessible via JDBC 
URL
 #'
 #' Additional JDBC database connection properties can be set (...)

http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/R/pkg/R/catalog.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/catalog.R b/R/pkg/R/catalog.R
index c2d0fc3..7641f8a 100644
--- a/R/pkg/R/catalog.R
+++ b/R/pkg/R/catalog.R
@@ -17,40 +17,6 @@
 
 # catalog.R: SparkSession catalog functions
 
-#' (Deprecated) Create an external table
-#'
-#' Creates an external table based on the dataset in a data source,
-#' Returns a SparkDataFrame associated with the external table.
-#'
-#' The data source is specified by the \code{source} and a set of options(...).
-#' If \code{source} is not specified, the default data source configured by
-#' "spark.sql.sources.default" will be used.
-#'
-#' @param tableName a name of the table.
-#' @param path the path of files to load.
-#' @param source the name of external data source.
-#' @param schema the schema of the data required for some data sources.
-#' @param ... additional argument(s) passed to the method.
-#' @return A SparkDataFrame.
-#' @rdname createExternalTable-deprecated
-#' @seealso \link{createTable}
-#' @examples
-#'\dontrun{
-#' sparkR.session()
-#' df <- createExternalTable("myjson", path="path/to/json", source="json", 
schema)
-#' }
-#' @name createExternalTable
-#' @method createExternalTable default
-#' @note createExternalTable since 1.4.0
-createExternalTable.default <- function(tableName, path = NULL, source = NULL, 
schema = NULL, ...) {
-  .Deprecated("createTable", old = "createExternalTable")
-  createTable(tableName, path, source, schema, ...)
-}
-
-createExternalTable <- function(x, ...) {
-  dispatchFunc("createExternalTable(tableName, path = NULL, source = NULL, 
...)", x, ...)
-}
-
 #' Creates a table based on the dataset in a data source
 #'
 #' Creates a table based on the dataset in a data source. Returns a 
SparkDataFrame associated with
@@ -116,18 +82,13 @@ createTable <- function(tableName, path = NULL, source = 
NULL, schema = NULL, ..
 #' cacheTable("table")
 #' }
 #' @name cacheTable
-#' @method cacheTable default
 #' @note cacheTable since 1.4.0
-cacheTable.default <- function(tableName) {
+cacheTable <- function(tableName) {
   sparkSession <- getSparkSession()
   catalog <- callJMethod(sparkSession, "catalog")
   invisible(handledCallJMethod(catalog, "cacheTable", tableName))
 }
 
-cacheTable <- function(x, ...) {
-  dispatchFunc("cacheTable(tableName)", x, ...)
-}
-
 #' Uncache Table
 #'
 #' Removes the specified table from the in-memory cache.
@@ -145,18 +106,13 @@ cacheTable <- function(x, ...) {
 #' uncacheTable("table")
 #' }
 #' @name uncacheTable
-#' @method uncacheTable default
 #' @note uncacheTable since 1.4.0
-uncacheTable.default <- function(tableName) {
+uncacheTable <- function(tableName) {
   sparkSession <- getSparkSession()
   catalog <- callJMethod(sparkSession, "catalog")
   invisible(handledCallJMethod(catalog, "uncacheTable", tableName))
 }
 
-uncacheTable <- function(x, ...) {
-  dispatchFunc("uncacheTable(tableName)", x, ...)
-}
-
 #' Clear Cache
 #'
 #' Removes all cached tables from the in-memory cache.
@@ -167,48 +123,13 @@ uncacheTable <- function(x, ...) {
 #' clearCache()
 #' }
 #' @name clearCache
-#' @method clearCache default
 #' @note clearCache since 1.4.0
-clearCache.default <- function() {
+clearCache <- function() {
   sparkSession <- getSparkSession()
   catalog <- callJMethod(sparkSession, "catalog")
   invisible(callJMethod(catalog, "clearCache"))
 }
 
-clearCache <- function() {
-  dispatchFunc("clearCache()")
-}
-
-#' (Deprecated) Drop Temporary Table
-#'
-#' Drops the temporary table with the given table name in the catalog.
-#' If the table has been cached/persisted before, it's also unpersisted.
-#'
-#' @param tableName The name of the SparkSQL table to be dropped.
-#' @seealso \link{dropTempView}
-#' @rdname dropTempTable-deprecated
-#' @examples
-#' \dontrun{
-#' sparkR.session()
-#' df <- read.df(path, "parquet")
-#' createOrReplaceTempView(df, "table")
-#' dropTempTable("table")
-#' }
-#' @name dropTempTable
-#' @method dropTempTable default
-#' @note dropTempTable since 1.4.0
-dropTempTable.default <- function(tableName) {
-  .Deprecated("dropTempView", old = "dropTempTable")
-  if (class(tableName) != "character") {
-    stop("tableName must be a string.")
-  }
-  dropTempView(tableName)
-}
-
-dropTempTable <- function(x, ...) {
-  dispatchFunc("dropTempView(viewName)", x, ...)
-}
-
 #' Drops the temporary view with the given view name in the catalog.
 #'
 #' Drops the temporary view with the given view name in the catalog.
@@ -249,17 +170,12 @@ dropTempView <- function(viewName) {
 #' tables("hive")
 #' }
 #' @name tables
-#' @method tables default
 #' @note tables since 1.4.0
-tables.default <- function(databaseName = NULL) {
+tables <- function(databaseName = NULL) {
   # rename column to match previous output schema
   withColumnRenamed(listTables(databaseName), "name", "tableName")
 }
 
-tables <- function(x, ...) {
-  dispatchFunc("tables(databaseName = NULL)", x, ...)
-}
-
 #' Table Names
 #'
 #' Returns the names of tables in the given database as an array.
@@ -273,9 +189,8 @@ tables <- function(x, ...) {
 #' tableNames("hive")
 #' }
 #' @name tableNames
-#' @method tableNames default
 #' @note tableNames since 1.4.0
-tableNames.default <- function(databaseName = NULL) {
+tableNames <- function(databaseName = NULL) {
   sparkSession <- getSparkSession()
   callJStatic("org.apache.spark.sql.api.r.SQLUtils",
               "getTableNames",
@@ -283,10 +198,6 @@ tableNames.default <- function(databaseName = NULL) {
               databaseName)
 }
 
-tableNames <- function(x, ...) {
-  dispatchFunc("tableNames(databaseName = NULL)", x, ...)
-}
-
 #' Returns the current default database
 #'
 #' Returns the current default database.

http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/R/pkg/R/generics.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R
index 045e075..76e17c1 100644
--- a/R/pkg/R/generics.R
+++ b/R/pkg/R/generics.R
@@ -528,9 +528,6 @@ setGeneric("persist", function(x, newLevel) { 
standardGeneric("persist") })
 #' @rdname printSchema
 setGeneric("printSchema", function(x) { standardGeneric("printSchema") })
 
-#' @rdname registerTempTable-deprecated
-setGeneric("registerTempTable", function(x, tableName) { 
standardGeneric("registerTempTable") })
-
 #' @rdname rename
 setGeneric("rename", function(x, ...) { standardGeneric("rename") })
 
@@ -595,9 +592,6 @@ setGeneric("write.parquet", function(x, path, ...) {
   standardGeneric("write.parquet")
 })
 
-#' @rdname write.parquet
-setGeneric("saveAsParquetFile", function(x, path) { 
standardGeneric("saveAsParquetFile") })
-
 #' @rdname write.stream
 setGeneric("write.stream", function(df, source = NULL, outputMode = NULL, ...) 
{
   standardGeneric("write.stream")
@@ -637,9 +631,6 @@ setGeneric("toRDD", function(x) { standardGeneric("toRDD") 
})
 #' @rdname union
 setGeneric("union", function(x, y) { standardGeneric("union") })
 
-#' @rdname union
-setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") })
-
 #' @rdname unionByName
 setGeneric("unionByName", function(x, y) { standardGeneric("unionByName") })
 

http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/R/pkg/R/sparkR.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R
index 038fefa..ac289d3 100644
--- a/R/pkg/R/sparkR.R
+++ b/R/pkg/R/sparkR.R
@@ -88,49 +88,6 @@ sparkR.stop <- function() {
   sparkR.session.stop()
 }
 
-#' (Deprecated) Initialize a new Spark Context
-#'
-#' This function initializes a new SparkContext.
-#'
-#' @param master The Spark master URL
-#' @param appName Application name to register with cluster manager
-#' @param sparkHome Spark Home directory
-#' @param sparkEnvir Named list of environment variables to set on worker nodes
-#' @param sparkExecutorEnv Named list of environment variables to be used when 
launching executors
-#' @param sparkJars Character vector of jar files to pass to the worker nodes
-#' @param sparkPackages Character vector of package coordinates
-#' @seealso \link{sparkR.session}
-#' @rdname sparkR.init-deprecated
-#' @examples
-#'\dontrun{
-#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark")
-#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark",
-#'                  list(spark.executor.memory="1g"))
-#' sc <- sparkR.init("yarn-client", "SparkR", "/home/spark",
-#'                  list(spark.executor.memory="4g"),
-#'                  list(LD_LIBRARY_PATH="/directory of JVM libraries 
(libjvm.so) on workers/"),
-#'                  c("one.jar", "two.jar", "three.jar"),
-#'                  c("com.databricks:spark-avro_2.11:2.0.1"))
-#'}
-#' @note sparkR.init since 1.4.0
-sparkR.init <- function(
-  master = "",
-  appName = "SparkR",
-  sparkHome = Sys.getenv("SPARK_HOME"),
-  sparkEnvir = list(),
-  sparkExecutorEnv = list(),
-  sparkJars = "",
-  sparkPackages = "") {
-  .Deprecated("sparkR.session")
-  sparkR.sparkContext(master,
-     appName,
-     sparkHome,
-     convertNamedListToEnv(sparkEnvir),
-     convertNamedListToEnv(sparkExecutorEnv),
-     sparkJars,
-     sparkPackages)
-}
-
 # Internal function to handle creating the SparkContext.
 sparkR.sparkContext <- function(
   master = "",
@@ -272,61 +229,6 @@ sparkR.sparkContext <- function(
   sc
 }
 
-#' (Deprecated) Initialize a new SQLContext
-#'
-#' This function creates a SparkContext from an existing JavaSparkContext and
-#' then uses it to initialize a new SQLContext
-#'
-#' Starting SparkR 2.0, a SparkSession is initialized and returned instead.
-#' This API is deprecated and kept for backward compatibility only.
-#'
-#' @param jsc The existing JavaSparkContext created with SparkR.init()
-#' @seealso \link{sparkR.session}
-#' @rdname sparkRSQL.init-deprecated
-#' @examples
-#'\dontrun{
-#' sc <- sparkR.init()
-#' sqlContext <- sparkRSQL.init(sc)
-#'}
-#' @note sparkRSQL.init since 1.4.0
-sparkRSQL.init <- function(jsc = NULL) {
-  .Deprecated("sparkR.session")
-
-  if (exists(".sparkRsession", envir = .sparkREnv)) {
-    return(get(".sparkRsession", envir = .sparkREnv))
-  }
-
-  # Default to without Hive support for backward compatibility.
-  sparkR.session(enableHiveSupport = FALSE)
-}
-
-#' (Deprecated) Initialize a new HiveContext
-#'
-#' This function creates a HiveContext from an existing JavaSparkContext
-#'
-#' Starting SparkR 2.0, a SparkSession is initialized and returned instead.
-#' This API is deprecated and kept for backward compatibility only.
-#'
-#' @param jsc The existing JavaSparkContext created with SparkR.init()
-#' @seealso \link{sparkR.session}
-#' @rdname sparkRHive.init-deprecated
-#' @examples
-#'\dontrun{
-#' sc <- sparkR.init()
-#' sqlContext <- sparkRHive.init(sc)
-#'}
-#' @note sparkRHive.init since 1.4.0
-sparkRHive.init <- function(jsc = NULL) {
-  .Deprecated("sparkR.session")
-
-  if (exists(".sparkRsession", envir = .sparkREnv)) {
-    return(get(".sparkRsession", envir = .sparkREnv))
-  }
-
-  # Default to without Hive support for backward compatibility.
-  sparkR.session(enableHiveSupport = TRUE)
-}
-
 #' Get the existing SparkSession or initialize a new SparkSession.
 #'
 #' SparkSession is the entry point into SparkR. \code{sparkR.session} gets the 
existing
@@ -482,26 +384,11 @@ sparkR.uiWebUrl <- function() {
 #' setJobGroup("myJobGroup", "My job group description", TRUE)
 #'}
 #' @note setJobGroup since 1.5.0
-#' @method setJobGroup default
-setJobGroup.default <- function(groupId, description, interruptOnCancel) {
+setJobGroup <- function(groupId, description, interruptOnCancel) {
   sc <- getSparkContext()
   invisible(callJMethod(sc, "setJobGroup", groupId, description, 
interruptOnCancel))
 }
 
-setJobGroup <- function(sc, groupId, description, interruptOnCancel) {
-  if (class(sc) == "jobj" && any(grepl("JavaSparkContext", 
getClassName.jobj(sc)))) {
-    .Deprecated("setJobGroup(groupId, description, interruptOnCancel)",
-                old = "setJobGroup(sc, groupId, description, 
interruptOnCancel)")
-    setJobGroup.default(groupId, description, interruptOnCancel)
-  } else {
-    # Parameter order is shifted
-    groupIdToUse <- sc
-    descriptionToUse <- groupId
-    interruptOnCancelToUse <- description
-    setJobGroup.default(groupIdToUse, descriptionToUse, interruptOnCancelToUse)
-  }
-}
-
 #' Clear current job group ID and its description
 #'
 #' @rdname clearJobGroup
@@ -512,22 +399,11 @@ setJobGroup <- function(sc, groupId, description, 
interruptOnCancel) {
 #' clearJobGroup()
 #'}
 #' @note clearJobGroup since 1.5.0
-#' @method clearJobGroup default
-clearJobGroup.default <- function() {
+clearJobGroup <- function() {
   sc <- getSparkContext()
   invisible(callJMethod(sc, "clearJobGroup"))
 }
 
-clearJobGroup <- function(sc) {
-  if (!missing(sc) &&
-      class(sc) == "jobj" &&
-      any(grepl("JavaSparkContext", getClassName.jobj(sc)))) {
-    .Deprecated("clearJobGroup()", old = "clearJobGroup(sc)")
-  }
-  clearJobGroup.default()
-}
-
-
 #' Cancel active jobs for the specified group
 #'
 #' @param groupId the ID of job group to be cancelled
@@ -539,23 +415,11 @@ clearJobGroup <- function(sc) {
 #' cancelJobGroup("myJobGroup")
 #'}
 #' @note cancelJobGroup since 1.5.0
-#' @method cancelJobGroup default
-cancelJobGroup.default <- function(groupId) {
+cancelJobGroup <- function(groupId) {
   sc <- getSparkContext()
   invisible(callJMethod(sc, "cancelJobGroup", groupId))
 }
 
-cancelJobGroup <- function(sc, groupId) {
-  if (class(sc) == "jobj" && any(grepl("JavaSparkContext", 
getClassName.jobj(sc)))) {
-    .Deprecated("cancelJobGroup(groupId)", old = "cancelJobGroup(sc, groupId)")
-    cancelJobGroup.default(groupId)
-  } else {
-    # Parameter order is shifted
-    groupIdToUse <- sc
-    cancelJobGroup.default(groupIdToUse)
-  }
-}
-
 #' Set a human readable description of the current job.
 #'
 #' Set a description that is shown as a job description in UI.

http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/R/pkg/tests/fulltests/test_context.R
----------------------------------------------------------------------
diff --git a/R/pkg/tests/fulltests/test_context.R 
b/R/pkg/tests/fulltests/test_context.R
index 288a271..eb8d2a7 100644
--- a/R/pkg/tests/fulltests/test_context.R
+++ b/R/pkg/tests/fulltests/test_context.R
@@ -54,15 +54,6 @@ test_that("Check masked functions", {
                sort(namesOfMaskedCompletely, na.last = TRUE))
 })
 
-test_that("repeatedly starting and stopping SparkR", {
-  for (i in 1:4) {
-    sc <- suppressWarnings(sparkR.init(master = sparkRTestMaster))
-    rdd <- parallelize(sc, 1:20, 2L)
-    expect_equal(countRDD(rdd), 20)
-    suppressWarnings(sparkR.stop())
-  }
-})
-
 test_that("repeatedly starting and stopping SparkSession", {
   for (i in 1:4) {
     sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE)
@@ -101,9 +92,6 @@ test_that("job group functions can be called", {
   cancelJobGroup("groupId")
   clearJobGroup()
 
-  suppressWarnings(setJobGroup(sc, "groupId", "job description", TRUE))
-  suppressWarnings(cancelJobGroup(sc, "groupId"))
-  suppressWarnings(clearJobGroup(sc))
   sparkR.session.stop()
 })
 

http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/R/pkg/tests/fulltests/test_sparkSQL.R
----------------------------------------------------------------------
diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R 
b/R/pkg/tests/fulltests/test_sparkSQL.R
index 68bf5ea..58e0a54 100644
--- a/R/pkg/tests/fulltests/test_sparkSQL.R
+++ b/R/pkg/tests/fulltests/test_sparkSQL.R
@@ -106,15 +106,6 @@ if (is_windows()) {
   Sys.setenv(TZ = "GMT")
 }
 
-test_that("calling sparkRSQL.init returns existing SQL context", {
-  sqlContext <- suppressWarnings(sparkRSQL.init(sc))
-  expect_equal(suppressWarnings(sparkRSQL.init(sc)), sqlContext)
-})
-
-test_that("calling sparkRSQL.init returns existing SparkSession", {
-  expect_equal(suppressWarnings(sparkRSQL.init(sc)), sparkSession)
-})
-
 test_that("calling sparkR.session returns existing SparkSession", {
   expect_equal(sparkR.session(), sparkSession)
 })
@@ -221,7 +212,7 @@ test_that("structField type strings", {
 
 test_that("create DataFrame from RDD", {
   rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) 
})
-  df <- createDataFrame(rdd, list("a", "b"))
+  df <- SparkR::createDataFrame(rdd, list("a", "b"))
   dfAsDF <- as.DataFrame(rdd, list("a", "b"))
   expect_is(df, "SparkDataFrame")
   expect_is(dfAsDF, "SparkDataFrame")
@@ -287,7 +278,7 @@ test_that("create DataFrame from RDD", {
 
   df <- as.DataFrame(cars, numPartitions = 2)
   expect_equal(getNumPartitions(df), 2)
-  df <- createDataFrame(cars, numPartitions = 3)
+  df <- SparkR::createDataFrame(cars, numPartitions = 3)
   expect_equal(getNumPartitions(df), 3)
   # validate limit by num of rows
   df <- createDataFrame(cars, numPartitions = 60)
@@ -308,7 +299,7 @@ test_that("create DataFrame from RDD", {
   sql("CREATE TABLE people (name string, age double, height float)")
   df <- read.df(jsonPathNa, "json", schema)
   insertInto(df, "people")
-  expect_equal(collect(sql("SELECT age from people WHERE name = 'Bob'"))$age,
+  expect_equal(collect(SparkR::sql("SELECT age from people WHERE name = 
'Bob'"))$age,
                c(16))
   expect_equal(collect(sql("SELECT height from people WHERE name 
='Bob'"))$height,
                c(176.5))
@@ -665,10 +656,10 @@ test_that("test tableNames and tables", {
   expect_true("tableName" %in% colnames(tables()))
   expect_true(all(c("tableName", "database", "isTemporary") %in% 
colnames(tables())))
 
-  suppressWarnings(registerTempTable(df, "table2"))
+  createOrReplaceTempView(df, "table2")
   tables <- listTables()
   expect_equal(count(tables), count + 2)
-  suppressWarnings(dropTempTable("table1"))
+  dropTempView("table1")
   expect_true(dropTempView("table2"))
 
   tables <- listTables()
@@ -2461,7 +2452,7 @@ test_that("union(), unionByName(), rbind(), except(), and 
intersect() on a DataF
   expect_is(unioned, "SparkDataFrame")
   expect_equal(count(unioned), 6)
   expect_equal(first(unioned)$name, "Michael")
-  expect_equal(count(arrange(suppressWarnings(unionAll(df, df2)), df$age)), 6)
+  expect_equal(count(arrange(suppressWarnings(union(df, df2)), df$age)), 6)
 
   df1 <- select(df2, "age", "name")
   unioned1 <- arrange(unionByName(df1, df), df1$age)
@@ -2640,11 +2631,11 @@ test_that("read/write Parquet files", {
     expect_is(df2, "SparkDataFrame")
     expect_equal(count(df2), 3)
 
-    # Test write.parquet/saveAsParquetFile and read.parquet
+    # Test write.parquet and read.parquet
     parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet")
     write.parquet(df, parquetPath2)
     parquetPath3 <- tempfile(pattern = "parquetPath3", fileext = ".parquet")
-    suppressWarnings(saveAsParquetFile(df, parquetPath3))
+    write.parquet(df, parquetPath3)
     parquetDF <- read.parquet(c(parquetPath2, parquetPath3))
     expect_is(parquetDF, "SparkDataFrame")
     expect_equal(count(parquetDF), count(df) * 2)
@@ -3456,39 +3447,6 @@ test_that("Window functions on a DataFrame", {
   expect_equal(result, expected)
 })
 
-test_that("createDataFrame sqlContext parameter backward compatibility", {
-  sqlContext <- suppressWarnings(sparkRSQL.init(sc))
-  a <- 1:3
-  b <- c("a", "b", "c")
-  ldf <- data.frame(a, b)
-  # Call function with namespace :: operator - SPARK-16538
-  df <- suppressWarnings(SparkR::createDataFrame(sqlContext, ldf))
-  expect_equal(columns(df), c("a", "b"))
-  expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
-  expect_equal(count(df), 3)
-  ldf2 <- collect(df)
-  expect_equal(ldf$a, ldf2$a)
-
-  df2 <- suppressWarnings(createDataFrame(sqlContext, iris))
-  expect_equal(count(df2), 150)
-  expect_equal(ncol(df2), 5)
-
-  df3 <- suppressWarnings(read.df(sqlContext, jsonPath, "json"))
-  expect_is(df3, "SparkDataFrame")
-  expect_equal(count(df3), 3)
-
-  before <- suppressWarnings(createDataFrame(sqlContext, iris))
-  after <- suppressWarnings(createDataFrame(iris))
-  expect_equal(collect(before), collect(after))
-
-  # more tests for SPARK-16538
-  createOrReplaceTempView(df, "table")
-  SparkR::listTables()
-  SparkR::sql("SELECT 1")
-  suppressWarnings(SparkR::sql(sqlContext, "SELECT * FROM table"))
-  suppressWarnings(SparkR::dropTempTable(sqlContext, "table"))
-})
-
 test_that("randomSplit", {
   num <- 4000
   df <- createDataFrame(data.frame(id = 1:num))
@@ -3675,7 +3633,7 @@ test_that("catalog APIs, listTables, listColumns, 
listFunctions", {
 
   createOrReplaceTempView(as.DataFrame(cars), "cars")
 
-  tb <- listTables()
+  tb <- SparkR::listTables()
   expect_equal(nrow(tb), count + 1)
   tbs <- collect(tb)
   expect_true(nrow(tbs[tbs$name == "cars", ]) > 0)

http://git-wip-us.apache.org/repos/asf/spark/blob/41e1416f/docs/sparkr.md
----------------------------------------------------------------------
diff --git a/docs/sparkr.md b/docs/sparkr.md
index 5882ed7..cc6bc6d 100644
--- a/docs/sparkr.md
+++ b/docs/sparkr.md
@@ -487,7 +487,7 @@ df2
 ##+-------+-----+
 ##only showing top 10 rows
 
-{% endhighlight %} 
+{% endhighlight %}
 </div>
 
 Note that to enable eager execution in `sparkR` shell, add 
`spark.sql.repl.eagerEval.enabled=true` configuration property to the `--conf` 
option.
@@ -717,4 +717,5 @@ You can inspect the search path in R with 
[`search()`](https://stat.ethz.ch/R-ma
 
 ## Upgrading to SparkR 3.0.0
 
- - The deprecated methods `parquetFile`, `jsonRDD` and `jsonFile` in 
`SQLContext` have been removed. Use `read.parquet` and `read.json`.
+ - The deprecated methods `sparkR.init`, `sparkRSQL.init`, `sparkRHive.init` 
have been removed. Use `sparkR.session` instead.
+ - The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, 
`registerTempTable`, `createExternalTable`, `dropTempTable`, `unionAll` have 
been removed. Use `read.parquet`, `write.parquet`, `read.json`, 
`createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead.


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to