Skip to content

Commit f53d8c6

Browse files
Michael ChiricoHyukjinKwon
Michael Chirico
authored andcommitted
[SPARK-31571][R] Overhaul stop/message/warning calls to be more canonical
### 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 #28365 from MichaelChirico/r-stop-paste. Authored-by: Michael Chirico <[email protected]> Signed-off-by: HyukjinKwon <[email protected]>
1 parent 13dddee commit f53d8c6

16 files changed

+103
-118
lines changed

R/pkg/R/DataFrame.R

Lines changed: 19 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -431,7 +431,7 @@ setMethod("coltypes",
431431
if (is.null(type)) {
432432
specialtype <- specialtypeshandle(x)
433433
if (is.null(specialtype)) {
434-
stop(paste("Unsupported data type: ", x))
434+
stop("Unsupported data type: ", x)
435435
}
436436
type <- PRIMITIVE_TYPES[[specialtype]]
437437
}
@@ -829,8 +829,8 @@ setMethod("repartitionByRange",
829829
jcol <- lapply(cols, function(c) { c@jc })
830830
sdf <- callJMethod(x@sdf, "repartitionByRange", numToInt(numPartitions), jcol)
831831
} else {
832-
stop(paste("numPartitions and col must be numeric and Column; however, got",
833-
class(numPartitions), "and", class(col)))
832+
stop("numPartitions and col must be numeric and Column; however, got ",
833+
class(numPartitions), " and ", class(col))
834834
}
835835
} else if (!is.null(col)) {
836836
# only columns are specified
@@ -839,7 +839,7 @@ setMethod("repartitionByRange",
839839
jcol <- lapply(cols, function(c) { c@jc })
840840
sdf <- callJMethod(x@sdf, "repartitionByRange", jcol)
841841
} else {
842-
stop(paste("col must be Column; however, got", class(col)))
842+
stop("col must be Column; however, got ", class(col))
843843
}
844844
} else if (!is.null(numPartitions)) {
845845
# only numPartitions is specified
@@ -1068,10 +1068,10 @@ setMethod("sample",
10681068
signature(x = "SparkDataFrame"),
10691069
function(x, withReplacement = FALSE, fraction, seed) {
10701070
if (!is.numeric(fraction)) {
1071-
stop(paste("fraction must be numeric; however, got", class(fraction)))
1071+
stop("fraction must be numeric; however, got ", class(fraction))
10721072
}
10731073
if (!is.logical(withReplacement)) {
1074-
stop(paste("withReplacement must be logical; however, got", class(withReplacement)))
1074+
stop("withReplacement must be logical; however, got ", class(withReplacement))
10751075
}
10761076

10771077
if (!missing(seed)) {
@@ -1211,11 +1211,10 @@ setMethod("collect",
12111211
checkSchemaInArrow(schema(x))
12121212
TRUE
12131213
}, error = function(e) {
1214-
warning(paste0("The conversion from Spark DataFrame to R DataFrame was attempted ",
1215-
"with Arrow optimization because ",
1216-
"'spark.sql.execution.arrow.sparkr.enabled' is set to true; ",
1217-
"however, failed, attempting non-optimization. Reason: ",
1218-
e))
1214+
warning("The conversion from Spark DataFrame to R DataFrame was attempted ",
1215+
"with Arrow optimization because ",
1216+
"'spark.sql.execution.arrow.sparkr.enabled' is set to true; ",
1217+
"however, failed, attempting non-optimization. Reason: ", e)
12191218
FALSE
12201219
})
12211220
}
@@ -1508,8 +1507,8 @@ dapplyInternal <- function(x, func, schema) {
15081507
if (inherits(schema, "structType")) {
15091508
checkSchemaInArrow(schema)
15101509
} else if (is.null(schema)) {
1511-
stop(paste0("Arrow optimization does not support 'dapplyCollect' yet. Please disable ",
1512-
"Arrow optimization or use 'collect' and 'dapply' APIs instead."))
1510+
stop("Arrow optimization does not support 'dapplyCollect' yet. Please disable ",
1511+
"Arrow optimization or use 'collect' and 'dapply' APIs instead.")
15131512
} else {
15141513
stop("'schema' should be DDL-formatted string or structType.")
15151514
}
@@ -2012,8 +2011,8 @@ setMethod("[", signature(x = "SparkDataFrame"),
20122011
x
20132012
} else {
20142013
if (class(i) != "Column") {
2015-
stop(paste0("Expressions other than filtering predicates are not supported ",
2016-
"in the first parameter of extract operator [ or subset() method."))
2014+
stop("Expressions other than filtering predicates are not supported ",
2015+
"in the first parameter of extract operator [ or subset() method.")
20172016
}
20182017
filter(x, i)
20192018
}
@@ -2604,18 +2603,17 @@ setMethod("join",
26042603
if (is.null(joinType)) {
26052604
sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc)
26062605
} else {
2607-
if (joinType %in% c("inner", "cross",
2606+
validJoinTypes <- c("inner", "cross",
26082607
"outer", "full", "fullouter", "full_outer",
26092608
"left", "leftouter", "left_outer",
26102609
"right", "rightouter", "right_outer",
2611-
"semi", "left_semi", "leftsemi", "anti", "left_anti", "leftanti")) {
2610+
"semi", "leftsemi", "left_semi", "anti", "leftanti", "left_anti")
2611+
if (joinType %in% validJoinTypes) {
26122612
joinType <- gsub("_", "", joinType, fixed = TRUE)
26132613
sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType)
26142614
} else {
2615-
stop(paste("joinType must be one of the following types:",
2616-
"'inner', 'cross', 'outer', 'full', 'fullouter', 'full_outer',",
2617-
"'left', 'leftouter', 'left_outer', 'right', 'rightouter', 'right_outer',",
2618-
"'semi', 'leftsemi', 'left_semi', 'anti', 'leftanti' or 'left_anti'."))
2615+
stop("joinType must be one of the following types: ",
2616+
"'", paste(validJoinTypes, collapse = "', '"), "'")
26192617
}
26202618
}
26212619
}

R/pkg/R/RDD.R

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -947,7 +947,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical",
947947
MAXINT <- .Machine$integer.max
948948

949949
if (num < 0)
950-
stop(paste("Negative number of elements requested"))
950+
stop("Negative number of elements requested")
951951

952952
if (initialCount > MAXINT - 1) {
953953
maxSelected <- MAXINT - 1

R/pkg/R/SQLContext.R

Lines changed: 8 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ getInternalType <- function(x) {
3434
Date = "date",
3535
POSIXlt = "timestamp",
3636
POSIXct = "timestamp",
37-
stop(paste("Unsupported type for SparkDataFrame:", class(x))))
37+
stop("Unsupported type for SparkDataFrame: ", class(x)))
3838
}
3939

4040
#' return the SparkSession
@@ -112,9 +112,9 @@ sparkR.conf <- function(key, defaultValue) {
112112
error = function(e) {
113113
estr <- as.character(e)
114114
if (any(grepl("java.util.NoSuchElementException", estr, fixed = TRUE))) {
115-
stop(paste0("Config '", key, "' is not set"))
115+
stop("Config '", key, "' is not set")
116116
} else {
117-
stop(paste0("Unknown error: ", estr))
117+
stop("Unknown error: ", estr)
118118
}
119119
})
120120
} else {
@@ -208,7 +208,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) {
208208
names <- lapply(names, function(n) {
209209
nn <- gsub(".", "_", n, fixed = TRUE)
210210
if (nn != n) {
211-
warning(paste("Use", nn, "instead of", n, "as column name"))
211+
warning("Use ", nn, " instead of ", n, " as column name")
212212
}
213213
nn
214214
})
@@ -290,10 +290,9 @@ createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0,
290290
TRUE
291291
},
292292
error = function(e) {
293-
warning(paste0("createDataFrame attempted Arrow optimization because ",
294-
"'spark.sql.execution.arrow.sparkr.enabled' is set to true; however, ",
295-
"failed, attempting non-optimization. Reason: ",
296-
e))
293+
warning("createDataFrame attempted Arrow optimization because ",
294+
"'spark.sql.execution.arrow.sparkr.enabled' is set to true; however, ",
295+
"failed, attempting non-optimization. Reason: ", e)
297296
FALSE
298297
})
299298
}
@@ -326,7 +325,7 @@ createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0,
326325
} else if (inherits(data, "RDD")) {
327326
rdd <- data
328327
} else {
329-
stop(paste("unexpected type:", class(data)))
328+
stop("unexpected type: ", class(data))
330329
}
331330

