From 85a871818ee1134deb29387c78c6ce21eb6d2acb Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Mon, 28 Dec 2015 21:28:32 -0800 Subject: [PATCH 01/33] [SPARK-11394][SQL] Throw IllegalArgumentException for unsupported types in postgresql If DataFrame has BYTE types, throws an exception: org.postgresql.util.PSQLException: ERROR: type "byte" does not exist Author: Takeshi YAMAMURO Closes #9350 from maropu/FixBugInPostgreJdbc. (cherry picked from commit 73862a1eb9744c3c32458c9c6f6431c23783786a) Signed-off-by: Yin Huai --- .../scala/org/apache/spark/sql/jdbc/PostgresDialect.scala | 1 + .../src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index 3cf80f576e92c..ad9e31690b2d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -64,6 +64,7 @@ private object PostgresDialect extends JdbcDialect { getJDBCType(et).map(_.databaseTypeDefinition) .orElse(JdbcUtils.getCommonJDBCType(et).map(_.databaseTypeDefinition)) .map(typeName => JdbcType(s"$typeName[]", java.sql.Types.ARRAY)) + case ByteType => throw new IllegalArgumentException(s"Unsupported type in postgresql: $dt"); case _ => None } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 8c24aa3151bc1..26c9ad079cf60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -460,6 +460,10 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") assert(Postgres.getCatalystType(java.sql.Types.OTHER, "json", 1, null) === Some(StringType)) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "jsonb", 1, null) === Some(StringType)) + val errMsg = intercept[IllegalArgumentException] { + Postgres.getJDBCType(ByteType) + } + assert(errMsg.getMessage contains "Unsupported type in postgresql: ByteType") } test("DerbyDialect jdbc type mapping") { From c069ffc2b13879f471e6d888116f45f6a8902236 Mon Sep 17 00:00:00 2001 From: Forest Fang Date: Tue, 29 Dec 2015 12:45:24 +0530 Subject: [PATCH 02/33] [SPARK-12526][SPARKR] ifelse`, `when`, `otherwise` unable to take Column as value `ifelse`, `when`, `otherwise` is unable to take `Column` typed S4 object as values. For example: ```r ifelse(lit(1) == lit(1), lit(2), lit(3)) ifelse(df$mpg > 0, df$mpg, 0) ``` will both fail with ```r attempt to replicate an object of type 'environment' ``` The PR replaces `ifelse` calls with `if ... else ...` inside the function implementations to avoid attempt to vectorize(i.e. `rep()`). It remains to be discussed whether we should instead support vectorization in these functions for consistency because `ifelse` in base R is vectorized but I cannot foresee any scenarios these functions will want to be vectorized in SparkR. For reference, added test cases which trigger failures: ```r . Error: when(), otherwise() and ifelse() with column on a DataFrame ---------- error in evaluating the argument 'x' in selecting a method for function 'collect': error in evaluating the argument 'col' in selecting a method for function 'select': attempt to replicate an object of type 'environment' Calls: when -> when -> ifelse -> ifelse 1: withCallingHandlers(eval(code, new_test_environment), error = capture_calls, message = function(c) invokeRestart("muffleMessage")) 2: eval(code, new_test_environment) 3: eval(expr, envir, enclos) 4: expect_equal(collect(select(df, when(df$a > 1 & df$b > 2, lit(1))))[, 1], c(NA, 1)) at test_sparkSQL.R:1126 5: expect_that(object, equals(expected, label = expected.label, ...), info = info, label = label) 6: condition(object) 7: compare(actual, expected, ...) 8: collect(select(df, when(df$a > 1 & df$b > 2, lit(1)))) Error: Test failures Execution halted ``` Author: Forest Fang Closes #10481 from saurfang/spark-12526. (cherry picked from commit d80cc90b5545cff82cd9b340f12d01eafc9ca524) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/column.R | 4 ++-- R/pkg/R/functions.R | 13 ++++++++----- R/pkg/inst/tests/testthat/test_sparkSQL.R | 8 ++++++++ 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index 7bb8ef2595b59..356bcee3cf5c6 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -215,7 +215,7 @@ setMethod("%in%", #' otherwise #' -#' If values in the specified column are null, returns the value. +#' If values in the specified column are null, returns the value. #' Can be used in conjunction with `when` to specify a default value for expressions. #' #' @rdname otherwise @@ -225,7 +225,7 @@ setMethod("%in%", setMethod("otherwise", signature(x = "Column", value = "ANY"), function(x, value) { - value <- ifelse(class(value) == "Column", value@jc, value) + value <- if (class(value) == "Column") { value@jc } else { value } jc <- callJMethod(x@jc, "otherwise", value) column(jc) }) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 09e4e04335a33..df36bc869acb4 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -37,7 +37,7 @@ setMethod("lit", signature("ANY"), function(x) { jc <- callJStatic("org.apache.spark.sql.functions", "lit", - ifelse(class(x) == "Column", x@jc, x)) + if (class(x) == "Column") { x@jc } else { x }) column(jc) }) @@ -2262,7 +2262,7 @@ setMethod("unix_timestamp", signature(x = "Column", format = "character"), setMethod("when", signature(condition = "Column", value = "ANY"), function(condition, value) { condition <- condition@jc - value <- ifelse(class(value) == "Column", value@jc, value) + value <- if (class(value) == "Column") { value@jc } else { value } jc <- callJStatic("org.apache.spark.sql.functions", "when", condition, value) column(jc) }) @@ -2277,13 +2277,16 @@ setMethod("when", signature(condition = "Column", value = "ANY"), #' @name ifelse #' @seealso \link{when} #' @export -#' @examples \dontrun{ifelse(df$a > 1 & df$b > 2, 0, 1)} +#' @examples \dontrun{ +#' ifelse(df$a > 1 & df$b > 2, 0, 1) +#' ifelse(df$a > 1, df$a, 1) +#' } setMethod("ifelse", signature(test = "Column", yes = "ANY", no = "ANY"), function(test, yes, no) { test <- test@jc - yes <- ifelse(class(yes) == "Column", yes@jc, yes) - no <- ifelse(class(no) == "Column", no@jc, no) + yes <- if (class(yes) == "Column") { yes@jc } else { yes } + no <- if (class(no) == "Column") { no@jc } else { no } jc <- callJMethod(callJStatic("org.apache.spark.sql.functions", "when", test, yes), diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 135c7576e5291..c2b6adbe3ae01 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1120,6 +1120,14 @@ test_that("when(), otherwise() and ifelse() on a DataFrame", { expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, 0, 1)))[, 1], c(1, 0)) }) +test_that("when(), otherwise() and ifelse() with column on a DataFrame", { + l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) + df <- createDataFrame(sqlContext, l) + expect_equal(collect(select(df, when(df$a > 1 & df$b > 2, lit(1))))[, 1], c(NA, 1)) + expect_equal(collect(select(df, otherwise(when(df$a > 1, lit(1)), lit(0))))[, 1], c(0, 1)) + expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, lit(0), lit(1))))[, 1], c(1, 0)) +}) + test_that("group by, agg functions", { df <- read.json(sqlContext, jsonPath) df1 <- agg(df, name = "max", age = "sum") From 8dc65497152f2c8949b08fddad853d31c4bd9ae5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 30 Dec 2015 11:14:47 -0800 Subject: [PATCH 03/33] [SPARK-12300] [SQL] [PYSPARK] fix schema inferance on local collections Current schema inference for local python collections halts as soon as there are no NullTypes. This is different than when we specify a sampling ratio of 1.0 on a distributed collection. This could result in incomplete schema information. Author: Holden Karau Closes #10275 from holdenk/SPARK-12300-fix-schmea-inferance-on-local-collections. (cherry picked from commit d1ca634db4ca9db7f0ba7ca38a0e03bcbfec23c9) Signed-off-by: Davies Liu --- python/pyspark/sql/context.py | 10 +++------- python/pyspark/sql/tests.py | 11 +++++++++++ 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index b05aa2f5c4cd7..ba6915a12347e 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -18,6 +18,7 @@ import sys import warnings import json +from functools import reduce if sys.version >= '3': basestring = unicode = str @@ -236,14 +237,9 @@ def _inferSchemaFromList(self, data): if type(first) is dict: warnings.warn("inferring schema from dict is deprecated," "please use pyspark.sql.Row instead") - schema = _infer_schema(first) + schema = reduce(_merge_type, map(_infer_schema, data)) if _has_nulltype(schema): - for r in data: - schema = _merge_type(schema, _infer_schema(r)) - if not _has_nulltype(schema): - break - else: - raise ValueError("Some of types cannot be determined after inferring") + raise ValueError("Some of types cannot be determined after inferring") return schema def _inferSchema(self, rdd, samplingRatio=None): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 9f5f7cfdf7a69..10b99175ad952 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -353,6 +353,17 @@ def test_apply_schema_to_row(self): df3 = self.sqlCtx.createDataFrame(rdd, df.schema) self.assertEqual(10, df3.count()) + def test_infer_schema_to_local(self): + input = [{"a": 1}, {"b": "coffee"}] + rdd = self.sc.parallelize(input) + df = self.sqlCtx.createDataFrame(input) + df2 = self.sqlCtx.createDataFrame(rdd, samplingRatio=1.0) + self.assertEqual(df.schema, df2.schema) + + rdd = self.sc.parallelize(range(10)).map(lambda x: Row(a=x)) + df3 = self.sqlCtx.createDataFrame(rdd, df.schema) + self.assertEqual(10, df3.count()) + def test_serialize_nested_array_and_map(self): d = [Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")})] rdd = self.sc.parallelize(d) From cd86075b52d6363f674dffc3eb71d90449563879 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Wed, 30 Dec 2015 13:49:10 -0800 Subject: [PATCH 04/33] [SPARK-12399] Display correct error message when accessing REST API with an unknown app Id I got an exception when accessing the below REST API with an unknown application Id. `http://:18080/api/v1/applications/xxx/jobs` Instead of an exception, I expect an error message "no such app: xxx" which is a similar error message when I access `/api/v1/applications/xxx` ``` org.spark-project.guava.util.concurrent.UncheckedExecutionException: java.util.NoSuchElementException: no app with key xxx at org.spark-project.guava.cache.LocalCache$Segment.get(LocalCache.java:2263) at org.spark-project.guava.cache.LocalCache.get(LocalCache.java:4000) at org.spark-project.guava.cache.LocalCache.getOrLoad(LocalCache.java:4004) at org.spark-project.guava.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) at org.apache.spark.deploy.history.HistoryServer.getSparkUI(HistoryServer.scala:116) at org.apache.spark.status.api.v1.UIRoot$class.withSparkUI(ApiRootResource.scala:226) at org.apache.spark.deploy.history.HistoryServer.withSparkUI(HistoryServer.scala:46) at org.apache.spark.status.api.v1.ApiRootResource.getJobs(ApiRootResource.scala:66) ``` Author: Carson Wang Closes #10352 from carsonwang/unknownAppFix. (cherry picked from commit b244297966be1d09f8e861cfe2d8e69f7bed84da) Signed-off-by: Marcelo Vanzin --- .../spark/deploy/history/HistoryServer.scala | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index f31fef0eccc3b..d11c9b7d18f14 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -21,6 +21,8 @@ import java.util.NoSuchElementException import java.util.zip.ZipOutputStream import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} +import scala.util.control.NonFatal + import com.google.common.cache._ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} @@ -115,7 +117,17 @@ class HistoryServer( } def getSparkUI(appKey: String): Option[SparkUI] = { - Option(appCache.get(appKey)) + try { + val ui = appCache.get(appKey) + Some(ui) + } catch { + case NonFatal(e) => e.getCause() match { + case nsee: NoSuchElementException => + None + + case cause: Exception => throw cause + } + } } initialize() @@ -195,7 +207,7 @@ class HistoryServer( appCache.get(appId + attemptId.map { id => s"/$id" }.getOrElse("")) true } catch { - case e: Exception => e.getCause() match { + case NonFatal(e) => e.getCause() match { case nsee: NoSuchElementException => false From 4e9dd16987b3cba19dcf6437f3b6c8aeb59e2e39 Mon Sep 17 00:00:00 2001 From: felixcheung Date: Sun, 3 Jan 2016 20:53:35 +0530 Subject: [PATCH 05/33] [SPARK-12327][SPARKR] fix code for lintr warning for commented code shivaram Author: felixcheung Closes #10408 from felixcheung/rcodecomment. (cherry picked from commit c3d505602de2fd2361633f90e4fff7e041849e28) Signed-off-by: Shivaram Venkataraman --- R/pkg/.lintr | 2 +- R/pkg/R/RDD.R | 40 +++++++++++++++++++++-- R/pkg/R/deserialize.R | 3 ++ R/pkg/R/pairRDD.R | 30 +++++++++++++++++ R/pkg/R/serialize.R | 2 ++ R/pkg/inst/tests/testthat/test_rdd.R | 4 +-- R/pkg/inst/tests/testthat/test_shuffle.R | 4 +-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 12 ++++--- R/pkg/inst/tests/testthat/test_utils.R | 2 ++ 9 files changed, 88 insertions(+), 11 deletions(-) diff --git a/R/pkg/.lintr b/R/pkg/.lintr index 39c872663ad44..038236fc149e6 100644 --- a/R/pkg/.lintr +++ b/R/pkg/.lintr @@ -1,2 +1,2 @@ -linters: with_defaults(line_length_linter(100), camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE), commented_code_linter = NULL) +linters: with_defaults(line_length_linter(100), camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE)) exclusions: list("inst/profile/general.R" = 1, "inst/profile/shell.R") diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 00c40c38cabc9..a78fbb714f2be 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -180,7 +180,7 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), } # Save the serialization flag after we create a RRDD rdd@env$serializedMode <- serializedMode - rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() + rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") rdd@env$jrdd_val }) @@ -225,7 +225,7 @@ setMethod("cache", #' #' Persist this RDD with the specified storage level. For details of the #' supported storage levels, refer to -#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#'\url{http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence}. #' #' @param x The RDD to persist #' @param newLevel The new storage level to be assigned @@ -382,11 +382,13 @@ setMethod("collectPartition", #' \code{collectAsMap} returns a named list as a map that contains all of the elements #' in a key-value pair RDD. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) #' collectAsMap(rdd) # list(`1` = 2, `3` = 4) #'} +# nolint end #' @rdname collect-methods #' @aliases collectAsMap,RDD-method #' @noRd @@ -442,11 +444,13 @@ setMethod("length", #' @return list of (value, count) pairs, where count is number of each unique #' value in rdd. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, c(1,2,3,2,1)) #' countByValue(rdd) # (1,2L), (2,2L), (3,1L) #'} +# nolint end #' @rdname countByValue #' @aliases countByValue,RDD-method #' @noRd @@ -597,11 +601,13 @@ setMethod("mapPartitionsWithIndex", #' @param x The RDD to be filtered. #' @param f A unary predicate function. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) #'} +# nolint end #' @rdname filterRDD #' @aliases filterRDD,RDD,function-method #' @noRd @@ -756,11 +762,13 @@ setMethod("foreachPartition", #' @param x The RDD to take elements from #' @param num Number of elements to take #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' take(rdd, 2L) # list(1, 2) #'} +# nolint end #' @rdname take #' @aliases take,RDD,numeric-method #' @noRd @@ -824,11 +832,13 @@ setMethod("first", #' @param x The RDD to remove duplicates from. #' @param numPartitions Number of partitions to create. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, c(1,2,2,3,3,3)) #' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) #'} +# nolint end #' @rdname distinct #' @aliases distinct,RDD-method #' @noRd @@ -974,11 +984,13 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", #' @param x The RDD. #' @param func The function to be applied. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(1, 2, 3)) #' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) #'} +# nolint end #' @rdname keyBy #' @aliases keyBy,RDD #' @noRd @@ -1113,11 +1125,13 @@ setMethod("saveAsTextFile", #' @param numPartitions Number of partitions to create. #' @return An RDD where all elements are sorted. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(3, 2, 1)) #' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) #'} +# nolint end #' @rdname sortBy #' @aliases sortBy,RDD,RDD-method #' @noRd @@ -1188,11 +1202,13 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { #' @param num Number of elements to return. #' @return The first N elements from the RDD in ascending order. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) #' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) #'} +# nolint end #' @rdname takeOrdered #' @aliases takeOrdered,RDD,RDD-method #' @noRd @@ -1209,11 +1225,13 @@ setMethod("takeOrdered", #' @return The top N elements from the RDD. #' @rdname top #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) #' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) #'} +# nolint end #' @aliases top,RDD,RDD-method #' @noRd setMethod("top", @@ -1261,6 +1279,7 @@ setMethod("fold", #' @rdname aggregateRDD #' @seealso reduce #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(1, 2, 3, 4)) @@ -1269,6 +1288,7 @@ setMethod("fold", #' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } #' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) #'} +# nolint end #' @aliases aggregateRDD,RDD,RDD-method #' @noRd setMethod("aggregateRDD", @@ -1367,12 +1387,14 @@ setMethod("setName", #' @return An RDD with zipped items. #' @seealso zipWithIndex #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) #' collect(zipWithUniqueId(rdd)) #' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) #'} +# nolint end #' @rdname zipWithUniqueId #' @aliases zipWithUniqueId,RDD #' @noRd @@ -1408,12 +1430,14 @@ setMethod("zipWithUniqueId", #' @return An RDD with zipped items. #' @seealso zipWithUniqueId #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) #' collect(zipWithIndex(rdd)) #' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) #'} +# nolint end #' @rdname zipWithIndex #' @aliases zipWithIndex,RDD #' @noRd @@ -1454,12 +1478,14 @@ setMethod("zipWithIndex", #' @return An RDD created by coalescing all elements within #' each partition into a list. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, as.list(1:4), 2L) #' collect(glom(rdd)) #' # list(list(1, 2), list(3, 4)) #'} +# nolint end #' @rdname glom #' @aliases glom,RDD #' @noRd @@ -1519,6 +1545,7 @@ setMethod("unionRDD", #' @param other Another RDD to be zipped. #' @return An RDD zipped from the two RDDs. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, 0:4) @@ -1526,6 +1553,7 @@ setMethod("unionRDD", #' collect(zipRDD(rdd1, rdd2)) #' # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) #'} +# nolint end #' @rdname zipRDD #' @aliases zipRDD,RDD #' @noRd @@ -1557,12 +1585,14 @@ setMethod("zipRDD", #' @param other An RDD. #' @return A new RDD which is the Cartesian product of these two RDDs. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:2) #' sortByKey(cartesian(rdd, rdd)) #' # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) #'} +# nolint end #' @rdname cartesian #' @aliases cartesian,RDD,RDD-method #' @noRd @@ -1587,6 +1617,7 @@ setMethod("cartesian", #' @param numPartitions Number of the partitions in the result RDD. #' @return An RDD with the elements from this that are not in other. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) @@ -1594,6 +1625,7 @@ setMethod("cartesian", #' collect(subtract(rdd1, rdd2)) #' # list(1, 1, 3) #'} +# nolint end #' @rdname subtract #' @aliases subtract,RDD #' @noRd @@ -1619,6 +1651,7 @@ setMethod("subtract", #' @param numPartitions The number of partitions in the result RDD. #' @return An RDD which is the intersection of these two RDDs. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) @@ -1626,6 +1659,7 @@ setMethod("subtract", #' collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) #' # list(1, 2, 3) #'} +# nolint end #' @rdname intersection #' @aliases intersection,RDD #' @noRd @@ -1653,6 +1687,7 @@ setMethod("intersection", #' Assumes that all the RDDs have the *same number of partitions*, but #' does *not* require them to have the same number of elements in each partition. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 @@ -1662,6 +1697,7 @@ setMethod("intersection", #' func = function(x, y, z) { list(list(x, y, z))} )) #' # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) #'} +# nolint end #' @rdname zipRDD #' @aliases zipPartitions,RDD #' @noRd diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index f7e56e43016ea..d8a0393275390 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -17,6 +17,7 @@ # Utility functions to deserialize objects from Java. +# nolint start # Type mapping from Java to R # # void -> NULL @@ -32,6 +33,8 @@ # # Array[T] -> list() # Object -> jobj +# +# nolint end readObject <- function(con) { # Read type first diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 334c11d2f89a1..f7131140feafb 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -30,12 +30,14 @@ NULL #' @param key The key to look up for #' @return a list of values in this RDD for key key #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' pairs <- list(c(1, 1), c(2, 2), c(1, 3)) #' rdd <- parallelize(sc, pairs) #' lookup(rdd, 1) # list(1, 3) #'} +# nolint end #' @rdname lookup #' @aliases lookup,RDD-method #' @noRd @@ -58,11 +60,13 @@ setMethod("lookup", #' @param x The RDD to count keys. #' @return list of (key, count) pairs, where count is number of each key in rdd. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) #' countByKey(rdd) # ("a", 2L), ("b", 1L) #'} +# nolint end #' @rdname countByKey #' @aliases countByKey,RDD-method #' @noRd @@ -77,11 +81,13 @@ setMethod("countByKey", #' #' @param x The RDD from which the keys of each tuple is returned. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) #' collect(keys(rdd)) # list(1, 3) #'} +# nolint end #' @rdname keys #' @aliases keys,RDD #' @noRd @@ -98,11 +104,13 @@ setMethod("keys", #' #' @param x The RDD from which the values of each tuple is returned. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) #' collect(values(rdd)) # list(2, 4) #'} +# nolint end #' @rdname values #' @aliases values,RDD #' @noRd @@ -348,6 +356,7 @@ setMethod("reduceByKey", #' @return A list of elements of type list(K, V') where V' is the merged value for each key #' @seealso reduceByKey #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) @@ -355,6 +364,7 @@ setMethod("reduceByKey", #' reduced <- reduceByKeyLocally(rdd, "+") #' reduced # list(list(1, 6), list(1.1, 3)) #'} +# nolint end #' @rdname reduceByKeyLocally #' @aliases reduceByKeyLocally,RDD,integer-method #' @noRd @@ -412,6 +422,7 @@ setMethod("reduceByKeyLocally", #' @return An RDD where each element is list(K, C) where C is the combined type #' @seealso groupByKey, reduceByKey #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) @@ -420,6 +431,7 @@ setMethod("reduceByKeyLocally", #' combined <- collect(parts) #' combined[[1]] # Should be a list(1, 6) #'} +# nolint end #' @rdname combineByKey #' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method #' @noRd @@ -473,6 +485,7 @@ setMethod("combineByKey", #' @return An RDD containing the aggregation result. #' @seealso foldByKey, combineByKey #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) @@ -482,6 +495,7 @@ setMethod("combineByKey", #' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) #' # list(list(1, list(3, 2)), list(2, list(7, 2))) #'} +# nolint end #' @rdname aggregateByKey #' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method #' @noRd @@ -509,11 +523,13 @@ setMethod("aggregateByKey", #' @return An RDD containing the aggregation result. #' @seealso aggregateByKey, combineByKey #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) #' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) #'} +# nolint end #' @rdname foldByKey #' @aliases foldByKey,RDD,ANY,ANY,integer-method #' @noRd @@ -540,12 +556,14 @@ setMethod("foldByKey", #' @return a new RDD containing all pairs of elements with matching keys in #' two input RDDs. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) #' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) #' join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) #'} +# nolint end #' @rdname join-methods #' @aliases join,RDD,RDD-method #' @noRd @@ -578,6 +596,7 @@ setMethod("join", #' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) #' if no elements in rdd2 have key k. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) @@ -585,6 +604,7 @@ setMethod("join", #' leftOuterJoin(rdd1, rdd2, 2L) #' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) #'} +# nolint end #' @rdname join-methods #' @aliases leftOuterJoin,RDD,RDD-method #' @noRd @@ -616,6 +636,7 @@ setMethod("leftOuterJoin", #' all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) #' if no elements in x have key k. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) @@ -623,6 +644,7 @@ setMethod("leftOuterJoin", #' rightOuterJoin(rdd1, rdd2, 2L) #' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) #'} +# nolint end #' @rdname join-methods #' @aliases rightOuterJoin,RDD,RDD-method #' @noRd @@ -655,6 +677,7 @@ setMethod("rightOuterJoin", #' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements #' in x/y have key k. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) @@ -664,6 +687,7 @@ setMethod("rightOuterJoin", #' # list(2, list(NULL, 4))) #' # list(3, list(3, NULL)), #'} +# nolint end #' @rdname join-methods #' @aliases fullOuterJoin,RDD,RDD-method #' @noRd @@ -688,6 +712,7 @@ setMethod("fullOuterJoin", #' @return a new RDD containing all pairs of elements with values in a list #' in all RDDs. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) @@ -695,6 +720,7 @@ setMethod("fullOuterJoin", #' cogroup(rdd1, rdd2, numPartitions = 2L) #' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) #'} +# nolint end #' @rdname cogroup #' @aliases cogroup,RDD-method #' @noRd @@ -740,11 +766,13 @@ setMethod("cogroup", #' @param numPartitions Number of partitions to create. #' @return An RDD where all (k, v) pair elements are sorted. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) #' collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) #'} +# nolint end #' @rdname sortByKey #' @aliases sortByKey,RDD,RDD-method #' @noRd @@ -805,6 +833,7 @@ setMethod("sortByKey", #' @param numPartitions Number of the partitions in the result RDD. #' @return An RDD with the pairs from x whose keys are not in other. #' @examples +# nolint start #'\dontrun{ #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), @@ -813,6 +842,7 @@ setMethod("sortByKey", #' collect(subtractByKey(rdd1, rdd2)) #' # list(list("b", 4), list("b", 5)) #'} +# nolint end #' @rdname subtractByKey #' @aliases subtractByKey,RDD #' @noRd diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 17082b4e52fcf..095ddb9aed2e7 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -17,6 +17,7 @@ # Utility functions to serialize R objects so they can be read in Java. +# nolint start # Type mapping from R to Java # # NULL -> Void @@ -31,6 +32,7 @@ # list[T] -> Array[T], where T is one of above mentioned types # environment -> Map[String, T], where T is a native type # jobj -> Object, where jobj is an object created in the backend +# nolint end getSerdeType <- function(object) { type <- class(object)[[1]] diff --git a/R/pkg/inst/tests/testthat/test_rdd.R b/R/pkg/inst/tests/testthat/test_rdd.R index 7423b4f2bed1f..1b3a22486e95f 100644 --- a/R/pkg/inst/tests/testthat/test_rdd.R +++ b/R/pkg/inst/tests/testthat/test_rdd.R @@ -223,14 +223,14 @@ test_that("takeSample() on RDDs", { s <- takeSample(data, TRUE, 100L, seed) expect_equal(length(s), 100L) # Chance of getting all distinct elements is astronomically low, so test we - # got < 100 + # got less than 100 expect_true(length(unique(s)) < 100L) } for (seed in 4:5) { s <- takeSample(data, TRUE, 200L, seed) expect_equal(length(s), 200L) # Chance of getting all distinct elements is still quite low, so test we - # got < 100 + # got less than 100 expect_true(length(unique(s)) < 100L) } }) diff --git a/R/pkg/inst/tests/testthat/test_shuffle.R b/R/pkg/inst/tests/testthat/test_shuffle.R index adf0b91d25fe9..d3d0f8a24d01c 100644 --- a/R/pkg/inst/tests/testthat/test_shuffle.R +++ b/R/pkg/inst/tests/testthat/test_shuffle.R @@ -176,8 +176,8 @@ test_that("partitionBy() partitions data correctly", { resultRDD <- partitionBy(numPairsRdd, 2L, partitionByMagnitude) - expected_first <- list(list(1, 100), list(2, 200)) # key < 3 - expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key >= 3 + expected_first <- list(list(1, 100), list(2, 200)) # key less than 3 + expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key greater than or equal 3 actual_first <- collectPartition(resultRDD, 0L) actual_second <- collectPartition(resultRDD, 1L) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index c2b6adbe3ae01..ee942d25ba4b5 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -494,9 +494,11 @@ test_that("table() returns a new DataFrame", { expect_equal(count(tabledf), 3) dropTempTable(sqlContext, "table1") + # nolint start # Test base::table is working #a <- letters[1:3] #expect_equal(class(table(a, sample(a))), "table") + # nolint end }) test_that("toRDD() returns an RRDD", { @@ -762,8 +764,10 @@ test_that("sample on a DataFrame", { sampled3 <- sample_frac(df, FALSE, 0.1, 0) # set seed for predictable result expect_true(count(sampled3) < 3) + # nolint start # Test base::sample is working #expect_equal(length(sample(1:12)), 12) + # nolint end }) test_that("select operators", { @@ -1048,8 +1052,8 @@ test_that("string operators", { df2 <- createDataFrame(sqlContext, l2) expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) expect_equal(collect(select(df2, locate("aa", df2$a, 1)))[1, 1], 2) - expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") - expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") + expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") # nolint + expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") # nolint l3 <- list(list(a = "a.b.c.d")) df3 <- createDataFrame(sqlContext, l3) @@ -1255,7 +1259,7 @@ test_that("filter() on a DataFrame", { expect_equal(count(filtered6), 2) # Test stats::filter is working - #expect_true(is.ts(filter(1:100, rep(1, 3)))) + #expect_true(is.ts(filter(1:100, rep(1, 3)))) # nolint }) test_that("join() and merge() on a DataFrame", { @@ -1655,7 +1659,7 @@ test_that("cov() and corr() on a DataFrame", { expect_true(abs(result - 1.0) < 1e-12) # Test stats::cov is working - #expect_true(abs(max(cov(swiss)) - 1739.295) < 1e-3) + #expect_true(abs(max(cov(swiss)) - 1739.295) < 1e-3) # nolint }) test_that("freqItems() on a DataFrame", { diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index 12df4cf4f65b7..56f14a3bce61e 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -95,7 +95,9 @@ test_that("cleanClosure on R functions", { # TODO(shivaram): length(ls(env)) is 4 here for some reason and `lapply` is included in `env`. # Disabling this test till we debug this. # + # nolint start # expect_equal(length(ls(env)), 3) # Only "g", "l" and "f". No "base", "field" or "defUse". + # nolint end expect_true("g" %in% ls(env)) expect_true("l" %in% ls(env)) expect_true("f" %in% ls(env)) From f7a322382a3c1eed7088541add55a7813813a958 Mon Sep 17 00:00:00 2001 From: Xiu Guo Date: Sun, 3 Jan 2016 20:48:56 -0800 Subject: [PATCH 06/33] [SPARK-12562][SQL] DataFrame.write.format(text) requires the column name to be called value Author: Xiu Guo Closes #10515 from xguo27/SPARK-12562. (cherry picked from commit 84f8492c1555bf8ab44c9818752278f61768eb16) Signed-off-by: Reynold Xin --- .../sql/execution/datasources/text/DefaultSource.scala | 9 +++++---- .../spark/sql/execution/datasources/text/TextSuite.scala | 4 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala index 4a1cbe4c38fa2..10529f2bdfd12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala @@ -50,7 +50,7 @@ class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { partitionColumns: Option[StructType], parameters: Map[String, String]): HadoopFsRelation = { dataSchema.foreach(verifySchema) - new TextRelation(None, partitionColumns, paths)(sqlContext) + new TextRelation(None, dataSchema, partitionColumns, paths)(sqlContext) } override def shortName(): String = "text" @@ -70,15 +70,16 @@ class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { private[sql] class TextRelation( val maybePartitionSpec: Option[PartitionSpec], + val textSchema: Option[StructType], override val userDefinedPartitionColumns: Option[StructType], override val paths: Array[String] = Array.empty[String], parameters: Map[String, String] = Map.empty[String, String]) (@transient val sqlContext: SQLContext) extends HadoopFsRelation(maybePartitionSpec, parameters) { - /** Data schema is always a single column, named "value". */ - override def dataSchema: StructType = new StructType().add("value", StringType) - + /** Data schema is always a single column, named "value" if original Data source has no schema. */ + override def dataSchema: StructType = + textSchema.getOrElse(new StructType().add("value", StringType)) /** This is an internal data source that outputs internal row format. */ override val needConversion: Boolean = false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 914e516613f9e..02c416af50cd7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -33,8 +33,8 @@ class TextSuite extends QueryTest with SharedSQLContext { verifyFrame(sqlContext.read.text(testFile)) } - test("writing") { - val df = sqlContext.read.text(testFile) + test("SPARK-12562 verify write.text() can handle column name beyond `value`") { + val df = sqlContext.read.text(testFile).withColumnRenamed("value", "adwrasdf") val tempFile = Utils.createTempDir() tempFile.delete() From cd02038198fa57da816211d7bc65921ff9f1e9bb Mon Sep 17 00:00:00 2001 From: Nong Li Date: Mon, 4 Jan 2016 10:37:56 -0800 Subject: [PATCH 07/33] [SPARK-12486] Worker should kill the executors more forcefully if possible. This patch updates the ExecutorRunner's terminate path to use the new java 8 API to terminate processes more forcefully if possible. If the executor is unhealthy, it would previously ignore the destroy() call. Presumably, the new java API was added to handle cases like this. We could update the termination path in the future to use OS specific commands for older java versions. Author: Nong Li Closes #10438 from nongli/spark-12486-executors. (cherry picked from commit 8f659393b270c46e940c4e98af2d996bd4fd6442) Signed-off-by: Andrew Or --- .../spark/deploy/worker/ExecutorRunner.scala | 17 ++-- .../scala/org/apache/spark/util/Utils.scala | 24 ++++++ .../org/apache/spark/util/UtilsSuite.scala | 83 +++++++++++++++++-- 3 files changed, 112 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 9a42487bb37aa..9c4b8cdc646b0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -23,13 +23,12 @@ import scala.collection.JavaConverters._ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files - -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.{SecurityManager, SparkConf, Logging} -import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.util.logging.FileAppender +import org.apache.spark.{Logging, SecurityManager, SparkConf} /** * Manages the execution of one executor process. @@ -60,6 +59,9 @@ private[deploy] class ExecutorRunner( private var stdoutAppender: FileAppender = null private var stderrAppender: FileAppender = null + // Timeout to wait for when trying to terminate an executor. + private val EXECUTOR_TERMINATE_TIMEOUT_MS = 10 * 1000 + // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. private var shutdownHook: AnyRef = null @@ -94,8 +96,11 @@ private[deploy] class ExecutorRunner( if (stderrAppender != null) { stderrAppender.stop() } - process.destroy() - exitCode = Some(process.waitFor()) + exitCode = Utils.terminateProcess(process, EXECUTOR_TERMINATE_TIMEOUT_MS) + if (exitCode.isEmpty) { + logWarning("Failed to terminate process: " + process + + ". This process will likely be orphaned.") + } } try { worker.send(ExecutorStateChanged(appId, execId, state, message, exitCode)) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6cb52fbde4153..e29c84162f91c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1695,6 +1695,30 @@ private[spark] object Utils extends Logging { new File(path).getName } + /** + * Terminates a process waiting for at most the specified duration. Returns whether + * the process terminated. + */ + def terminateProcess(process: Process, timeoutMs: Long): Option[Int] = { + try { + // Java8 added a new API which will more forcibly kill the process. Use that if available. + val destroyMethod = process.getClass().getMethod("destroyForcibly"); + destroyMethod.setAccessible(true) + destroyMethod.invoke(process) + } catch { + case NonFatal(e) => + if (!e.isInstanceOf[NoSuchMethodException]) { + logWarning("Exception when attempting to kill process", e) + } + process.destroy() + } + if (waitForProcess(process, timeoutMs)) { + Option(process.exitValue()) + } else { + None + } + } + /** * Wait for a process to terminate for at most the specified duration. * Return whether the process actually terminated after the given timeout. diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index fdb51d440eff6..7de995af512db 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -17,26 +17,24 @@ package org.apache.spark.util -import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File, FileOutputStream} import java.lang.{Double => JDouble, Float => JFloat} import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.text.DecimalFormatSymbols -import java.util.concurrent.TimeUnit import java.util.Locale +import java.util.concurrent.TimeUnit import scala.collection.mutable.ListBuffer import scala.util.Random import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files - +import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path - import org.apache.spark.network.util.ByteUnit -import org.apache.spark.{Logging, SparkFunSuite} -import org.apache.spark.SparkConf +import org.apache.spark.{Logging, SparkConf, SparkFunSuite} class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { @@ -745,4 +743,77 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.decodeFileNameInURI(new URI("files:///abc")) === "abc") assert(Utils.decodeFileNameInURI(new URI("files:///abc%20xyz")) === "abc xyz") } + + test("Kill process") { + // Verify that we can terminate a process even if it is in a bad state. This is only run + // on UNIX since it does some OS specific things to verify the correct behavior. + if (SystemUtils.IS_OS_UNIX) { + def getPid(p: Process): Int = { + val f = p.getClass().getDeclaredField("pid") + f.setAccessible(true) + f.get(p).asInstanceOf[Int] + } + + def pidExists(pid: Int): Boolean = { + val p = Runtime.getRuntime.exec(s"kill -0 $pid") + p.waitFor() + p.exitValue() == 0 + } + + def signal(pid: Int, s: String): Unit = { + val p = Runtime.getRuntime.exec(s"kill -$s $pid") + p.waitFor() + } + + // Start up a process that runs 'sleep 10'. Terminate the process and assert it takes + // less time and the process is no longer there. + val startTimeMs = System.currentTimeMillis() + val process = new ProcessBuilder("sleep", "10").start() + val pid = getPid(process) + try { + assert(pidExists(pid)) + val terminated = Utils.terminateProcess(process, 5000) + assert(terminated.isDefined) + Utils.waitForProcess(process, 5000) + val durationMs = System.currentTimeMillis() - startTimeMs + assert(durationMs < 5000) + assert(!pidExists(pid)) + } finally { + // Forcibly kill the test process just in case. + signal(pid, "SIGKILL") + } + + val v: String = System.getProperty("java.version") + if (v >= "1.8.0") { + // Java8 added a way to forcibly terminate a process. We'll make sure that works by + // creating a very misbehaving process. It ignores SIGTERM and has been SIGSTOPed. On + // older versions of java, this will *not* terminate. + val file = File.createTempFile("temp-file-name", ".tmp") + val cmd = + s""" + |#!/bin/bash + |trap "" SIGTERM + |sleep 10 + """.stripMargin + Files.write(cmd.getBytes(), file) + file.getAbsoluteFile.setExecutable(true) + + val process = new ProcessBuilder(file.getAbsolutePath).start() + val pid = getPid(process) + assert(pidExists(pid)) + try { + signal(pid, "SIGSTOP") + val start = System.currentTimeMillis() + val terminated = Utils.terminateProcess(process, 5000) + assert(terminated.isDefined) + Utils.waitForProcess(process, 5000) + val duration = System.currentTimeMillis() - start + assert(duration < 5000) + assert(!pidExists(pid)) + } finally { + signal(pid, "SIGKILL") + } + } + } + } } From b5a1f564a3c099ef0b674599f0b012d9346115a3 Mon Sep 17 00:00:00 2001 From: Pete Robbins Date: Mon, 4 Jan 2016 10:43:21 -0800 Subject: [PATCH 08/33] [SPARK-12470] [SQL] Fix size reduction calculation also only allocate required buffer size Author: Pete Robbins Closes #10421 from robbinspg/master. (cherry picked from commit b504b6a90a95a723210beb0031ed41a75d702f66) Signed-off-by: Davies Liu Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala --- .../expressions/codegen/GenerateUnsafeRowJoiner.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index da602d9b4bce1..fb3c7b1bb4f72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -61,9 +61,9 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U val outputBitsetWords = (schema1.size + schema2.size + 63) / 64 val bitset1Remainder = schema1.size % 64 - // The number of words we can reduce when we concat two rows together. + // The number of bytes we can reduce when we concat two rows together. // The only reduction comes from merging the bitset portion of the two rows, saving 1 word. - val sizeReduction = bitset1Words + bitset2Words - outputBitsetWords + val sizeReduction = (bitset1Words + bitset2Words - outputBitsetWords) * 8 // --------------------- copy bitset from row 1 and row 2 --------------------------- // val copyBitset = Seq.tabulate(outputBitsetWords) { i => @@ -171,7 +171,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U | // row1: ${schema1.size} fields, $bitset1Words words in bitset | // row2: ${schema2.size}, $bitset2Words words in bitset | // output: ${schema1.size + schema2.size} fields, $outputBitsetWords words in bitset - | final int sizeInBytes = row1.getSizeInBytes() + row2.getSizeInBytes(); + | final int sizeInBytes = row1.getSizeInBytes() + row2.getSizeInBytes() - $sizeReduction; | if (sizeInBytes > buf.length) { | buf = new byte[sizeInBytes]; | } @@ -188,7 +188,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U | $copyVariableLengthRow2 | $updateOffset | - | out.pointTo(buf, ${schema1.size + schema2.size}, sizeInBytes - $sizeReduction); + | out.pointTo(buf, ${schema1.size + schema2.size}, sizeInBytes); | | return out; | } From 7f37c1e45d52b7823d566349e2be21366d73651f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 4 Jan 2016 10:39:42 -0800 Subject: [PATCH 09/33] [SPARK-12579][SQL] Force user-specified JDBC driver to take precedence Spark SQL's JDBC data source allows users to specify an explicit JDBC driver to load (using the `driver` argument), but in the current code it's possible that the user-specified driver will not be used when it comes time to actually create a JDBC connection. In a nutshell, the problem is that you might have multiple JDBC drivers on the classpath that claim to be able to handle the same subprotocol, so simply registering the user-provided driver class with the our `DriverRegistry` and JDBC's `DriverManager` is not sufficient to ensure that it's actually used when creating the JDBC connection. This patch addresses this issue by first registering the user-specified driver with the DriverManager, then iterating over the driver manager's loaded drivers in order to obtain the correct driver and use it to create a connection (previously, we just called `DriverManager.getConnection()` directly). If a user did not specify a JDBC driver to use, then we call `DriverManager.getDriver` to figure out the class of the driver to use, then pass that class's name to executors; this guards against corner-case bugs in situations where the driver and executor JVMs might have different sets of JDBC drivers on their classpaths (previously, there was the (rare) potential for `DriverManager.getConnection()` to use different drivers on the driver and executors if the user had not explicitly specified a JDBC driver class and the classpaths were different). This patch is inspired by a similar patch that I made to the `spark-redshift` library (https://github.com/databricks/spark-redshift/pull/143), which contains its own modified fork of some of Spark's JDBC data source code (for cross-Spark-version compatibility reasons). Author: Josh Rosen Closes #10519 from JoshRosen/jdbc-driver-precedence. (cherry picked from commit 6c83d938cc61bd5fabaf2157fcc3936364a83f02) Signed-off-by: Yin Huai --- docs/sql-programming-guide.md | 4 +-- .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../datasources/jdbc/DefaultSource.scala | 3 -- .../datasources/jdbc/DriverRegistry.scala | 5 --- .../execution/datasources/jdbc/JDBCRDD.scala | 33 +++-------------- .../datasources/jdbc/JDBCRelation.scala | 2 -- .../datasources/jdbc/JdbcUtils.scala | 35 +++++++++++++++---- 7 files changed, 34 insertions(+), 50 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 3f9a831eddc88..b058833616433 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1895,9 +1895,7 @@ the Data Sources API. The following options are supported: driver - The class name of the JDBC driver needed to connect to this URL. This class will be loaded - on the master and workers before running an JDBC commands to allow the driver to - register itself with the JDBC subsystem. + The class name of the JDBC driver to use to connect to this URL. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index ab362539e2982..9f59c0f3a74db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -275,7 +275,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { } // connectionProperties should override settings in extraOptions props.putAll(connectionProperties) - val conn = JdbcUtils.createConnection(url, props) + val conn = JdbcUtils.createConnectionFactory(url, props)() try { var tableExists = JdbcUtils.tableExists(conn, url, table) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala index f522303be94ad..5ae6cff9b5584 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala @@ -31,15 +31,12 @@ class DefaultSource extends RelationProvider with DataSourceRegister { sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { val url = parameters.getOrElse("url", sys.error("Option 'url' not specified")) - val driver = parameters.getOrElse("driver", null) val table = parameters.getOrElse("dbtable", sys.error("Option 'dbtable' not specified")) val partitionColumn = parameters.getOrElse("partitionColumn", null) val lowerBound = parameters.getOrElse("lowerBound", null) val upperBound = parameters.getOrElse("upperBound", null) val numPartitions = parameters.getOrElse("numPartitions", null) - if (driver != null) DriverRegistry.register(driver) - if (partitionColumn != null && (lowerBound == null || upperBound == null || numPartitions == null)) { sys.error("Partitioning incompletely specified") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala index 7ccd61ed469e9..65af397451c5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala @@ -51,10 +51,5 @@ object DriverRegistry extends Logging { } } } - - def getDriverClassName(url: String): String = DriverManager.getDriver(url) match { - case wrapper: DriverWrapper => wrapper.wrapped.getClass.getCanonicalName - case driver => driver.getClass.getCanonicalName - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index c2f2a312a4fbd..fad482b427355 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.sql.{Connection, Date, DriverManager, ResultSet, ResultSetMetaData, SQLException, Timestamp} +import java.sql.{Connection, Date, ResultSet, ResultSetMetaData, SQLException, Timestamp} import java.util.Properties import org.apache.commons.lang3.StringUtils @@ -39,7 +39,6 @@ private[sql] case class JDBCPartition(whereClause: String, idx: Int) extends Par override def index: Int = idx } - private[sql] object JDBCRDD extends Logging { /** @@ -118,7 +117,7 @@ private[sql] object JDBCRDD extends Logging { */ def resolveTable(url: String, table: String, properties: Properties): StructType = { val dialect = JdbcDialects.get(url) - val conn: Connection = getConnector(properties.getProperty("driver"), url, properties)() + val conn: Connection = JdbcUtils.createConnectionFactory(url, properties)() try { val statement = conn.prepareStatement(s"SELECT * FROM $table WHERE 1=0") try { @@ -170,36 +169,13 @@ private[sql] object JDBCRDD extends Logging { new StructType(columns map { name => fieldMap(name) }) } - /** - * Given a driver string and an url, return a function that loads the - * specified driver string then returns a connection to the JDBC url. - * getConnector is run on the driver code, while the function it returns - * is run on the executor. - * - * @param driver - The class name of the JDBC driver for the given url, or null if the class name - * is not necessary. - * @param url - The JDBC url to connect to. - * - * @return A function that loads the driver and connects to the url. - */ - def getConnector(driver: String, url: String, properties: Properties): () => Connection = { - () => { - try { - if (driver != null) DriverRegistry.register(driver) - } catch { - case e: ClassNotFoundException => - logWarning(s"Couldn't find class $driver", e) - } - DriverManager.getConnection(url, properties) - } - } + /** * Build and return JDBCRDD from the given information. * * @param sc - Your SparkContext. * @param schema - The Catalyst schema of the underlying database table. - * @param driver - The class name of the JDBC driver for the given url. * @param url - The JDBC url to connect to. * @param fqTable - The fully-qualified table name (or paren'd SQL query) to use. * @param requiredColumns - The names of the columns to SELECT. @@ -212,7 +188,6 @@ private[sql] object JDBCRDD extends Logging { def scanTable( sc: SparkContext, schema: StructType, - driver: String, url: String, properties: Properties, fqTable: String, @@ -223,7 +198,7 @@ private[sql] object JDBCRDD extends Logging { val quotedColumns = requiredColumns.map(colName => dialect.quoteIdentifier(colName)) new JDBCRDD( sc, - getConnector(driver, url, properties), + JdbcUtils.createConnectionFactory(url, properties), pruneSchema(schema, requiredColumns), fqTable, quotedColumns, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index f9300dc2cb529..375266f1be42c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -91,12 +91,10 @@ private[sql] case class JDBCRelation( override val schema: StructType = JDBCRDD.resolveTable(url, table, properties) override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { - val driver: String = DriverRegistry.getDriverClassName(url) // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row] JDBCRDD.scanTable( sqlContext.sparkContext, schema, - driver, url, properties, table, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 46f2670eee010..10f650693f288 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.sql.{Connection, PreparedStatement} +import java.sql.{Connection, Driver, DriverManager, PreparedStatement} import java.util.Properties +import scala.collection.JavaConverters._ import scala.util.Try import scala.util.control.NonFatal @@ -34,10 +35,31 @@ import org.apache.spark.sql.{DataFrame, Row} object JdbcUtils extends Logging { /** - * Establishes a JDBC connection. + * Returns a factory for creating connections to the given JDBC URL. + * + * @param url the JDBC url to connect to. + * @param properties JDBC connection properties. */ - def createConnection(url: String, connectionProperties: Properties): Connection = { - JDBCRDD.getConnector(connectionProperties.getProperty("driver"), url, connectionProperties)() + def createConnectionFactory(url: String, properties: Properties): () => Connection = { + val userSpecifiedDriverClass = Option(properties.getProperty("driver")) + userSpecifiedDriverClass.foreach(DriverRegistry.register) + // Performing this part of the logic on the driver guards against the corner-case where the + // driver returned for a URL is different on the driver and executors due to classpath + // differences. + val driverClass: String = userSpecifiedDriverClass.getOrElse { + DriverManager.getDriver(url).getClass.getCanonicalName + } + () => { + userSpecifiedDriverClass.foreach(DriverRegistry.register) + val driver: Driver = DriverManager.getDrivers.asScala.collectFirst { + case d: DriverWrapper if d.wrapped.getClass.getCanonicalName == driverClass => d + case d if d.getClass.getCanonicalName == driverClass => d + }.getOrElse { + throw new IllegalStateException( + s"Did not find registered driver with class $driverClass") + } + driver.connect(url, properties) + } } /** @@ -242,15 +264,14 @@ object JdbcUtils extends Logging { df: DataFrame, url: String, table: String, - properties: Properties = new Properties()) { + properties: Properties) { val dialect = JdbcDialects.get(url) val nullTypes: Array[Int] = df.schema.fields.map { field => getJdbcType(field.dataType, dialect).jdbcNullType } val rddSchema = df.schema - val driver: String = DriverRegistry.getDriverClassName(url) - val getConnection: () => Connection = JDBCRDD.getConnector(driver, url, properties) + val getConnection: () => Connection = createConnectionFactory(url, properties) val batchSize = properties.getProperty("batchsize", "1000").toInt df.foreachPartition { iterator => savePartition(getConnection, table, iterator, rddSchema, nullTypes, batchSize, dialect) From 1005ee396f74dc4fcf127613b65e1abdb7f1934c Mon Sep 17 00:00:00 2001 From: tedyu Date: Mon, 4 Jan 2016 12:38:04 -0800 Subject: [PATCH 10/33] [DOC] Adjust coverage for partitionBy() This is the related thread: http://search-hadoop.com/m/q3RTtO3ReeJ1iF02&subj=Re+partitioning+json+data+in+spark Michael suggested fixing the doc. Please review. Author: tedyu Closes #10499 from ted-yu/master. (cherry picked from commit 40d03960d79debdff5cef21997417c4f8a8ce2e9) Signed-off-by: Michael Armbrust --- .../src/main/scala/org/apache/spark/sql/DataFrameWriter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 9f59c0f3a74db..9afa6856907a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -119,7 +119,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * Partitions the output by the given columns on the file system. If specified, the output is * laid out on the file system similar to Hive's partitioning scheme. * - * This is only applicable for Parquet at the moment. + * This was initially applicable for Parquet but in 1.5+ covers JSON, text, ORC and avro as well. * * @since 1.4.0 */ From 8ac9198096d1cef9fbc062df8b8bd94fb9e96829 Mon Sep 17 00:00:00 2001 From: Nong Li Date: Mon, 4 Jan 2016 14:58:24 -0800 Subject: [PATCH 11/33] [SPARK-12589][SQL] Fix UnsafeRowParquetRecordReader to properly set the row length. The reader was previously not setting the row length meaning it was wrong if there were variable length columns. This problem does not manifest usually, since the value in the column is correct and projecting the row fixes the issue. Author: Nong Li Closes #10576 from nongli/spark-12589. (cherry picked from commit 34de24abb518e95c4312b77aa107d061ce02c835) Signed-off-by: Yin Huai Conflicts: sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java --- .../sql/catalyst/expressions/UnsafeRow.java | 3 +++ .../parquet/UnsafeRowParquetRecordReader.java | 9 +++++++ .../datasources/parquet/ParquetIOSuite.scala | 24 +++++++++++++++++++ 3 files changed, 36 insertions(+) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index b6979d0c82977..af687eaeb8ab4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -214,6 +214,9 @@ public void pointTo(byte[] buf, int sizeInBytes) { pointTo(buf, numFields, sizeInBytes); } + public void setTotalSize(int sizeInBytes) { + this.sizeInBytes = sizeInBytes; + } public void setNotNullAt(int i) { assertIndexIsValid(i); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java index dade488ca281b..b0d07745fd870 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java @@ -264,6 +264,15 @@ private boolean loadBatch() throws IOException { numBatched = num; batchIdx = 0; } + + // Update the total row lengths if the schema contained variable length. We did not maintain + // this as we populated the columns. + if (containsVarLenFields) { + for (int i = 0; i < numBatched; ++i) { + rows[i].setTotalSize(rowWriters[i].holder().totalSize()); + } + } + return true; } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 159c3c6284899..4c3d3f9fd2433 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -40,6 +40,7 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -626,6 +627,29 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { readResourceParquetFile("dec-in-fixed-len.parquet"), sqlContext.range(1 << 4).select('id % 10 cast DecimalType(10, 2) as 'fixed_len_dec)) } + + test("SPARK-12589 copy() on rows returned from reader works for strings") { + withTempPath { dir => + val data = (1, "abc") ::(2, "helloabcde") :: Nil + data.toDF().write.parquet(dir.getCanonicalPath) + var hash1: Int = 0 + var hash2: Int = 0 + (false :: true :: Nil).foreach { v => + withSQLConf(SQLConf.PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED.key -> v.toString) { + val df = sqlContext.read.parquet(dir.getCanonicalPath) + val rows = df.queryExecution.toRdd.map(_.copy()).collect() + val unsafeRows = rows.map(_.asInstanceOf[UnsafeRow]) + if (!v) { + hash1 = unsafeRows(0).hashCode() + hash2 = unsafeRows(1).hashCode() + } else { + assert(hash1 == unsafeRows(0).hashCode()) + assert(hash2 == unsafeRows(1).hashCode()) + } + } + } + } + } } class JobCommitFailureParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) From 8950482ee5e9132d11dc5b5d41132bb1fe1e7ba2 Mon Sep 17 00:00:00 2001 From: felixcheung Date: Tue, 5 Jan 2016 08:39:58 +0530 Subject: [PATCH 12/33] [SPARKR][DOC] minor doc update for version in migration guide checked that the change is in Spark 1.6.0. shivaram Author: felixcheung Closes #10574 from felixcheung/rwritemodedoc. (cherry picked from commit 8896ec9f02a6747917f3ae42a517ff0e3742eaf6) Signed-off-by: Shivaram Venkataraman --- docs/sparkr.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sparkr.md b/docs/sparkr.md index 9ddd2eda3fe8b..ea81532c611e2 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -385,12 +385,12 @@ The following functions are masked by the SparkR package: Since part of SparkR is modeled on the `dplyr` package, certain functions in SparkR share the same names with those in `dplyr`. Depending on the load order of the two packages, some functions from the package loaded first are masked by those in the package loaded after. In such case, prefix such calls with the package name, for instance, `SparkR::cume_dist(x)` or `dplyr::cume_dist(x)`. - + You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-manual/R-devel/library/base/html/search.html) # Migration Guide -## Upgrading From SparkR 1.6 to 1.7 +## Upgrading From SparkR 1.5.x to 1.6 - - Until Spark 1.6, the default mode for writes was `append`. It was changed in Spark 1.7 to `error` to match the Scala API. + - Before Spark 1.6, the default mode for writes was `append`. It was changed in Spark 1.6.0 to `error` to match the Scala API. From d9e4438b5c7b3569662a50973164955332463d05 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 4 Jan 2016 23:23:41 -0800 Subject: [PATCH 13/33] [SPARK-12568][SQL] Add BINARY to Encoders Author: Michael Armbrust Closes #10516 from marmbrus/datasetCleanup. (cherry picked from commit 53beddc5bf04a35ab73de99158919c2fdd5d4508) Signed-off-by: Michael Armbrust --- .../src/main/scala/org/apache/spark/sql/Encoder.scala | 6 ++++++ .../spark/sql/catalyst/encoders/ExpressionEncoder.scala | 9 +++++++++ .../sql/catalyst/encoders/ExpressionEncoderSuite.scala | 6 +++--- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala index bb0fdc4c3d83b..22b7e1ea0c4cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala @@ -157,6 +157,12 @@ object Encoders { */ def TIMESTAMP: Encoder[java.sql.Timestamp] = ExpressionEncoder() + /** + * An encoder for arrays of bytes. + * @since 1.6.1 + */ + def BINARY: Encoder[Array[Byte]] = ExpressionEncoder() + /** * Creates an encoder for Java Bean of type T. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 3e8420ecb9ccf..0f6dc2c2235f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -198,6 +198,15 @@ case class ExpressionEncoder[T]( @transient private lazy val constructProjection = GenerateSafeProjection.generate(fromRowExpression :: Nil) + /** + * Returns this encoder where it has been bound to its own output (i.e. no remaping of columns + * is performed). + */ + def defaultBinding: ExpressionEncoder[T] = { + val attrs = schema.toAttributes + resolve(attrs, OuterScopes.outerScopes).bind(attrs) + } + /** * Returns an encoded version of `t` as a Spark SQL row. Note that multiple calls to * toRow are allowed to return the same actual [[InternalRow]] object. Thus, the caller should diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index d6ca138672ef1..47b07df80879e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -77,6 +77,8 @@ class JavaSerializable(val value: Int) extends Serializable { } class ExpressionEncoderSuite extends SparkFunSuite { + OuterScopes.outerScopes.put(getClass.getName, this) + implicit def encoder[T : TypeTag]: ExpressionEncoder[T] = ExpressionEncoder() // test flat encoders @@ -274,8 +276,6 @@ class ExpressionEncoderSuite extends SparkFunSuite { } } - private val outers: ConcurrentMap[String, AnyRef] = new MapMaker().weakValues().makeMap() - outers.put(getClass.getName, this) private def encodeDecodeTest[T : ExpressionEncoder]( input: T, testName: String): Unit = { @@ -283,7 +283,7 @@ class ExpressionEncoderSuite extends SparkFunSuite { val encoder = implicitly[ExpressionEncoder[T]] val row = encoder.toRow(input) val schema = encoder.schema.toAttributes - val boundEncoder = encoder.resolve(schema, outers).bind(schema) + val boundEncoder = encoder.defaultBinding val convertedBack = try boundEncoder.fromRow(row) catch { case e: Exception => fail( From 5afa62b20090e763ba10d9939ec214a11466087b Mon Sep 17 00:00:00 2001 From: Pete Robbins Date: Tue, 5 Jan 2016 13:10:21 -0800 Subject: [PATCH 14/33] [SPARK-12647][SQL] Fix o.a.s.sqlexecution.ExchangeCoordinatorSuite.determining the number of reducers: aggregate operator change expected partition sizes Author: Pete Robbins Closes #10599 from robbinspg/branch-1.6. --- .../apache/spark/sql/execution/ExchangeCoordinatorSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 180050bdac00f..59bb35443b8c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -318,7 +318,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } } - withSQLContext(test, 1536, minNumPostShufflePartitions) + withSQLContext(test, 1700, minNumPostShufflePartitions) } test(s"determining the number of reducers: join operator$testNameNote") { From f31d0fd9ea12bfe94434671fbcfe3d0e06a4a97d Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 5 Jan 2016 13:10:46 -0800 Subject: [PATCH 15/33] [SPARK-12617] [PYSPARK] Clean up the leak sockets of Py4J This patch added Py4jCallbackConnectionCleaner to clean the leak sockets of Py4J every 30 seconds. This is a workaround before Py4J fixes the leak issue https://github.com/bartdag/py4j/issues/187 Author: Shixiong Zhu Closes #10579 from zsxwing/SPARK-12617. (cherry picked from commit 047a31bb1042867b20132b347b1e08feab4562eb) Signed-off-by: Davies Liu --- python/pyspark/context.py | 61 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 61 insertions(+) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 529d16b480399..5e4aeac330c5a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -54,6 +54,64 @@ } +class Py4jCallbackConnectionCleaner(object): + + """ + A cleaner to clean up callback connections that are not closed by Py4j. See SPARK-12617. + It will scan all callback connections every 30 seconds and close the dead connections. + """ + + def __init__(self, gateway): + self._gateway = gateway + self._stopped = False + self._timer = None + self._lock = RLock() + + def start(self): + if self._stopped: + return + + def clean_closed_connections(): + from py4j.java_gateway import quiet_close, quiet_shutdown + + callback_server = self._gateway._callback_server + with callback_server.lock: + try: + closed_connections = [] + for connection in callback_server.connections: + if not connection.isAlive(): + quiet_close(connection.input) + quiet_shutdown(connection.socket) + quiet_close(connection.socket) + closed_connections.append(connection) + + for closed_connection in closed_connections: + callback_server.connections.remove(closed_connection) + except Exception: + import traceback + traceback.print_exc() + + self._start_timer(clean_closed_connections) + + self._start_timer(clean_closed_connections) + + def _start_timer(self, f): + from threading import Timer + + with self._lock: + if not self._stopped: + self._timer = Timer(30.0, f) + self._timer.daemon = True + self._timer.start() + + def stop(self): + with self._lock: + self._stopped = True + if self._timer: + self._timer.cancel() + self._timer = None + + class SparkContext(object): """ @@ -68,6 +126,7 @@ class SparkContext(object): _active_spark_context = None _lock = RLock() _python_includes = None # zip and egg files that need to be added to PYTHONPATH + _py4j_cleaner = None PACKAGE_EXTENSIONS = ('.zip', '.egg', '.jar') @@ -244,6 +303,8 @@ def _ensure_initialized(cls, instance=None, gateway=None): if not SparkContext._gateway: SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm + _py4j_cleaner = Py4jCallbackConnectionCleaner(SparkContext._gateway) + _py4j_cleaner.start() if instance: if (SparkContext._active_spark_context and From 83fe5cf9a2621d7e53b5792a7c7549c9da7f130a Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 5 Jan 2016 13:48:47 -0800 Subject: [PATCH 16/33] [SPARK-12511] [PYSPARK] [STREAMING] Make sure PythonDStream.registerSerializer is called only once There is an issue that Py4J's PythonProxyHandler.finalize blocks forever. (https://github.com/bartdag/py4j/pull/184) Py4j will create a PythonProxyHandler in Java for "transformer_serializer" when calling "registerSerializer". If we call "registerSerializer" twice, the second PythonProxyHandler will override the first one, then the first one will be GCed and trigger "PythonProxyHandler.finalize". To avoid that, we should not call"registerSerializer" more than once, so that "PythonProxyHandler" in Java side won't be GCed. Author: Shixiong Zhu Closes #10514 from zsxwing/SPARK-12511. (cherry picked from commit 6cfe341ee89baa952929e91d33b9ecbca73a3ea0) Signed-off-by: Davies Liu --- python/pyspark/streaming/context.py | 33 ++++++++++++++----- python/pyspark/streaming/util.py | 3 +- .../spark/streaming/StreamingContext.scala | 12 +++++++ 3 files changed, 38 insertions(+), 10 deletions(-) diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 1388b6d044e04..06346e5fa2325 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -98,8 +98,28 @@ def _ensure_initialized(cls): # register serializer for TransformFunction # it happens before creating SparkContext when loading from checkpointing - cls._transformerSerializer = TransformFunctionSerializer( - SparkContext._active_spark_context, CloudPickleSerializer(), gw) + if cls._transformerSerializer is None: + transformer_serializer = TransformFunctionSerializer() + transformer_serializer.init( + SparkContext._active_spark_context, CloudPickleSerializer(), gw) + # SPARK-12511 streaming driver with checkpointing unable to finalize leading to OOM + # There is an issue that Py4J's PythonProxyHandler.finalize blocks forever. + # (https://github.com/bartdag/py4j/pull/184) + # + # Py4j will create a PythonProxyHandler in Java for "transformer_serializer" when + # calling "registerSerializer". If we call "registerSerializer" twice, the second + # PythonProxyHandler will override the first one, then the first one will be GCed and + # trigger "PythonProxyHandler.finalize". To avoid that, we should not call + # "registerSerializer" more than once, so that "PythonProxyHandler" in Java side won't + # be GCed. + # + # TODO Once Py4J fixes this issue, we should upgrade Py4j to the latest version. + transformer_serializer.gateway.jvm.PythonDStream.registerSerializer( + transformer_serializer) + cls._transformerSerializer = transformer_serializer + else: + cls._transformerSerializer.init( + SparkContext._active_spark_context, CloudPickleSerializer(), gw) @classmethod def getOrCreate(cls, checkpointPath, setupFunc): @@ -116,16 +136,13 @@ def getOrCreate(cls, checkpointPath, setupFunc): gw = SparkContext._gateway # Check whether valid checkpoint information exists in the given path - if gw.jvm.CheckpointReader.read(checkpointPath).isEmpty(): + ssc_option = gw.jvm.StreamingContextPythonHelper().tryRecoverFromCheckpoint(checkpointPath) + if ssc_option.isEmpty(): ssc = setupFunc() ssc.checkpoint(checkpointPath) return ssc - try: - jssc = gw.jvm.JavaStreamingContext(checkpointPath) - except Exception: - print("failed to load StreamingContext from checkpoint", file=sys.stderr) - raise + jssc = gw.jvm.JavaStreamingContext(ssc_option.get()) # If there is already an active instance of Python SparkContext use it, or create a new one if not SparkContext._active_spark_context: diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py index abbbf6eb9394f..e617fc9ce9eec 100644 --- a/python/pyspark/streaming/util.py +++ b/python/pyspark/streaming/util.py @@ -89,11 +89,10 @@ class TransformFunctionSerializer(object): it uses this class to invoke Python, which returns the serialized function as a byte array. """ - def __init__(self, ctx, serializer, gateway=None): + def init(self, ctx, serializer, gateway=None): self.ctx = ctx self.serializer = serializer self.gateway = gateway or self.ctx._gateway - self.gateway.jvm.PythonDStream.registerSerializer(self) self.failure = None def dumps(self, id): diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 53324e79a04e6..d2dd3ad6d5133 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -897,3 +897,15 @@ object StreamingContext extends Logging { result } } + +private class StreamingContextPythonHelper { + + /** + * This is a private method only for Python to implement `getOrCreate`. + */ + def tryRecoverFromCheckpoint(checkpointPath: String): Option[StreamingContext] = { + val checkpointOption = CheckpointReader.read( + checkpointPath, new SparkConf(), SparkHadoopUtil.get.conf, false) + checkpointOption.map(new StreamingContext(null, _, null)) + } +} From 0afad6678431846a6eebda8d5891da9115884915 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Tue, 5 Jan 2016 15:05:04 -0800 Subject: [PATCH 17/33] [SPARK-12450][MLLIB] Un-persist broadcasted variables in KMeans SPARK-12450 . Un-persist broadcasted variables in KMeans. Author: RJ Nowling Closes #10415 from rnowling/spark-12450. (cherry picked from commit 78015a8b7cc316343e302eeed6fe30af9f2961e8) Signed-off-by: Joseph K. Bradley --- .../scala/org/apache/spark/mllib/clustering/KMeans.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 2895db7c9061b..e47c4db62955d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -301,6 +301,8 @@ class KMeans private ( contribs.iterator }.reduceByKey(mergeContribs).collectAsMap() + bcActiveCenters.unpersist(blocking = false) + // Update the cluster centers and costs for each active run for ((run, i) <- activeRuns.zipWithIndex) { var changed = false @@ -419,7 +421,10 @@ class KMeans private ( s0 } ) + + bcNewCenters.unpersist(blocking = false) preCosts.unpersist(blocking = false) + val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointsWithCosts) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) pointsWithCosts.flatMap { case (p, c) => @@ -448,6 +453,9 @@ class KMeans private ( ((r, KMeans.findClosest(bcCenters.value(r), p)._1), 1.0) } }.reduceByKey(_ + _).collectAsMap() + + bcCenters.unpersist(blocking = false) + val finalCenters = (0 until runs).par.map { r => val myCenters = centers(r).toArray val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray From bf3dca2df4dd3be264691be1321e0c700d4f4e32 Mon Sep 17 00:00:00 2001 From: BrianLondon Date: Tue, 5 Jan 2016 23:15:07 +0000 Subject: [PATCH 18/33] [SPARK-12453][STREAMING] Remove explicit dependency on aws-java-sdk Successfully ran kinesis demo on a live, aws hosted kinesis stream against master and 1.6 branches. For reasons I don't entirely understand it required a manual merge to 1.5 which I did as shown here: https://github.com/BrianLondon/spark/commit/075c22e89bc99d5e99be21f40e0d72154a1e23a2 The demo ran successfully on the 1.5 branch as well. According to `mvn dependency:tree` it is still pulling a fairly old version of the aws-java-sdk (1.9.37), but this appears to have fixed the kinesis regression in 1.5.2. Author: BrianLondon Closes #10492 from BrianLondon/remove-only. (cherry picked from commit ff89975543b153d0d235c0cac615d45b34aa8fe7) Signed-off-by: Sean Owen --- extras/kinesis-asl/pom.xml | 5 ----- .../org/apache/spark/streaming/kinesis/KinesisReceiver.scala | 1 + pom.xml | 1 - 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 519a920279c97..eeba869ca5808 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -59,11 +59,6 @@ amazon-kinesis-client ${aws.kinesis.client.version} - - com.amazonaws - aws-java-sdk - ${aws.java.sdk.version} - com.amazonaws amazon-kinesis-producer diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 80edda59e1719..abb9b6cd32f1c 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -185,6 +185,7 @@ private[kinesis] class KinesisReceiver[T]( workerThread.setName(s"Kinesis Receiver ${streamId}") workerThread.setDaemon(true) workerThread.start() + logInfo(s"Started receiver with workerId $workerId") } diff --git a/pom.xml b/pom.xml index 4050b43d68561..6104c9d7d45d7 100644 --- a/pom.xml +++ b/pom.xml @@ -153,7 +153,6 @@ 1.7.7 hadoop2 0.7.1 - 1.9.40 1.4.0 0.10.1 From c3135d02176cdd679b4a0e4883895b9e9f001a55 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 6 Jan 2016 12:05:41 +0530 Subject: [PATCH 19/33] [SPARK-12393][SPARKR] Add read.text and write.text for SparkR Add ```read.text``` and ```write.text``` for SparkR. cc sun-rui felixcheung shivaram Author: Yanbo Liang Closes #10348 from yanboliang/spark-12393. (cherry picked from commit d1fea41363c175a67b97cb7b3fe89f9043708739) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 4 +++- R/pkg/R/DataFrame.R | 28 +++++++++++++++++++++++ R/pkg/R/SQLContext.R | 26 +++++++++++++++++++++ R/pkg/R/generics.R | 4 ++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 21 +++++++++++++++++ 5 files changed, 82 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index ccc01fe169601..beacc39500aaa 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -94,7 +94,8 @@ exportMethods("arrange", "withColumnRenamed", "write.df", "write.json", - "write.parquet") + "write.parquet", + "write.text") exportClasses("Column") @@ -274,6 +275,7 @@ export("as.DataFrame", "parquetFile", "read.df", "read.parquet", + "read.text", "sql", "table", "tableNames", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 0cfa12b997d69..7a7aef27ccb24 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -661,6 +661,34 @@ setMethod("saveAsParquetFile", write.parquet(x, path) }) +#' 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". +#' Each row becomes a new line in the output file. +#' +#' @param x A SparkSQL DataFrame +#' @param path The directory where the file is saved +#' +#' @family DataFrame functions +#' @rdname write.text +#' @name write.text +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRSQL.init(sc) +#' path <- "path/to/file.txt" +#' df <- read.text(sqlContext, path) +#' write.text(df, "/tmp/sparkr-tmp/") +#'} +setMethod("write.text", + signature(x = "DataFrame", path = "character"), + function(x, path) { + write <- callJMethod(x@sdf, "write") + invisible(callJMethod(write, "text", path)) + }) + #' Distinct #' #' Return a new DataFrame containing the distinct rows in this DataFrame. diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 9243d70e66f75..443666164b2b4 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -295,6 +295,32 @@ parquetFile <- function(sqlContext, ...) { dataFrame(sdf) } +#' Create a DataFrame 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. +#' +#' @param sqlContext SQLContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @return DataFrame +#' @rdname read.text +#' @name read.text +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRSQL.init(sc) +#' path <- "path/to/file.txt" +#' df <- read.text(sqlContext, path) +#' } +read.text <- function(sqlContext, path) { + # Allow the user to have a more flexible definiton of the text file path + paths <- as.list(suppressWarnings(normalizePath(path))) + read <- callJMethod(sqlContext, "read") + sdf <- callJMethod(read, "text", paths) + dataFrame(sdf) +} + #' SQL Query #' #' Executes a SQL query using Spark, returning the result as a DataFrame. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 62be2ddc8f522..ba6861709754d 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -549,6 +549,10 @@ setGeneric("write.parquet", function(x, path) { standardGeneric("write.parquet") #' @export setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") }) +#' @rdname write.text +#' @export +setGeneric("write.text", function(x, path) { standardGeneric("write.text") }) + #' @rdname schema #' @export setGeneric("schema", function(x) { standardGeneric("schema") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index ee942d25ba4b5..73f311e2684a2 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1493,6 +1493,27 @@ test_that("read/write Parquet files", { unlink(parquetPath4) }) +test_that("read/write text files", { + # Test write.df and read.df + df <- read.df(sqlContext, jsonPath, "text") + expect_is(df, "DataFrame") + expect_equal(colnames(df), c("value")) + expect_equal(count(df), 3) + textPath <- tempfile(pattern = "textPath", fileext = ".txt") + write.df(df, textPath, "text", mode="overwrite") + + # Test write.text and read.text + textPath2 <- tempfile(pattern = "textPath2", fileext = ".txt") + write.text(df, textPath2) + df2 <- read.text(sqlContext, c(textPath, textPath2)) + expect_is(df2, "DataFrame") + expect_equal(colnames(df2), c("value")) + expect_equal(count(df2), count(df) * 2) + + unlink(textPath) + unlink(textPath2) +}) + test_that("describe() and summarize() on a DataFrame", { df <- read.json(sqlContext, jsonPath) stats <- describe(df, "age") From 175681914af953b7ce1b2971fef83a2445de1f94 Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 6 Jan 2016 11:58:33 -0800 Subject: [PATCH 20/33] [SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None If initial model passed to GMM is not empty it causes `net.razorvine.pickle.PickleException`. It can be fixed by converting `initialModel.weights` to `list`. Author: zero323 Closes #9986 from zero323/SPARK-12006. (cherry picked from commit fcd013cf70e7890aa25a8fe3cb6c8b36bf0e1f04) Signed-off-by: Joseph K. Bradley --- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/tests.py | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index c9e6f1dec6bf8..48daa87e82d13 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -346,7 +346,7 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia if initialModel.k != k: raise Exception("Mismatched cluster count, initialModel.k = %s, however k = %s" % (initialModel.k, k)) - initialModelWeights = initialModel.weights + initialModelWeights = list(initialModel.weights) initialModelMu = [initialModel.gaussians[i].mu for i in range(initialModel.k)] initialModelSigma = [initialModel.gaussians[i].sigma for i in range(initialModel.k)] java_model = callMLlibFunc("trainGaussianMixtureModel", rdd.map(_convert_to_vector), diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index f8e8e0e0adbea..e9e7a9057afe9 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -474,6 +474,18 @@ def test_gmm_deterministic(self): for c1, c2 in zip(clusters1.weights, clusters2.weights): self.assertEqual(round(c1, 7), round(c2, 7)) + def test_gmm_with_initial_model(self): + from pyspark.mllib.clustering import GaussianMixture + data = self.sc.parallelize([ + (-10, -5), (-9, -4), (10, 5), (9, 4) + ]) + + gmm1 = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=10, seed=63) + gmm2 = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=10, seed=63, initialModel=gmm1) + self.assertAlmostEqual((gmm1.weights - gmm2.weights).sum(), 0.0) + def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes from pyspark.mllib.tree import DecisionTree, DecisionTreeModel, RandomForest,\ From d821fae0ecca6393d3632977797d72ba594d26a9 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 6 Jan 2016 12:03:01 -0800 Subject: [PATCH 21/33] [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to Streaming Move Py4jCallbackConnectionCleaner to Streaming because the callback server starts only in StreamingContext. Author: Shixiong Zhu Closes #10621 from zsxwing/SPARK-12617-2. (cherry picked from commit 1e6648d62fb82b708ea54c51cd23bfe4f542856e) Signed-off-by: Shixiong Zhu --- python/pyspark/context.py | 61 ---------------------------- python/pyspark/streaming/context.py | 63 +++++++++++++++++++++++++++++ 2 files changed, 63 insertions(+), 61 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5e4aeac330c5a..529d16b480399 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -54,64 +54,6 @@ } -class Py4jCallbackConnectionCleaner(object): - - """ - A cleaner to clean up callback connections that are not closed by Py4j. See SPARK-12617. - It will scan all callback connections every 30 seconds and close the dead connections. - """ - - def __init__(self, gateway): - self._gateway = gateway - self._stopped = False - self._timer = None - self._lock = RLock() - - def start(self): - if self._stopped: - return - - def clean_closed_connections(): - from py4j.java_gateway import quiet_close, quiet_shutdown - - callback_server = self._gateway._callback_server - with callback_server.lock: - try: - closed_connections = [] - for connection in callback_server.connections: - if not connection.isAlive(): - quiet_close(connection.input) - quiet_shutdown(connection.socket) - quiet_close(connection.socket) - closed_connections.append(connection) - - for closed_connection in closed_connections: - callback_server.connections.remove(closed_connection) - except Exception: - import traceback - traceback.print_exc() - - self._start_timer(clean_closed_connections) - - self._start_timer(clean_closed_connections) - - def _start_timer(self, f): - from threading import Timer - - with self._lock: - if not self._stopped: - self._timer = Timer(30.0, f) - self._timer.daemon = True - self._timer.start() - - def stop(self): - with self._lock: - self._stopped = True - if self._timer: - self._timer.cancel() - self._timer = None - - class SparkContext(object): """ @@ -126,7 +68,6 @@ class SparkContext(object): _active_spark_context = None _lock = RLock() _python_includes = None # zip and egg files that need to be added to PYTHONPATH - _py4j_cleaner = None PACKAGE_EXTENSIONS = ('.zip', '.egg', '.jar') @@ -303,8 +244,6 @@ def _ensure_initialized(cls, instance=None, gateway=None): if not SparkContext._gateway: SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm - _py4j_cleaner = Py4jCallbackConnectionCleaner(SparkContext._gateway) - _py4j_cleaner.start() if instance: if (SparkContext._active_spark_context and diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 06346e5fa2325..874cb3feb8c98 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -19,6 +19,7 @@ import os import sys +from threading import RLock, Timer from py4j.java_gateway import java_import, JavaObject @@ -32,6 +33,63 @@ __all__ = ["StreamingContext"] +class Py4jCallbackConnectionCleaner(object): + + """ + A cleaner to clean up callback connections that are not closed by Py4j. See SPARK-12617. + It will scan all callback connections every 30 seconds and close the dead connections. + """ + + def __init__(self, gateway): + self._gateway = gateway + self._stopped = False + self._timer = None + self._lock = RLock() + + def start(self): + if self._stopped: + return + + def clean_closed_connections(): + from py4j.java_gateway import quiet_close, quiet_shutdown + + callback_server = self._gateway._callback_server + if callback_server: + with callback_server.lock: + try: + closed_connections = [] + for connection in callback_server.connections: + if not connection.isAlive(): + quiet_close(connection.input) + quiet_shutdown(connection.socket) + quiet_close(connection.socket) + closed_connections.append(connection) + + for closed_connection in closed_connections: + callback_server.connections.remove(closed_connection) + except Exception: + import traceback + traceback.print_exc() + + self._start_timer(clean_closed_connections) + + self._start_timer(clean_closed_connections) + + def _start_timer(self, f): + with self._lock: + if not self._stopped: + self._timer = Timer(30.0, f) + self._timer.daemon = True + self._timer.start() + + def stop(self): + with self._lock: + self._stopped = True + if self._timer: + self._timer.cancel() + self._timer = None + + class StreamingContext(object): """ Main entry point for Spark Streaming functionality. A StreamingContext @@ -47,6 +105,9 @@ class StreamingContext(object): # Reference to a currently active StreamingContext _activeContext = None + # A cleaner to clean leak sockets of callback server every 30 seconds + _py4j_cleaner = None + def __init__(self, sparkContext, batchDuration=None, jssc=None): """ Create a new StreamingContext. @@ -95,6 +156,8 @@ def _ensure_initialized(cls): jgws = JavaObject("GATEWAY_SERVER", gw._gateway_client) # update the port of CallbackClient with real port gw.jvm.PythonDStream.updatePythonGatewayPort(jgws, gw._python_proxy_port) + _py4j_cleaner = Py4jCallbackConnectionCleaner(gw) + _py4j_cleaner.start() # register serializer for TransformFunction # it happens before creating SparkContext when loading from checkpointing From 8f0ead3e79beb2c5f2731ceaa34fe1c133763386 Mon Sep 17 00:00:00 2001 From: huangzhaowei Date: Wed, 6 Jan 2016 12:48:57 -0800 Subject: [PATCH 22/33] [SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url. Author: huangzhaowei Closes #10617 from SaintBacchus/SPARK-12672. --- .../org/apache/spark/streaming/scheduler/JobScheduler.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 1ed6fb0aa9d52..2c57706636fa5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -26,7 +26,8 @@ import org.apache.spark.Logging import org.apache.spark.rdd.PairRDDFunctions import org.apache.spark.streaming._ import org.apache.spark.streaming.ui.UIUtils -import org.apache.spark.util.{EventLoop, ThreadUtils, Utils} +import org.apache.spark.ui.{UIUtils => SparkUIUtils} +import org.apache.spark.util.{EventLoop, ThreadUtils} private[scheduler] sealed trait JobSchedulerEvent @@ -203,7 +204,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { try { val formattedTime = UIUtils.formatBatchTime( job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) - val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" + val batchUrl = s"${SparkUIUtils.uiRoot}/streaming/batch/?id=${job.time.milliseconds}" val batchLinkText = s"[output operation ${job.outputOpId}, batch time ${formattedTime}]" ssc.sc.setJobDescription( From 39b0a348008b6ab532768b90fd578b77711af98c Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 6 Jan 2016 13:53:25 -0800 Subject: [PATCH 23/33] Revert "[SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url." This reverts commit 8f0ead3e79beb2c5f2731ceaa34fe1c133763386. Will merge #10618 instead. --- .../org/apache/spark/streaming/scheduler/JobScheduler.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 2c57706636fa5..1ed6fb0aa9d52 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -26,8 +26,7 @@ import org.apache.spark.Logging import org.apache.spark.rdd.PairRDDFunctions import org.apache.spark.streaming._ import org.apache.spark.streaming.ui.UIUtils -import org.apache.spark.ui.{UIUtils => SparkUIUtils} -import org.apache.spark.util.{EventLoop, ThreadUtils} +import org.apache.spark.util.{EventLoop, ThreadUtils, Utils} private[scheduler] sealed trait JobSchedulerEvent @@ -204,7 +203,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { try { val formattedTime = UIUtils.formatBatchTime( job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) - val batchUrl = s"${SparkUIUtils.uiRoot}/streaming/batch/?id=${job.time.milliseconds}" + val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" val batchLinkText = s"[output operation ${job.outputOpId}, batch time ${formattedTime}]" ssc.sc.setJobDescription( From 11b901b22b1cdaa6d19b1b73885627ac601be275 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 14 Dec 2015 09:59:42 -0800 Subject: [PATCH 24/33] [SPARK-12016] [MLLIB] [PYSPARK] Wrap Word2VecModel when loading it in pyspark JIRA: https://issues.apache.org/jira/browse/SPARK-12016 We should not directly use Word2VecModel in pyspark. We need to wrap it in a Word2VecModelWrapper when loading it in pyspark. Author: Liang-Chi Hsieh Closes #10100 from viirya/fix-load-py-wordvecmodel. (cherry picked from commit b51a4cdff3a7e640a8a66f7a9c17021f3056fd34) Signed-off-by: Joseph K. Bradley --- .../mllib/api/python/PythonMLLibAPI.scala | 33 ---------- .../api/python/Word2VecModelWrapper.scala | 62 +++++++++++++++++++ python/pyspark/mllib/feature.py | 6 +- 3 files changed, 67 insertions(+), 34 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 8d546e3d6099b..29160a10e16b3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -680,39 +680,6 @@ private[python] class PythonMLLibAPI extends Serializable { } } - private[python] class Word2VecModelWrapper(model: Word2VecModel) { - def transform(word: String): Vector = { - model.transform(word) - } - - /** - * Transforms an RDD of words to its vector representation - * @param rdd an RDD of words - * @return an RDD of vector representations of words - */ - def transform(rdd: JavaRDD[String]): JavaRDD[Vector] = { - rdd.rdd.map(model.transform) - } - - def findSynonyms(word: String, num: Int): JList[Object] = { - val vec = transform(word) - findSynonyms(vec, num) - } - - def findSynonyms(vector: Vector, num: Int): JList[Object] = { - val result = model.findSynonyms(vector, num) - val similarity = Vectors.dense(result.map(_._2)) - val words = result.map(_._1) - List(words, similarity).map(_.asInstanceOf[Object]).asJava - } - - def getVectors: JMap[String, JList[Float]] = { - model.getVectors.map({case (k, v) => (k, v.toList.asJava)}).asJava - } - - def save(sc: SparkContext, path: String): Unit = model.save(sc, path) - } - /** * Java stub for Python mllib DecisionTree.train(). * This stub returns a handle to the Java object instead of the content of the Java object. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala new file mode 100644 index 0000000000000..0f55980481dcb --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.api.python + +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkContext +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.feature.Word2VecModel +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +/** + * Wrapper around Word2VecModel to provide helper methods in Python + */ +private[python] class Word2VecModelWrapper(model: Word2VecModel) { + def transform(word: String): Vector = { + model.transform(word) + } + + /** + * Transforms an RDD of words to its vector representation + * @param rdd an RDD of words + * @return an RDD of vector representations of words + */ + def transform(rdd: JavaRDD[String]): JavaRDD[Vector] = { + rdd.rdd.map(model.transform) + } + + def findSynonyms(word: String, num: Int): JList[Object] = { + val vec = transform(word) + findSynonyms(vec, num) + } + + def findSynonyms(vector: Vector, num: Int): JList[Object] = { + val result = model.findSynonyms(vector, num) + val similarity = Vectors.dense(result.map(_._2)) + val words = result.map(_._1) + List(words, similarity).map(_.asInstanceOf[Object]).asJava + } + + def getVectors: JMap[String, JList[Float]] = { + model.getVectors.map({case (k, v) => (k, v.toList.asJava)}).asJava + } + + def save(sc: SparkContext, path: String): Unit = model.save(sc, path) +} diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index c14095de0b5a0..acd7ec57d69da 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -502,7 +502,8 @@ def load(cls, sc, path): """ jmodel = sc._jvm.org.apache.spark.mllib.feature \ .Word2VecModel.load(sc._jsc.sc(), path) - return Word2VecModel(jmodel) + model = sc._jvm.Word2VecModelWrapper(jmodel) + return Word2VecModel(model) @ignore_unicode_prefix @@ -544,6 +545,9 @@ class Word2Vec(object): >>> sameModel = Word2VecModel.load(sc, path) >>> model.transform("a") == sameModel.transform("a") True + >>> syms = sameModel.findSynonyms("a", 2) + >>> [s[0] for s in syms] + [u'b', u'c'] >>> from shutil import rmtree >>> try: ... rmtree(path) From 94af69c9be70b9d2cd95c26288e2af9599d61e5c Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 6 Jan 2016 21:28:29 -0800 Subject: [PATCH 25/33] [SPARK-12673][UI] Add missing uri prepending for job description Otherwise the url will be failed to proxy to the right one if in YARN mode. Here is the screenshot: ![screen shot 2016-01-06 at 5 28 26 pm](https://cloud.githubusercontent.com/assets/850797/12139632/bbe78ecc-b49c-11e5-8932-94e8b3622a09.png) Author: jerryshao Closes #10618 from jerryshao/SPARK-12673. (cherry picked from commit 174e72ceca41a6ac17ad05d50832ee9c561918c0) Signed-off-by: Shixiong Zhu --- .../main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index d467dd9e1f29d..e65ab2c02c969 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -224,10 +224,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") - val jobDescription = UIUtils.makeDescription(lastStageDescription, parent.basePath) + val basePathUri = UIUtils.prependBaseUri(parent.basePath) + val jobDescription = UIUtils.makeDescription(lastStageDescription, basePathUri) - val detailUrl = - "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) + val detailUrl = "%s/jobs/job?id=%s".format(basePathUri, job.jobId) {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} From d061b852274c12784f3feb96c0cdcab39989f8e7 Mon Sep 17 00:00:00 2001 From: Guillaume Poulin Date: Wed, 6 Jan 2016 21:34:46 -0800 Subject: [PATCH 26/33] [SPARK-12678][CORE] MapPartitionsRDD clearDependencies MapPartitionsRDD was keeping a reference to `prev` after a call to `clearDependencies` which could lead to memory leak. Author: Guillaume Poulin Closes #10623 from gpoulin/map_partition_deps. (cherry picked from commit b6738520374637347ab5ae6c801730cdb6b35daa) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 4312d3a417759..e4587c96eae1c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -25,7 +25,7 @@ import org.apache.spark.{Partition, TaskContext} * An RDD that applies the provided function to every partition of the parent RDD. */ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( - prev: RDD[T], + var prev: RDD[T], f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator) preservesPartitioning: Boolean = false) extends RDD[U](prev) { @@ -36,4 +36,9 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[U] = f(context, split.index, firstParent[T].iterator(split, context)) + + override def clearDependencies() { + super.clearDependencies() + prev = null + } } From 34effc46cd54735cc660d8b43f0a190e91747a06 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 6 Jan 2016 22:03:31 -0800 Subject: [PATCH 27/33] Revert "[SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None" This reverts commit fcd013cf70e7890aa25a8fe3cb6c8b36bf0e1f04. Author: Yin Huai Closes #10632 from yhuai/pythonStyle. (cherry picked from commit e5cde7ab11a43334fa01b1bb8904da5c0774bc62) Signed-off-by: Yin Huai --- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/tests.py | 12 ------------ 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 48daa87e82d13..c9e6f1dec6bf8 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -346,7 +346,7 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia if initialModel.k != k: raise Exception("Mismatched cluster count, initialModel.k = %s, however k = %s" % (initialModel.k, k)) - initialModelWeights = list(initialModel.weights) + initialModelWeights = initialModel.weights initialModelMu = [initialModel.gaussians[i].mu for i in range(initialModel.k)] initialModelSigma = [initialModel.gaussians[i].sigma for i in range(initialModel.k)] java_model = callMLlibFunc("trainGaussianMixtureModel", rdd.map(_convert_to_vector), diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index e9e7a9057afe9..f8e8e0e0adbea 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -474,18 +474,6 @@ def test_gmm_deterministic(self): for c1, c2 in zip(clusters1.weights, clusters2.weights): self.assertEqual(round(c1, 7), round(c2, 7)) - def test_gmm_with_initial_model(self): - from pyspark.mllib.clustering import GaussianMixture - data = self.sc.parallelize([ - (-10, -5), (-9, -4), (10, 5), (9, 4) - ]) - - gmm1 = GaussianMixture.train(data, 2, convergenceTol=0.001, - maxIterations=10, seed=63) - gmm2 = GaussianMixture.train(data, 2, convergenceTol=0.001, - maxIterations=10, seed=63, initialModel=gmm1) - self.assertAlmostEqual((gmm1.weights - gmm2.weights).sum(), 0.0) - def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes from pyspark.mllib.tree import DecisionTree, DecisionTreeModel, RandomForest,\ From 47a58c799206d011587e03178a259974be47d3bc Mon Sep 17 00:00:00 2001 From: zzcclp Date: Wed, 6 Jan 2016 23:06:21 -0800 Subject: [PATCH 28/33] [DOC] fix 'spark.memory.offHeap.enabled' default value to false modify 'spark.memory.offHeap.enabled' default value to false Author: zzcclp Closes #10633 from zzcclp/fix_spark.memory.offHeap.enabled_default_value. (cherry picked from commit 84e77a15df18ba3f1cc871a3c52c783b46e52369) Signed-off-by: Reynold Xin --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 488dc52ed61ed..995af6a555cb5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -740,7 +740,7 @@ Apart from these, the following properties are also available, and may be useful spark.memory.offHeap.enabled - true + false If true, Spark will attempt to use off-heap memory for certain operations. If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive. From 69a885a71cfe7c62179e784e7d9eee023d3bb6eb Mon Sep 17 00:00:00 2001 From: zero323 Date: Thu, 7 Jan 2016 10:32:56 -0800 Subject: [PATCH 29/33] [SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None If initial model passed to GMM is not empty it causes net.razorvine.pickle.PickleException. It can be fixed by converting initialModel.weights to list. Author: zero323 Closes #10644 from zero323/SPARK-12006. (cherry picked from commit 592f64985d0d58b4f6a0366bf975e04ca496bdbe) Signed-off-by: Joseph K. Bradley --- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/tests.py | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index c9e6f1dec6bf8..48daa87e82d13 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -346,7 +346,7 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia if initialModel.k != k: raise Exception("Mismatched cluster count, initialModel.k = %s, however k = %s" % (initialModel.k, k)) - initialModelWeights = initialModel.weights + initialModelWeights = list(initialModel.weights) initialModelMu = [initialModel.gaussians[i].mu for i in range(initialModel.k)] initialModelSigma = [initialModel.gaussians[i].sigma for i in range(initialModel.k)] java_model = callMLlibFunc("trainGaussianMixtureModel", rdd.map(_convert_to_vector), diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index f8e8e0e0adbea..26db87ea546a0 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -474,6 +474,18 @@ def test_gmm_deterministic(self): for c1, c2 in zip(clusters1.weights, clusters2.weights): self.assertEqual(round(c1, 7), round(c2, 7)) + def test_gmm_with_initial_model(self): + from pyspark.mllib.clustering import GaussianMixture + data = self.sc.parallelize([ + (-10, -5), (-9, -4), (10, 5), (9, 4) + ]) + + gmm1 = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=10, seed=63) + gmm2 = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=10, seed=63, initialModel=gmm1) + self.assertAlmostEqual((gmm1.weights - gmm2.weights).sum(), 0.0) + def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes from pyspark.mllib.tree import DecisionTree, DecisionTreeModel, RandomForest,\ From 017b73e69693cd151516f92640a95a4a66e02dff Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Thu, 7 Jan 2016 10:37:15 -0800 Subject: [PATCH 30/33] [SPARK-12662][SQL] Fix DataFrame.randomSplit to avoid creating overlapping splits https://issues.apache.org/jira/browse/SPARK-12662 cc yhuai Author: Sameer Agarwal Closes #10626 from sameeragarwal/randomsplit. (cherry picked from commit f194d9911a93fc3a78be820096d4836f22d09976) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/DataFrame.scala | 7 +++++- .../apache/spark/sql/DataFrameStatSuite.scala | 22 +++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 74f937080f176..318004947ef1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1107,10 +1107,15 @@ class DataFrame private[sql]( * @since 1.4.0 */ def randomSplit(weights: Array[Double], seed: Long): Array[DataFrame] = { + // It is possible that the underlying dataframe doesn't guarantee the ordering of rows in its + // constituent partitions each time a split is materialized which could result in + // overlapping splits. To prevent this, we explicitly sort each input partition to make the + // ordering deterministic. + val sorted = Sort(logicalPlan.output.map(SortOrder(_, Ascending)), global = false, logicalPlan) val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => - new DataFrame(sqlContext, Sample(x(0), x(1), withReplacement = false, seed, logicalPlan)) + new DataFrame(sqlContext, Sample(x(0), x(1), withReplacement = false, seed, sorted)) }.toArray } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index b15af42caa3ab..63ad6c439a870 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -62,6 +62,28 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { } } + test("randomSplit on reordered partitions") { + // This test ensures that randomSplit does not create overlapping splits even when the + // underlying dataframe (such as the one below) doesn't guarantee a deterministic ordering of + // rows in each partition. + val data = + sparkContext.parallelize(1 to 600, 2).mapPartitions(scala.util.Random.shuffle(_)).toDF("id") + val splits = data.randomSplit(Array[Double](2, 3), seed = 1) + + assert(splits.length == 2, "wrong number of splits") + + // Verify that the splits span the entire dataset + assert(splits.flatMap(_.collect()).toSet == data.collect().toSet) + + // Verify that the splits don't overalap + assert(splits(0).intersect(splits(1)).collect().isEmpty) + + // Verify that the results are deterministic across multiple runs + val firstRun = splits.toSeq.map(_.collect().toSeq) + val secondRun = data.randomSplit(Array[Double](2, 3), seed = 1).toSeq.map(_.collect().toSeq) + assert(firstRun == secondRun) + } + test("pearson correlation") { val df = Seq.tabulate(10)(i => (i, 2 * i, i * -1.0)).toDF("a", "b", "c") val corr1 = df.stat.corr("a", "b", "pearson") From 6ef823544dfbc8c9843bdedccfda06147a1a74fe Mon Sep 17 00:00:00 2001 From: Darek Blasiak Date: Thu, 7 Jan 2016 21:15:40 +0000 Subject: [PATCH 31/33] [SPARK-12598][CORE] bug in setMinPartitions There is a bug in the calculation of ```maxSplitSize```. The ```totalLen``` should be divided by ```minPartitions``` and not by ```files.size```. Author: Darek Blasiak Closes #10546 from datafarmer/setminpartitionsbug. (cherry picked from commit 8346518357f4a3565ae41e9a5ccd7e2c3ed6c468) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/input/PortableDataStream.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 280e7a5fe893c..8e97584e898ff 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -43,9 +43,8 @@ private[spark] abstract class StreamFileInputFormat[T] * which is set through setMaxSplitSize */ def setMinPartitions(context: JobContext, minPartitions: Int) { - val files = listStatus(context).asScala - val totalLen = files.map(file => if (file.isDir) 0L else file.getLen).sum - val maxSplitSize = Math.ceil(totalLen * 1.0 / files.size).toLong + val totalLen = listStatus(context).asScala.filterNot(_.isDir).map(_.getLen).sum + val maxSplitSize = math.ceil(totalLen / math.max(minPartitions, 1.0)).toLong super.setMaxSplitSize(maxSplitSize) } From a7c36362fb9532183b7b6a0ad5020f02b816a9b3 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 7 Jan 2016 17:37:46 -0800 Subject: [PATCH 32/33] [SPARK-12507][STREAMING][DOCUMENT] Expose closeFileAfterWrite and allowBatching configurations for Streaming /cc tdas brkyvz Author: Shixiong Zhu Closes #10453 from zsxwing/streaming-conf. (cherry picked from commit c94199e977279d9b4658297e8108b46bdf30157b) Signed-off-by: Tathagata Das --- docs/configuration.md | 18 ++++++++++++++++++ docs/streaming-programming-guide.md | 12 +++++------- 2 files changed, 23 insertions(+), 7 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 995af6a555cb5..ee7101ba3df66 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1596,6 +1596,24 @@ Apart from these, the following properties are also available, and may be useful How many batches the Spark Streaming UI and status APIs remember before garbage collecting. + + spark.streaming.driver.writeAheadLog.closeFileAfterWrite + false + + Whether to close the file after writing a write ahead log record on the driver. Set this to 'true' + when you want to use S3 (or any file system that does not support flushing) for the metadata WAL + on the driver. + + + + spark.streaming.receiver.writeAheadLog.closeFileAfterWrite + false + + Whether to close the file after writing a write ahead log record on the receivers. Set this to 'true' + when you want to use S3 (or any file system that does not support flushing) for the data WAL + on the receivers. + + #### SparkR diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 3b071c7da5596..1edc0fe34706b 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1985,7 +1985,11 @@ To run a Spark Streaming applications, you need to have the following. to increase aggregate throughput. Additionally, it is recommended that the replication of the received data within Spark be disabled when the write ahead log is enabled as the log is already stored in a replicated storage system. This can be done by setting the storage level for the - input stream to `StorageLevel.MEMORY_AND_DISK_SER`. + input stream to `StorageLevel.MEMORY_AND_DISK_SER`. While using S3 (or any file system that + does not support flushing) for _write ahead logs_, please remember to enable + `spark.streaming.driver.writeAheadLog.closeFileAfterWrite` and + `spark.streaming.receiver.writeAheadLog.closeFileAfterWrite`. See + [Spark Streaming Configuration](configuration.html#spark-streaming) for more details. - *Setting the max receiving rate* - If the cluster resources is not large enough for the streaming application to process data as fast as it is being received, the receivers can be rate limited @@ -2023,12 +2027,6 @@ contains serialized Scala/Java/Python objects and trying to deserialize objects modified classes may lead to errors. In this case, either start the upgraded app with a different checkpoint directory, or delete the previous checkpoint directory. -### Other Considerations -{:.no_toc} -If the data is being received by the receivers faster than what can be processed, -you can limit the rate by setting the [configuration parameter](configuration.html#spark-streaming) -`spark.streaming.receiver.maxRate`. - *** ## Monitoring Applications From 0d96c54534d8bfca191c892b98397a176bc46152 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 8 Jan 2016 02:02:06 -0800 Subject: [PATCH 33/33] [SPARK-12591][STREAMING] Register OpenHashMapBasedStateMap for Kryo (branch 1.6) backport #10609 to branch 1.6 Author: Shixiong Zhu Closes #10656 from zsxwing/SPARK-12591-branch-1.6. --- .../spark/serializer/KryoSerializer.scala | 24 ++++- .../serializer/KryoSerializerSuite.scala | 20 +++- project/MimaExcludes.scala | 4 + .../spark/streaming/util/StateMap.scala | 71 ++++++++++---- .../spark/streaming/StateMapSuite.scala | 96 ++++++++++++++++--- 5 files changed, 174 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index e8268a577df70..87021ec979070 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -17,7 +17,7 @@ package org.apache.spark.serializer -import java.io.{DataInput, DataOutput, EOFException, IOException, InputStream, OutputStream} +import java.io._ import java.nio.ByteBuffer import javax.annotation.Nullable @@ -378,18 +378,24 @@ private[serializer] object KryoSerializer { private val toRegisterSerializer = Map[Class[_], KryoClassSerializer[_]]( classOf[RoaringBitmap] -> new KryoClassSerializer[RoaringBitmap]() { override def write(kryo: Kryo, output: KryoOutput, bitmap: RoaringBitmap): Unit = { - bitmap.serialize(new KryoOutputDataOutputBridge(output)) + bitmap.serialize(new KryoOutputObjectOutputBridge(kryo, output)) } override def read(kryo: Kryo, input: KryoInput, cls: Class[RoaringBitmap]): RoaringBitmap = { val ret = new RoaringBitmap - ret.deserialize(new KryoInputDataInputBridge(input)) + ret.deserialize(new KryoInputObjectInputBridge(kryo, input)) ret } } ) } -private[serializer] class KryoInputDataInputBridge(input: KryoInput) extends DataInput { +/** + * This is a bridge class to wrap KryoInput as an InputStream and ObjectInput. It forwards all + * methods of InputStream and ObjectInput to KryoInput. It's usually helpful when an API expects + * an InputStream or ObjectInput but you want to use Kryo. + */ +private[spark] class KryoInputObjectInputBridge( + kryo: Kryo, input: KryoInput) extends FilterInputStream(input) with ObjectInput { override def readLong(): Long = input.readLong() override def readChar(): Char = input.readChar() override def readFloat(): Float = input.readFloat() @@ -408,9 +414,16 @@ private[serializer] class KryoInputDataInputBridge(input: KryoInput) extends Dat override def readBoolean(): Boolean = input.readBoolean() override def readUnsignedByte(): Int = input.readByteUnsigned() override def readDouble(): Double = input.readDouble() + override def readObject(): AnyRef = kryo.readClassAndObject(input) } -private[serializer] class KryoOutputDataOutputBridge(output: KryoOutput) extends DataOutput { +/** + * This is a bridge class to wrap KryoOutput as an OutputStream and ObjectOutput. It forwards all + * methods of OutputStream and ObjectOutput to KryoOutput. It's usually helpful when an API expects + * an OutputStream or ObjectOutput but you want to use Kryo. + */ +private[spark] class KryoOutputObjectOutputBridge( + kryo: Kryo, output: KryoOutput) extends FilterOutputStream(output) with ObjectOutput { override def writeFloat(v: Float): Unit = output.writeFloat(v) // There is no "readChars" counterpart, except maybe "readLine", which is not supported override def writeChars(s: String): Unit = throw new UnsupportedOperationException("writeChars") @@ -426,6 +439,7 @@ private[serializer] class KryoOutputDataOutputBridge(output: KryoOutput) extends override def writeChar(v: Int): Unit = output.writeChar(v.toChar) override def writeLong(v: Long): Unit = output.writeLong(v) override def writeByte(v: Int): Unit = output.writeByte(v) + override def writeObject(obj: AnyRef): Unit = kryo.writeClassAndObject(output, obj) } /** diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index f700f1dd1df3d..ad5e1e28624f6 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -363,19 +363,35 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { bitmap.add(1) bitmap.add(3) bitmap.add(5) - bitmap.serialize(new KryoOutputDataOutputBridge(output)) + // Ignore Kryo because it doesn't use writeObject + bitmap.serialize(new KryoOutputObjectOutputBridge(null, output)) output.flush() output.close() val inStream = new FileInputStream(tmpfile) val input = new KryoInput(inStream) val ret = new RoaringBitmap - ret.deserialize(new KryoInputDataInputBridge(input)) + // Ignore Kryo because it doesn't use readObject + ret.deserialize(new KryoInputObjectInputBridge(null, input)) input.close() assert(ret == bitmap) Utils.deleteRecursively(dir) } + test("KryoOutputObjectOutputBridge.writeObject and KryoInputObjectInputBridge.readObject") { + val kryo = new KryoSerializer(conf).newKryo() + + val bytesOutput = new ByteArrayOutputStream() + val objectOutput = new KryoOutputObjectOutputBridge(kryo, new KryoOutput(bytesOutput)) + objectOutput.writeObject("test") + objectOutput.close() + + val bytesInput = new ByteArrayInputStream(bytesOutput.toByteArray) + val objectInput = new KryoInputObjectInputBridge(kryo, new KryoInput(bytesInput)) + assert(objectInput.readObject() === "test") + objectInput.close() + } + test("getAutoReset") { val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance] assert(ser.getAutoReset) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index d3a3c0ceb68c8..08b4a2349ac4b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -159,6 +159,10 @@ object MimaExcludes { // SPARK-3580 Add getNumPartitions method to JavaRDD ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.api.java.JavaRDDLike.getNumPartitions") + ) ++ Seq( + // SPARK-12591 Register OpenHashMapBasedStateMap for Kryo + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoInputDataInputBridge"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoOutputDataOutputBridge") ) case v if v.startsWith("1.5") => Seq( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala index 3f139ad138c88..4e5baebaae04b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala @@ -17,16 +17,20 @@ package org.apache.spark.streaming.util -import java.io.{ObjectInputStream, ObjectOutputStream} +import java.io._ import scala.reflect.ClassTag +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} + import org.apache.spark.SparkConf +import org.apache.spark.serializer.{KryoOutputObjectOutputBridge, KryoInputObjectInputBridge} import org.apache.spark.streaming.util.OpenHashMapBasedStateMap._ import org.apache.spark.util.collection.OpenHashMap /** Internal interface for defining the map that keeps track of sessions. */ -private[streaming] abstract class StateMap[K: ClassTag, S: ClassTag] extends Serializable { +private[streaming] abstract class StateMap[K, S] extends Serializable { /** Get the state for a key if it exists */ def get(key: K): Option[S] @@ -54,7 +58,7 @@ private[streaming] abstract class StateMap[K: ClassTag, S: ClassTag] extends Ser /** Companion object for [[StateMap]], with utility methods */ private[streaming] object StateMap { - def empty[K: ClassTag, S: ClassTag]: StateMap[K, S] = new EmptyStateMap[K, S] + def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S] def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = { val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold", @@ -64,7 +68,7 @@ private[streaming] object StateMap { } /** Implementation of StateMap interface representing an empty map */ -private[streaming] class EmptyStateMap[K: ClassTag, S: ClassTag] extends StateMap[K, S] { +private[streaming] class EmptyStateMap[K, S] extends StateMap[K, S] { override def put(key: K, session: S, updateTime: Long): Unit = { throw new NotImplementedError("put() should not be called on an EmptyStateMap") } @@ -77,21 +81,26 @@ private[streaming] class EmptyStateMap[K: ClassTag, S: ClassTag] extends StateMa } /** Implementation of StateMap based on Spark's [[org.apache.spark.util.collection.OpenHashMap]] */ -private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S: ClassTag]( +private[streaming] class OpenHashMapBasedStateMap[K, S]( @transient @volatile var parentStateMap: StateMap[K, S], - initialCapacity: Int = DEFAULT_INITIAL_CAPACITY, - deltaChainThreshold: Int = DELTA_CHAIN_LENGTH_THRESHOLD - ) extends StateMap[K, S] { self => + private var initialCapacity: Int = DEFAULT_INITIAL_CAPACITY, + private var deltaChainThreshold: Int = DELTA_CHAIN_LENGTH_THRESHOLD + )(implicit private var keyClassTag: ClassTag[K], private var stateClassTag: ClassTag[S]) + extends StateMap[K, S] with KryoSerializable { self => - def this(initialCapacity: Int, deltaChainThreshold: Int) = this( + def this(initialCapacity: Int, deltaChainThreshold: Int) + (implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = this( new EmptyStateMap[K, S], initialCapacity = initialCapacity, deltaChainThreshold = deltaChainThreshold) - def this(deltaChainThreshold: Int) = this( + def this(deltaChainThreshold: Int) + (implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = this( initialCapacity = DEFAULT_INITIAL_CAPACITY, deltaChainThreshold = deltaChainThreshold) - def this() = this(DELTA_CHAIN_LENGTH_THRESHOLD) + def this()(implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = { + this(DELTA_CHAIN_LENGTH_THRESHOLD) + } require(initialCapacity >= 1, "Invalid initial capacity") require(deltaChainThreshold >= 1, "Invalid delta chain threshold") @@ -206,11 +215,7 @@ private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S: ClassTag]( * Serialize the map data. Besides serialization, this method actually compact the deltas * (if needed) in a single pass over all the data in the map. */ - - private def writeObject(outputStream: ObjectOutputStream): Unit = { - // Write all the non-transient fields, especially class tags, etc. - outputStream.defaultWriteObject() - + private def writeObjectInternal(outputStream: ObjectOutput): Unit = { // Write the data in the delta of this state map outputStream.writeInt(deltaMap.size) val deltaMapIterator = deltaMap.iterator @@ -262,11 +267,7 @@ private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S: ClassTag]( } /** Deserialize the map data. */ - private def readObject(inputStream: ObjectInputStream): Unit = { - - // Read the non-transient fields, especially class tags, etc. - inputStream.defaultReadObject() - + private def readObjectInternal(inputStream: ObjectInput): Unit = { // Read the data of the delta val deltaMapSize = inputStream.readInt() deltaMap = if (deltaMapSize != 0) { @@ -309,6 +310,34 @@ private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S: ClassTag]( } parentStateMap = newParentSessionStore } + + private def writeObject(outputStream: ObjectOutputStream): Unit = { + // Write all the non-transient fields, especially class tags, etc. + outputStream.defaultWriteObject() + writeObjectInternal(outputStream) + } + + private def readObject(inputStream: ObjectInputStream): Unit = { + // Read the non-transient fields, especially class tags, etc. + inputStream.defaultReadObject() + readObjectInternal(inputStream) + } + + override def write(kryo: Kryo, output: Output): Unit = { + output.writeInt(initialCapacity) + output.writeInt(deltaChainThreshold) + kryo.writeClassAndObject(output, keyClassTag) + kryo.writeClassAndObject(output, stateClassTag) + writeObjectInternal(new KryoOutputObjectOutputBridge(kryo, output)) + } + + override def read(kryo: Kryo, input: Input): Unit = { + initialCapacity = input.readInt() + deltaChainThreshold = input.readInt() + keyClassTag = kryo.readClassAndObject(input).asInstanceOf[ClassTag[K]] + stateClassTag = kryo.readClassAndObject(input).asInstanceOf[ClassTag[S]] + readObjectInternal(new KryoInputObjectInputBridge(kryo, input)) + } } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala index c4a01eaea739e..ea32bbf95ce59 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala @@ -17,15 +17,23 @@ package org.apache.spark.streaming +import org.apache.spark.streaming.rdd.MapWithStateRDDRecord + import scala.collection.{immutable, mutable, Map} +import scala.reflect.ClassTag import scala.util.Random -import org.apache.spark.SparkFunSuite +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Output, Input} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer._ import org.apache.spark.streaming.util.{EmptyStateMap, OpenHashMapBasedStateMap, StateMap} -import org.apache.spark.util.Utils class StateMapSuite extends SparkFunSuite { + private val conf = new SparkConf() + test("EmptyStateMap") { val map = new EmptyStateMap[Int, Int] intercept[scala.NotImplementedError] { @@ -128,17 +136,17 @@ class StateMapSuite extends SparkFunSuite { map1.put(2, 200, 2) testSerialization(map1, "error deserializing and serialized map with data + no delta") - val map2 = map1.copy() + val map2 = map1.copy().asInstanceOf[OpenHashMapBasedStateMap[Int, Int]] // Do not test compaction - assert(map2.asInstanceOf[OpenHashMapBasedStateMap[_, _]].shouldCompact === false) + assert(map2.shouldCompact === false) testSerialization(map2, "error deserializing and serialized map with 1 delta + no new data") map2.put(3, 300, 3) map2.put(4, 400, 4) testSerialization(map2, "error deserializing and serialized map with 1 delta + new data") - val map3 = map2.copy() - assert(map3.asInstanceOf[OpenHashMapBasedStateMap[_, _]].shouldCompact === false) + val map3 = map2.copy().asInstanceOf[OpenHashMapBasedStateMap[Int, Int]] + assert(map3.shouldCompact === false) testSerialization(map3, "error deserializing and serialized map with 2 delta + no new data") map3.put(3, 600, 3) map3.remove(2) @@ -267,18 +275,25 @@ class StateMapSuite extends SparkFunSuite { assertMap(stateMap, refMap.toMap, time, "Final state map does not match reference map") } - private def testSerialization[MapType <: StateMap[Int, Int]]( - map: MapType, msg: String): MapType = { - val deserMap = Utils.deserialize[MapType]( - Utils.serialize(map), Thread.currentThread().getContextClassLoader) + private def testSerialization[T: ClassTag]( + map: OpenHashMapBasedStateMap[T, T], msg: String): OpenHashMapBasedStateMap[T, T] = { + testSerialization(new JavaSerializer(conf), map, msg) + testSerialization(new KryoSerializer(conf), map, msg) + } + + private def testSerialization[T : ClassTag]( + serializer: Serializer, + map: OpenHashMapBasedStateMap[T, T], + msg: String): OpenHashMapBasedStateMap[T, T] = { + val deserMap = serializeAndDeserialize(serializer, map) assertMap(deserMap, map, 1, msg) deserMap } // Assert whether all the data and operations on a state map matches that of a reference state map - private def assertMap( - mapToTest: StateMap[Int, Int], - refMapToTestWith: StateMap[Int, Int], + private def assertMap[T]( + mapToTest: StateMap[T, T], + refMapToTestWith: StateMap[T, T], time: Long, msg: String): Unit = { withClue(msg) { @@ -321,4 +336,59 @@ class StateMapSuite extends SparkFunSuite { } } } + + test("OpenHashMapBasedStateMap - serializing and deserializing with KryoSerializable states") { + val map = new OpenHashMapBasedStateMap[KryoState, KryoState]() + map.put(new KryoState("a"), new KryoState("b"), 1) + testSerialization( + new KryoSerializer(conf), map, "error deserializing and serialized KryoSerializable states") + } + + test("EmptyStateMap - serializing and deserializing") { + val map = StateMap.empty[KryoState, KryoState] + // Since EmptyStateMap doesn't contains any date, KryoState won't break JavaSerializer. + assert(serializeAndDeserialize(new JavaSerializer(conf), map). + isInstanceOf[EmptyStateMap[KryoState, KryoState]]) + assert(serializeAndDeserialize(new KryoSerializer(conf), map). + isInstanceOf[EmptyStateMap[KryoState, KryoState]]) + } + + test("MapWithStateRDDRecord - serializing and deserializing with KryoSerializable states") { + val map = new OpenHashMapBasedStateMap[KryoState, KryoState]() + map.put(new KryoState("a"), new KryoState("b"), 1) + + val record = + MapWithStateRDDRecord[KryoState, KryoState, KryoState](map, Seq(new KryoState("c"))) + val deserRecord = serializeAndDeserialize(new KryoSerializer(conf), record) + assert(!(record eq deserRecord)) + assert(record.stateMap.getAll().toSeq === deserRecord.stateMap.getAll().toSeq) + assert(record.mappedData === deserRecord.mappedData) + } + + private def serializeAndDeserialize[T: ClassTag](serializer: Serializer, t: T): T = { + val serializerInstance = serializer.newInstance() + serializerInstance.deserialize[T]( + serializerInstance.serialize(t), Thread.currentThread().getContextClassLoader) + } +} + +/** A class that only supports Kryo serialization. */ +private[streaming] final class KryoState(var state: String) extends KryoSerializable { + + override def write(kryo: Kryo, output: Output): Unit = { + kryo.writeClassAndObject(output, state) + } + + override def read(kryo: Kryo, input: Input): Unit = { + state = kryo.readClassAndObject(input).asInstanceOf[String] + } + + override def equals(other: Any): Boolean = other match { + case that: KryoState => state == that.state + case _ => false + } + + override def hashCode(): Int = { + if (state == null) 0 else state.hashCode() + } }