Skip to content

Commit

Permalink
[SPARK-31571][R] Overhaul stop/message/warning calls to be more canon…
Browse files Browse the repository at this point in the history
…ical

### What changes were proposed in this pull request?

Internal usages like `{stop,warning,message}({paste,paste0,sprintf}` and `{stop,warning,message}(some_literal_string_as_variable` have been removed and replaced as appropriate.

### Why are the changes needed?

CRAN policy recommends against using such constructions to build error messages, in particular because it makes the process of creating portable error messages for the package more onerous.

### Does this PR introduce any user-facing change?

There may be some small grammatical changes visible in error messaging.

### How was this patch tested?

Not done

Closes apache#28365 from MichaelChirico/r-stop-paste.

Authored-by: Michael Chirico <michael.chirico@grabtaxi.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
  • Loading branch information
Michael Chirico authored and huaxingao committed May 4, 2020
1 parent ca9f9eb commit 0c8146a
Show file tree
Hide file tree
Showing 16 changed files with 103 additions and 118 deletions.
40 changes: 19 additions & 21 deletions R/pkg/R/DataFrame.R
Expand Up @@ -431,7 +431,7 @@ setMethod("coltypes",
if (is.null(type)) {
specialtype <- specialtypeshandle(x)
if (is.null(specialtype)) {
stop(paste("Unsupported data type: ", x))
stop("Unsupported data type: ", x)
}
type <- PRIMITIVE_TYPES[[specialtype]]
}
Expand Down Expand Up @@ -829,8 +829,8 @@ setMethod("repartitionByRange",
jcol <- lapply(cols, function(c) { c@jc })
sdf <- callJMethod(x@sdf, "repartitionByRange", numToInt(numPartitions), jcol)
} else {
stop(paste("numPartitions and col must be numeric and Column; however, got",
class(numPartitions), "and", class(col)))
stop("numPartitions and col must be numeric and Column; however, got ",
class(numPartitions), " and ", class(col))
}
} else if (!is.null(col)) {
# only columns are specified
Expand All @@ -839,7 +839,7 @@ setMethod("repartitionByRange",
jcol <- lapply(cols, function(c) { c@jc })
sdf <- callJMethod(x@sdf, "repartitionByRange", jcol)
} else {
stop(paste("col must be Column; however, got", class(col)))
stop("col must be Column; however, got ", class(col))
}
} else if (!is.null(numPartitions)) {
# only numPartitions is specified
Expand Down Expand Up @@ -1068,10 +1068,10 @@ setMethod("sample",
signature(x = "SparkDataFrame"),
function(x, withReplacement = FALSE, fraction, seed) {
if (!is.numeric(fraction)) {
stop(paste("fraction must be numeric; however, got", class(fraction)))
stop("fraction must be numeric; however, got ", class(fraction))
}
if (!is.logical(withReplacement)) {
stop(paste("withReplacement must be logical; however, got", class(withReplacement)))
stop("withReplacement must be logical; however, got ", class(withReplacement))
}

if (!missing(seed)) {
Expand Down Expand Up @@ -1211,11 +1211,10 @@ setMethod("collect",
checkSchemaInArrow(schema(x))
TRUE
}, error = function(e) {
warning(paste0("The conversion from Spark DataFrame to R DataFrame was attempted ",
"with Arrow optimization because ",
"'spark.sql.execution.arrow.sparkr.enabled' is set to true; ",
"however, failed, attempting non-optimization. Reason: ",
e))
warning("The conversion from Spark DataFrame to R DataFrame was attempted ",
"with Arrow optimization because ",
"'spark.sql.execution.arrow.sparkr.enabled' is set to true; ",
"however, failed, attempting non-optimization. Reason: ", e)
FALSE
})
}
Expand Down Expand Up @@ -1508,8 +1507,8 @@ dapplyInternal <- function(x, func, schema) {
if (inherits(schema, "structType")) {
checkSchemaInArrow(schema)
} else if (is.null(schema)) {
stop(paste0("Arrow optimization does not support 'dapplyCollect' yet. Please disable ",
"Arrow optimization or use 'collect' and 'dapply' APIs instead."))
stop("Arrow optimization does not support 'dapplyCollect' yet. Please disable ",
"Arrow optimization or use 'collect' and 'dapply' APIs instead.")
} else {
stop("'schema' should be DDL-formatted string or structType.")
}
Expand Down Expand Up @@ -2012,8 +2011,8 @@ setMethod("[", signature(x = "SparkDataFrame"),
x
} else {
if (class(i) != "Column") {
stop(paste0("Expressions other than filtering predicates are not supported ",
"in the first parameter of extract operator [ or subset() method."))
stop("Expressions other than filtering predicates are not supported ",
"in the first parameter of extract operator [ or subset() method.")
}
filter(x, i)
}
Expand Down Expand Up @@ -2604,18 +2603,17 @@ setMethod("join",
if (is.null(joinType)) {
sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc)
} else {
if (joinType %in% c("inner", "cross",
validJoinTypes <- c("inner", "cross",
"outer", "full", "fullouter", "full_outer",
"left", "leftouter", "left_outer",
"right", "rightouter", "right_outer",
"semi", "left_semi", "leftsemi", "anti", "left_anti", "leftanti")) {
"semi", "leftsemi", "left_semi", "anti", "leftanti", "left_anti")
if (joinType %in% validJoinTypes) {
joinType <- gsub("_", "", joinType, fixed = TRUE)
sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType)
} else {
stop(paste("joinType must be one of the following types:",
"'inner', 'cross', 'outer', 'full', 'fullouter', 'full_outer',",
"'left', 'leftouter', 'left_outer', 'right', 'rightouter', 'right_outer',",
"'semi', 'leftsemi', 'left_semi', 'anti', 'leftanti' or 'left_anti'."))
stop("joinType must be one of the following types: ",
"'", paste(validJoinTypes, collapse = "', '"), "'")
}
}
}
Expand Down
2 changes: 1 addition & 1 deletion R/pkg/R/RDD.R
Expand Up @@ -947,7 +947,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical",
MAXINT <- .Machine$integer.max

if (num < 0)
stop(paste("Negative number of elements requested"))
stop("Negative number of elements requested")

if (initialCount > MAXINT - 1) {
maxSelected <- MAXINT - 1
Expand Down
17 changes: 8 additions & 9 deletions R/pkg/R/SQLContext.R
Expand Up @@ -34,7 +34,7 @@ getInternalType <- function(x) {
Date = "date",
POSIXlt = "timestamp",
POSIXct = "timestamp",
stop(paste("Unsupported type for SparkDataFrame:", class(x))))
stop("Unsupported type for SparkDataFrame: ", class(x)))
}

#' return the SparkSession
Expand Down Expand Up @@ -112,9 +112,9 @@ sparkR.conf <- function(key, defaultValue) {
error = function(e) {
estr <- as.character(e)
if (any(grepl("java.util.NoSuchElementException", estr, fixed = TRUE))) {
stop(paste0("Config '", key, "' is not set"))
stop("Config '", key, "' is not set")
} else {
stop(paste0("Unknown error: ", estr))
stop("Unknown error: ", estr)
}
})
} else {
Expand Down Expand Up @@ -208,7 +208,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) {
names <- lapply(names, function(n) {
nn <- gsub(".", "_", n, fixed = TRUE)
if (nn != n) {
warning(paste("Use", nn, "instead of", n, "as column name"))
warning("Use ", nn, " instead of ", n, " as column name")
}
nn
})
Expand Down Expand Up @@ -290,10 +290,9 @@ createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0,
TRUE
},
error = function(e) {
warning(paste0("createDataFrame attempted Arrow optimization because ",
"'spark.sql.execution.arrow.sparkr.enabled' is set to true; however, ",
"failed, attempting non-optimization. Reason: ",
e))
warning("createDataFrame attempted Arrow optimization because ",
"'spark.sql.execution.arrow.sparkr.enabled' is set to true; however, ",
"failed, attempting non-optimization. Reason: ", e)
FALSE
})
}
Expand Down Expand Up @@ -326,7 +325,7 @@ createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0,
} else if (inherits(data, "RDD")) {
rdd <- data
} else {
stop(paste("unexpected type:", class(data)))
stop("unexpected type: ", class(data))
}