332331
schema <- getSchema(schema, firstRow, rdd)

R/pkg/R/client.R

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -102,10 +102,9 @@ checkJavaVersion <- function() {
102102
javaVersionNum <- as.integer(versions[1])
103103
}
104104
if (javaVersionNum < minJavaVersion || javaVersionNum >= maxJavaVersion) {
105-
stop(paste0("Java version, greater than or equal to ", minJavaVersion,
106-
" and less than ", maxJavaVersion,
107-
", is required for this package; found version: ",
108-
javaVersionStr))
105+
stop("Java version, greater than or equal to ", minJavaVersion,
106+
" and less than ", maxJavaVersion, ", is required for this ",
107+
"package; found version: ", javaVersionStr)
109108
}
110109
return(javaVersionNum)
111110
}

R/pkg/R/context.R

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -144,13 +144,13 @@ parallelize <- function(sc, coll, numSlices = 1) {
144144
if ((!is.list(coll) && !is.vector(coll)) || is.data.frame(coll)) {
145145
# nolint end
146146
if (is.data.frame(coll)) {
147-
message(paste("context.R: A data frame is parallelized by columns."))
147+
message("context.R: A data frame is parallelized by columns.")
148148
} else {
149149
if (is.matrix(coll)) {
150-
message(paste("context.R: A matrix is parallelized by elements."))
150+
message("context.R: A matrix is parallelized by elements.")
151151
} else {
152-
message(paste("context.R: parallelize() currently only supports lists and vectors.",
153-
"Calling as.list() to coerce coll into a list."))
152+
message("context.R: parallelize() currently only supports lists and vectors. ",
153+
"Calling as.list() to coerce coll into a list.")
154154
}
155155
}
156156
coll <- as.list(coll)

R/pkg/R/deserialize.R

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -57,7 +57,7 @@ readTypedObject <- function(con, type) {
5757
"s" = readStruct(con),
5858
"n" = NULL,
5959
"j" = getJobj(readString(con)),
60-
stop(paste("Unsupported type for deserialization", type)))
60+
stop("Unsupported type for deserialization ", type))
6161
}
6262

