Skip to content

Commit a33c4ea

Browse files
committed
merge master & resolve conflicts
2 parents e0f4ce6 + 20adf9a commit a33c4ea

File tree

382 files changed

+12744
-2413
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

382 files changed

+12744
-2413
lines changed

R/pkg/R/DataFrame.R

Lines changed: 27 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -986,10 +986,10 @@ setMethod("unique",
986986
#' @param x A SparkDataFrame
987987
#' @param withReplacement Sampling with replacement or not
988988
#' @param fraction The (rough) sample target fraction
989-
#' @param seed Randomness seed value
989+
#' @param seed Randomness seed value. Default is a random seed.
990990
#'
991991
#' @family SparkDataFrame functions
992-
#' @aliases sample,SparkDataFrame,logical,numeric-method
992+
#' @aliases sample,SparkDataFrame-method
993993
#' @rdname sample
994994
#' @name sample
995995
#' @export
@@ -998,33 +998,47 @@ setMethod("unique",
998998
#' sparkR.session()
999999
#' path <- "path/to/file.json"
10001000
#' df <- read.json(path)
1001+
#' collect(sample(df, fraction = 0.5))
10011002
#' collect(sample(df, FALSE, 0.5))
1002-
#' collect(sample(df, TRUE, 0.5))
1003+
#' collect(sample(df, TRUE, 0.5, seed = 3))
10031004
#'}
10041005
#' @note sample since 1.4.0
10051006
setMethod("sample",
1006-
signature(x = "SparkDataFrame", withReplacement = "logical",
1007-
fraction = "numeric"),
1008-
function(x, withReplacement, fraction, seed) {
1009-
if (fraction < 0.0) stop(cat("Negative fraction value:", fraction))
1007+
signature(x = "SparkDataFrame"),
1008+
function(x, withReplacement = FALSE, fraction, seed) {
1009+
if (!is.numeric(fraction)) {
1010+
stop(paste("fraction must be numeric; however, got", class(fraction)))
1011+
}
1012+
if (!is.logical(withReplacement)) {
1013+
stop(paste("withReplacement must be logical; however, got", class(withReplacement)))
1014+
}
1015+
10101016
if (!missing(seed)) {
1017+
if (is.null(seed)) {
1018+
stop("seed must not be NULL or NA; however, got NULL")
1019+
}
1020+
if (is.na(seed)) {
1021+
stop("seed must not be NULL or NA; however, got NA")
1022+
}
1023+
10111024
# TODO : Figure out how to send integer as java.lang.Long to JVM so
10121025
# we can send seed as an argument through callJMethod
1013-
sdf <- callJMethod(x@sdf, "sample", withReplacement, fraction, as.integer(seed))
1026+
sdf <- handledCallJMethod(x@sdf, "sample", as.logical(withReplacement),
1027+
as.numeric(fraction), as.integer(seed))
10141028
} else {
1015-
sdf <- callJMethod(x@sdf, "sample", withReplacement, fraction)
1029+
sdf <- handledCallJMethod(x@sdf, "sample",
1030+
as.logical(withReplacement), as.numeric(fraction))
10161031
}
10171032
dataFrame(sdf)
10181033
})
10191034

10201035
#' @rdname sample
1021-
#' @aliases sample_frac,SparkDataFrame,logical,numeric-method
1036+
#' @aliases sample_frac,SparkDataFrame-method
10221037
#' @name sample_frac
10231038
#' @note sample_frac since 1.4.0
10241039
setMethod("sample_frac",
1025-
signature(x = "SparkDataFrame", withReplacement = "logical",
1026-
fraction = "numeric"),
1027-
function(x, withReplacement, fraction, seed) {
1040+
signature(x = "SparkDataFrame"),
1041+
function(x, withReplacement = FALSE, fraction, seed) {
10281042
sample(x, withReplacement, fraction, seed)
10291043
})
10301044

R/pkg/R/functions.R

Lines changed: 19 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -176,7 +176,8 @@ NULL
176176
#'
177177
#' @param x Column to compute on. Note the difference in the following methods:
178178
#' \itemize{
179-
#' \item \code{to_json}: it is the column containing the struct or array of the structs.
179+
#' \item \code{to_json}: it is the column containing the struct, array of the structs,
180+
#' the map or array of maps.
180181
#' \item \code{from_json}: it is the column containing the JSON string.
181182
#' }
182183
#' @param ... additional argument(s). In \code{to_json} and \code{from_json}, this contains
@@ -1700,8 +1701,9 @@ setMethod("to_date",
17001701
})
17011702

17021703
#' @details
1703-
#' \code{to_json}: Converts a column containing a \code{structType} or array of \code{structType}
1704-
#' into a Column of JSON string. Resolving the Column can fail if an unsupported type is encountered.
1704+
#' \code{to_json}: Converts a column containing a \code{structType}, array of \code{structType},
1705+
#' a \code{mapType} or array of \code{mapType} into a Column of JSON string.
1706+
#' Resolving the Column can fail if an unsupported type is encountered.
17051707
#'
17061708
#' @rdname column_collection_functions
17071709
#' @aliases to_json to_json,Column-method
@@ -1715,6 +1717,14 @@ setMethod("to_date",
17151717
#'
17161718
#' # Converts an array of structs into a JSON array
17171719
#' df2 <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people")
1720+
#' df2 <- mutate(df2, people_json = to_json(df2$people))
1721+
#'
1722+
#' # Converts a map into a JSON object
1723+
#' df2 <- sql("SELECT map('name', 'Bob')) as people")
1724+
#' df2 <- mutate(df2, people_json = to_json(df2$people))
1725+
#'
1726+
#' # Converts an array of maps into a JSON array
1727+
#' df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
17181728
#' df2 <- mutate(df2, people_json = to_json(df2$people))}
17191729
#' @note to_json since 2.2.0
17201730
setMethod("to_json", signature(x = "Column"),
@@ -2216,8 +2226,9 @@ setMethod("from_json", signature(x = "Column", schema = "characterOrstructType")
22162226
})
22172227

22182228
#' @details
2219-
#' \code{from_utc_timestamp}: Given a timestamp, which corresponds to a certain time of day in UTC,
2220-
#' returns another timestamp that corresponds to the same time of day in the given timezone.
2229+
#' \code{from_utc_timestamp}: Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a
2230+
#' time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1'
2231+
#' would yield '2017-07-14 03:40:00.0'.
22212232
#'
22222233
#' @rdname column_datetime_diff_functions
22232234
#'
@@ -2276,8 +2287,9 @@ setMethod("next_day", signature(y = "Column", x = "character"),
22762287
})
22772288

22782289
#' @details
2279-
#' \code{to_utc_timestamp}: Given a timestamp, which corresponds to a certain time of day
2280-
#' in the given timezone, returns another timestamp that corresponds to the same time of day in UTC.
2290+
#' \code{to_utc_timestamp}: Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a
2291+
#' time in the given time zone, and renders that time as a timestamp in UTC. For example, 'GMT+1'
2292+
#' would yield '2017-07-14 01:40:00.0'.
22812293
#'
22822294
#' @rdname column_datetime_diff_functions
22832295
#' @aliases to_utc_timestamp to_utc_timestamp,Column,character-method

R/pkg/R/generics.R

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -645,7 +645,7 @@ setGeneric("repartition", function(x, ...) { standardGeneric("repartition") })
645645
#' @rdname sample
646646
#' @export
647647
setGeneric("sample",
648-
function(x, withReplacement, fraction, seed) {
648+
function(x, withReplacement = FALSE, fraction, seed) {
649649
standardGeneric("sample")
650650
})
651651

@@ -656,7 +656,7 @@ setGeneric("rollup", function(x, ...) { standardGeneric("rollup") })
656656
#' @rdname sample
657657
#' @export
658658
setGeneric("sample_frac",
659-
function(x, withReplacement, fraction, seed) { standardGeneric("sample_frac") })
659+
function(x, withReplacement = FALSE, fraction, seed) { standardGeneric("sample_frac") })
660660

661661
#' @rdname sampleBy
662662
#' @export

R/pkg/tests/fulltests/test_sparkSQL.R

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1116,6 +1116,20 @@ test_that("sample on a DataFrame", {
11161116
sampled3 <- sample_frac(df, FALSE, 0.1, 0) # set seed for predictable result
11171117
expect_true(count(sampled3) < 3)
11181118

1119+
# Different arguments
1120+
df <- createDataFrame(as.list(seq(10)))
1121+
expect_equal(count(sample(df, fraction = 0.5, seed = 3)), 4)
1122+
expect_equal(count(sample(df, withReplacement = TRUE, fraction = 0.5, seed = 3)), 2)
1123+
expect_equal(count(sample(df, fraction = 1.0)), 10)
1124+
expect_equal(count(sample(df, fraction = 1L)), 10)
1125+
expect_equal(count(sample(df, FALSE, fraction = 1.0)), 10)
1126+
1127+
expect_error(sample(df, fraction = "a"), "fraction must be numeric")
1128+
expect_error(sample(df, "a", fraction = 0.1), "however, got character")
1129+
expect_error(sample(df, fraction = 1, seed = NA), "seed must not be NULL or NA; however, got NA")
1130+
expect_error(sample(df, fraction = -1.0),
1131+
"illegal argument - requirement failed: Sampling fraction \\(-1.0\\)")
1132+
11191133
# nolint start
11201134
# Test base::sample is working
11211135
#expect_equal(length(sample(1:12)), 12)
@@ -1491,6 +1505,14 @@ test_that("column functions", {
14911505
j <- collect(select(df, alias(to_json(df$people), "json")))
14921506
expect_equal(j[order(j$json), ][1], "[{\"name\":\"Bob\"},{\"name\":\"Alice\"}]")
14931507

1508+
df <- sql("SELECT map('name', 'Bob') as people")
1509+
j <- collect(select(df, alias(to_json(df$people), "json")))
1510+
expect_equal(j[order(j$json), ][1], "{\"name\":\"Bob\"}")
1511+
1512+
df <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
1513+
j <- collect(select(df, alias(to_json(df$people), "json")))
1514+
expect_equal(j[order(j$json), ][1], "[{\"name\":\"Bob\"},{\"name\":\"Alice\"}]")
1515+
14941516
df <- read.json(mapTypeJsonPath)
14951517
j <- collect(select(df, alias(to_json(df$info), "json")))
14961518
expect_equal(j[order(j$json), ][1], "{\"age\":16,\"height\":176.5}")

appveyor.yml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ only_commits:
3232
- sql/core/src/main/scala/org/apache/spark/sql/api/r/
3333
- core/src/main/scala/org/apache/spark/api/r/
3434
- mllib/src/main/scala/org/apache/spark/ml/r/
35+
- core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
3536

3637
cache:
3738
- C:\Users\appveyor\.m2

assembly/pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -187,7 +187,7 @@
187187
<plugin>
188188
<groupId>org.apache.maven.plugins</groupId>
189189
<artifactId>maven-assembly-plugin</artifactId>
190-
<version>3.0.0</version>
190+
<version>3.1.0</version>
191191
<executions>
192192
<execution>
193193
<id>dist</id>

bin/spark-class2.cmd

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ if "x%1"=="x" (
2929
)
3030

3131
rem Find Spark jars.
32-
if exist "%SPARK_HOME%\RELEASE" (
32+
if exist "%SPARK_HOME%\jars" (
3333
set SPARK_JARS_DIR="%SPARK_HOME%\jars"
3434
) else (
3535
set SPARK_JARS_DIR="%SPARK_HOME%\assembly\target\scala-%SPARK_SCALA_VERSION%\jars"

common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -249,7 +249,7 @@ Index parent() {
249249
* calculated only once, avoiding redundant work when multiple child indices of the
250250
* same parent index exist.
251251
*/
252-
byte[] childPrefix(Object value) throws Exception {
252+
byte[] childPrefix(Object value) {
253253
Preconditions.checkState(parent == null, "Not a parent index.");
254254
return buildKey(name, toParentKey(value));
255255
}
@@ -295,7 +295,7 @@ byte[] end(byte[] prefix) {
295295
}
296296

297297
/** The key for the end marker for entries with the given value. */
298-
byte[] end(byte[] prefix, Object value) throws Exception {
298+
byte[] end(byte[] prefix, Object value) {
299299
checkParent(prefix);
300300
return (parent != null) ? buildKey(false, prefix, name, toKey(value), END_MARKER)
301301
: buildKey(name, toKey(value), END_MARKER);
@@ -313,7 +313,7 @@ byte[] entityKey(byte[] prefix, Object entity) throws Exception {
313313
return entityKey;
314314
}
315315

316-
private void updateCount(WriteBatch batch, byte[] key, long delta) throws Exception {
316+
private void updateCount(WriteBatch batch, byte[] key, long delta) {
317317
long updated = getCount(key) + delta;
318318
if (updated > 0) {
319319
batch.put(key, db.serializer.serialize(updated));
@@ -431,7 +431,7 @@ void remove(
431431
addOrRemove(batch, entity, null, null, naturalKey, prefix);
432432
}
433433

434-
long getCount(byte[] key) throws Exception {
434+
long getCount(byte[] key) {
435435
byte[] data = db.db().get(key);
436436
return data != null ? db.serializer.deserializeLong(data) : 0;
437437
}

common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,7 @@ class AuthEngine implements Closeable {
8181
*
8282
* @return A challenge to be sent the remote side.
8383
*/
84-
ClientChallenge challenge() throws GeneralSecurityException, IOException {
84+
ClientChallenge challenge() throws GeneralSecurityException {
8585
this.authNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE);
8686
SecretKeySpec authKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(),
8787
authNonce, conf.encryptionKeyLength());
@@ -105,7 +105,7 @@ ClientChallenge challenge() throws GeneralSecurityException, IOException {
105105
* @return A response to be sent to the client.
106106
*/
107107
ServerResponse respond(ClientChallenge clientChallenge)
108-
throws GeneralSecurityException, IOException {
108+
throws GeneralSecurityException {
109109

110110
SecretKeySpec authKey = generateKey(clientChallenge.kdf, clientChallenge.iterations,
111111
clientChallenge.nonce, clientChallenge.keyLength);

common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717

1818
package org.apache.spark.network.sasl;
1919

20-
import java.io.IOException;
2120
import java.util.Map;
2221
import javax.security.auth.callback.Callback;
2322
import javax.security.auth.callback.CallbackHandler;
@@ -125,7 +124,7 @@ public synchronized void dispose() {
125124
*/
126125
private class ClientCallbackHandler implements CallbackHandler {
127126
@Override
128-
public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
127+
public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
129128

130129
for (Callback callback : callbacks) {
131130
if (callback instanceof NameCallback) {

0 commit comments

Comments
 (0)