schema <- getSchema(schema, firstRow, rdd)
Expand Down
7 changes: 3 additions & 4 deletions R/pkg/R/client.R
Expand Up @@ -102,10 +102,9 @@ checkJavaVersion <- function() {
javaVersionNum <- as.integer(versions[1])
}
if (javaVersionNum < minJavaVersion || javaVersionNum >= maxJavaVersion) {
stop(paste0("Java version, greater than or equal to ", minJavaVersion,
" and less than ", maxJavaVersion,
", is required for this package; found version: ",
javaVersionStr))
stop("Java version, greater than or equal to ", minJavaVersion,
" and less than ", maxJavaVersion, ", is required for this ",
"package; found version: ", javaVersionStr)
}
return(javaVersionNum)
}
Expand Down
8 changes: 4 additions & 4 deletions R/pkg/R/context.R
Expand Up @@ -144,13 +144,13 @@ parallelize <- function(sc, coll, numSlices = 1) {
if ((!is.list(coll) && !is.vector(coll)) || is.data.frame(coll)) {
# nolint end
if (is.data.frame(coll)) {
message(paste("context.R: A data frame is parallelized by columns."))
message("context.R: A data frame is parallelized by columns.")
} else {
if (is.matrix(coll)) {
message(paste("context.R: A matrix is parallelized by elements."))
message("context.R: A matrix is parallelized by elements.")
} else {
message(paste("context.R: parallelize() currently only supports lists and vectors.",
"Calling as.list() to coerce coll into a list."))
message("context.R: parallelize() currently only supports lists and vectors. ",
"Calling as.list() to coerce coll into a list.")
}
}
coll <- as.list(coll)
Expand Down
2 changes: 1 addition & 1 deletion R/pkg/R/deserialize.R
Expand Up @@ -57,7 +57,7 @@ readTypedObject <- function(con, type) {
"s" = readStruct(con),
"n" = NULL,
"j" = getJobj(readString(con)),
stop(paste("Unsupported type for deserialization", type)))
stop("Unsupported type for deserialization ", type))
}