6363
readStringData <- function(con, len) {

R/pkg/R/group.R

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -234,8 +234,8 @@ gapplyInternal <- function(x, func, schema) {
234234
if (inherits(schema, "structType")) {
235235
checkSchemaInArrow(schema)
236236
} else if (is.null(schema)) {
237-
stop(paste0("Arrow optimization does not support 'gapplyCollect' yet. Please disable ",
238-
"Arrow optimization or use 'collect' and 'gapply' APIs instead."))
237+
stop("Arrow optimization does not support 'gapplyCollect' yet. Please disable ",
238+
"Arrow optimization or use 'collect' and 'gapply' APIs instead.")
239239
} else {
240240
stop("'schema' should be DDL-formatted string or structType.")
241241
}

R/pkg/R/install.R

Lines changed: 26 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -89,8 +89,8 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL,
8989
}
9090

9191
if (overwrite) {
92-
message(paste0("Overwrite = TRUE: download and overwrite the tar file",
93-
"and Spark package directory if they exist."))
92+
message("Overwrite = TRUE: download and overwrite the tar file",
93+
"and Spark package directory if they exist.")
9494
}
9595

9696
releaseUrl <- Sys.getenv("SPARKR_RELEASE_DOWNLOAD_URL")
@@ -103,12 +103,11 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL,
103103
# can use dir.exists(packageLocalDir) under R 3.2.0 or later
104104
if (!is.na(file.info(packageLocalDir)$isdir) && !overwrite) {
105105
if (releaseUrl != "") {
106-
message(paste(packageName, "found, setting SPARK_HOME to", packageLocalDir))
106+
message(packageName, " found, setting SPARK_HOME to ", packageLocalDir)
107107
} else {
108-
fmt <- "%s for Hadoop %s found, setting SPARK_HOME to %s"
109-
msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion),
110-
packageLocalDir)
111-
message(msg)
108+
message(version, " for Hadoop ",
109+
if (hadoopVersion == "without") "Free build" else hadoopVersion,
110+
" found, setting SPARK_HOME to ", packageLocalDir)
112111
}
113112
Sys.setenv(SPARK_HOME = packageLocalDir)
114113
return(invisible(packageLocalDir))
@@ -127,26 +126,23 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL,
127126
success <- downloadUrl(releaseUrl, packageLocalPath)
128127
if (!success) {
129128
unlink(packageLocalPath)
130-
stop(paste0("Fetch failed from ", releaseUrl))
129+
stop("Fetch failed from ", releaseUrl)
131130
}
132131
} else {
133132
robustDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath)
134133
}
135134
}
136135

137-
message(sprintf("Installing to %s", localDir))
136+
message("Installing to ", localDir)
138137
# There are two ways untar can fail - untar could stop() on errors like incomplete block on file
139138
# or, tar command can return failure code
140139
success <- tryCatch(untar(tarfile = packageLocalPath, exdir = localDir) == 0,
141140
error = function(e) {
142-
message(e)
143-
message()
141+
message(e, "\n")
144142
FALSE
145143
},
146144
warning = function(w) {
147-
# Treat warning as error, add an empty line with message()
148-
message(w)
149-
message()
145+
message(w, "\n")
150146
FALSE
151147
})
152148
if (!tarExists || overwrite || !success) {
@@ -160,7 +156,7 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL,
160156
if (!success) stop("Extract archive failed.")
161157
message("DONE.")
162158
Sys.setenv(SPARK_HOME = packageLocalDir)
163-
message(paste("SPARK_HOME set to", packageLocalDir))
159+
message("SPARK_HOME set to ", packageLocalDir)
164160
invisible(packageLocalDir)
165161
}
166162