readStringData <- function(con, len) {
Expand Down
4 changes: 2 additions & 2 deletions R/pkg/R/group.R
Expand Up @@ -234,8 +234,8 @@ gapplyInternal <- function(x, func, schema) {
if (inherits(schema, "structType")) {
checkSchemaInArrow(schema)
} else if (is.null(schema)) {
stop(paste0("Arrow optimization does not support 'gapplyCollect' yet. Please disable ",
"Arrow optimization or use 'collect' and 'gapply' APIs instead."))
stop("Arrow optimization does not support 'gapplyCollect' yet. Please disable ",
"Arrow optimization or use 'collect' and 'gapply' APIs instead.")
} else {
stop("'schema' should be DDL-formatted string or structType.")
}
Expand Down
62 changes: 26 additions & 36 deletions R/pkg/R/install.R
Expand Up @@ -89,8 +89,8 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL,
}

if (overwrite) {
message(paste0("Overwrite = TRUE: download and overwrite the tar file",
"and Spark package directory if they exist."))
message("Overwrite = TRUE: download and overwrite the tar file",
"and Spark package directory if they exist.")
}

releaseUrl <- Sys.getenv("SPARKR_RELEASE_DOWNLOAD_URL")
Expand All @@ -103,12 +103,11 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL,
# can use dir.exists(packageLocalDir) under R 3.2.0 or later
if (!is.na(file.info(packageLocalDir)$isdir) && !overwrite) {
if (releaseUrl != "") {
message(paste(packageName, "found, setting SPARK_HOME to", packageLocalDir))
message(packageName, " found, setting SPARK_HOME to ", packageLocalDir)
} else {
fmt <- "%s for Hadoop %s found, setting SPARK_HOME to %s"
msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion),
packageLocalDir)
message(msg)
message(version, " for Hadoop ",
if (hadoopVersion == "without") "Free build" else hadoopVersion,
" found, setting SPARK_HOME to ", packageLocalDir)
}
Sys.setenv(SPARK_HOME = packageLocalDir)
return(invisible(packageLocalDir))
Expand All @@ -127,26 +126,23 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL,
success <- downloadUrl(releaseUrl, packageLocalPath)
if (!success) {
unlink(packageLocalPath)
stop(paste0("Fetch failed from ", releaseUrl))
stop("Fetch failed from ", releaseUrl)
}
} else {
robustDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath)
}
}

message(sprintf("Installing to %s", localDir))
message("Installing to ", localDir)
# There are two ways untar can fail - untar could stop() on errors like incomplete block on file
# or, tar command can return failure code
success <- tryCatch(untar(tarfile = packageLocalPath, exdir = localDir) == 0,
error = function(e) {
message(e)
message()
message(e, "\n")
FALSE
},
warning = function(w) {
# Treat warning as error, add an empty line with message()
message(w)
message()
message(w, "\n")
FALSE
})
if (!tarExists || overwrite || !success) {
Expand All @@ -160,7 +156,7 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL,
if (!success) stop("Extract archive failed.")
message("DONE.")
Sys.setenv(SPARK_HOME = packageLocalDir)
message(paste("SPARK_HOME set to", packageLocalDir))
message("SPARK_HOME set to ", packageLocalDir)
invisible(packageLocalDir)
}

Expand All @@ -173,7 +169,7 @@ robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, pa
if (success) {
return()
} else {
message(paste0("Unable to download from mirrorUrl: ", mirrorUrl))
message("Unable to download from mirrorUrl: ", mirrorUrl)
}
} else {
message("MirrorUrl not provided.")
Expand Down Expand Up @@ -201,11 +197,9 @@ robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, pa
# remove any partially downloaded file
unlink(packageLocalPath)
message("Unable to download from default mirror site: ", mirrorUrl)
msg <- sprintf(paste("Unable to download Spark %s for Hadoop %s.",
"Please check network connection, Hadoop version,",
"or provide other mirror sites."),
version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion))
stop(msg)
stop("Unable to download Spark ", version,
" for Hadoop ", if (hadoopVersion == "without") "Free build" else hadoopVersion,
". Please check network connection, Hadoop version, or provide other mirror sites.")
}
}

Expand All @@ -222,7 +216,7 @@ getPreferredMirror <- function(version, packageName) {
endPos <- matchInfo + attr(matchInfo, "match.length") - 2
mirrorPreferred <- base::substr(linePreferred, startPos, endPos)
mirrorPreferred <- paste0(mirrorPreferred, "spark")
message(sprintf("Preferred mirror site found: %s", mirrorPreferred))
message("Preferred mirror site found: ", mirrorPreferred)
} else {
mirrorPreferred <- NULL
}
Expand All @@ -231,24 +225,20 @@ getPreferredMirror <- function(version, packageName) {

directDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) {
packageRemotePath <- paste0(file.path(mirrorUrl, version, packageName), ".tgz")
fmt <- "Downloading %s for Hadoop %s from:\n- %s"
msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion),
packageRemotePath)
message(msg)
message("Downloading ", version, " for Hadoop ",
if (hadoopVersion == "without") "Free build" else hadoopVersion,
" from:\n- ", packageRemotePath)
downloadUrl(packageRemotePath, packageLocalPath)
}

downloadUrl <- function(remotePath, localPath) {
isFail <- tryCatch(download.file(remotePath, localPath),
error = function(e) {
message(e)
message()
message(e, "\n")
TRUE
},
warning = function(w) {
# Treat warning as error, add an empty line with message()
message(w)
message()
message(w, "\n")
TRUE
})
!isFail
Expand Down Expand Up @@ -279,9 +269,9 @@ sparkCachePath <- function() {
winAppPath <- Sys.getenv("USERPROFILE", unset = NA)
}
if (is.na(winAppPath)) {
stop(paste("%LOCALAPPDATA% and %USERPROFILE% not found.",
"Please define the environment variable",
"or restart and enter an installation path in localDir."))
stop("%LOCALAPPDATA% and %USERPROFILE% not found. ",
"Please define the environment variable ",
"or restart and enter an installation path in localDir.")
} else {
path <- file.path(winAppPath, "Apache", "Spark", "Cache")
}
Expand All @@ -293,7 +283,7 @@ sparkCachePath <- function() {
Sys.getenv("XDG_CACHE_HOME", file.path(Sys.getenv("HOME"), ".cache")), "spark")
}
} else {
stop(sprintf("Unknown OS: %s", .Platform$OS.type))
stop("Unknown OS: ", .Platform$OS.type)
}
normalizePath(path, mustWork = FALSE)
}
Expand Down Expand Up @@ -322,7 +312,7 @@ installInstruction <- function(mode) {
"If you need further help, ",
"contact the administrators of the cluster.")
} else {
stop(paste0("No instruction found for ", mode, " mode."))
stop("No instruction found for mode ", mode)
}
}

Expand Down
4 changes: 2 additions & 2 deletions R/pkg/R/mllib_classification.R
Expand Up @@ -337,8 +337,8 @@ setMethod("spark.logit", signature(data = "SparkDataFrame", formula = "formula")

if (!is.null(lowerBoundsOnCoefficients) && (row != nrow(upperBoundsOnCoefficients)
|| col != ncol(upperBoundsOnCoefficients))) {
stop(paste0("dimension of upperBoundsOnCoefficients ",
"is not the same as lowerBoundsOnCoefficients"))
stop("dimension of upperBoundsOnCoefficients ",
"is not the same as lowerBoundsOnCoefficients")
}

if (is.null(lowerBoundsOnCoefficients)) {
Expand Down

0 comments on commit 0c8146a

Please sign in to comment.