@@ -173,7 +169,7 @@ robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, pa
173169
if (success) {
174170
return()
175171
} else {
176-
message(paste0("Unable to download from mirrorUrl: ", mirrorUrl))
172+
message("Unable to download from mirrorUrl: ", mirrorUrl)
177173
}
178174
} else {
179175
message("MirrorUrl not provided.")
@@ -201,11 +197,9 @@ robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, pa
201197
# remove any partially downloaded file
202198
unlink(packageLocalPath)
203199
message("Unable to download from default mirror site: ", mirrorUrl)
204-
msg <- sprintf(paste("Unable to download Spark %s for Hadoop %s.",
205-
"Please check network connection, Hadoop version,",
206-
"or provide other mirror sites."),
207-
version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion))
208-
stop(msg)
200+
stop("Unable to download Spark ", version,
201+
" for Hadoop ", if (hadoopVersion == "without") "Free build" else hadoopVersion,
202+
". Please check network connection, Hadoop version, or provide other mirror sites.")
209203
}
210204
}
211205

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

232226
directDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) {
233227
packageRemotePath <- paste0(file.path(mirrorUrl, version, packageName), ".tgz")
234-
fmt <- "Downloading %s for Hadoop %s from:\n- %s"
235-
msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion),
236-
packageRemotePath)
237-
message(msg)
228+
message("Downloading ", version, " for Hadoop ",
229+
if (hadoopVersion == "without") "Free build" else hadoopVersion,
230+
" from:\n- ", packageRemotePath)
238231
downloadUrl(packageRemotePath, packageLocalPath)
239232
}
240233

241234
downloadUrl <- function(remotePath, localPath) {
242235
isFail <- tryCatch(download.file(remotePath, localPath),
243236
error = function(e) {
244-
message(e)
245-
message()
237+
message(e, "\n")
246238
TRUE
247239
},
248240
warning = function(w) {
249-
# Treat warning as error, add an empty line with message()
250-
message(w)
251-
message()
241+
message(w, "\n")
252242
TRUE
253243
})
254244
!isFail
@@ -279,9 +269,9 @@ sparkCachePath <- function() {
279269
winAppPath <- Sys.getenv("USERPROFILE", unset = NA)
280270
}
281271
if (is.na(winAppPath)) {
282-
stop(paste("%LOCALAPPDATA% and %USERPROFILE% not found.",
283-
"Please define the environment variable",
284-
"or restart and enter an installation path in localDir."))
272+
stop("%LOCALAPPDATA% and %USERPROFILE% not found. ",
273+
"Please define the environment variable ",
274+
"or restart and enter an installation path in localDir.")
285275
} else {
286276
path <- file.path(winAppPath, "Apache", "Spark", "Cache")
287277
}
@@ -293,7 +283,7 @@ sparkCachePath <- function() {
293283
Sys.getenv("XDG_CACHE_HOME", file.path(Sys.getenv("HOME"), ".cache")), "spark")
294284
}
295285
} else {
296-
stop(sprintf("Unknown OS: %s", .Platform$OS.type))
286+
stop("Unknown OS: ", .Platform$OS.type)
297287
}
298288
normalizePath(path, mustWork = FALSE)
299289
}
@@ -322,7 +312,7 @@ installInstruction <- function(mode) {
322312
"If you need further help, ",
323313
"contact the administrators of the cluster.")
324314
} else {
325-
stop(paste0("No instruction found for ", mode, " mode."))
315+
stop("No instruction found for mode ", mode)
326316
}
327317
}
328318

R/pkg/R/mllib_classification.R

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -337,8 +337,8 @@ setMethod("spark.logit", signature(data = "SparkDataFrame", formula = "formula")
337337

338338
if (!is.null(lowerBoundsOnCoefficients) && (row != nrow(upperBoundsOnCoefficients)
339339
|| col != ncol(upperBoundsOnCoefficients))) {
340-
stop(paste0("dimension of upperBoundsOnCoefficients ",
341-
"is not the same as lowerBoundsOnCoefficients"))
340+
stop("dimension of upperBoundsOnCoefficients ",
341+
"is not the same as lowerBoundsOnCoefficients")
342342
}
343343

344344
if (is.null(lowerBoundsOnCoefficients)) {

0 commit comments

Comments
 (0)