diff --git a/.gitignore b/.gitignore index e4c44d0590d5..d5cf66d1db12 100644 --- a/.gitignore +++ b/.gitignore @@ -94,3 +94,6 @@ spark-warehouse/ *.Rproj.* .Rproj.user + +# For SBT +.jvmopts diff --git a/LICENSE b/LICENSE index b771bd552b76..150ccc54ec6c 100644 --- a/LICENSE +++ b/LICENSE @@ -222,7 +222,7 @@ Python Software Foundation License ---------------------------------- pyspark/heapq3.py - +python/docs/_static/copybutton.js BSD 3-Clause ------------ @@ -258,4 +258,4 @@ data/mllib/images/kittens/29.5.a_b_EGDP022204.jpg data/mllib/images/kittens/54893.jpg data/mllib/images/kittens/DP153539.jpg data/mllib/images/kittens/DP802813.jpg -data/mllib/images/multi-channel/chr30.4.184.jpg \ No newline at end of file +data/mllib/images/multi-channel/chr30.4.184.jpg diff --git a/LICENSE-binary b/LICENSE-binary index b94ea90de08b..2ff881fac5fb 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -209,10 +209,10 @@ org.apache.zookeeper:zookeeper oro:oro commons-configuration:commons-configuration commons-digester:commons-digester -com.chuusai:shapeless_2.11 +com.chuusai:shapeless_2.12 com.googlecode.javaewah:JavaEWAH com.twitter:chill-java -com.twitter:chill_2.11 +com.twitter:chill_2.12 com.univocity:univocity-parsers javax.jdo:jdo-api joda-time:joda-time @@ -220,23 +220,23 @@ net.sf.opencsv:opencsv org.apache.derby:derby org.objenesis:objenesis org.roaringbitmap:RoaringBitmap -org.scalanlp:breeze-macros_2.11 -org.scalanlp:breeze_2.11 -org.typelevel:macro-compat_2.11 +org.scalanlp:breeze-macros_2.12 +org.scalanlp:breeze_2.12 +org.typelevel:macro-compat_2.12 org.yaml:snakeyaml org.apache.xbean:xbean-asm5-shaded com.squareup.okhttp3:logging-interceptor com.squareup.okhttp3:okhttp com.squareup.okio:okio -org.apache.spark:spark-catalyst_2.11 -org.apache.spark:spark-kvstore_2.11 -org.apache.spark:spark-launcher_2.11 -org.apache.spark:spark-mllib-local_2.11 -org.apache.spark:spark-network-common_2.11 -org.apache.spark:spark-network-shuffle_2.11 -org.apache.spark:spark-sketch_2.11 -org.apache.spark:spark-tags_2.11 -org.apache.spark:spark-unsafe_2.11 +org.apache.spark:spark-catalyst_2.12 +org.apache.spark:spark-kvstore_2.12 +org.apache.spark:spark-launcher_2.12 +org.apache.spark:spark-mllib-local_2.12 +org.apache.spark:spark-network-common_2.12 +org.apache.spark:spark-network-shuffle_2.12 +org.apache.spark:spark-sketch_2.12 +org.apache.spark:spark-tags_2.12 +org.apache.spark:spark-unsafe_2.12 commons-httpclient:commons-httpclient com.vlkan:flatbuffers com.ning:compress-lzf @@ -260,9 +260,6 @@ net.sf.supercsv:super-csv org.apache.arrow:arrow-format org.apache.arrow:arrow-memory org.apache.arrow:arrow-vector -org.apache.calcite:calcite-avatica -org.apache.calcite:calcite-core -org.apache.calcite:calcite-linq4j org.apache.commons:commons-crypto org.apache.commons:commons-lang3 org.apache.hadoop:hadoop-annotations @@ -287,10 +284,10 @@ org.apache.orc:orc-mapreduce org.mortbay.jetty:jetty org.mortbay.jetty:jetty-util com.jolbox:bonecp -org.json4s:json4s-ast_2.11 -org.json4s:json4s-core_2.11 -org.json4s:json4s-jackson_2.11 -org.json4s:json4s-scalap_2.11 +org.json4s:json4s-ast_2.12 +org.json4s:json4s-core_2.12 +org.json4s:json4s-jackson_2.12 +org.json4s:json4s-scalap_2.12 com.carrotsearch:hppc com.fasterxml.jackson.core:jackson-annotations com.fasterxml.jackson.core:jackson-core @@ -298,14 +295,13 @@ com.fasterxml.jackson.core:jackson-databind com.fasterxml.jackson.dataformat:jackson-dataformat-yaml com.fasterxml.jackson.module:jackson-module-jaxb-annotations com.fasterxml.jackson.module:jackson-module-paranamer -com.fasterxml.jackson.module:jackson-module-scala_2.11 +com.fasterxml.jackson.module:jackson-module-scala_2.12 com.github.mifmif:generex com.google.code.findbugs:jsr305 com.google.code.gson:gson com.google.inject:guice com.google.inject.extensions:guice-servlet com.twitter:parquet-hadoop-bundle -commons-beanutils:commons-beanutils-core commons-cli:commons-cli commons-dbcp:commons-dbcp commons-io:commons-io @@ -415,8 +411,8 @@ com.thoughtworks.paranamer:paranamer org.scala-lang:scala-compiler org.scala-lang:scala-library org.scala-lang:scala-reflect -org.scala-lang.modules:scala-parser-combinators_2.11 -org.scala-lang.modules:scala-xml_2.11 +org.scala-lang.modules:scala-parser-combinators_2.12 +org.scala-lang.modules:scala-xml_2.12 org.fusesource.leveldbjni:leveldbjni-all net.sourceforge.f2j:arpack_combined_all xmlenc:xmlenc @@ -437,15 +433,15 @@ is distributed under the 3-Clause BSD license. MIT License ----------- -org.spire-math:spire-macros_2.11 -org.spire-math:spire_2.11 -org.typelevel:machinist_2.11 +org.spire-math:spire-macros_2.12 +org.spire-math:spire_2.12 +org.typelevel:machinist_2.12 net.razorvine:pyrolite org.slf4j:jcl-over-slf4j org.slf4j:jul-to-slf4j org.slf4j:slf4j-api org.slf4j:slf4j-log4j12 -com.github.scopt:scopt_2.11 +com.github.scopt:scopt_2.12 core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js core/src/main/resources/org/apache/spark/ui/static/*dataTables* @@ -487,6 +483,14 @@ org.glassfish.jersey.core:jersey-server org.glassfish.jersey.media:jersey-media-jaxb +Eclipse Distribution License (EDL) 1.0 +-------------------------------------- + +org.glassfish.jaxb:jaxb-runtime +jakarta.xml.bind:jakarta.xml.bind-api +com.sun.istack:istack-commons-runtime + + Mozilla Public License (MPL) 1.1 -------------------------------- diff --git a/NOTICE-binary b/NOTICE-binary index b707c436983f..df416181a1c1 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -792,15 +792,6 @@ Copyright 2005-2006 The Apache Software Foundation Apache Jakarta HttpClient Copyright 1999-2007 The Apache Software Foundation -Calcite Avatica -Copyright 2012-2015 The Apache Software Foundation - -Calcite Core -Copyright 2012-2015 The Apache Software Foundation - -Calcite Linq4j -Copyright 2012-2015 The Apache Software Foundation - Apache HttpClient Copyright 1999-2017 The Apache Software Foundation diff --git a/R/CRAN_RELEASE.md b/R/CRAN_RELEASE.md index d6084c7a7cc9..4d9b6416c01c 100644 --- a/R/CRAN_RELEASE.md +++ b/R/CRAN_RELEASE.md @@ -1,3 +1,21 @@ +--- +license: | + 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. +--- + # SparkR CRAN Release To release SparkR as a package to CRAN, we would use the `devtools` package. Please work with the diff --git a/R/DOCUMENTATION.md b/R/DOCUMENTATION.md index 7314a1fcccda..fd0c7644189d 100644 --- a/R/DOCUMENTATION.md +++ b/R/DOCUMENTATION.md @@ -1,3 +1,21 @@ +--- +license: | + 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. +--- + # SparkR Documentation SparkR documentation is generated by using in-source comments and annotated by using diff --git a/R/README.md b/R/README.md index d77a1ecffc99..e238a0efe4b5 100644 --- a/R/README.md +++ b/R/README.md @@ -39,15 +39,7 @@ To set other options like driver memory, executor memory etc. you can pass in th #### Using SparkR from RStudio -If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example -```R -# Set this to where Spark is installed -Sys.setenv(SPARK_HOME="/Users/username/spark") -# This line loads SparkR from the installed directory -.libPaths(c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"), .libPaths())) -library(SparkR) -sparkR.session() -``` +If you wish to use SparkR from RStudio, please refer [SparkR documentation](https://spark.apache.org/docs/latest/sparkr.html#starting-up-from-rstudio). #### Making changes to SparkR diff --git a/R/WINDOWS.md b/R/WINDOWS.md index 33a4c850cfda..92442f9b655f 100644 --- a/R/WINDOWS.md +++ b/R/WINDOWS.md @@ -1,3 +1,21 @@ +--- +license: | + 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. +--- + ## Building SparkR on Windows To build SparkR on Windows, the following steps are required diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 736da46eaa8d..3d31be809be6 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,8 +1,8 @@ Package: SparkR Type: Package Version: 3.0.0 -Title: R Frontend for Apache Spark -Description: Provides an R Frontend for Apache Spark. +Title: R Front End for 'Apache Spark' +Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), person("Xiangrui", "Meng", role = "aut", @@ -11,8 +11,8 @@ Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "felixcheung@apache.org"), person(family = "The Apache Software Foundation", role = c("aut", "cph"))) License: Apache License (== 2.0) -URL: http://www.apache.org/ http://spark.apache.org/ -BugReports: http://spark.apache.org/contributing.html +URL: https://www.apache.org/ https://spark.apache.org/ +BugReports: https://spark.apache.org/contributing.html SystemRequirements: Java (== 8) Depends: R (>= 3.1), diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index cdeafdd90ce4..f9d9494ca6fa 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -67,7 +67,8 @@ exportMethods("glm", "spark.fpGrowth", "spark.freqItemsets", "spark.associationRules", - "spark.findFrequentSequentialPatterns") + "spark.findFrequentSequentialPatterns", + "spark.assignClusters") # Job group lifecycle management methods export("setJobGroup", @@ -311,8 +312,10 @@ exportMethods("%<=>%", "lower", "lpad", "ltrim", + "map_concat", "map_entries", "map_from_arrays", + "map_from_entries", "map_keys", "map_values", "max", @@ -351,6 +354,8 @@ exportMethods("%<=>%", "row_number", "rpad", "rtrim", + "schema_of_csv", + "schema_of_json", "second", "sha1", "sha2", @@ -403,6 +408,7 @@ exportMethods("%<=>%", "weekofyear", "when", "window", + "xxhash64", "year") exportClasses("GroupedData") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ad9cd845f696..774a2b2202cc 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -766,7 +766,6 @@ setMethod("repartition", #' \item{2.} {Return a new SparkDataFrame range partitioned by the given column(s), #' using \code{spark.sql.shuffle.partitions} as number of partitions.} #'} -#' #' At least one partition-by expression must be specified. #' When no explicit sort order is specified, "ascending nulls first" is assumed. #' @@ -828,7 +827,6 @@ setMethod("repartitionByRange", #' toJSON #' #' Converts a SparkDataFrame into a SparkDataFrame of JSON string. -#' #' Each row is turned into a JSON document with columns as different fields. #' The returned SparkDataFrame has a single character column with the name \code{value} #' @@ -952,7 +950,7 @@ setMethod("write.parquet", #' #' Save the content of the SparkDataFrame in a text file at the specified path. #' The SparkDataFrame must have only one column of string type with the name "value". -#' Each row becomes a new line in the output file. +#' Each row becomes a new line in the output file. The text files will be encoded as UTF-8. #' #' @param x A SparkDataFrame #' @param path The directory where the file is saved @@ -1179,11 +1177,50 @@ setMethod("dim", setMethod("collect", signature(x = "SparkDataFrame"), function(x, stringsAsFactors = FALSE) { + connectionTimeout <- as.numeric(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) + useArrow <- FALSE + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] == "true" + if (arrowEnabled) { + useArrow <- tryCatch({ + checkSchemaInArrow(schema(x)) + TRUE + }, error = function(e) { + warning(paste0("The conversion from Spark DataFrame to R DataFrame was attempted ", + "with Arrow optimization because ", + "'spark.sql.execution.arrow.enabled' is set to true; however, ", + "failed, attempting non-optimization. Reason: ", + e)) + FALSE + }) + } + dtypes <- dtypes(x) ncol <- length(dtypes) if (ncol <= 0) { # empty data.frame with 0 columns and 0 rows data.frame() + } else if (useArrow) { + requireNamespace1 <- requireNamespace + if (requireNamespace1("arrow", quietly = TRUE)) { + read_arrow <- get("read_arrow", envir = asNamespace("arrow"), inherits = FALSE) + as_tibble <- get("as_tibble", envir = asNamespace("arrow")) + + portAuth <- callJMethod(x@sdf, "collectAsArrowToR") + port <- portAuth[[1]] + authSecret <- portAuth[[2]] + conn <- socketConnection( + port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout) + output <- tryCatch({ + doServerAuth(conn, authSecret) + arrowTable <- read_arrow(readRaw(conn)) + as.data.frame(as_tibble(arrowTable), stringsAsFactors = stringsAsFactors) + }, finally = { + close(conn) + }) + return(output) + } else { + stop("'arrow' package should be installed.") + } } else { # listCols is a list of columns listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf) @@ -1439,6 +1476,18 @@ dapplyInternal <- function(x, func, schema) { schema <- structType(schema) } + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] == "true" + if (arrowEnabled) { + if (inherits(schema, "structType")) { + checkSchemaInArrow(schema) + } else if (is.null(schema)) { + stop(paste0("Arrow optimization does not support 'dapplyCollect' yet. Please disable ", + "Arrow optimization or use 'collect' and 'dapply' APIs instead.")) + } else { + stop("'schema' should be DDL-formatted string or structType.") + } + } + packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) @@ -2094,6 +2143,11 @@ setMethod("selectExpr", #' Return a new SparkDataFrame by adding a column or replacing the existing column #' that has the same name. #' +#' Note: This method introduces a projection internally. Therefore, calling it multiple times, +#' for instance, via loops in order to add multiple columns can generate big plans which +#' can cause performance issues and even \code{StackOverflowException}. To avoid this, +#' use \code{select} with the multiple columns at once. +#' #' @param x a SparkDataFrame. #' @param colName a column name. #' @param col a Column expression (which must refer only to this SparkDataFrame), or an atomic @@ -2471,8 +2525,9 @@ setMethod("dropDuplicates", #' Column expression. If joinExpr is omitted, the default, inner join is attempted and an error is #' thrown if it would be a Cartesian Product. For Cartesian join, use crossJoin instead. #' @param joinType The type of join to perform, default 'inner'. -#' Must be one of: 'inner', 'cross', 'outer', 'full', 'full_outer', -#' 'left', 'left_outer', 'right', 'right_outer', 'left_semi', or 'left_anti'. +#' Must be one of: 'inner', 'cross', 'outer', 'full', 'fullouter', 'full_outer', +#' 'left', 'leftouter', 'left_outer', 'right', 'rightouter', 'right_outer', 'semi', +#' 'leftsemi', 'left_semi', 'anti', 'leftanti', 'left_anti'. #' @return A SparkDataFrame containing the result of the join operation. #' @family SparkDataFrame functions #' @aliases join,SparkDataFrame,SparkDataFrame-method @@ -2504,14 +2559,14 @@ setMethod("join", "outer", "full", "fullouter", "full_outer", "left", "leftouter", "left_outer", "right", "rightouter", "right_outer", - "left_semi", "leftsemi", "left_anti", "leftanti")) { + "semi", "left_semi", "leftsemi", "anti", "left_anti", "leftanti")) { joinType <- gsub("_", "", joinType) sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType) } else { - stop("joinType must be one of the following types: ", - "'inner', 'cross', 'outer', 'full', 'full_outer',", - "'left', 'left_outer', 'right', 'right_outer',", - "'left_semi', or 'left_anti'.") + stop(paste("joinType must be one of the following types:", + "'inner', 'cross', 'outer', 'full', 'fullouter', 'full_outer',", + "'left', 'leftouter', 'left_outer', 'right', 'rightouter', 'right_outer',", + "'semi', 'leftsemi', 'left_semi', 'anti', 'leftanti' or 'left_anti'.")) } } } @@ -2732,13 +2787,25 @@ setMethod("union", dataFrame(unioned) }) -#' Return a new SparkDataFrame containing the union of rows +#' Return a new SparkDataFrame containing the union of rows. #' -#' This is an alias for `union`. +#' This is an alias for \code{union}. #' -#' @rdname union -#' @name unionAll +#' @param x a SparkDataFrame. +#' @param y a SparkDataFrame. +#' @return A SparkDataFrame containing the result of the unionAll operation. +#' @family SparkDataFrame functions #' @aliases unionAll,SparkDataFrame,SparkDataFrame-method +#' @rdname unionAll +#' @name unionAll +#' @seealso \link{union} +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) +#' unionAllDF <- unionAll(df1, df2) +#' } #' @note unionAll since 1.4.0 setMethod("unionAll", signature(x = "SparkDataFrame", y = "SparkDataFrame"), diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index afcdd6faa849..592866188e79 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -147,6 +147,93 @@ getDefaultSqlSource <- function() { l[["spark.sql.sources.default"]] } +writeToFileInArrow <- function(fileName, rdf, numPartitions) { + requireNamespace1 <- requireNamespace + + # R API in Arrow is not yet released in CRAN. CRAN requires to add the + # package in requireNamespace at DESCRIPTION. Later, CRAN checks if the package is available + # or not. Therefore, it works around by avoiding direct requireNamespace. + # Currently, as of Arrow 0.12.0, it can be installed by install_github. See ARROW-3204. + if (requireNamespace1("arrow", quietly = TRUE)) { + record_batch <- get("record_batch", envir = asNamespace("arrow"), inherits = FALSE) + RecordBatchStreamWriter <- get( + "RecordBatchStreamWriter", envir = asNamespace("arrow"), inherits = FALSE) + FileOutputStream <- get( + "FileOutputStream", envir = asNamespace("arrow"), inherits = FALSE) + + numPartitions <- if (!is.null(numPartitions)) { + numToInt(numPartitions) + } else { + 1 + } + + rdf_slices <- if (numPartitions > 1) { + split(rdf, makeSplits(numPartitions, nrow(rdf))) + } else { + list(rdf) + } + + stream_writer <- NULL + tryCatch({ + for (rdf_slice in rdf_slices) { + batch <- record_batch(rdf_slice) + if (is.null(stream_writer)) { + stream <- FileOutputStream(fileName) + schema <- batch$schema + stream_writer <- RecordBatchStreamWriter(stream, schema) + } + + stream_writer$write_batch(batch) + } + }, + finally = { + if (!is.null(stream_writer)) { + stream_writer$close() + } + }) + + } else { + stop("'arrow' package should be installed.") + } +} + +getSchema <- function(schema, firstRow = NULL, rdd = NULL) { + if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { + if (is.null(firstRow)) { + stopifnot(!is.null(rdd)) + firstRow <- firstRDD(rdd) + } + names <- if (is.null(schema)) { + names(firstRow) + } else { + as.list(schema) + } + if (is.null(names)) { + names <- lapply(1:length(firstRow), function(x) { + paste0("_", as.character(x)) + }) + } + + # SPAKR-SQL does not support '.' in column name, so replace it with '_' + # TODO(davies): remove this once SPARK-2775 is fixed + names <- lapply(names, function(n) { + nn <- gsub("[.]", "_", n) + if (nn != n) { + warning(paste("Use", nn, "instead of", n, "as column name")) + } + nn + }) + + types <- lapply(firstRow, infer_type) + fields <- lapply(1:length(firstRow), function(i) { + structField(names[[i]], types[[i]], TRUE) + }) + schema <- do.call(structType, fields) + } else { + schema + } +} + #' Create a SparkDataFrame #' #' Converts R data.frame or list into SparkDataFrame. @@ -172,36 +259,75 @@ getDefaultSqlSource <- function() { createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0, numPartitions = NULL) { sparkSession <- getSparkSession() + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] == "true" + useArrow <- FALSE + firstRow <- NULL if (is.data.frame(data)) { - # Convert data into a list of rows. Each row is a list. - - # get the names of columns, they will be put into RDD - if (is.null(schema)) { - schema <- names(data) - } + # get the names of columns, they will be put into RDD + if (is.null(schema)) { + schema <- names(data) + } - # get rid of factor type - cleanCols <- function(x) { - if (is.factor(x)) { - as.character(x) - } else { - x - } + # get rid of factor type + cleanCols <- function(x) { + if (is.factor(x)) { + as.character(x) + } else { + x } + } + data[] <- lapply(data, cleanCols) + + args <- list(FUN = list, SIMPLIFY = FALSE, USE.NAMES = FALSE) + if (arrowEnabled) { + useArrow <- tryCatch({ + stopifnot(length(data) > 0) + firstRow <- do.call(mapply, append(args, head(data, 1)))[[1]] + schema <- getSchema(schema, firstRow = firstRow) + checkSchemaInArrow(schema) + fileName <- tempfile(pattern = "sparwriteToFileInArrowk-arrow", fileext = ".tmp") + tryCatch({ + writeToFileInArrow(fileName, data, numPartitions) + jrddInArrow <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "readArrowStreamFromFile", + sparkSession, + fileName) + }, + finally = { + # File might not be created. + suppressWarnings(file.remove(fileName)) + }) + TRUE + }, + error = function(e) { + warning(paste0("createDataFrame attempted Arrow optimization because ", + "'spark.sql.execution.arrow.enabled' is set to true; however, ", + "failed, attempting non-optimization. Reason: ", + e)) + FALSE + }) + } + if (!useArrow) { + # Convert data into a list of rows. Each row is a list. # drop factors and wrap lists - data <- setNames(lapply(data, cleanCols), NULL) + data <- setNames(as.list(data), NULL) # check if all columns have supported type lapply(data, getInternalType) # convert to rows - args <- list(FUN = list, SIMPLIFY = FALSE, USE.NAMES = FALSE) data <- do.call(mapply, append(args, data)) + if (length(data) > 0) { + firstRow <- data[[1]] + } + } } - if (is.list(data)) { + if (useArrow) { + rdd <- jrddInArrow + } else if (is.list(data)) { sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) if (!is.null(numPartitions)) { rdd <- parallelize(sc, data, numSlices = numToInt(numPartitions)) @@ -214,42 +340,19 @@ createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0, stop(paste("unexpected type:", class(data))) } - if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { - row <- firstRDD(rdd) - names <- if (is.null(schema)) { - names(row) - } else { - as.list(schema) - } - if (is.null(names)) { - names <- lapply(1:length(row), function(x) { - paste("_", as.character(x), sep = "") - }) - } - - # SPAKR-SQL does not support '.' in column name, so replace it with '_' - # TODO(davies): remove this once SPARK-2775 is fixed - names <- lapply(names, function(n) { - nn <- gsub("[.]", "_", n) - if (nn != n) { - warning(paste("Use", nn, "instead of", n, " as column name")) - } - nn - }) - - types <- lapply(row, infer_type) - fields <- lapply(1:length(row), function(i) { - structField(names[[i]], types[[i]], TRUE) - }) - schema <- do.call(structType, fields) - } + schema <- getSchema(schema, firstRow, rdd) stopifnot(class(schema) == "structType") - jrdd <- getJRDD(lapply(rdd, function(x) x), "row") - srdd <- callJMethod(jrdd, "rdd") - sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", - srdd, schema$jobj, sparkSession) + if (useArrow) { + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "toDataFrame", rdd, schema$jobj, sparkSession) + } else { + jrdd <- getJRDD(lapply(rdd, function(x) x), "row") + srdd <- callJMethod(jrdd, "rdd") + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", + srdd, schema$jobj, sparkSession) + } dataFrame(sdf) } @@ -358,7 +461,7 @@ read.parquet <- function(path, ...) { #' #' Loads text files and returns a SparkDataFrame whose schema starts with #' a string column named "value", and followed by partitioned columns if -#' there are any. +#' there are any. The text files must be encoded as UTF-8. #' #' Each line in the text file is a new row in the resulting SparkDataFrame. #' diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index ee7f4adf726e..037809cd0923 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -127,6 +127,16 @@ setMethod("orderBy", #' "0" means "current row", while "-1" means the row before the current row, and "5" means the #' fifth row after the current row. #' +#' We recommend users use \code{Window.unboundedPreceding}, \code{Window.unboundedFollowing}, +#' and \code{Window.currentRow} to specify special boundary values, rather than using long values +#' directly. +#' +#' A row based boundary is based on the position of the row within the partition. +#' An offset indicates the number of rows above or below the current row, the frame for the +#' current row starts or ends. For instance, given a row based sliding frame with a lower bound +#' offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from +#' index 4 to index 6. +#' #' @param x a WindowSpec #' @param start boundary start, inclusive. #' The frame is unbounded if this is the minimum long value. @@ -139,7 +149,14 @@ setMethod("orderBy", #' @family windowspec_method #' @examples #' \dontrun{ -#' rowsBetween(ws, 0, 3) +#' id <- c(rep(1, 3), rep(2, 3), 3) +#' desc <- c('New', 'New', 'Good', 'New', 'Good', 'Good', 'New') +#' df <- data.frame(id, desc) +#' df <- createDataFrame(df) +#' w1 <- orderBy(windowPartitionBy('desc'), df$id) +#' w2 <- rowsBetween(w1, 0, 3) +#' df1 <- withColumn(df, "sum", over(sum(df$id), w2)) +#' head(df1) #' } #' @note rowsBetween since 2.0.0 setMethod("rowsBetween", @@ -158,6 +175,19 @@ setMethod("rowsBetween", #' "current row", while "-1" means one off before the current row, and "5" means the five off #' after the current row. #' +#' We recommend users use \code{Window.unboundedPreceding}, \code{Window.unboundedFollowing}, +#' and \code{Window.currentRow} to specify special boundary values, rather than using long values +#' directly. +#' +#' A range-based boundary is based on the actual value of the ORDER BY +#' expression(s). An offset is used to alter the value of the ORDER BY expression, +#' for instance if the current ORDER BY expression has a value of 10 and the lower bound offset +#' is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a +#' number of constraints on the ORDER BY expressions: there can be only one expression and this +#' expression must have a numerical data type. An exception can be made when the offset is +#' unbounded, because no value modification is needed, in this case multiple and non-numeric +#' ORDER BY expression are allowed. +#' #' @param x a WindowSpec #' @param start boundary start, inclusive. #' The frame is unbounded if this is the minimum long value. @@ -170,7 +200,14 @@ setMethod("rowsBetween", #' @family windowspec_method #' @examples #' \dontrun{ -#' rangeBetween(ws, 0, 3) +#' id <- c(rep(1, 3), rep(2, 3), 3) +#' desc <- c('New', 'New', 'Good', 'New', 'Good', 'Good', 'New') +#' df <- data.frame(id, desc) +#' df <- createDataFrame(df) +#' w1 <- orderBy(windowPartitionBy('desc'), df$id) +#' w2 <- rangeBetween(w1, 0, 3) +#' df1 <- withColumn(df, "sum", over(sum(df$id), w2)) +#' head(df1) #' } #' @note rangeBetween since 2.0.0 setMethod("rangeBetween", diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index e99136723f65..619153645d92 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -29,7 +29,7 @@ getMinPartitions <- function(sc, minPartitions) { #' #' This function reads a text file from HDFS, a local file system (available on all #' nodes), or any Hadoop-supported file system URI, and creates an -#' RDD of strings from it. +#' RDD of strings from it. The text files must be encoded as UTF-8. #' #' @param sc SparkContext to use #' @param path Path of file to read. A vector of multiple paths is allowed. @@ -81,13 +81,33 @@ objectFile <- function(sc, path, minPartitions = NULL) { RDD(jrdd, "byte") } +makeSplits <- function(numSerializedSlices, length) { + # Generate the slice ids to put each row + # For instance, for numSerializedSlices of 22, length of 50 + # [1] 0 0 2 2 4 4 6 6 6 9 9 11 11 13 13 15 15 15 18 18 20 20 22 22 22 + # [26] 25 25 27 27 29 29 31 31 31 34 34 36 36 38 38 40 40 40 43 43 45 45 47 47 47 + # Notice the slice group with 3 slices (ie. 6, 15, 22) are roughly evenly spaced. + # We are trying to reimplement the calculation in the positions method in ParallelCollectionRDD + if (numSerializedSlices > 0) { + unlist(lapply(0: (numSerializedSlices - 1), function(x) { + # nolint start + start <- trunc((as.numeric(x) * length) / numSerializedSlices) + end <- trunc(((as.numeric(x) + 1) * length) / numSerializedSlices) + # nolint end + rep(start, end - start) + })) + } else { + 1 + } +} + #' Create an RDD from a homogeneous list or vector. #' #' This function creates an RDD from a local homogeneous list in R. The elements #' in the list are split into \code{numSlices} slices and distributed to nodes #' in the cluster. #' -#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MB), the function +#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MiB), the function #' will write it to disk and send the file name to JVM. Also to make sure each slice is not #' larger than that limit, number of slices may be increased. #' @@ -143,25 +163,7 @@ parallelize <- function(sc, coll, numSlices = 1) { # For large objects we make sure the size of each slice is also smaller than sizeLimit numSerializedSlices <- min(len, max(numSlices, ceiling(objectSize / sizeLimit))) - # Generate the slice ids to put each row - # For instance, for numSerializedSlices of 22, length of 50 - # [1] 0 0 2 2 4 4 6 6 6 9 9 11 11 13 13 15 15 15 18 18 20 20 22 22 22 - # [26] 25 25 27 27 29 29 31 31 31 34 34 36 36 38 38 40 40 40 43 43 45 45 47 47 47 - # Notice the slice group with 3 slices (ie. 6, 15, 22) are roughly evenly spaced. - # We are trying to reimplement the calculation in the positions method in ParallelCollectionRDD - splits <- if (numSerializedSlices > 0) { - unlist(lapply(0: (numSerializedSlices - 1), function(x) { - # nolint start - start <- trunc((as.numeric(x) * len) / numSerializedSlices) - end <- trunc(((as.numeric(x) + 1) * len) / numSerializedSlices) - # nolint end - rep(start, end - start) - })) - } else { - 1 - } - - slices <- split(coll, splits) + slices <- split(coll, makeSplits(numSerializedSlices, len)) # Serialize each slice: obtain a list of raws, or a list of lists (slices) of # 2-tuples of raws @@ -173,13 +175,15 @@ parallelize <- function(sc, coll, numSlices = 1) { if (objectSize < sizeLimit) { jrdd <- callJStatic("org.apache.spark.api.r.RRDD", "createRDDFromArray", sc, serializedSlices) } else { - if (callJStatic("org.apache.spark.api.r.RUtils", "getEncryptionEnabled", sc)) { + if (callJStatic("org.apache.spark.api.r.RUtils", "isEncryptionEnabled", sc)) { + connectionTimeout <- as.numeric(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) # the length of slices here is the parallelism to use in the jvm's sc.parallelize() parallelism <- as.integer(numSlices) jserver <- newJObject("org.apache.spark.api.r.RParallelizeServer", sc, parallelism) authSecret <- callJMethod(jserver, "secret") port <- callJMethod(jserver, "port") - conn <- socketConnection(port = port, blocking = TRUE, open = "wb", timeout = 1500) + conn <- socketConnection( + port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout) doServerAuth(conn, authSecret) writeToConnection(serializedSlices, conn) jrdd <- callJMethod(jserver, "getResult") diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index cb03f1667629..191c51e2e41f 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -231,6 +231,37 @@ readMultipleObjectsWithKeys <- function(inputCon) { list(keys = keys, data = data) # this is a list of keys and corresponding data } +readDeserializeInArrow <- function(inputCon) { + # This is a hack to avoid CRAN check. Arrow is not uploaded into CRAN now. See ARROW-3204. + requireNamespace1 <- requireNamespace + if (requireNamespace1("arrow", quietly = TRUE)) { + RecordBatchStreamReader <- get( + "RecordBatchStreamReader", envir = asNamespace("arrow"), inherits = FALSE) + as_tibble <- get("as_tibble", envir = asNamespace("arrow")) + + # Currently, there looks no way to read batch by batch by socket connection in R side, + # See ARROW-4512. Therefore, it reads the whole Arrow streaming-formatted binary at once + # for now. + dataLen <- readInt(inputCon) + arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") + batches <- RecordBatchStreamReader(arrowData)$batches() + + # Read all groupped batches. Tibble -> data.frame is cheap. + lapply(batches, function(batch) as.data.frame(as_tibble(batch))) + } else { + stop("'arrow' package should be installed.") + } +} + +readDeserializeWithKeysInArrow <- function(inputCon) { + data <- readDeserializeInArrow(inputCon) + + keys <- readMultipleObjects(inputCon) + + # Read keys to map with each groupped batch later. + list(keys = keys, data = data) +} + readRowList <- function(obj) { # readRowList is meant for use inside an lapply. As a result, it is # necessary to open a standalone connection for the row and consume diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index f72645a25779..3bd1f544d77a 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -80,6 +80,11 @@ NULL #' \item \code{from_utc_timestamp}, \code{to_utc_timestamp}: time zone to use. #' \item \code{next_day}: day of the week string. #' } +#' @param ... additional argument(s). +#' \itemize{ +#' \item \code{months_between}, this contains an optional parameter to specify the +#' the result is rounded off to 8 digits. +#' } #' #' @name column_datetime_diff_functions #' @rdname column_datetime_diff_functions @@ -202,14 +207,23 @@ NULL #' \itemize{ #' \item \code{from_json}: a structType object to use as the schema to use #' when parsing the JSON string. Since Spark 2.3, the DDL-formatted string is -#' also supported for the schema. -#' \item \code{from_csv}: a DDL-formatted string +#' also supported for the schema. Since Spark 3.0, \code{schema_of_json} or +#' the DDL-formatted string literal can also be accepted. +#' \item \code{from_csv}: a structType object, DDL-formatted string or \code{schema_of_csv} +#' } +#' @param ... additional argument(s). +#' \itemize{ +#' \item \code{to_json}, \code{from_json} and \code{schema_of_json}: this contains +#' additional named properties to control how it is converted and accepts the +#' same options as the JSON data source. +#' \item \code{to_json}: it supports the "pretty" option which enables pretty +#' JSON generation. +#' \item \code{to_csv}, \code{from_csv} and \code{schema_of_csv}: this contains +#' additional named properties to control how it is converted and accepts the +#' same options as the CSV data source. +#' \item \code{arrays_zip}, this contains additional Columns of arrays to be merged. +#' \item \code{map_concat}, this contains additional Columns of maps to be unioned. #' } -#' @param ... additional argument(s). In \code{to_json}, \code{to_csv} and \code{from_json}, -#' this contains additional named properties to control how it is converted, accepts -#' the same options as the JSON/CSV data source. Additionally \code{to_json} supports -#' the "pretty" option which enables pretty JSON generation. In \code{arrays_zip}, -#' this contains additional Columns of arrays to be merged. #' @name column_collection_functions #' @rdname column_collection_functions #' @family collection functions @@ -221,7 +235,7 @@ NULL #' head(select(tmp, array_contains(tmp$v1, 21), size(tmp$v1), shuffle(tmp$v1))) #' head(select(tmp, array_max(tmp$v1), array_min(tmp$v1), array_distinct(tmp$v1))) #' head(select(tmp, array_position(tmp$v1, 21), array_repeat(df$mpg, 3), array_sort(tmp$v1))) -#' head(select(tmp, flatten(tmp$v1), reverse(tmp$v1), array_remove(tmp$v1, 21))) +#' head(select(tmp, reverse(tmp$v1), array_remove(tmp$v1, 21))) #' tmp2 <- mutate(tmp, v2 = explode(tmp$v1)) #' head(tmp2) #' head(select(tmp, posexplode(tmp$v1))) @@ -230,15 +244,21 @@ NULL #' head(select(tmp, sort_array(tmp$v1, asc = FALSE))) #' tmp3 <- mutate(df, v3 = create_map(df$model, df$cyl)) #' head(select(tmp3, map_entries(tmp3$v3), map_keys(tmp3$v3), map_values(tmp3$v3))) -#' head(select(tmp3, element_at(tmp3$v3, "Valiant"))) +#' head(select(tmp3, element_at(tmp3$v3, "Valiant"), map_concat(tmp3$v3, tmp3$v3))) #' tmp4 <- mutate(df, v4 = create_array(df$mpg, df$cyl), v5 = create_array(df$cyl, df$hp)) #' head(select(tmp4, concat(tmp4$v4, tmp4$v5), arrays_overlap(tmp4$v4, tmp4$v5))) #' head(select(tmp4, array_except(tmp4$v4, tmp4$v5), array_intersect(tmp4$v4, tmp4$v5))) #' head(select(tmp4, array_union(tmp4$v4, tmp4$v5))) -#' head(select(tmp4, arrays_zip(tmp4$v4, tmp4$v5), map_from_arrays(tmp4$v4, tmp4$v5))) +#' head(select(tmp4, arrays_zip(tmp4$v4, tmp4$v5))) #' head(select(tmp, concat(df$mpg, df$cyl, df$hp))) #' tmp5 <- mutate(df, v6 = create_array(df$model, df$model)) -#' head(select(tmp5, array_join(tmp5$v6, "#"), array_join(tmp5$v6, "#", "NULL")))} +#' head(select(tmp5, array_join(tmp5$v6, "#"), array_join(tmp5$v6, "#", "NULL"))) +#' tmp6 <- mutate(df, v7 = create_array(create_array(df$model, df$model))) +#' head(select(tmp6, flatten(tmp6$v7))) +#' tmp7 <- mutate(df, v8 = create_array(df$model, df$cyl), v9 = create_array(df$model, df$hp)) +#' head(select(tmp7, map_from_arrays(tmp7$v8, tmp7$v9))) +#' tmp8 <- mutate(df, v10 = create_array(struct(df$model, df$cyl))) +#' head(select(tmp8, map_from_entries(tmp8$v10)))} NULL #' Window functions for Column operations @@ -715,6 +735,25 @@ setMethod("hash", column(jc) }) +#' @details +#' \code{xxhash64}: Calculates the hash code of given columns using the 64-bit +#' variant of the xxHash algorithm, and returns the result as a long +#' column. +#' +#' @rdname column_misc_functions +#' @aliases xxhash64 xxhash64,Column-method +#' @note xxhash64 since 3.0.0 +setMethod("xxhash64", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function(x) { + stopifnot(class(x) == "Column") + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "xxhash64", jcols) + column(jc) + }) + #' @details #' \code{dayofmonth}: Extracts the day of the month as an integer from a #' given date/timestamp/string. @@ -1716,7 +1755,7 @@ setMethod("radians", #' @details #' \code{to_date}: Converts the column into a DateType. You may optionally specify #' a format according to the rules in: -#' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. +#' \url{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}. #' If the string cannot be parsed according to the specified format (or default), #' the value of the column will be null. #' By default, it follows casting rules to a DateType if the format is omitted @@ -1771,12 +1810,16 @@ setMethod("to_date", #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' #' # Converts a map into a JSON object -#' df2 <- sql("SELECT map('name', 'Bob')) as people") +#' df2 <- sql("SELECT map('name', 'Bob') as people") #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' #' # Converts an array of maps into a JSON array #' df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people") -#' df2 <- mutate(df2, people_json = to_json(df2$people))} +#' df2 <- mutate(df2, people_json = to_json(df2$people)) +#' +#' # Converts a map into a pretty JSON object +#' df2 <- sql("SELECT map('name', 'Bob') as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people, pretty = TRUE))} #' @note to_json since 2.2.0 setMethod("to_json", signature(x = "Column"), function(x, ...) { @@ -1808,7 +1851,7 @@ setMethod("to_csv", signature(x = "Column"), #' @details #' \code{to_timestamp}: Converts the column into a TimestampType. You may optionally specify #' a format according to the rules in: -#' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. +#' \url{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}. #' If the string cannot be parsed according to the specified format (or default), #' the value of the column will be null. #' By default, it follows casting rules to a TimestampType if the format is omitted @@ -2062,15 +2105,21 @@ setMethod("levenshtein", signature(y = "Column"), #' are on the same day of month, or both are the last day of month, time of day will be ignored. #' Otherwise, the difference is calculated based on 31 days per month, and rounded to 8 digits. #' +#' @param roundOff an optional parameter to specify if the result is rounded off to 8 digits #' @rdname column_datetime_diff_functions #' @aliases months_between months_between,Column-method #' @note months_between since 1.5.0 setMethod("months_between", signature(y = "Column"), - function(y, x) { + function(y, x, roundOff = NULL) { if (class(x) == "Column") { x <- x@jc } - jc <- callJStatic("org.apache.spark.sql.functions", "months_between", y@jc, x) + jc <- if (is.null(roundOff)) { + callJStatic("org.apache.spark.sql.functions", "months_between", y@jc, x) + } else { + callJStatic("org.apache.spark.sql.functions", "months_between", y@jc, x, + as.logical(roundOff)) + } column(jc) }) @@ -2229,7 +2278,7 @@ setMethod("n", signature(x = "Column"), #' \code{date_format}: Converts a date/timestamp/string to a value of string in the format #' specified by the date format given by the second argument. A pattern could be for instance #' \code{dd.MM.yyyy} and could return a string like '18.03.1993'. All -#' pattern letters of \code{java.text.SimpleDateFormat} can be used. +#' pattern letters of \code{java.time.format.DateTimeFormatter} can be used. #' Note: Use when ever possible specialized functions like \code{year}. These benefit from a #' specialized implementation. #' @@ -2243,6 +2292,8 @@ setMethod("date_format", signature(y = "Column", x = "character"), column(jc) }) +setClassUnion("characterOrstructTypeOrColumn", c("character", "structType", "Column")) + #' @details #' \code{from_json}: Parses a column containing a JSON string into a Column of \code{structType} #' with the specified \code{schema} or array of \code{structType} if \code{as.json.array} is set @@ -2250,7 +2301,7 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' #' @rdname column_collection_functions #' @param as.json.array indicating if input string is JSON array of objects or a single object. -#' @aliases from_json from_json,Column,characterOrstructType-method +#' @aliases from_json from_json,Column,characterOrstructTypeOrColumn-method #' @examples #' #' \dontrun{ @@ -2258,25 +2309,37 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' df2 <- mutate(df2, d2 = to_json(df2$d, dateFormat = 'dd/MM/yyyy')) #' schema <- structType(structField("date", "string")) #' head(select(df2, from_json(df2$d2, schema, dateFormat = 'dd/MM/yyyy'))) - #' df2 <- sql("SELECT named_struct('name', 'Bob') as people") #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' schema <- structType(structField("name", "string")) #' head(select(df2, from_json(df2$people_json, schema))) -#' head(select(df2, from_json(df2$people_json, "name STRING")))} +#' head(select(df2, from_json(df2$people_json, "name STRING"))) +#' head(select(df2, from_json(df2$people_json, schema_of_json(head(df2)$people_json))))} #' @note from_json since 2.2.0 -setMethod("from_json", signature(x = "Column", schema = "characterOrstructType"), +setMethod("from_json", signature(x = "Column", schema = "characterOrstructTypeOrColumn"), function(x, schema, as.json.array = FALSE, ...) { if (is.character(schema)) { - schema <- structType(schema) + jschema <- structType(schema)$jobj + } else if (class(schema) == "structType") { + jschema <- schema$jobj + } else { + jschema <- schema@jc } if (as.json.array) { - jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", - "createArrayType", - schema$jobj) - } else { - jschema <- schema$jobj + # This case is R-specifically different. Unlike Scala and Python side, + # R side has 'as.json.array' option to indicate if the schema should be + # treated as struct or element type of array in order to make it more + # R-friendly. + if (class(schema) == "Column") { + jschema <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createArrayType", + jschema) + } else { + jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", + "createArrayType", + jschema) + } } options <- varargsToStrEnv(...) jc <- callJStatic("org.apache.spark.sql.functions", @@ -2285,28 +2348,59 @@ setMethod("from_json", signature(x = "Column", schema = "characterOrstructType") column(jc) }) +#' @details +#' \code{schema_of_json}: Parses a JSON string and infers its schema in DDL format. +#' +#' @rdname column_collection_functions +#' @aliases schema_of_json schema_of_json,characterOrColumn-method +#' @examples +#' +#' \dontrun{ +#' json <- "{\"name\":\"Bob\"}" +#' df <- sql("SELECT * FROM range(1)") +#' head(select(df, schema_of_json(json)))} +#' @note schema_of_json since 3.0.0 +setMethod("schema_of_json", signature(x = "characterOrColumn"), + function(x, ...) { + if (class(x) == "character") { + col <- callJStatic("org.apache.spark.sql.functions", "lit", x) + } else { + col <- x@jc + } + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", + "schema_of_json", + col, options) + column(jc) + }) + #' @details #' \code{from_csv}: Parses a column containing a CSV string into a Column of \code{structType} #' with the specified \code{schema}. #' If the string is unparseable, the Column will contain the value NA. #' #' @rdname column_collection_functions -#' @aliases from_csv from_csv,Column,character-method +#' @aliases from_csv from_csv,Column,characterOrstructTypeOrColumn-method #' @examples #' #' \dontrun{ -#' df <- sql("SELECT 'Amsterdam,2018' as csv") +#' csv <- "Amsterdam,2018" +#' df <- sql(paste0("SELECT '", csv, "' as csv")) #' schema <- "city STRING, year INT" -#' head(select(df, from_csv(df$csv, schema)))} +#' head(select(df, from_csv(df$csv, schema))) +#' head(select(df, from_csv(df$csv, structType(schema)))) +#' head(select(df, from_csv(df$csv, schema_of_csv(csv))))} #' @note from_csv since 3.0.0 -setMethod("from_csv", signature(x = "Column", schema = "characterOrColumn"), +setMethod("from_csv", signature(x = "Column", schema = "characterOrstructTypeOrColumn"), function(x, schema, ...) { - if (class(schema) == "Column") { - jschema <- schema@jc - } else if (is.character(schema)) { + if (class(schema) == "structType") { + schema <- callJMethod(schema$jobj, "toDDL") + } + + if (is.character(schema)) { jschema <- callJStatic("org.apache.spark.sql.functions", "lit", schema) } else { - stop("schema argument should be a column or character") + jschema <- schema@jc } options <- varargsToStrEnv(...) jc <- callJStatic("org.apache.spark.sql.functions", @@ -2315,6 +2409,32 @@ setMethod("from_csv", signature(x = "Column", schema = "characterOrColumn"), column(jc) }) +#' @details +#' \code{schema_of_csv}: Parses a CSV string and infers its schema in DDL format. +#' +#' @rdname column_collection_functions +#' @aliases schema_of_csv schema_of_csv,characterOrColumn-method +#' @examples +#' +#' \dontrun{ +#' csv <- "Amsterdam,2018" +#' df <- sql("SELECT * FROM range(1)") +#' head(select(df, schema_of_csv(csv)))} +#' @note schema_of_csv since 3.0.0 +setMethod("schema_of_csv", signature(x = "characterOrColumn"), + function(x, ...) { + if (class(x) == "character") { + col <- callJStatic("org.apache.spark.sql.functions", "lit", x) + } else { + col <- x@jc + } + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", + "schema_of_csv", + col, options) + column(jc) + }) + #' @details #' \code{from_utc_timestamp}: This is a common function for databases supporting TIMESTAMP WITHOUT #' TIMEZONE. This function takes a timestamp which is timezone-agnostic, and interprets it as a @@ -2339,6 +2459,7 @@ setMethod("from_csv", signature(x = "Column", schema = "characterOrColumn"), #' @note from_utc_timestamp since 1.5.0 setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { + .Deprecated(msg = "from_utc_timestamp is deprecated. See SPARK-25496.") jc <- callJStatic("org.apache.spark.sql.functions", "from_utc_timestamp", y@jc, x) column(jc) }) @@ -2397,6 +2518,7 @@ setMethod("next_day", signature(y = "Column", x = "character"), #' @note to_utc_timestamp since 1.5.0 setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { + .Deprecated(msg = "to_utc_timestamp is deprecated. See SPARK-25496.") jc <- callJStatic("org.apache.spark.sql.functions", "to_utc_timestamp", y@jc, x) column(jc) }) @@ -2603,7 +2725,7 @@ setMethod("format_string", signature(format = "character", x = "Column"), #' \code{from_unixtime}: Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) #' to a string representing the timestamp of that moment in the current system time zone in the JVM #' in the given format. -#' See \href{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}{ +#' See \href{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}{ #' Customizing Formats} for available options. #' #' @rdname column_datetime_functions @@ -3365,6 +3487,23 @@ setMethod("flatten", column(jc) }) +#' @details +#' \code{map_concat}: Returns the union of all the given maps. +#' +#' @rdname column_collection_functions +#' @aliases map_concat map_concat,Column-method +#' @note map_concat since 3.0.0 +setMethod("map_concat", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function(arg) { + stopifnot(class(arg) == "Column") + arg@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "map_concat", jcols) + column(jc) + }) + #' @details #' \code{map_entries}: Returns an unordered array of all entries in the given map. #' @@ -3393,6 +3532,19 @@ setMethod("map_from_arrays", column(jc) }) +#' @details +#' \code{map_from_entries}: Returns a map created from the given array of entries. +#' +#' @rdname column_collection_functions +#' @aliases map_from_entries map_from_entries,Column-method +#' @note map_from_entries since 3.0.0 +setMethod("map_from_entries", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "map_from_entries", x@jc) + column(jc) + }) + #' @details #' \code{map_keys}: Returns an unordered array containing the keys of the map. #' @@ -3437,6 +3589,8 @@ setMethod("element_at", #' @details #' \code{explode}: Creates a new row for each element in the given array or map column. +#' Uses the default column name \code{col} for elements in the array and +#' \code{key} and \code{value} for elements in the map unless specified otherwise. #' #' @rdname column_collection_functions #' @aliases explode explode,Column-method @@ -3497,7 +3651,9 @@ setMethod("sort_array", #' @details #' \code{posexplode}: Creates a new row for each element with position in the given array -#' or map column. +#' or map column. Uses the default column name \code{pos} for position, and \code{col} +#' for elements in the array and \code{key} and \code{value} for elements in the map +#' unless specified otherwise. #' #' @rdname column_collection_functions #' @aliases posexplode posexplode,Column-method @@ -3638,7 +3794,8 @@ setMethod("repeat_string", #' \code{explode}: Creates a new row for each element in the given array or map column. #' Unlike \code{explode}, if the array/map is \code{null} or empty #' then \code{null} is produced. -#' +#' Uses the default column name \code{col} for elements in the array and +#' \code{key} and \code{value} for elements in the map unless specified otherwise. #' #' @rdname column_collection_functions #' @aliases explode_outer explode_outer,Column-method @@ -3663,6 +3820,9 @@ setMethod("explode_outer", #' \code{posexplode_outer}: Creates a new row for each element with position in the given #' array or map column. Unlike \code{posexplode}, if the array/map is \code{null} or empty #' then the row (\code{null}, \code{null}) is produced. +#' Uses the default column name \code{pos} for position, and \code{col} +#' for elements in the array and \code{key} and \code{value} for elements in the map +#' unless specified otherwise. #' #' @rdname column_collection_functions #' @aliases posexplode_outer posexplode_outer,Column-method diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index b2ca6e62175e..f849dd172247 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -631,7 +631,7 @@ setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) #' @rdname union setGeneric("union", function(x, y) { standardGeneric("union") }) -#' @rdname union +#' @rdname unionAll setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) #' @rdname unionByName @@ -1078,6 +1078,10 @@ setGeneric("lpad", function(x, len, pad) { standardGeneric("lpad") }) #' @name NULL setGeneric("ltrim", function(x, trimString) { standardGeneric("ltrim") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("map_concat", function(x, ...) { standardGeneric("map_concat") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("map_entries", function(x) { standardGeneric("map_entries") }) @@ -1086,6 +1090,10 @@ setGeneric("map_entries", function(x) { standardGeneric("map_entries") }) #' @name NULL setGeneric("map_from_arrays", function(x, y) { standardGeneric("map_from_arrays") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("map_from_entries", function(x) { standardGeneric("map_from_entries") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("map_keys", function(x) { standardGeneric("map_keys") }) @@ -1113,7 +1121,7 @@ setGeneric("month", function(x) { standardGeneric("month") }) #' @rdname column_datetime_diff_functions #' @name NULL -setGeneric("months_between", function(y, x) { standardGeneric("months_between") }) +setGeneric("months_between", function(y, x, ...) { standardGeneric("months_between") }) #' @rdname count setGeneric("n", function(x) { standardGeneric("n") }) @@ -1206,6 +1214,14 @@ setGeneric("rpad", function(x, len, pad) { standardGeneric("rpad") }) #' @name NULL setGeneric("rtrim", function(x, trimString) { standardGeneric("rtrim") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("schema_of_csv", function(x, ...) { standardGeneric("schema_of_csv") }) + +#' @rdname column_collection_functions +#' @name NULL +setGeneric("schema_of_json", function(x, ...) { standardGeneric("schema_of_json") }) + #' @rdname column_aggregate_functions #' @name NULL setGeneric("sd", function(x, na.rm = FALSE) { standardGeneric("sd") }) @@ -1378,6 +1394,10 @@ setGeneric("weekofyear", function(x) { standardGeneric("weekofyear") }) #' @name NULL setGeneric("window", function(x, ...) { standardGeneric("window") }) +#' @rdname column_misc_functions +#' @name NULL +setGeneric("xxhash64", function(x, ...) { standardGeneric("xxhash64") }) + #' @rdname column_datetime_functions #' @name NULL setGeneric("year", function(x) { standardGeneric("year") }) @@ -1471,6 +1491,10 @@ setGeneric("spark.associationRules", function(object) { standardGeneric("spark.a setGeneric("spark.findFrequentSequentialPatterns", function(data, ...) { standardGeneric("spark.findFrequentSequentialPatterns") }) +#' @rdname spark.powerIterationClustering +setGeneric("spark.assignClusters", + function(data, ...) { standardGeneric("spark.assignClusters") }) + #' @param object a fitted ML model object. #' @param path the directory where the model is saved. #' @param ... additional argument(s) passed to the method. diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index f751b952f391..7b3913cdd1cd 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -229,6 +229,18 @@ gapplyInternal <- function(x, func, schema) { if (is.character(schema)) { schema <- structType(schema) } + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] == "true" + if (arrowEnabled) { + if (inherits(schema, "structType")) { + checkSchemaInArrow(schema) + } else if (is.null(schema)) { + stop(paste0("Arrow optimization does not support 'gapplyCollect' yet. Please disable ", + "Arrow optimization or use 'collect' and 'gapply' APIs instead.")) + } else { + stop("'schema' should be DDL-formatted string or structType.") + } + } + packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), diff --git a/R/pkg/R/mllib_clustering.R b/R/pkg/R/mllib_clustering.R index 900be685824d..9b32b71d34fe 100644 --- a/R/pkg/R/mllib_clustering.R +++ b/R/pkg/R/mllib_clustering.R @@ -41,6 +41,12 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' @note LDAModel since 2.1.0 setClass("LDAModel", representation(jobj = "jobj")) +#' S4 class that represents a PowerIterationClustering +#' +#' @param jobj a Java object reference to the backing Scala PowerIterationClustering +#' @note PowerIterationClustering since 3.0.0 +setClass("PowerIterationClustering", slots = list(jobj = "jobj")) + #' Bisecting K-Means Clustering Model #' #' Fits a bisecting k-means clustering model against a SparkDataFrame. @@ -610,3 +616,56 @@ setMethod("write.ml", signature(object = "LDAModel", path = "character"), function(object, path, overwrite = FALSE) { write_internal(object, path, overwrite) }) + +#' PowerIterationClustering +#' +#' A scalable graph clustering algorithm. Users can call \code{spark.assignClusters} to +#' return a cluster assignment for each input vertex. +#' Run the PIC algorithm and returns a cluster assignment for each input vertex. +#' @param data a SparkDataFrame. +#' @param k the number of clusters to create. +#' @param initMode the initialization algorithm; "random" or "degree" +#' @param maxIter the maximum number of iterations. +#' @param sourceCol the name of the input column for source vertex IDs. +#' @param destinationCol the name of the input column for destination vertex IDs +#' @param weightCol weight column name. If this is not set or \code{NULL}, +#' we treat all instance weights as 1.0. +#' @param ... additional argument(s) passed to the method. +#' @return A dataset that contains columns of vertex id and the corresponding cluster for the id. +#' The schema of it will be: \code{id: integer}, \code{cluster: integer} +#' @rdname spark.powerIterationClustering +#' @aliases spark.assignClusters,SparkDataFrame-method +#' @examples +#' \dontrun{ +#' df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), +#' list(1L, 2L, 1.0), list(3L, 4L, 1.0), +#' list(4L, 0L, 0.1)), +#' schema = c("src", "dst", "weight")) +#' clusters <- spark.assignClusters(df, initMode = "degree", weightCol = "weight") +#' showDF(clusters) +#' } +#' @note spark.assignClusters(SparkDataFrame) since 3.0.0 +setMethod("spark.assignClusters", + signature(data = "SparkDataFrame"), + function(data, k = 2L, initMode = c("random", "degree"), maxIter = 20L, + sourceCol = "src", destinationCol = "dst", weightCol = NULL) { + if (!is.integer(k) || k < 1) { + stop("k should be a number with value >= 1.") + } + if (!is.integer(maxIter) || maxIter <= 0) { + stop("maxIter should be a number with value > 0.") + } + initMode <- match.arg(initMode) + if (!is.null(weightCol) && weightCol == "") { + weightCol <- NULL + } else if (!is.null(weightCol)) { + weightCol <- as.character(weightCol) + } + jobj <- callJStatic("org.apache.spark.ml.r.PowerIterationClusteringWrapper", + "getPowerIterationClustering", + as.integer(k), initMode, + as.integer(maxIter), as.character(sourceCol), + as.character(destinationCol), weightCol) + object <- new("PowerIterationClustering", jobj = jobj) + dataFrame(callJMethod(object@jobj, "assignClusters", data@sdf)) + }) diff --git a/R/pkg/R/mllib_fpm.R b/R/pkg/R/mllib_fpm.R index ac37580c6b37..0cc7a16c302d 100644 --- a/R/pkg/R/mllib_fpm.R +++ b/R/pkg/R/mllib_fpm.R @@ -183,16 +183,17 @@ setMethod("write.ml", signature(object = "FPGrowthModel", path = "character"), #' @return A complete set of frequent sequential patterns in the input sequences of itemsets. #' The returned \code{SparkDataFrame} contains columns of sequence and corresponding #' frequency. The schema of it will be: -#' \code{sequence: ArrayType(ArrayType(T))} (T is the item type) -#' \code{freq: Long} +#' \code{sequence: ArrayType(ArrayType(T))}, \code{freq: integer} +#' where T is the item type #' @rdname spark.prefixSpan #' @aliases findFrequentSequentialPatterns,PrefixSpan,SparkDataFrame-method #' @examples #' \dontrun{ #' df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), -#' list(list(list(1L), list(3L, 2L), list(1L, 2L))), -#' list(list(list(1L, 2L), list(5L))), -#' list(list(list(6L)))), schema = c("sequence")) +#' list(list(list(1L), list(3L, 2L), list(1L, 2L))), +#' list(list(list(1L, 2L), list(5L))), +#' list(list(list(6L)))), +#' schema = c("sequence")) #' frequency <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, #' maxLocalProjDBSize = 32000000L) #' showDF(frequency) diff --git a/R/pkg/R/mllib_tree.R b/R/pkg/R/mllib_tree.R index 0e60842dd44c..9844061cfd07 100644 --- a/R/pkg/R/mllib_tree.R +++ b/R/pkg/R/mllib_tree.R @@ -157,7 +157,7 @@ print.summary.decisionTree <- function(x) { #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the @@ -382,7 +382,7 @@ setMethod("write.ml", signature(object = "GBTClassificationModel", path = "chara #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the @@ -588,7 +588,7 @@ setMethod("write.ml", signature(object = "RandomForestClassificationModel", path #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 263b9b576c0c..0d6f32c8f7e1 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -220,3 +220,18 @@ writeArgs <- function(con, args) { } } } + +writeSerializeInArrow <- function(conn, df) { + # This is a hack to avoid CRAN check. Arrow is not uploaded into CRAN now. See ARROW-3204. + requireNamespace1 <- requireNamespace + if (requireNamespace1("arrow", quietly = TRUE)) { + write_arrow <- get("write_arrow", envir = asNamespace("arrow"), inherits = FALSE) + + # There looks no way to send each batch in streaming format via socket + # connection. See ARROW-4512. + # So, it writes the whole Arrow streaming-formatted binary at once for now. + writeRaw(conn, write_arrow(df, raw())) + } else { + stop("'arrow' package should be installed.") + } +} diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index ac289d38d01b..31b986c326d0 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -269,7 +269,7 @@ sparkR.sparkContext <- function( #' sparkR.session("yarn-client", "SparkR", "/home/spark", #' list(spark.executor.memory="4g"), #' c("one.jar", "two.jar", "three.jar"), -#' c("com.databricks:spark-avro_2.11:2.0.1")) +#' c("com.databricks:spark-avro_2.12:2.0.1")) #' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g") #'} #' @note sparkR.session since 2.0.0 diff --git a/R/pkg/R/types.R b/R/pkg/R/types.R index ade0f05c0254..55f75508e88f 100644 --- a/R/pkg/R/types.R +++ b/R/pkg/R/types.R @@ -83,3 +83,38 @@ specialtypeshandle <- function(type) { } returntype } + +# Helper function that checks supported types in Arrow. +checkSchemaInArrow <- function(schema) { + stopifnot(inherits(schema, "structType")) + + requireNamespace1 <- requireNamespace + if (!requireNamespace1("arrow", quietly = TRUE)) { + stop("'arrow' package should be installed.") + } + + # Both cases below produce a corrupt value for unknown reason. It needs to be investigated. + if (any(sapply(schema$fields(), function(x) x$dataType.toString() == "FloatType"))) { + stop("Arrow optimization in R does not support float type yet.") + } + if (any(sapply(schema$fields(), function(x) x$dataType.toString() == "BinaryType"))) { + stop("Arrow optimization in R does not support binary type yet.") + } + if (any(sapply(schema$fields(), + function(x) startsWith(x$dataType.toString(), + "ArrayType")))) { + stop("Arrow optimization in R does not support array type yet.") + } + + # Arrow optimization in Spark does not yet support both cases below. + if (any(sapply(schema$fields(), + function(x) startsWith(x$dataType.toString(), + "StructType")))) { + stop("Arrow optimization in R does not support nested struct type yet.") + } + if (any(sapply(schema$fields(), + function(x) startsWith(x$dataType.toString(), + "MapType")))) { + stop("Arrow optimization in R does not support map type yet.") + } +} diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R index 32eb3671b594..e4e0d032997d 100644 --- a/R/pkg/inst/profile/shell.R +++ b/R/pkg/inst/profile/shell.R @@ -33,19 +33,19 @@ sc <- SparkR:::callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", spark) assign("sc", sc, envir = .GlobalEnv) sparkVer <- SparkR:::callJMethod(sc, "version") - cat("\n Welcome to") + cat("\nWelcome to") cat("\n") - cat(" ____ __", "\n") - cat(" / __/__ ___ _____/ /__", "\n") - cat(" _\\ \\/ _ \\/ _ `/ __/ '_/", "\n") - cat(" /___/ .__/\\_,_/_/ /_/\\_\\") + cat(" ____ __", "\n") + cat(" / __/__ ___ _____/ /__", "\n") + cat(" _\\ \\/ _ \\/ _ `/ __/ '_/", "\n") + cat(" /___/ .__/\\_,_/_/ /_/\\_\\") if (nchar(sparkVer) == 0) { cat("\n") } else { - cat(" version ", sparkVer, "\n") + cat(" version", sparkVer, "\n") } - cat(" /_/", "\n") + cat(" /_/", "\n") cat("\n") - cat("\n SparkSession available as 'spark'.\n") + cat("\nSparkSession available as 'spark'.\n") } diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index c2adf613acb0..80dc4ee63451 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -49,7 +49,7 @@ compute <- function(mode, partition, serializer, deserializer, key, names(inputData) <- colNames } else { # Check to see if inputData is a valid data.frame - stopifnot(deserializer == "byte") + stopifnot(deserializer == "byte" || deserializer == "arrow") stopifnot(class(inputData) == "data.frame") } @@ -63,7 +63,7 @@ compute <- function(mode, partition, serializer, deserializer, key, output <- split(output, seq(nrow(output))) } else { # Serialize the output to a byte array - stopifnot(serializer == "byte") + stopifnot(serializer == "byte" || serializer == "arrow") } } else { output <- computeFunc(partition, inputData) @@ -76,6 +76,8 @@ outputResult <- function(serializer, output, outputCon) { SparkR:::writeRawSerialize(outputCon, output) } else if (serializer == "row") { SparkR:::writeRowSerialize(outputCon, output) + } else if (serializer == "arrow") { + SparkR:::writeSerializeInArrow(outputCon, output) } else { # write lines one-by-one with flag lapply(output, function(line) SparkR:::writeString(outputCon, line)) @@ -171,6 +173,16 @@ if (isEmpty != 0) { data <- dataWithKeys$data } else if (deserializer == "row") { data <- SparkR:::readMultipleObjects(inputCon) + } else if (deserializer == "arrow" && mode == 2) { + dataWithKeys <- SparkR:::readDeserializeWithKeysInArrow(inputCon) + keys <- dataWithKeys$keys + data <- dataWithKeys$data + } else if (deserializer == "arrow" && mode == 1) { + data <- SparkR:::readDeserializeInArrow(inputCon) + # See https://stat.ethz.ch/pipermail/r-help/2010-September/252046.html + # rbind.fill might be an anternative to make it faster if plyr is installed. + # Also, note that, 'dapply' applies a function to each partition. + data <- do.call("rbind", data) } # Timing reading input data for execution @@ -181,17 +193,29 @@ if (isEmpty != 0) { colNames, computeFunc, data) } else { # gapply mode + outputs <- list() for (i in 1:length(data)) { # Timing reading input data for execution inputElap <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], colNames, computeFunc, data[[i]]) computeElap <- elapsedSecs() - outputResult(serializer, output, outputCon) + if (serializer == "arrow") { + outputs[[length(outputs) + 1L]] <- output + } else { + outputResult(serializer, output, outputCon) + } outputElap <- elapsedSecs() computeInputElapsDiff <- computeInputElapsDiff + (computeElap - inputElap) outputComputeElapsDiff <- outputComputeElapsDiff + (outputElap - computeElap) } + + if (serializer == "arrow") { + # See https://stat.ethz.ch/pipermail/r-help/2010-September/252046.html + # rbind.fill might be an anternative to make it faster if plyr is installed. + combined <- do.call("rbind", outputs) + SparkR:::writeSerializeInArrow(outputCon, combined) + } } } else { output <- compute(mode, partition, serializer, deserializer, NULL, diff --git a/R/pkg/tests/fulltests/test_Serde.R b/R/pkg/tests/fulltests/test_Serde.R index 1525bdb2f5c8..e01f6ee00521 100644 --- a/R/pkg/tests/fulltests/test_Serde.R +++ b/R/pkg/tests/fulltests/test_Serde.R @@ -138,7 +138,7 @@ test_that("createDataFrame large objects", { enableHiveSupport = FALSE)) sc <- getSparkContext() - actual <- callJStatic("org.apache.spark.api.r.RUtils", "getEncryptionEnabled", sc) + actual <- callJStatic("org.apache.spark.api.r.RUtils", "isEncryptionEnabled", sc) expected <- as.logical(encryptionEnabled) expect_equal(actual, expected) diff --git a/R/pkg/tests/fulltests/test_client.R b/R/pkg/tests/fulltests/test_client.R index de624b572cc2..9798627ffc55 100644 --- a/R/pkg/tests/fulltests/test_client.R +++ b/R/pkg/tests/fulltests/test_client.R @@ -37,7 +37,7 @@ test_that("multiple packages don't produce a warning", { test_that("sparkJars sparkPackages as character vectors", { args <- generateSparkSubmitArgs("", "", c("one.jar", "two.jar", "three.jar"), "", - c("com.databricks:spark-avro_2.11:2.0.1")) + c("com.databricks:spark-avro_2.12:2.0.1")) expect_match(args, "--jars one.jar,two.jar,three.jar") - expect_match(args, "--packages com.databricks:spark-avro_2.11:2.0.1") + expect_match(args, "--packages com.databricks:spark-avro_2.12:2.0.1") }) diff --git a/R/pkg/tests/fulltests/test_mllib_classification.R b/R/pkg/tests/fulltests/test_mllib_classification.R index 023686e75d50..1f1b187aef56 100644 --- a/R/pkg/tests/fulltests/test_mllib_classification.R +++ b/R/pkg/tests/fulltests/test_mllib_classification.R @@ -299,7 +299,7 @@ test_that("spark.mlp", { df <- read.df(absoluteSparkPath("data/mllib/sample_multiclass_classification_data.txt"), source = "libsvm") model <- spark.mlp(df, label ~ features, blockSize = 128, layers = c(4, 5, 4, 3), - solver = "l-bfgs", maxIter = 100, tol = 0.5, stepSize = 1, seed = 1) + solver = "l-bfgs", maxIter = 100, tol = 0.00001, stepSize = 1, seed = 1) # Test summary method summary <- summary(model) @@ -307,13 +307,13 @@ test_that("spark.mlp", { expect_equal(summary$numOfOutputs, 3) expect_equal(summary$layers, c(4, 5, 4, 3)) expect_equal(length(summary$weights), 64) - expect_equal(head(summary$weights, 5), list(-0.878743, 0.2154151, -1.16304, -0.6583214, 1.009825), + expect_equal(head(summary$weights, 5), list(-24.28415, 107.8701, 16.86376, 1.103736, 9.244488), tolerance = 1e-6) # Test predict method mlpTestDF <- df mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) - expect_equal(head(mlpPredictions$prediction, 6), c("1.0", "0.0", "0.0", "0.0", "0.0", "0.0")) + expect_equal(head(mlpPredictions$prediction, 6), c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0")) # Test model save/load if (windows_with_hadoop()) { @@ -348,12 +348,12 @@ test_that("spark.mlp", { # Test random seed # default seed - model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 10) + model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 100) mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) expect_equal(head(mlpPredictions$prediction, 10), c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0", "2.0", "2.0", "1.0", "0.0")) # seed equals 10 - model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 10, seed = 10) + model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 100, seed = 10) mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) expect_equal(head(mlpPredictions$prediction, 10), c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0", "2.0", "2.0", "1.0", "0.0")) diff --git a/R/pkg/tests/fulltests/test_mllib_clustering.R b/R/pkg/tests/fulltests/test_mllib_clustering.R index 4110e13da494..028ad574b813 100644 --- a/R/pkg/tests/fulltests/test_mllib_clustering.R +++ b/R/pkg/tests/fulltests/test_mllib_clustering.R @@ -153,7 +153,7 @@ test_that("spark.kmeans", { model <- spark.kmeans(data = training, ~ ., k = 2, maxIter = 10, initMode = "random") sample <- take(select(predict(model, training), "prediction"), 1) expect_equal(typeof(sample$prediction), "integer") - expect_equal(sample$prediction, 1) + expect_equal(sample$prediction, 0) # Test stats::kmeans is working statsModel <- kmeans(x = newIris, centers = 2) @@ -319,4 +319,17 @@ test_that("spark.posterior and spark.perplexity", { expect_equal(length(local.posterior), sum(unlist(local.posterior))) }) +test_that("spark.assignClusters", { + df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), + list(1L, 2L, 1.0), list(3L, 4L, 1.0), + list(4L, 0L, 0.1)), + schema = c("src", "dst", "weight")) + clusters <- spark.assignClusters(df, initMode = "degree", weightCol = "weight") + expected_result <- createDataFrame(list(list(4L, 1L), list(0L, 0L), + list(1L, 0L), list(3L, 1L), + list(2L, 0L)), + schema = c("id", "cluster")) + expect_equivalent(expected_result, clusters) +}) + sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_fpm.R b/R/pkg/tests/fulltests/test_mllib_fpm.R index daf9ff97a821..bc1e17538d41 100644 --- a/R/pkg/tests/fulltests/test_mllib_fpm.R +++ b/R/pkg/tests/fulltests/test_mllib_fpm.R @@ -84,19 +84,20 @@ test_that("spark.fpGrowth", { }) test_that("spark.prefixSpan", { - df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), - list(list(list(1L), list(3L, 2L), list(1L, 2L))), - list(list(list(1L, 2L), list(5L))), - list(list(list(6L)))), schema = c("sequence")) - result1 <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, - maxLocalProjDBSize = 32000000L) - - expected_result <- createDataFrame(list(list(list(list(1L)), 3L), - list(list(list(3L)), 2L), - list(list(list(2L)), 3L), - list(list(list(1L, 2L)), 3L), - list(list(list(1L), list(3L)), 2L)), - schema = c("sequence", "freq")) - }) + df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), + list(list(list(1L), list(3L, 2L), list(1L, 2L))), + list(list(list(1L, 2L), list(5L))), + list(list(list(6L)))), + schema = c("sequence")) + result <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, + maxLocalProjDBSize = 32000000L) + + expected_result <- createDataFrame(list(list(list(list(1L)), 3L), list(list(list(3L)), 2L), + list(list(list(2L)), 3L), list(list(list(1L, 2L)), 3L), + list(list(list(1L), list(3L)), 2L)), + schema = c("sequence", "freq")) + + expect_equivalent(expected_result, result) +}) sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_recommendation.R b/R/pkg/tests/fulltests/test_mllib_recommendation.R index 4d919c9d746b..d50de4123aeb 100644 --- a/R/pkg/tests/fulltests/test_mllib_recommendation.R +++ b/R/pkg/tests/fulltests/test_mllib_recommendation.R @@ -27,13 +27,13 @@ test_that("spark.als", { list(2, 1, 1.0), list(2, 2, 5.0)) df <- createDataFrame(data, c("user", "item", "score")) model <- spark.als(df, ratingCol = "score", userCol = "user", itemCol = "item", - rank = 10, maxIter = 5, seed = 0, regParam = 0.1) + rank = 10, maxIter = 15, seed = 0, regParam = 0.1) stats <- summary(model) expect_equal(stats$rank, 10) test <- createDataFrame(list(list(0, 2), list(1, 0), list(2, 0)), c("user", "item")) predictions <- collect(predict(model, test)) - expect_equal(predictions$prediction, c(-0.1380762, 2.6258414, -1.5018409), + expect_equal(predictions$prediction, c(0.6324540, 3.6218479, -0.4568263), tolerance = 1e-4) # Test model save/load diff --git a/R/pkg/tests/fulltests/test_mllib_regression.R b/R/pkg/tests/fulltests/test_mllib_regression.R index 23daca75fcc2..b40c4cb9a969 100644 --- a/R/pkg/tests/fulltests/test_mllib_regression.R +++ b/R/pkg/tests/fulltests/test_mllib_regression.R @@ -102,10 +102,18 @@ test_that("spark.glm and predict", { }) test_that("spark.glm summary", { + # prepare dataset + Sepal.Length <- c(2.0, 1.5, 1.8, 3.4, 5.1, 1.8, 1.0, 2.3) + Sepal.Width <- c(2.1, 2.3, 5.4, 4.7, 3.1, 2.1, 3.1, 5.5) + Petal.Length <- c(1.8, 2.1, 7.1, 2.5, 3.7, 6.3, 2.2, 7.2) + Species <- c("setosa", "versicolor", "versicolor", "versicolor", "virginica", "virginica", + "versicolor", "virginica") + dataset <- data.frame(Sepal.Length, Sepal.Width, Petal.Length, Species, stringsAsFactors = TRUE) + # gaussian family - training <- suppressWarnings(createDataFrame(iris)) + training <- suppressWarnings(createDataFrame(dataset)) stats <- summary(spark.glm(training, Sepal_Width ~ Sepal_Length + Species)) - rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)) + rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = dataset)) # test summary coefficients return matrix type expect_true(class(stats$coefficients) == "matrix") @@ -126,15 +134,15 @@ test_that("spark.glm summary", { out <- capture.output(print(stats)) expect_match(out[2], "Deviance Residuals:") - expect_true(any(grepl("AIC: 59.22", out))) + expect_true(any(grepl("AIC: 35.84", out))) # binomial family - df <- suppressWarnings(createDataFrame(iris)) + df <- suppressWarnings(createDataFrame(dataset)) training <- df[df$Species %in% c("versicolor", "virginica"), ] stats <- summary(spark.glm(training, Species ~ Sepal_Length + Sepal_Width, family = binomial(link = "logit"))) - rTraining <- iris[iris$Species %in% c("versicolor", "virginica"), ] + rTraining <- dataset[dataset$Species %in% c("versicolor", "virginica"), ] rStats <- summary(glm(Species ~ Sepal.Length + Sepal.Width, data = rTraining, family = binomial(link = "logit"))) @@ -174,17 +182,17 @@ test_that("spark.glm summary", { expect_equal(stats$aic, rStats$aic) # Test spark.glm works with offset - training <- suppressWarnings(createDataFrame(iris)) + training <- suppressWarnings(createDataFrame(dataset)) stats <- summary(spark.glm(training, Sepal_Width ~ Sepal_Length + Species, family = poisson(), offsetCol = "Petal_Length")) rStats <- suppressWarnings(summary(glm(Sepal.Width ~ Sepal.Length + Species, - data = iris, family = poisson(), offset = iris$Petal.Length))) + data = dataset, family = poisson(), offset = dataset$Petal.Length))) expect_true(all(abs(rStats$coefficients - stats$coefficients) < 1e-3)) # Test summary works on base GLM models - baseModel <- stats::glm(Sepal.Width ~ Sepal.Length + Species, data = iris) + baseModel <- stats::glm(Sepal.Width ~ Sepal.Length + Species, data = dataset) baseSummary <- summary(baseModel) - expect_true(abs(baseSummary$deviance - 12.19313) < 1e-4) + expect_true(abs(baseSummary$deviance - 11.84013) < 1e-4) # Test spark.glm works with regularization parameter data <- as.data.frame(cbind(a1, a2, b)) @@ -300,11 +308,19 @@ test_that("glm and predict", { }) test_that("glm summary", { + # prepare dataset + Sepal.Length <- c(2.0, 1.5, 1.8, 3.4, 5.1, 1.8, 1.0, 2.3) + Sepal.Width <- c(2.1, 2.3, 5.4, 4.7, 3.1, 2.1, 3.1, 5.5) + Petal.Length <- c(1.8, 2.1, 7.1, 2.5, 3.7, 6.3, 2.2, 7.2) + Species <- c("setosa", "versicolor", "versicolor", "versicolor", "virginica", "virginica", + "versicolor", "virginica") + dataset <- data.frame(Sepal.Length, Sepal.Width, Petal.Length, Species, stringsAsFactors = TRUE) + # gaussian family - training <- suppressWarnings(createDataFrame(iris)) + training <- suppressWarnings(createDataFrame(dataset)) stats <- summary(glm(Sepal_Width ~ Sepal_Length + Species, data = training)) - rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)) + rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = dataset)) coefs <- stats$coefficients rCoefs <- rStats$coefficients @@ -320,12 +336,12 @@ test_that("glm summary", { expect_equal(stats$aic, rStats$aic) # binomial family - df <- suppressWarnings(createDataFrame(iris)) + df <- suppressWarnings(createDataFrame(dataset)) training <- df[df$Species %in% c("versicolor", "virginica"), ] stats <- summary(glm(Species ~ Sepal_Length + Sepal_Width, data = training, family = binomial(link = "logit"))) - rTraining <- iris[iris$Species %in% c("versicolor", "virginica"), ] + rTraining <- dataset[dataset$Species %in% c("versicolor", "virginica"), ] rStats <- summary(glm(Species ~ Sepal.Length + Sepal.Width, data = rTraining, family = binomial(link = "logit"))) diff --git a/R/pkg/tests/fulltests/test_mllib_tree.R b/R/pkg/tests/fulltests/test_mllib_tree.R index facd3a941cf1..ad68700c7ff4 100644 --- a/R/pkg/tests/fulltests/test_mllib_tree.R +++ b/R/pkg/tests/fulltests/test_mllib_tree.R @@ -148,10 +148,10 @@ test_that("spark.randomForest", { model <- spark.randomForest(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, numTrees = 20, seed = 123) predictions <- collect(predict(model, data)) - expect_equal(predictions$prediction, c(60.32820, 61.22315, 60.69025, 62.11070, - 63.53160, 64.05470, 65.12710, 64.30450, - 66.70910, 67.86125, 68.08700, 67.21865, - 68.89275, 69.53180, 69.39640, 69.68250), + expect_equal(predictions$prediction, c(60.32495, 61.06495, 60.52120, 61.98500, + 63.64450, 64.21910, 65.00810, 64.30450, + 66.70910, 67.96875, 68.22140, 67.21865, + 68.89275, 69.55900, 69.30160, 69.93050), tolerance = 1e-4) stats <- summary(model) expect_equal(stats$numTrees, 20) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 77a29c9ecad8..fdc747482065 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1390,7 +1390,7 @@ test_that("column functions", { c9 <- signum(c) + sin(c) + sinh(c) + size(c) + stddev(c) + soundex(c) + sqrt(c) + sum(c) c10 <- sumDistinct(c) + tan(c) + tanh(c) + degrees(c) + radians(c) c11 <- to_date(c) + trim(c) + unbase64(c) + unhex(c) + upper(c) - c12 <- variance(c) + ltrim(c, "a") + rtrim(c, "b") + trim(c, "c") + c12 <- variance(c) + xxhash64(c) + ltrim(c, "a") + rtrim(c, "b") + trim(c, "c") c13 <- lead("col", 1) + lead(c, 1) + lag("col", 1) + lag(c, 1) c14 <- cume_dist() + ntile(1) + corr(c, c1) c15 <- dense_rank() + percent_rank() + rank() + row_number() @@ -1440,6 +1440,14 @@ test_that("column functions", { df5 <- createDataFrame(list(list(a = "010101"))) expect_equal(collect(select(df5, conv(df5$a, 2, 16)))[1, 1], "15") + # Test months_between() + df <- createDataFrame(list(list(a = as.Date("1997-02-28"), + b = as.Date("1996-10-30")))) + result1 <- collect(select(df, alias(months_between(df[[1]], df[[2]]), "month")))[[1]] + expect_equal(result1, 3.93548387) + result2 <- collect(select(df, alias(months_between(df[[1]], df[[2]], FALSE), "month")))[[1]] + expect_equal(result2, 3.935483870967742) + # Test array_contains(), array_max(), array_min(), array_position(), element_at() and reverse() df <- createDataFrame(list(list(list(1L, 2L, 3L)), list(list(6L, 5L, 4L)))) result <- collect(select(df, array_contains(df[[1]], 1L)))[[1]] @@ -1485,6 +1493,13 @@ test_that("column functions", { expected_entries <- list(as.environment(list(x = 1, y = 2))) expect_equal(result, expected_entries) + # Test map_from_entries() + df <- createDataFrame(list(list(list(listToStruct(list(c1 = "x", c2 = 1L)), + listToStruct(list(c1 = "y", c2 = 2L)))))) + result <- collect(select(df, map_from_entries(df[[1]])))[[1]] + expected_entries <- list(as.environment(list(x = 1L, y = 2L))) + expect_equal(result, expected_entries) + # Test array_repeat() df <- createDataFrame(list(list("a", 3L), list("b", 2L))) result <- collect(select(df, array_repeat(df[[1]], df[[2]])))[[1]] @@ -1543,6 +1558,13 @@ test_that("column functions", { result <- collect(select(df, flatten(df[[1]])))[[1]] expect_equal(result, list(list(1L, 2L, 3L, 4L), list(5L, 6L, 7L, 8L))) + # Test map_concat + df <- createDataFrame(list(list(map1 = as.environment(list(x = 1, y = 2)), + map2 = as.environment(list(a = 3, b = 4))))) + result <- collect(select(df, map_concat(df[[1]], df[[2]])))[[1]] + expected_entries <- list(as.environment(list(x = 1, y = 2, a = 3, b = 4))) + expect_equal(result, expected_entries) + # Test map_entries(), map_keys(), map_values() and element_at() df <- createDataFrame(list(list(map = as.environment(list(x = 1, y = 2))))) result <- collect(select(df, map_entries(df$map)))[[1]] @@ -1620,14 +1642,26 @@ test_that("column functions", { expect_equal(collect(select(df, bround(df$x, 0)))[[1]][1], 2) expect_equal(collect(select(df, bround(df$x, 0)))[[1]][2], 4) - # Test from_csv() + # Test from_csv(), schema_of_csv() df <- as.DataFrame(list(list("col" = "1"))) c <- collect(select(df, alias(from_csv(df$col, "a INT"), "csv"))) expect_equal(c[[1]][[1]]$a, 1) c <- collect(select(df, alias(from_csv(df$col, lit("a INT")), "csv"))) expect_equal(c[[1]][[1]]$a, 1) + c <- collect(select(df, alias(from_csv(df$col, structType("a INT")), "csv"))) + expect_equal(c[[1]][[1]]$a, 1) + c <- collect(select(df, alias(from_csv(df$col, schema_of_csv("1")), "csv"))) + expect_equal(c[[1]][[1]]$`_c0`, 1) + c <- collect(select(df, alias(from_csv(df$col, schema_of_csv(lit("1"))), "csv"))) + expect_equal(c[[1]][[1]]$`_c0`, 1) + + df <- as.DataFrame(list(list("col" = "1"))) + c <- collect(select(df, schema_of_csv("Amsterdam,2018"))) + expect_equal(c[[1]], "struct<_c0:string,_c1:int>") + c <- collect(select(df, schema_of_csv(lit("Amsterdam,2018")))) + expect_equal(c[[1]], "struct<_c0:string,_c1:int>") - # Test to_json(), from_json() + # Test to_json(), from_json(), schema_of_json() df <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people") j <- collect(select(df, alias(to_json(df$people), "json"))) expect_equal(j[order(j$json), ][1], "[{\"name\":\"Bob\"},{\"name\":\"Alice\"}]") @@ -1645,7 +1679,9 @@ test_that("column functions", { expect_equal(j[order(j$json), ][1], "{\"age\":16,\"height\":176.5}") df <- as.DataFrame(j) schemas <- list(structType(structField("age", "integer"), structField("height", "double")), - "age INT, height DOUBLE") + "age INT, height DOUBLE", + schema_of_json("{\"age\":16,\"height\":176.5}"), + schema_of_json(lit("{\"age\":16,\"height\":176.5}"))) for (schema in schemas) { s <- collect(select(df, alias(from_json(df$json, schema), "structcol"))) expect_equal(ncol(s), 1) @@ -1654,6 +1690,12 @@ test_that("column functions", { expect_true(any(apply(s, 1, function(x) { x[[1]]$age == 16 }))) } + df <- as.DataFrame(list(list("col" = "1"))) + c <- collect(select(df, schema_of_json('{"name":"Bob"}'))) + expect_equal(c[[1]], "struct") + c <- collect(select(df, schema_of_json(lit('{"name":"Bob"}')))) + expect_equal(c[[1]], "struct") + # Test to_json() supports arrays of primitive types and arrays df <- sql("SELECT array(19, 42, 70) as age") j <- collect(select(df, alias(to_json(df$age), "json"))) @@ -1674,12 +1716,16 @@ test_that("column functions", { # check for unparseable df <- as.DataFrame(list(list("a" = ""))) - expect_equal(collect(select(df, from_json(df$a, schema)))[[1]][[1]]$a, NA) + expect_equal(collect(select(df, from_json(df$a, schema)))[[1]][[1]], NA) # check if array type in string is correctly supported. jsonArr <- "[{\"name\":\"Bob\"}, {\"name\":\"Alice\"}]" df <- as.DataFrame(list(list("people" = jsonArr))) - for (schema in list(structType(structField("name", "string")), "name STRING")) { + schemas <- list(structType(structField("name", "string")), + "name STRING", + schema_of_json("{\"name\":\"Alice\"}"), + schema_of_json(lit("{\"name\":\"Bob\"}"))) + for (schema in schemas) { arr <- collect(select(df, alias(from_json(df$people, schema, as.json.array = TRUE), "arrcol"))) expect_equal(ncol(arr), 1) expect_equal(nrow(arr), 1) @@ -1740,9 +1786,9 @@ test_that("column binary mathfunctions", { expect_equal(collect(select(df, shiftRight(df$b, 1)))[4, 1], 4) expect_equal(collect(select(df, shiftRightUnsigned(df$b, 1)))[4, 1], 4) expect_equal(class(collect(select(df, rand()))[2, 1]), "numeric") - expect_equal(collect(select(df, rand(1)))[1, 1], 0.134, tolerance = 0.01) + expect_equal(collect(select(df, rand(1)))[1, 1], 0.636, tolerance = 0.01) expect_equal(class(collect(select(df, randn()))[2, 1]), "numeric") - expect_equal(collect(select(df, randn(1)))[1, 1], -1.03, tolerance = 0.01) + expect_equal(collect(select(df, randn(1)))[1, 1], 1.68, tolerance = 0.01) }) test_that("string operators", { @@ -1859,10 +1905,20 @@ test_that("date functions on a DataFrame", { df2 <- createDataFrame(l2) expect_equal(collect(select(df2, minute(df2$b)))[, 1], c(34, 24)) expect_equal(collect(select(df2, second(df2$b)))[, 1], c(0, 34)) - expect_equal(collect(select(df2, from_utc_timestamp(df2$b, "JST")))[, 1], - c(as.POSIXct("2012-12-13 21:34:00 UTC"), as.POSIXct("2014-12-15 10:24:34 UTC"))) - expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], - c(as.POSIXct("2012-12-13 03:34:00 UTC"), as.POSIXct("2014-12-14 16:24:34 UTC"))) + conf <- callJMethod(sparkSession, "conf") + isUtcTimestampFuncEnabled <- callJMethod(conf, "get", "spark.sql.legacy.utcTimestampFunc.enabled") + callJMethod(conf, "set", "spark.sql.legacy.utcTimestampFunc.enabled", "true") + tryCatch({ + # Both from_utc_timestamp and to_utc_timestamp are deprecated as of SPARK-25496 + expect_equal(suppressWarnings(collect(select(df2, from_utc_timestamp(df2$b, "JST"))))[, 1], + c(as.POSIXct("2012-12-13 21:34:00 UTC"), as.POSIXct("2014-12-15 10:24:34 UTC"))) + expect_equal(suppressWarnings(collect(select(df2, to_utc_timestamp(df2$b, "JST"))))[, 1], + c(as.POSIXct("2012-12-13 03:34:00 UTC"), as.POSIXct("2014-12-14 16:24:34 UTC"))) + }, + finally = { + # Reverting the conf back + callJMethod(conf, "set", "spark.sql.legacy.utcTimestampFunc.enabled", isUtcTimestampFuncEnabled) + }) expect_gt(collect(select(df2, unix_timestamp()))[1, 1], 0) expect_gt(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) @@ -2310,39 +2366,94 @@ test_that("join(), crossJoin() and merge() on a DataFrame", { expect_equal(names(joined2), c("age", "name", "name", "test")) expect_equal(count(joined2), 3) - joined3 <- join(df, df2, df$name == df2$name, "rightouter") + joined3 <- join(df, df2, df$name == df2$name, "right") expect_equal(names(joined3), c("age", "name", "name", "test")) expect_equal(count(joined3), 4) expect_true(is.na(collect(orderBy(joined3, joined3$age))$age[2])) - joined4 <- select(join(df, df2, df$name == df2$name, "outer"), - alias(df$age + 5, "newAge"), df$name, df2$test) - expect_equal(names(joined4), c("newAge", "name", "test")) + joined4 <- join(df, df2, df$name == df2$name, "right_outer") + expect_equal(names(joined4), c("age", "name", "name", "test")) expect_equal(count(joined4), 4) - expect_equal(collect(orderBy(joined4, joined4$name))$newAge[3], 24) + expect_true(is.na(collect(orderBy(joined4, joined4$age))$age[2])) - joined5 <- join(df, df2, df$name == df2$name, "leftouter") + joined5 <- join(df, df2, df$name == df2$name, "rightouter") expect_equal(names(joined5), c("age", "name", "name", "test")) - expect_equal(count(joined5), 3) - expect_true(is.na(collect(orderBy(joined5, joined5$age))$age[1])) + expect_equal(count(joined5), 4) + expect_true(is.na(collect(orderBy(joined5, joined5$age))$age[2])) - joined6 <- join(df, df2, df$name == df2$name, "inner") - expect_equal(names(joined6), c("age", "name", "name", "test")) - expect_equal(count(joined6), 3) - joined7 <- join(df, df2, df$name == df2$name, "leftsemi") - expect_equal(names(joined7), c("age", "name")) - expect_equal(count(joined7), 3) + joined6 <- select(join(df, df2, df$name == df2$name, "outer"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined6), c("newAge", "name", "test")) + expect_equal(count(joined6), 4) + expect_equal(collect(orderBy(joined6, joined6$name))$newAge[3], 24) + + joined7 <- select(join(df, df2, df$name == df2$name, "full"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined7), c("newAge", "name", "test")) + expect_equal(count(joined7), 4) + expect_equal(collect(orderBy(joined7, joined7$name))$newAge[3], 24) - joined8 <- join(df, df2, df$name == df2$name, "left_outer") - expect_equal(names(joined8), c("age", "name", "name", "test")) - expect_equal(count(joined8), 3) - expect_true(is.na(collect(orderBy(joined8, joined8$age))$age[1])) + joined8 <- select(join(df, df2, df$name == df2$name, "fullouter"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined8), c("newAge", "name", "test")) + expect_equal(count(joined8), 4) + expect_equal(collect(orderBy(joined8, joined8$name))$newAge[3], 24) - joined9 <- join(df, df2, df$name == df2$name, "right_outer") - expect_equal(names(joined9), c("age", "name", "name", "test")) + joined9 <- select(join(df, df2, df$name == df2$name, "full_outer"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined9), c("newAge", "name", "test")) expect_equal(count(joined9), 4) - expect_true(is.na(collect(orderBy(joined9, joined9$age))$age[2])) + expect_equal(collect(orderBy(joined9, joined9$name))$newAge[3], 24) + + joined10 <- join(df, df2, df$name == df2$name, "left") + expect_equal(names(joined10), c("age", "name", "name", "test")) + expect_equal(count(joined10), 3) + expect_true(is.na(collect(orderBy(joined10, joined10$age))$age[1])) + + joined11 <- join(df, df2, df$name == df2$name, "leftouter") + expect_equal(names(joined11), c("age", "name", "name", "test")) + expect_equal(count(joined11), 3) + expect_true(is.na(collect(orderBy(joined11, joined11$age))$age[1])) + + joined12 <- join(df, df2, df$name == df2$name, "left_outer") + expect_equal(names(joined12), c("age", "name", "name", "test")) + expect_equal(count(joined12), 3) + expect_true(is.na(collect(orderBy(joined12, joined12$age))$age[1])) + + joined13 <- join(df, df2, df$name == df2$name, "inner") + expect_equal(names(joined13), c("age", "name", "name", "test")) + expect_equal(count(joined13), 3) + + joined14 <- join(df, df2, df$name == df2$name, "semi") + expect_equal(names(joined14), c("age", "name")) + expect_equal(count(joined14), 3) + + joined14 <- join(df, df2, df$name == df2$name, "leftsemi") + expect_equal(names(joined14), c("age", "name")) + expect_equal(count(joined14), 3) + + joined15 <- join(df, df2, df$name == df2$name, "left_semi") + expect_equal(names(joined15), c("age", "name")) + expect_equal(count(joined15), 3) + + joined16 <- join(df2, df, df2$name == df$name, "anti") + expect_equal(names(joined16), c("name", "test")) + expect_equal(count(joined16), 1) + + joined17 <- join(df2, df, df2$name == df$name, "leftanti") + expect_equal(names(joined17), c("name", "test")) + expect_equal(count(joined17), 1) + + joined18 <- join(df2, df, df2$name == df$name, "left_anti") + expect_equal(names(joined18), c("name", "test")) + expect_equal(count(joined18), 1) + + error_msg <- paste("joinType must be one of the following types:", + "'inner', 'cross', 'outer', 'full', 'fullouter', 'full_outer',", + "'left', 'leftouter', 'left_outer', 'right', 'rightouter', 'right_outer',", + "'semi', 'leftsemi', 'left_semi', 'anti', 'leftanti' or 'left_anti'.") + expect_error(join(df2, df, df2$name == df$name, "invalid"), error_msg) merged <- merge(df, df2, by.x = "name", by.y = "name", all.x = TRUE, all.y = TRUE) expect_equal(count(merged), 4) @@ -2925,7 +3036,7 @@ test_that("sampleBy() on a DataFrame", { sample <- sampleBy(df, "key", fractions, 0) result <- collect(orderBy(count(groupBy(sample, "key")), "key")) expect_identical(as.list(result[1, ]), list(key = "0", count = 3)) - expect_identical(as.list(result[2, ]), list(key = "1", count = 7)) + expect_identical(as.list(result[2, ]), list(key = "1", count = 8)) }) test_that("approxQuantile() on a DataFrame", { diff --git a/R/pkg/tests/fulltests/test_sparkSQL_arrow.R b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R new file mode 100644 index 000000000000..25a6d3c6ce36 --- /dev/null +++ b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R @@ -0,0 +1,315 @@ +# +# 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. +# + +library(testthat) + +context("SparkSQL Arrow optimization") + +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +test_that("createDataFrame/collect Arrow optimization", { + skip_if_not_installed("arrow") + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "false") + tryCatch({ + expected <- collect(createDataFrame(mtcars)) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + expect_equal(collect(createDataFrame(mtcars)), expected) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("createDataFrame/collect Arrow optimization - many partitions (partition order test)", { + skip_if_not_installed("arrow") + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + expect_equal(collect(createDataFrame(mtcars, numPartitions = 32)), + collect(createDataFrame(mtcars, numPartitions = 1))) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("createDataFrame/collect Arrow optimization - type specification", { + skip_if_not_installed("arrow") + rdf <- data.frame(list(list(a = 1, + b = "a", + c = TRUE, + d = 1.1, + e = 1L, + f = as.Date("1990-02-24"), + g = as.POSIXct("1990-02-24 12:34:56")))) + + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + conf <- callJMethod(sparkSession, "conf") + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "false") + tryCatch({ + expected <- collect(createDataFrame(rdf)) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + expect_equal(collect(createDataFrame(rdf)), expected) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("dapply() Arrow optimization", { + skip_if_not_installed("arrow") + df <- createDataFrame(mtcars) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "false") + tryCatch({ + ret <- dapply(df, + function(rdf) { + stopifnot(class(rdf) == "data.frame") + rdf + }, + schema(df)) + expected <- collect(ret) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + ret <- dapply(df, + function(rdf) { + stopifnot(class(rdf) == "data.frame") + # mtcars' hp is more then 50. + stopifnot(all(rdf$hp > 50)) + rdf + }, + schema(df)) + actual <- collect(ret) + expect_equal(actual, expected) + expect_equal(count(ret), nrow(mtcars)) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("dapply() Arrow optimization - type specification", { + skip_if_not_installed("arrow") + # Note that regular dapply() seems not supporting date and timestamps + # whereas Arrow-optimized dapply() does. + rdf <- data.frame(list(list(a = 1, + b = "a", + c = TRUE, + d = 1.1, + e = 1L))) + # numPartitions are set to 8 intentionally to test empty partitions as well. + df <- createDataFrame(rdf, numPartitions = 8) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "false") + tryCatch({ + ret <- dapply(df, function(rdf) { rdf }, schema(df)) + expected <- collect(ret) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + ret <- dapply(df, function(rdf) { rdf }, schema(df)) + actual <- collect(ret) + expect_equal(actual, expected) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("dapply() Arrow optimization - type specification (date and timestamp)", { + skip_if_not_installed("arrow") + rdf <- data.frame(list(list(a = as.Date("1990-02-24"), + b = as.POSIXct("1990-02-24 12:34:56")))) + df <- createDataFrame(rdf) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + ret <- dapply(df, function(rdf) { rdf }, schema(df)) + expect_equal(collect(ret), rdf) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("gapply() Arrow optimization", { + skip_if_not_installed("arrow") + df <- createDataFrame(mtcars) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "false") + tryCatch({ + ret <- gapply(df, + "gear", + function(key, grouped) { + if (length(key) > 0) { + stopifnot(is.numeric(key[[1]])) + } + stopifnot(class(grouped) == "data.frame") + grouped + }, + schema(df)) + expected <- collect(ret) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + ret <- gapply(df, + "gear", + function(key, grouped) { + if (length(key) > 0) { + stopifnot(is.numeric(key[[1]])) + } + stopifnot(class(grouped) == "data.frame") + stopifnot(length(colnames(grouped)) == 11) + # mtcars' hp is more then 50. + stopifnot(all(grouped$hp > 50)) + grouped + }, + schema(df)) + actual <- collect(ret) + expect_equal(actual, expected) + expect_equal(count(ret), nrow(mtcars)) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("gapply() Arrow optimization - type specification", { + skip_if_not_installed("arrow") + # Note that regular gapply() seems not supporting date and timestamps + # whereas Arrow-optimized gapply() does. + rdf <- data.frame(list(list(a = 1, + b = "a", + c = TRUE, + d = 1.1, + e = 1L))) + df <- createDataFrame(rdf) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "false") + tryCatch({ + ret <- gapply(df, + "a", + function(key, grouped) { grouped }, schema(df)) + expected <- collect(ret) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) + + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + ret <- gapply(df, + "a", + function(key, grouped) { grouped }, schema(df)) + actual <- collect(ret) + expect_equal(actual, expected) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("gapply() Arrow optimization - type specification (date and timestamp)", { + skip_if_not_installed("arrow") + rdf <- data.frame(list(list(a = as.Date("1990-02-24"), + b = as.POSIXct("1990-02-24 12:34:56")))) + df <- createDataFrame(rdf) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + ret <- gapply(df, + "a", + function(key, grouped) { grouped }, schema(df)) + expect_equal(collect(ret), rdf) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +test_that("Arrow optimization - unsupported types", { + skip_if_not_installed("arrow") + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.enabled")[[1]] + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", "true") + tryCatch({ + expect_error(checkSchemaInArrow(structType("a FLOAT")), "not support float type") + expect_error(checkSchemaInArrow(structType("a BINARY")), "not support binary type") + expect_error(checkSchemaInArrow(structType("a ARRAY")), "not support array type") + expect_error(checkSchemaInArrow(structType("a MAP")), "not support map type") + expect_error(checkSchemaInArrow(structType("a STRUCT")), + "not support nested struct type") + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.enabled", arrowEnabled) + }) +}) + +sparkR.session.stop() diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index f80b45b4f36a..9e48ae34634e 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -219,7 +219,7 @@ SparkR supports operating on a variety of data sources through the `SparkDataFra The general method for creating `SparkDataFrame` from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active Spark Session will be used automatically. SparkR supports reading CSV, JSON and Parquet files natively and through Spark Packages you can find data source connectors for popular file formats like Avro. These packages can be added with `sparkPackages` parameter when initializing SparkSession using `sparkR.session`. ```{r, eval=FALSE} -sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") +sparkR.session(sparkPackages = "com.databricks:spark-avro_2.12:3.0.0") ``` We can see how to use data sources using an example CSV input file. For more information please refer to SparkR [read.df](https://spark.apache.org/docs/latest/api/R/read.df.html) API documentation. @@ -549,6 +549,8 @@ SparkR supports the following machine learning models and algorithms. * Latent Dirichlet Allocation (LDA) +* Power Iteration Clustering (PIC) + #### Collaborative Filtering * Alternating Least Squares (ALS) @@ -982,6 +984,18 @@ predicted <- predict(model, df) head(predicted) ``` +#### Power Iteration Clustering + +Power Iteration Clustering (PIC) is a scalable graph clustering algorithm. `spark.assignClusters` method runs the PIC algorithm and returns a cluster assignment for each input vertex. + +```{r} +df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), + list(1L, 2L, 1.0), list(3L, 4L, 1.0), + list(4L, 0L, 0.1)), + schema = c("src", "dst", "weight")) +head(spark.assignClusters(df, initMode = "degree", weightCol = "weight")) +``` + #### FP-growth `spark.fpGrowth` executes FP-growth algorithm to mine frequent itemsets on a `SparkDataFrame`. `itemsCol` should be an array of values. @@ -1019,9 +1033,10 @@ head(predict(fpm, df)) ```{r} df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), - list(list(list(1L), list(3L, 2L), list(1L, 2L))), - list(list(list(1L, 2L), list(5L))), - list(list(list(6L)))), schema = c("sequence")) + list(list(list(1L), list(3L, 2L), list(1L, 2L))), + list(list(list(1L, 2L), list(5L))), + list(list(list(6L)))), + schema = c("sequence")) head(spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L)) ``` diff --git a/README.md b/README.md index fd8c7f656968..482c00764380 100644 --- a/README.md +++ b/README.md @@ -1,14 +1,18 @@ # Apache Spark -Spark is a fast and general cluster computing system for Big Data. It provides +Spark is a unified analytics engine for large-scale data processing. It provides high-level APIs in Scala, Java, Python, and R, and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and DataFrames, MLlib for machine learning, GraphX for graph processing, -and Spark Streaming for stream processing. +and Structured Streaming for stream processing. +[![Jenkins Build](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/badge/icon)](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7) +[![AppVeyor Build](https://img.shields.io/appveyor/ci/ApacheSoftwareFoundation/spark/master.svg?style=plastic&logo=appveyor)](https://ci.appveyor.com/project/ApacheSoftwareFoundation/spark) +[![PySpark Coverage](https://img.shields.io/badge/dynamic/xml.svg?label=pyspark%20coverage&url=https%3A%2F%2Fspark-test.github.io%2Fpyspark-coverage-site&query=%2Fhtml%2Fbody%2Fdiv%5B1%5D%2Fdiv%2Fh1%2Fspan&colorB=brightgreen&style=plastic)](https://spark-test.github.io/pyspark-coverage-site) + ## Online Documentation @@ -37,9 +41,9 @@ The easiest way to start using Spark is through the Scala shell: ./bin/spark-shell -Try the following command, which should return 1000: +Try the following command, which should return 1,000,000,000: - scala> sc.parallelize(1 to 1000).count() + scala> spark.range(1000 * 1000 * 1000).count() ## Interactive Python Shell @@ -47,9 +51,9 @@ Alternatively, if you prefer Python, you can use the Python shell: ./bin/pyspark -And run the following command, which should also return 1000: +And run the following command, which should also return 1,000,000,000: - >>> sc.parallelize(range(1000)).count() + >>> spark.range(1000 * 1000 * 1000).count() ## Example Programs diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index 9f735f1148da..68fafbb84800 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -67,6 +67,16 @@ function docker_push { fi } +function resolve_file { + local FILE=$1 + if [ -n "$FILE" ]; then + local DIR=$(dirname $FILE) + DIR=$(cd $DIR && pwd) + FILE="${DIR}/$(basename $FILE)" + fi + echo $FILE +} + # Create a smaller build context for docker in dev builds to make the build faster. Docker # uploads all of the current directory to the daemon, and it can get pretty big with dev # builds that contain test log files and other artifacts. @@ -107,6 +117,7 @@ function create_dev_build_context {( "$PYSPARK_CTX/kubernetes/dockerfiles" mkdir "$PYSPARK_CTX/python" cp -r "python/lib" "$PYSPARK_CTX/python/lib" + cp -r "python/pyspark" "$PYSPARK_CTX/python/pyspark" local R_CTX="$CTX_DIR/sparkr" mkdir -p "$R_CTX/kubernetes" @@ -146,11 +157,18 @@ function build { fi local BUILD_ARGS=(${BUILD_PARAMS}) + + # If a custom SPARK_UID was set add it to build arguments + if [ -n "$SPARK_UID" ]; then + BUILD_ARGS+=(--build-arg spark_uid=$SPARK_UID) + fi + local BINDING_BUILD_ARGS=( - ${BUILD_PARAMS} + ${BUILD_ARGS[@]} --build-arg base_img=$(image_ref spark) ) + local BASEDOCKERFILE=${BASEDOCKERFILE:-"kubernetes/dockerfiles/spark/Dockerfile"} local PYDOCKERFILE=${PYDOCKERFILE:-false} local RDOCKERFILE=${RDOCKERFILE:-false} @@ -207,8 +225,10 @@ Options: -t tag Tag to apply to the built image, or to identify the image to be pushed. -m Use minikube's Docker daemon. -n Build docker image with --no-cache - -b arg Build arg to build or push the image. For multiple build args, this option needs to - be used separately for each build arg. + -u uid UID to use in the USER directive to set the user the main Spark process runs as inside the + resulting container + -b arg Build arg to build or push the image. For multiple build args, this option needs to + be used separately for each build arg. Using minikube when building images will do so directly into minikube's Docker daemon. There is no need to push the images into minikube in that case, they'll be automatically @@ -243,13 +263,14 @@ PYDOCKERFILE= RDOCKERFILE= NOCACHEARG= BUILD_PARAMS= -while getopts f:p:R:mr:t:nb: option +SPARK_UID= +while getopts f:p:R:mr:t:nb:u: option do case "${option}" in - f) BASEDOCKERFILE=${OPTARG};; - p) PYDOCKERFILE=${OPTARG};; - R) RDOCKERFILE=${OPTARG};; + f) BASEDOCKERFILE=$(resolve_file ${OPTARG});; + p) PYDOCKERFILE=$(resolve_file ${OPTARG});; + R) RDOCKERFILE=$(resolve_file ${OPTARG});; r) REPO=${OPTARG};; t) TAG=${OPTARG};; n) NOCACHEARG="--no-cache";; @@ -261,8 +282,9 @@ do if ! minikube status 1>/dev/null; then error "Cannot contact minikube. Make sure it's running." fi - eval $(minikube docker-env) + eval $(minikube docker-env --shell bash) ;; + u) SPARK_UID=${OPTARG};; esac done diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index cefa513b6fb7..5f98cc34b6ba 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -21,6 +21,7 @@ rem This script loads spark-env.cmd if it exists, and ensures it is only loaded rem spark-env.cmd is loaded from SPARK_CONF_DIR if set, or within the current directory's rem conf\ subdirectory. +set SPARK_ENV_CMD=spark-env.cmd if [%SPARK_ENV_LOADED%] == [] ( set SPARK_ENV_LOADED=1 @@ -28,30 +29,34 @@ if [%SPARK_ENV_LOADED%] == [] ( set SPARK_CONF_DIR=%~dp0..\conf ) - call :LoadSparkEnv + set SPARK_ENV_CMD=%SPARK_CONF_DIR%\%SPARK_ENV_CMD% + if exist %SPARK_ENV_CMD% ( + call %SPARK_ENV_CMD% + ) ) rem Setting SPARK_SCALA_VERSION if not already set. -set ASSEMBLY_DIR2="%SPARK_HOME%\assembly\target\scala-2.11" -set ASSEMBLY_DIR1="%SPARK_HOME%\assembly\target\scala-2.12" - -if [%SPARK_SCALA_VERSION%] == [] ( - - if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( - echo "Presence of build for multiple Scala versions detected." - echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." - exit 1 - ) - if exist %ASSEMBLY_DIR2% ( - set SPARK_SCALA_VERSION=2.11 - ) else ( - set SPARK_SCALA_VERSION=2.12 - ) -) +rem TODO: revisit for Scala 2.13 support +set SPARK_SCALA_VERSION=2.12 +rem if [%SPARK_SCALA_VERSION%] == [] ( +rem set SCALA_VERSION_1=2.12 +rem set SCALA_VERSION_2=2.11 +rem +rem set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% +rem set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% +rem set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables +rem if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( +rem echo "Presence of build for multiple Scala versions detected (%ASSEMBLY_DIR1% and %ASSEMBLY_DIR2%)." +rem echo "Remove one of them or, set SPARK_SCALA_VERSION=%SCALA_VERSION_1% in %SPARK_ENV_CMD%." +rem echo "Visit %ENV_VARIABLE_DOC% for more details about setting environment variables in spark-env.cmd." +rem echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." +rem exit 1 +rem ) +rem if exist %ASSEMBLY_DIR1% ( +rem set SPARK_SCALA_VERSION=%SCALA_VERSION_1% +rem ) else ( +rem set SPARK_SCALA_VERSION=%SCALA_VERSION_2% +rem ) +rem ) exit /b 0 - -:LoadSparkEnv -if exist "%SPARK_CONF_DIR%\spark-env.cmd" ( - call "%SPARK_CONF_DIR%\spark-env.cmd" -) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 0ada5d8d0fc1..107e7991c28b 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -43,23 +43,25 @@ fi # Setting SPARK_SCALA_VERSION if not already set. -if [ -z "$SPARK_SCALA_VERSION" ]; then - SCALA_VERSION_1=2.12 - SCALA_VERSION_2=2.11 - - ASSEMBLY_DIR_1="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_1}" - ASSEMBLY_DIR_2="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_2}" - ENV_VARIABLE_DOC="https://spark.apache.org/docs/latest/configuration.html#environment-variables" - if [[ -d "$ASSEMBLY_DIR_1" && -d "$ASSEMBLY_DIR_2" ]]; then - echo "Presence of build for multiple Scala versions detected ($ASSEMBLY_DIR_1 and $ASSEMBLY_DIR_2)." 1>&2 - echo "Remove one of them or, export SPARK_SCALA_VERSION=$SCALA_VERSION_1 in ${SPARK_ENV_SH}." 1>&2 - echo "Visit ${ENV_VARIABLE_DOC} for more details about setting environment variables in spark-env.sh." 1>&2 - exit 1 - fi - - if [[ -d "$ASSEMBLY_DIR_1" ]]; then - export SPARK_SCALA_VERSION=${SCALA_VERSION_1} - else - export SPARK_SCALA_VERSION=${SCALA_VERSION_2} - fi -fi +# TODO: revisit for Scala 2.13 support +export SPARK_SCALA_VERSION=2.12 +#if [ -z "$SPARK_SCALA_VERSION" ]; then +# SCALA_VERSION_1=2.12 +# SCALA_VERSION_2=2.11 +# +# ASSEMBLY_DIR_1="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_1}" +# ASSEMBLY_DIR_2="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_2}" +# ENV_VARIABLE_DOC="https://spark.apache.org/docs/latest/configuration.html#environment-variables" +# if [[ -d "$ASSEMBLY_DIR_1" && -d "$ASSEMBLY_DIR_2" ]]; then +# echo "Presence of build for multiple Scala versions detected ($ASSEMBLY_DIR_1 and $ASSEMBLY_DIR_2)." 1>&2 +# echo "Remove one of them or, export SPARK_SCALA_VERSION=$SCALA_VERSION_1 in ${SPARK_ENV_SH}." 1>&2 +# echo "Visit ${ENV_VARIABLE_DOC} for more details about setting environment variables in spark-env.sh." 1>&2 +# exit 1 +# fi +# +# if [[ -d "$ASSEMBLY_DIR_1" ]]; then +# export SPARK_SCALA_VERSION=${SCALA_VERSION_1} +# else +# export SPARK_SCALA_VERSION=${SCALA_VERSION_2} +# fi +#fi diff --git a/bin/pyspark b/bin/pyspark index 1dcddcc6196b..44891aee2e0a 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -38,22 +38,15 @@ if [[ -n "$IPYTHON" || -n "$IPYTHON_OPTS" ]]; then fi # Default to standard python interpreter unless told otherwise -if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then - PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"python"}" -fi - -WORKS_WITH_IPYTHON=$(python -c 'import sys; print(sys.version_info >= (2, 7, 0))') - -# Determine the Python executable to use for the executors: if [[ -z "$PYSPARK_PYTHON" ]]; then - if [[ $PYSPARK_DRIVER_PYTHON == *ipython* && ! $WORKS_WITH_IPYTHON ]]; then - echo "IPython requires Python 2.7+; please install python2.7 or set PYSPARK_PYTHON" 1>&2 - exit 1 - else - PYSPARK_PYTHON=python - fi + PYSPARK_PYTHON=python +fi +if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then + PYSPARK_DRIVER_PYTHON=$PYSPARK_PYTHON fi export PYSPARK_PYTHON +export PYSPARK_DRIVER_PYTHON +export PYSPARK_DRIVER_PYTHON_OPTS # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" @@ -72,6 +65,4 @@ if [[ -n "$SPARK_TESTING" ]]; then exit fi -export PYSPARK_DRIVER_PYTHON -export PYSPARK_DRIVER_PYTHON_OPTS exec "${SPARK_HOME}"/bin/spark-submit pyspark-shell-main --name "PySparkShell" "$@" diff --git a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java index 480b52652de5..d99b9bdbce39 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java @@ -17,9 +17,11 @@ package org.apache.spark.network; +import java.io.Closeable; import java.util.ArrayList; import java.util.List; +import com.codahale.metrics.Counter; import io.netty.channel.Channel; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; @@ -59,13 +61,14 @@ * channel. As each TransportChannelHandler contains a TransportClient, this enables server * processes to send messages back to the client on an existing channel. */ -public class TransportContext { +public class TransportContext implements Closeable { private static final Logger logger = LoggerFactory.getLogger(TransportContext.class); private final TransportConf conf; private final RpcHandler rpcHandler; private final boolean closeIdleConnections; - private final boolean isClientOnly; + // Number of registered connections to the shuffle service + private Counter registeredConnections = new Counter(); /** * Force to create MessageEncoder and MessageDecoder so that we can make sure they will be created @@ -85,7 +88,7 @@ public class TransportContext { // Separate thread pool for handling ChunkFetchRequest. This helps to enable throttling // max number of TransportServer worker threads that are blocked on writing response // of ChunkFetchRequest message back to the client via the underlying channel. - private static EventLoopGroup chunkFetchWorkers; + private final EventLoopGroup chunkFetchWorkers; public TransportContext(TransportConf conf, RpcHandler rpcHandler) { this(conf, rpcHandler, false, false); @@ -117,18 +120,16 @@ public TransportContext( this.conf = conf; this.rpcHandler = rpcHandler; this.closeIdleConnections = closeIdleConnections; - this.isClientOnly = isClientOnly; - - synchronized(TransportContext.class) { - if (chunkFetchWorkers == null && - conf.getModuleName() != null && - conf.getModuleName().equalsIgnoreCase("shuffle") && - !isClientOnly) { - chunkFetchWorkers = NettyUtils.createEventLoop( - IOMode.valueOf(conf.ioMode()), - conf.chunkFetchHandlerThreads(), - "shuffle-chunk-fetch-handler"); - } + + if (conf.getModuleName() != null && + conf.getModuleName().equalsIgnoreCase("shuffle") && + !isClientOnly) { + chunkFetchWorkers = NettyUtils.createEventLoop( + IOMode.valueOf(conf.ioMode()), + conf.chunkFetchHandlerThreads(), + "shuffle-chunk-fetch-handler"); + } else { + chunkFetchWorkers = null; } } @@ -198,9 +199,7 @@ public TransportChannelHandler initializePipeline( // would require more logic to guarantee if this were not part of the same event loop. .addLast("handler", channelHandler); // Use a separate EventLoopGroup to handle ChunkFetchRequest messages for shuffle rpcs. - if (conf.getModuleName() != null && - conf.getModuleName().equalsIgnoreCase("shuffle") - && !isClientOnly) { + if (chunkFetchWorkers != null) { pipeline.addLast(chunkFetchWorkers, "chunkFetchHandler", chunkFetchHandler); } return channelHandler; @@ -221,7 +220,7 @@ private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, rpcHandler, conf.maxChunksBeingTransferred()); return new TransportChannelHandler(client, responseHandler, requestHandler, - conf.connectionTimeoutMs(), closeIdleConnections); + conf.connectionTimeoutMs(), closeIdleConnections, this); } /** @@ -234,4 +233,14 @@ private ChunkFetchRequestHandler createChunkFetchHandler(TransportChannelHandler } public TransportConf getConf() { return conf; } + + public Counter getRegisteredConnections() { + return registeredConnections; + } + + public void close() { + if (chunkFetchWorkers != null) { + chunkFetchWorkers.shutdownGracefully(); + } + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 16d242dbb2c4..a8e27157f42f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -84,7 +84,7 @@ private static class ClientPool { private final Class socketChannelClass; private EventLoopGroup workerGroup; - private PooledByteBufAllocator pooledAllocator; + private final PooledByteBufAllocator pooledAllocator; private final NettyMemoryMetrics metrics; public TransportClientFactory( @@ -103,8 +103,13 @@ public TransportClientFactory( ioMode, conf.clientThreads(), conf.getModuleName() + "-client"); - this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( - conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); + if (conf.sharedByteBufAllocators()) { + this.pooledAllocator = NettyUtils.getSharedPooledByteBufAllocator( + conf.preferDirectBufsForSharedByteBufAllocators(), false /* allowCache */); + } else { + this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); + } this.metrics = new NettyMemoryMetrics( this.pooledAllocator, conf.getModuleName() + "-client", conf); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 596b0ea5dba9..2f143f77fa4a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -91,7 +91,7 @@ public void removeRpcRequest(long requestId) { } public void addStreamCallback(String streamId, StreamCallback callback) { - timeOfLastRequestNs.set(System.nanoTime()); + updateTimeOfLastRequest(); streamCallbacks.offer(ImmutablePair.of(streamId, callback)); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java index 3c263783a610..77b167d15e91 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.security.GeneralSecurityException; +import java.util.concurrent.TimeoutException; import com.google.common.base.Throwables; import io.netty.buffer.ByteBuf; @@ -82,13 +83,19 @@ public void doBootstrap(TransportClient client, Channel channel) { } catch (RuntimeException e) { // There isn't a good exception that can be caught here to know whether it's really // OK to switch back to SASL (because the server doesn't speak the new protocol). So - // try it anyway, and in the worst case things will fail again. - if (conf.saslFallback()) { - LOG.warn("New auth protocol failed, trying SASL.", e); - doSaslAuth(client, channel); - } else { + // try it anyway, unless it's a timeout, which is locally fatal. In the worst case + // things will fail again. + if (!conf.saslFallback() || e.getCause() instanceof TimeoutException) { throw e; } + + if (LOG.isDebugEnabled()) { + Throwable cause = e.getCause() != null ? e.getCause() : e; + LOG.debug("New auth protocol failed, trying SASL.", cause); + } else { + LOG.info("New auth protocol failed, trying SASL."); + } + doSaslAuth(client, channel); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java index 2745052265f7..8995bbc940f6 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -44,7 +44,8 @@ public class TransportCipher { @VisibleForTesting static final String ENCRYPTION_HANDLER_NAME = "TransportEncryption"; private static final String DECRYPTION_HANDLER_NAME = "TransportDecryption"; - private static final int STREAM_BUFFER_SIZE = 1024 * 32; + @VisibleForTesting + static final int STREAM_BUFFER_SIZE = 1024 * 32; private final Properties conf; private final String cipher; @@ -84,7 +85,8 @@ public byte[] getOutputIv() { return outIv; } - private CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException { + @VisibleForTesting + CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException { return new CryptoOutputStream(cipher, conf, ch, key, new IvParameterSpec(outIv)); } @@ -104,7 +106,8 @@ public void addToChannel(Channel ch) throws IOException { .addFirst(DECRYPTION_HANDLER_NAME, new DecryptionHandler(this)); } - private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { + @VisibleForTesting + static class EncryptionHandler extends ChannelOutboundHandlerAdapter { private final ByteArrayWritableChannel byteChannel; private final CryptoOutputStream cos; private boolean isCipherValid; @@ -118,7 +121,12 @@ private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { @Override public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - ctx.write(new EncryptedMessage(this, cos, msg, byteChannel), promise); + ctx.write(createEncryptedMessage(msg), promise); + } + + @VisibleForTesting + EncryptedMessage createEncryptedMessage(Object msg) { + return new EncryptedMessage(this, cos, msg, byteChannel); } @Override @@ -190,12 +198,14 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } } - private static class EncryptedMessage extends AbstractFileRegion { + @VisibleForTesting + static class EncryptedMessage extends AbstractFileRegion { private final boolean isByteBuf; private final ByteBuf buf; private final FileRegion region; private final CryptoOutputStream cos; private final EncryptionHandler handler; + private final long count; private long transferred; // Due to streaming issue CRYPTO-125: https://issues.apache.org/jira/browse/CRYPTO-125, it has @@ -221,11 +231,12 @@ private static class EncryptedMessage extends AbstractFileRegion { this.byteRawChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); this.cos = cos; this.byteEncChannel = ch; + this.count = isByteBuf ? buf.readableBytes() : region.count(); } @Override public long count() { - return isByteBuf ? buf.readableBytes() : region.count(); + return count; } @Override @@ -277,22 +288,38 @@ public boolean release(int decrement) { public long transferTo(WritableByteChannel target, long position) throws IOException { Preconditions.checkArgument(position == transferred(), "Invalid position."); + if (transferred == count) { + return 0; + } + + long totalBytesWritten = 0L; do { if (currentEncrypted == null) { encryptMore(); } - int bytesWritten = currentEncrypted.remaining(); - target.write(currentEncrypted); - bytesWritten -= currentEncrypted.remaining(); - transferred += bytesWritten; - if (!currentEncrypted.hasRemaining()) { + long remaining = currentEncrypted.remaining(); + if (remaining == 0) { + // Just for safety to avoid endless loop. It usually won't happen, but since the + // underlying `region.transferTo` is allowed to transfer 0 bytes, we should handle it for + // safety. currentEncrypted = null; byteEncChannel.reset(); + return totalBytesWritten; } - } while (transferred < count()); - return transferred; + long bytesWritten = target.write(currentEncrypted); + totalBytesWritten += bytesWritten; + transferred += bytesWritten; + if (bytesWritten < remaining) { + // break as the underlying buffer in "target" is full + break; + } + currentEncrypted = null; + byteEncChannel.reset(); + } while (transferred < count); + + return totalBytesWritten; } private void encryptMore() throws IOException { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index 997f74e1a21b..06dc447309dd 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -79,7 +79,7 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro // sent. int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); long frameLength = headerLength + (isBodyInFrame ? bodyLength : 0); - ByteBuf header = ctx.alloc().heapBuffer(headerLength); + ByteBuf header = ctx.alloc().buffer(headerLength); header.writeLong(frameLength); msgType.encode(header); in.encode(header); diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java index f08d8b0f984c..43c3d23b6304 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java @@ -90,7 +90,6 @@ protected void channelRead0( ManagedBuffer buf; try { streamManager.checkAuthorization(client, msg.streamChunkId.streamId); - streamManager.registerChannel(channel, msg.streamChunkId.streamId); buf = streamManager.getChunk(msg.streamChunkId.streamId, msg.streamChunkId.chunkIndex); } catch (Exception e) { logger.error(String.format("Error opening block %s for request from %s", diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index 0f6a8824d95e..6fafcc131fa2 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.netty.channel.Channel; import org.apache.commons.lang3.tuple.ImmutablePair; @@ -49,7 +50,7 @@ private static class StreamState { final Iterator buffers; // The channel associated to the stream - Channel associatedChannel = null; + final Channel associatedChannel; // Used to keep track of the index of the buffer that the user has retrieved, just to ensure // that the caller only requests each chunk one at a time, in order. @@ -58,9 +59,10 @@ private static class StreamState { // Used to keep track of the number of chunks being transferred and not finished yet. volatile long chunksBeingTransferred = 0L; - StreamState(String appId, Iterator buffers) { + StreamState(String appId, Iterator buffers, Channel channel) { this.appId = appId; this.buffers = Preconditions.checkNotNull(buffers); + this.associatedChannel = channel; } } @@ -71,13 +73,6 @@ public OneForOneStreamManager() { streams = new ConcurrentHashMap<>(); } - @Override - public void registerChannel(Channel channel, long streamId) { - if (streams.containsKey(streamId)) { - streams.get(streamId).associatedChannel = channel; - } - } - @Override public ManagedBuffer getChunk(long streamId, int chunkIndex) { StreamState state = streams.get(streamId); @@ -195,11 +190,19 @@ public long chunksBeingTransferred() { * * If an app ID is provided, only callers who've authenticated with the given app ID will be * allowed to fetch from this stream. + * + * This method also associates the stream with a single client connection, which is guaranteed + * to be the only reader of the stream. Once the connection is closed, the stream will never + * be used again, enabling cleanup by `connectionTerminated`. */ - public long registerStream(String appId, Iterator buffers) { + public long registerStream(String appId, Iterator buffers, Channel channel) { long myStreamId = nextStreamId.getAndIncrement(); - streams.put(myStreamId, new StreamState(appId, buffers)); + streams.put(myStreamId, new StreamState(appId, buffers, channel)); return myStreamId; } + @VisibleForTesting + public int numStreamStates() { + return streams.size(); + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java index c53529583160..e48d27be1126 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java @@ -60,16 +60,6 @@ public ManagedBuffer openStream(String streamId) { throw new UnsupportedOperationException(); } - /** - * Associates a stream with a single client connection, which is guaranteed to be the only reader - * of the stream. The getChunk() method will be called serially on this connection and once the - * connection is closed, the stream will never be used again, enabling cleanup. - * - * This must be called before the first getChunk() on the stream, but it may be invoked multiple - * times with the same channel and stream id. - */ - public void registerChannel(Channel channel, long streamId) { } - /** * Indicates that the given channel has been terminated. After this occurs, we are guaranteed not * to read from the associated streams again, so any state can be cleaned up. diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java index c824a7b0d474..31371f6970ff 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -21,6 +21,7 @@ import io.netty.channel.SimpleChannelInboundHandler; import io.netty.handler.timeout.IdleState; import io.netty.handler.timeout.IdleStateEvent; +import org.apache.spark.network.TransportContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,18 +58,21 @@ public class TransportChannelHandler extends SimpleChannelInboundHandler 0; boolean isActuallyOverdue = System.nanoTime() - responseHandler.getTimeOfLastRequestNs() > requestTimeoutNs; if (e.state() == IdleState.ALL_IDLE && isActuallyOverdue) { - if (responseHandler.numOutstandingRequests() > 0) { + if (hasInFlightRequests) { String address = getRemoteAddress(ctx.channel()); logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + "requests. Assuming connection is dead; please adjust spark.network.timeout if " + @@ -176,4 +181,16 @@ public TransportResponseHandler getResponseHandler() { return responseHandler; } + @Override + public void channelRegistered(ChannelHandlerContext ctx) throws Exception { + transportContext.getRegisteredConnections().inc(); + super.channelRegistered(ctx); + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + transportContext.getRegisteredConnections().dec(); + super.channelUnregistered(ctx); + } + } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java index 9c85ab2f5f06..9b327d5404b4 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import com.codahale.metrics.Counter; import com.codahale.metrics.MetricSet; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -53,6 +54,7 @@ public class TransportServer implements Closeable { private ServerBootstrap bootstrap; private ChannelFuture channelFuture; private int port = -1; + private final PooledByteBufAllocator pooledAllocator; private NettyMemoryMetrics metrics; /** @@ -68,6 +70,13 @@ public TransportServer( this.context = context; this.conf = context.getConf(); this.appRpcHandler = appRpcHandler; + if (conf.sharedByteBufAllocators()) { + this.pooledAllocator = NettyUtils.getSharedPooledByteBufAllocator( + conf.preferDirectBufsForSharedByteBufAllocators(), true /* allowCache */); + } else { + this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads()); + } this.bootstraps = Lists.newArrayList(Preconditions.checkNotNull(bootstraps)); boolean shouldClose = true; @@ -95,18 +104,15 @@ private void init(String hostToBind, int portToBind) { NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); EventLoopGroup workerGroup = bossGroup; - PooledByteBufAllocator allocator = NettyUtils.createPooledByteBufAllocator( - conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads()); - bootstrap = new ServerBootstrap() .group(bossGroup, workerGroup) .channel(NettyUtils.getServerChannelClass(ioMode)) - .option(ChannelOption.ALLOCATOR, allocator) + .option(ChannelOption.ALLOCATOR, pooledAllocator) .option(ChannelOption.SO_REUSEADDR, !SystemUtils.IS_OS_WINDOWS) - .childOption(ChannelOption.ALLOCATOR, allocator); + .childOption(ChannelOption.ALLOCATOR, pooledAllocator); this.metrics = new NettyMemoryMetrics( - allocator, conf.getModuleName() + "-server", conf); + pooledAllocator, conf.getModuleName() + "-server", conf); if (conf.backLog() > 0) { bootstrap.option(ChannelOption.SO_BACKLOG, conf.backLog()); @@ -120,6 +126,10 @@ private void init(String hostToBind, int portToBind) { bootstrap.childOption(ChannelOption.SO_SNDBUF, conf.sendBuf()); } + if (conf.enableTcpKeepAlive()) { + bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true); + } + bootstrap.childHandler(new ChannelInitializer() { @Override protected void initChannel(SocketChannel ch) { @@ -159,4 +169,8 @@ public void close() { } bootstrap = null; } + + public Counter getRegisteredConnections() { + return context.getRegisteredConnections(); + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java index 984575acaf51..6f7925c26094 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java @@ -18,11 +18,11 @@ public enum ByteUnit { BYTE(1), - KiB(1024L), - MiB((long) Math.pow(1024L, 2L)), - GiB((long) Math.pow(1024L, 3L)), - TiB((long) Math.pow(1024L, 4L)), - PiB((long) Math.pow(1024L, 5L)); + KiB(1L << 10), + MiB(1L << 20), + GiB(1L << 30), + TiB(1L << 40), + PiB(1L << 50); ByteUnit(long multiplier) { this.multiplier = multiplier; @@ -50,7 +50,7 @@ public long convertTo(long d, ByteUnit u) { } } - public double toBytes(long d) { + public long toBytes(long d) { if (d < 0) { throw new IllegalArgumentException("Negative size value. Size must be positive: " + d); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java index 33d6eb4a83a0..423cc0c70ea0 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -36,6 +36,22 @@ * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO. */ public class NettyUtils { + + /** + * Specifies an upper bound on the number of Netty threads that Spark requires by default. + * In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core + * that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes + * at a premium. + * + * Thus, this value should still retain maximum throughput and reduce wasted off-heap memory + * allocation. It can be overridden by setting the number of serverThreads and clientThreads + * manually in Spark's configuration. + */ + private static int MAX_DEFAULT_NETTY_THREADS = 8; + + private static final PooledByteBufAllocator[] _sharedPooledByteBufAllocator = + new PooledByteBufAllocator[2]; + /** Creates a new ThreadFactory which prefixes each thread with the given name. */ public static ThreadFactory createThreadFactory(String threadPoolPrefix) { return new DefaultThreadFactory(threadPoolPrefix, true); @@ -95,6 +111,38 @@ public static String getRemoteAddress(Channel channel) { return ""; } + /** + * Returns the default number of threads for both the Netty client and server thread pools. + * If numUsableCores is 0, we will use Runtime get an approximate number of available cores. + */ + public static int defaultNumThreads(int numUsableCores) { + final int availableCores; + if (numUsableCores > 0) { + availableCores = numUsableCores; + } else { + availableCores = Runtime.getRuntime().availableProcessors(); + } + return Math.min(availableCores, MAX_DEFAULT_NETTY_THREADS); + } + + /** + * Returns the lazily created shared pooled ByteBuf allocator for the specified allowCache + * parameter value. + */ + public static synchronized PooledByteBufAllocator getSharedPooledByteBufAllocator( + boolean allowDirectBufs, + boolean allowCache) { + final int index = allowCache ? 0 : 1; + if (_sharedPooledByteBufAllocator[index] == null) { + _sharedPooledByteBufAllocator[index] = + createPooledByteBufAllocator( + allowDirectBufs, + allowCache, + defaultNumThreads(0)); + } + return _sharedPooledByteBufAllocator[index]; + } + /** * Create a pooled ByteBuf allocator but disables the thread-local cache. Thread-local caches * are disabled for TransportClients because the ByteBufs are allocated by the event loop thread, diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 43a6bc7dc3d0..3628da68f1c6 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -42,6 +42,7 @@ public class TransportConf { private final String SPARK_NETWORK_IO_RETRYWAIT_KEY; private final String SPARK_NETWORK_IO_LAZYFD_KEY; private final String SPARK_NETWORK_VERBOSE_METRICS; + private final String SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY; private final ConfigProvider conf; @@ -64,6 +65,7 @@ public TransportConf(String module, ConfigProvider conf) { SPARK_NETWORK_IO_RETRYWAIT_KEY = getConfKey("io.retryWait"); SPARK_NETWORK_IO_LAZYFD_KEY = getConfKey("io.lazyFD"); SPARK_NETWORK_VERBOSE_METRICS = getConfKey("io.enableVerboseMetrics"); + SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY = getConfKey("io.enableTcpKeepAlive"); } public int getInt(String name, int defaultValue) { @@ -173,6 +175,14 @@ public boolean verboseMetrics() { return conf.getBoolean(SPARK_NETWORK_VERBOSE_METRICS, false); } + /** + * Whether to enable TCP keep-alive. If true, the TCP keep-alives are enabled, which removes + * connections that are idle for too long. + */ + public boolean enableTcpKeepAlive() { + return conf.getBoolean(SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY, false); + } + /** * Maximum number of retries when binding to a port before giving up. */ @@ -265,6 +275,23 @@ public boolean saslServerAlwaysEncrypt() { return conf.getBoolean("spark.network.sasl.serverAlwaysEncrypt", false); } + /** + * Flag indicating whether to share the pooled ByteBuf allocators between the different Netty + * channels. If enabled then only two pooled ByteBuf allocators are created: one where caching + * is allowed (for transport servers) and one where not (for transport clients). + * When disabled a new allocator is created for each transport servers and clients. + */ + public boolean sharedByteBufAllocators() { + return conf.getBoolean("spark.network.sharedByteBufAllocators.enabled", true); + } + + /** + * If enabled then off-heap byte buffers will be prefered for the shared ByteBuf allocators. + */ + public boolean preferDirectBufsForSharedByteBufAllocators() { + return conf.getBoolean("spark.network.io.preferDirectBufs", true); + } + /** * The commons-crypto configuration for the module. */ @@ -309,8 +336,9 @@ public int chunkFetchHandlerThreads() { } int chunkFetchHandlerThreadsPercent = conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 100); - return (int)Math.ceil( - (this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors()) * - chunkFetchHandlerThreadsPercent/(double)100); + int threads = + this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors(); + return (int) Math.ceil(threads * (chunkFetchHandlerThreadsPercent / 100.0)); } + } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java index 8e73ab077a5c..1980361a1552 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java @@ -19,6 +19,7 @@ import java.util.LinkedList; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; @@ -48,14 +49,30 @@ public class TransportFrameDecoder extends ChannelInboundHandlerAdapter { private static final int LENGTH_SIZE = 8; private static final int MAX_FRAME_SIZE = Integer.MAX_VALUE; private static final int UNKNOWN_FRAME_SIZE = -1; + private static final long CONSOLIDATE_THRESHOLD = 20 * 1024 * 1024; private final LinkedList buffers = new LinkedList<>(); private final ByteBuf frameLenBuf = Unpooled.buffer(LENGTH_SIZE, LENGTH_SIZE); + private final long consolidateThreshold; + + private CompositeByteBuf frameBuf = null; + private long consolidatedFrameBufSize = 0; + private int consolidatedNumComponents = 0; private long totalSize = 0; private long nextFrameSize = UNKNOWN_FRAME_SIZE; + private int frameRemainingBytes = UNKNOWN_FRAME_SIZE; private volatile Interceptor interceptor; + public TransportFrameDecoder() { + this(CONSOLIDATE_THRESHOLD); + } + + @VisibleForTesting + TransportFrameDecoder(long consolidateThreshold) { + this.consolidateThreshold = consolidateThreshold; + } + @Override public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { ByteBuf in = (ByteBuf) data; @@ -123,30 +140,56 @@ private long decodeFrameSize() { private ByteBuf decodeNext() { long frameSize = decodeFrameSize(); - if (frameSize == UNKNOWN_FRAME_SIZE || totalSize < frameSize) { + if (frameSize == UNKNOWN_FRAME_SIZE) { return null; } - // Reset size for next frame. - nextFrameSize = UNKNOWN_FRAME_SIZE; - - Preconditions.checkArgument(frameSize < MAX_FRAME_SIZE, "Too large frame: %s", frameSize); - Preconditions.checkArgument(frameSize > 0, "Frame length should be positive: %s", frameSize); + if (frameBuf == null) { + Preconditions.checkArgument(frameSize < MAX_FRAME_SIZE, + "Too large frame: %s", frameSize); + Preconditions.checkArgument(frameSize > 0, + "Frame length should be positive: %s", frameSize); + frameRemainingBytes = (int) frameSize; - // If the first buffer holds the entire frame, return it. - int remaining = (int) frameSize; - if (buffers.getFirst().readableBytes() >= remaining) { - return nextBufferForFrame(remaining); + // If buffers is empty, then return immediately for more input data. + if (buffers.isEmpty()) { + return null; + } + // Otherwise, if the first buffer holds the entire frame, we attempt to + // build frame with it and return. + if (buffers.getFirst().readableBytes() >= frameRemainingBytes) { + // Reset buf and size for next frame. + frameBuf = null; + nextFrameSize = UNKNOWN_FRAME_SIZE; + return nextBufferForFrame(frameRemainingBytes); + } + // Other cases, create a composite buffer to manage all the buffers. + frameBuf = buffers.getFirst().alloc().compositeBuffer(Integer.MAX_VALUE); } - // Otherwise, create a composite buffer. - CompositeByteBuf frame = buffers.getFirst().alloc().compositeBuffer(Integer.MAX_VALUE); - while (remaining > 0) { - ByteBuf next = nextBufferForFrame(remaining); - remaining -= next.readableBytes(); - frame.addComponent(next).writerIndex(frame.writerIndex() + next.readableBytes()); + while (frameRemainingBytes > 0 && !buffers.isEmpty()) { + ByteBuf next = nextBufferForFrame(frameRemainingBytes); + frameRemainingBytes -= next.readableBytes(); + frameBuf.addComponent(true, next); } - assert remaining == 0; + // If the delta size of frameBuf exceeds the threshold, then we do consolidation + // to reduce memory consumption. + if (frameBuf.capacity() - consolidatedFrameBufSize > consolidateThreshold) { + int newNumComponents = frameBuf.numComponents() - consolidatedNumComponents; + frameBuf.consolidate(consolidatedNumComponents, newNumComponents); + consolidatedFrameBufSize = frameBuf.capacity(); + consolidatedNumComponents = frameBuf.numComponents(); + } + if (frameRemainingBytes > 0) { + return null; + } + + // Reset buf and size for next frame. + ByteBuf frame = frameBuf; + frameBuf = null; + consolidatedFrameBufSize = 0; + consolidatedNumComponents = 0; + nextFrameSize = UNKNOWN_FRAME_SIZE; return frame; } diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index 37a8664a5266..5999b6255b37 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -56,6 +56,7 @@ public class ChunkFetchIntegrationSuite { static final int BUFFER_CHUNK_INDEX = 0; static final int FILE_CHUNK_INDEX = 1; + static TransportContext context; static TransportServer server; static TransportClientFactory clientFactory; static StreamManager streamManager; @@ -117,7 +118,7 @@ public StreamManager getStreamManager() { return streamManager; } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); } @@ -127,6 +128,7 @@ public static void tearDown() { bufferChunk.release(); server.close(); clientFactory.close(); + context.close(); testFile.delete(); } @@ -172,7 +174,7 @@ public void onFailure(int chunkIndex, Throwable e) { for (int chunkIndex : chunkIndices) { client.fetchChunk(STREAM_ID, chunkIndex, callback); } - if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { + if (!sem.tryAcquire(chunkIndices.size(), 60, TimeUnit.SECONDS)) { fail("Timeout getting response from the server"); } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java index 2c72c53a33ae..6c9239606bb8 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java @@ -64,8 +64,7 @@ public void handleChunkFetchRequest() throws Exception { managedBuffers.add(new TestManagedBuffer(20)); managedBuffers.add(new TestManagedBuffer(30)); managedBuffers.add(new TestManagedBuffer(40)); - long streamId = streamManager.registerStream("test-app", managedBuffers.iterator()); - streamManager.registerChannel(channel, streamId); + long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); TransportClient reverseClient = mock(TransportClient.class); ChunkFetchRequestHandler requestHandler = new ChunkFetchRequestHandler(reverseClient, rpcHandler.getStreamManager(), 2L); diff --git a/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java index c0724e018263..15a28ba249b8 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java @@ -48,6 +48,7 @@ */ public class RequestTimeoutIntegrationSuite { + private TransportContext context; private TransportServer server; private TransportClientFactory clientFactory; @@ -79,6 +80,9 @@ public void tearDown() { if (clientFactory != null) { clientFactory.close(); } + if (context != null) { + context.close(); + } } // Basic suite: First request completes quickly, and second waits for longer than network timeout. @@ -106,7 +110,7 @@ public StreamManager getStreamManager() { } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); @@ -153,7 +157,7 @@ public StreamManager getStreamManager() { } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); @@ -204,7 +208,7 @@ public StreamManager getStreamManager() { } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index 1c0aa4da27ff..117f1e4d00fe 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -44,6 +44,7 @@ public class RpcIntegrationSuite { static TransportConf conf; + static TransportContext context; static TransportServer server; static TransportClientFactory clientFactory; static RpcHandler rpcHandler; @@ -90,7 +91,7 @@ public void receive(TransportClient client, ByteBuffer message) { @Override public StreamManager getStreamManager() { return new OneForOneStreamManager(); } }; - TransportContext context = new TransportContext(conf, rpcHandler); + context = new TransportContext(conf, rpcHandler); server = context.createServer(); clientFactory = context.createClientFactory(); oneWayMsgs = new ArrayList<>(); @@ -160,6 +161,7 @@ public String getID() { public static void tearDown() { server.close(); clientFactory.close(); + context.close(); testData.cleanup(); } diff --git a/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java b/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java index f3050cb79cdf..485d8ad55414 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java @@ -51,6 +51,7 @@ public class StreamSuite { private static final String[] STREAMS = StreamTestHelper.STREAMS; private static StreamTestHelper testData; + private static TransportContext context; private static TransportServer server; private static TransportClientFactory clientFactory; @@ -93,7 +94,7 @@ public StreamManager getStreamManager() { return streamManager; } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); } @@ -103,6 +104,7 @@ public static void tearDown() { server.close(); clientFactory.close(); testData.cleanup(); + context.close(); } @Test diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index e95d25fe6ae9..2c621140db83 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -64,6 +64,7 @@ public void setUp() { public void tearDown() { JavaUtils.closeQuietly(server1); JavaUtils.closeQuietly(server2); + JavaUtils.closeQuietly(context); } /** @@ -80,49 +81,50 @@ private void testClientReuse(int maxConnections, boolean concurrent) TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); RpcHandler rpcHandler = new NoOpRpcHandler(); - TransportContext context = new TransportContext(conf, rpcHandler); - TransportClientFactory factory = context.createClientFactory(); - Set clients = Collections.synchronizedSet( - new HashSet()); - - AtomicInteger failed = new AtomicInteger(); - Thread[] attempts = new Thread[maxConnections * 10]; - - // Launch a bunch of threads to create new clients. - for (int i = 0; i < attempts.length; i++) { - attempts[i] = new Thread(() -> { - try { - TransportClient client = - factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - assertTrue(client.isActive()); - clients.add(client); - } catch (IOException e) { - failed.incrementAndGet(); - } catch (InterruptedException e) { - throw new RuntimeException(e); + try (TransportContext context = new TransportContext(conf, rpcHandler)) { + TransportClientFactory factory = context.createClientFactory(); + Set clients = Collections.synchronizedSet( + new HashSet()); + + AtomicInteger failed = new AtomicInteger(); + Thread[] attempts = new Thread[maxConnections * 10]; + + // Launch a bunch of threads to create new clients. + for (int i = 0; i < attempts.length; i++) { + attempts[i] = new Thread(() -> { + try { + TransportClient client = + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertTrue(client.isActive()); + clients.add(client); + } catch (IOException e) { + failed.incrementAndGet(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + if (concurrent) { + attempts[i].start(); + } else { + attempts[i].run(); } - }); + } - if (concurrent) { - attempts[i].start(); - } else { - attempts[i].run(); + // Wait until all the threads complete. + for (Thread attempt : attempts) { + attempt.join(); } - } - // Wait until all the threads complete. - for (Thread attempt : attempts) { - attempt.join(); - } + Assert.assertEquals(0, failed.get()); + Assert.assertEquals(clients.size(), maxConnections); - Assert.assertEquals(0, failed.get()); - Assert.assertEquals(clients.size(), maxConnections); + for (TransportClient client : clients) { + client.close(); + } - for (TransportClient client : clients) { - client.close(); + factory.close(); } - - factory.close(); } @Test @@ -204,8 +206,8 @@ public Iterable> getAll() { throw new UnsupportedOperationException(); } }); - TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); - try (TransportClientFactory factory = context.createClientFactory()) { + try (TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); + TransportClientFactory factory = context.createClientFactory()) { TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); assertTrue(c1.isActive()); long expiredTime = System.currentTimeMillis() + 10000; // 10 seconds diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java index ad640415a8e6..a87f6c11a2bf 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java @@ -58,8 +58,10 @@ public void handleStreamRequest() throws Exception { managedBuffers.add(new TestManagedBuffer(20)); managedBuffers.add(new TestManagedBuffer(30)); managedBuffers.add(new TestManagedBuffer(40)); - long streamId = streamManager.registerStream("test-app", managedBuffers.iterator()); - streamManager.registerChannel(channel, streamId); + long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); + + assert streamManager.numStreamStates() == 1; + TransportClient reverseClient = mock(TransportClient.class); TransportRequestHandler requestHandler = new TransportRequestHandler(channel, reverseClient, rpcHandler, 2L); @@ -94,5 +96,8 @@ public void handleStreamRequest() throws Exception { requestHandler.handle(request3); verify(channel, times(1)).close(); assert responseAndPromisePairs.size() == 3; + + streamManager.connectionTerminated(channel); + assert streamManager.numStreamStates() == 0; } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java index c0aa298a4017..0790f0079c2b 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java @@ -17,16 +17,27 @@ package org.apache.spark.network.crypto; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; import java.util.Arrays; import java.util.Map; import java.security.InvalidKeyException; +import java.util.Random; + import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.collect.ImmutableMap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.FileRegion; import org.junit.BeforeClass; import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import static org.junit.Assert.*; +import static org.mockito.Mockito.*; +import org.apache.spark.network.util.ByteArrayWritableChannel; import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -121,4 +132,77 @@ public void testBadKeySize() throws Exception { } } + @Test + public void testEncryptedMessage() throws Exception { + AuthEngine client = new AuthEngine("appId", "secret", conf); + AuthEngine server = new AuthEngine("appId", "secret", conf); + try { + ClientChallenge clientChallenge = client.challenge(); + ServerResponse serverResponse = server.respond(clientChallenge); + client.validate(serverResponse); + + TransportCipher cipher = server.sessionCipher(); + TransportCipher.EncryptionHandler handler = new TransportCipher.EncryptionHandler(cipher); + + byte[] data = new byte[TransportCipher.STREAM_BUFFER_SIZE + 1]; + new Random().nextBytes(data); + ByteBuf buf = Unpooled.wrappedBuffer(data); + + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(data.length); + TransportCipher.EncryptedMessage emsg = handler.createEncryptedMessage(buf); + while (emsg.transfered() < emsg.count()) { + emsg.transferTo(channel, emsg.transfered()); + } + assertEquals(data.length, channel.length()); + } finally { + client.close(); + server.close(); + } + } + + @Test + public void testEncryptedMessageWhenTransferringZeroBytes() throws Exception { + AuthEngine client = new AuthEngine("appId", "secret", conf); + AuthEngine server = new AuthEngine("appId", "secret", conf); + try { + ClientChallenge clientChallenge = client.challenge(); + ServerResponse serverResponse = server.respond(clientChallenge); + client.validate(serverResponse); + + TransportCipher cipher = server.sessionCipher(); + TransportCipher.EncryptionHandler handler = new TransportCipher.EncryptionHandler(cipher); + + int testDataLength = 4; + FileRegion region = mock(FileRegion.class); + when(region.count()).thenReturn((long) testDataLength); + // Make `region.transferTo` do nothing in first call and transfer 4 bytes in the second one. + when(region.transferTo(any(), anyLong())).thenAnswer(new Answer() { + + private boolean firstTime = true; + + @Override + public Long answer(InvocationOnMock invocationOnMock) throws Throwable { + if (firstTime) { + firstTime = false; + return 0L; + } else { + WritableByteChannel channel = invocationOnMock.getArgument(0); + channel.write(ByteBuffer.wrap(new byte[testDataLength])); + return (long) testDataLength; + } + } + }); + + TransportCipher.EncryptedMessage emsg = handler.createEncryptedMessage(region); + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(testDataLength); + // "transferTo" should act correctly when the underlying FileRegion transfers 0 bytes. + assertEquals(0L, emsg.transferTo(channel, emsg.transfered())); + assertEquals(testDataLength, emsg.transferTo(channel, emsg.transfered())); + assertEquals(emsg.transfered(), emsg.count()); + assertEquals(4, channel.length()); + } finally { + client.close(); + server.close(); + } + } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java index 8751944a1c2a..2f9dd629df1b 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java @@ -124,6 +124,42 @@ public void testAuthReplay() throws Exception { } } + @Test + public void testLargeMessageEncryption() throws Exception { + // Use a big length to create a message that cannot be put into the encryption buffer completely + final int testErrorMessageLength = TransportCipher.STREAM_BUFFER_SIZE; + ctx = new AuthTestCtx(new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + char[] longMessage = new char[testErrorMessageLength]; + Arrays.fill(longMessage, 'D'); + callback.onFailure(new RuntimeException(new String(longMessage))); + } + + @Override + public StreamManager getStreamManager() { + return null; + } + }); + ctx.createServer("secret"); + ctx.createClient("secret"); + + try { + ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + fail("Should have failed unencrypted RPC."); + } catch (Exception e) { + assertTrue(ctx.authRpcHandler.doDelegate); + assertTrue(e.getMessage() + " is not an expected error", e.getMessage().contains("DDDDD")); + // Verify we receive the complete error message + int messageStart = e.getMessage().indexOf("DDDDD"); + int messageEnd = e.getMessage().lastIndexOf("DDDDD") + 5; + assertEquals(testErrorMessageLength, messageEnd - messageStart); + } + } + private class AuthTestCtx { private final String appId = "testAppId"; @@ -136,10 +172,7 @@ private class AuthTestCtx { volatile AuthRpcHandler authRpcHandler; AuthTestCtx() throws Exception { - Map testConf = ImmutableMap.of("spark.network.crypto.enabled", "true"); - this.conf = new TransportConf("rpc", new MapConfigProvider(testConf)); - - RpcHandler rpcHandler = new RpcHandler() { + this(new RpcHandler() { @Override public void receive( TransportClient client, @@ -153,8 +186,12 @@ public void receive( public StreamManager getStreamManager() { return null; } - }; + }); + } + AuthTestCtx(RpcHandler rpcHandler) throws Exception { + Map testConf = ImmutableMap.of("spark.network.crypto.enabled", "true"); + this.conf = new TransportConf("rpc", new MapConfigProvider(testConf)); this.ctx = new TransportContext(conf, rpcHandler); } @@ -196,6 +233,9 @@ void close() { if (server != null) { server.close(); } + if (ctx != null) { + ctx.close(); + } } private SecretKeyHolder createKeyHolder(String secret) { diff --git a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index 6f15718bd870..cf2d72f71e8d 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -347,10 +347,10 @@ public void testRpcHandlerDelegate() throws Exception { verify(handler).getStreamManager(); saslHandler.channelInactive(null); - verify(handler).channelInactive(any(TransportClient.class)); + verify(handler).channelInactive(isNull()); saslHandler.exceptionCaught(null, null); - verify(handler).exceptionCaught(any(Throwable.class), any(TransportClient.class)); + verify(handler).exceptionCaught(isNull(), isNull()); } @Test @@ -365,6 +365,7 @@ private static class SaslTestCtx { final TransportClient client; final TransportServer server; + final TransportContext ctx; private final boolean encrypt; private final boolean disableClientEncryption; @@ -396,7 +397,7 @@ private static class SaslTestCtx { when(keyHolder.getSaslUser(anyString())).thenReturn("user"); when(keyHolder.getSecretKey(anyString())).thenReturn("secret"); - TransportContext ctx = new TransportContext(conf, rpcHandler); + this.ctx = new TransportContext(conf, rpcHandler); this.checker = new EncryptionCheckerBootstrap(SaslEncryption.ENCRYPTION_HANDLER_NAME); @@ -431,6 +432,9 @@ void close() { if (server != null) { server.close(); } + if (ctx != null) { + ctx.close(); + } } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java index c647525d8f1b..4248762c3238 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -37,14 +37,15 @@ public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); buffers.add(buffer1); buffers.add(buffer2); - long streamId = manager.registerStream("appId", buffers.iterator()); Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); - manager.registerChannel(dummyChannel, streamId); + manager.registerStream("appId", buffers.iterator(), dummyChannel); + assert manager.numStreamStates() == 1; manager.connectionTerminated(dummyChannel); Mockito.verify(buffer1, Mockito.times(1)).release(); Mockito.verify(buffer2, Mockito.times(1)).release(); + assert manager.numStreamStates() == 0; } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java index 400b385c9703..f049cad83ff6 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java @@ -60,11 +60,14 @@ public void tearDown() { JavaUtils.closeQuietly(clientFactory); clientFactory = null; } - if (server != null) { JavaUtils.closeQuietly(server); server = null; } + if (context != null) { + JavaUtils.closeQuietly(context); + context = null; + } } @Test diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java index b53e41303751..4b67aa80351d 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.network.util; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -28,11 +27,15 @@ import io.netty.channel.ChannelHandlerContext; import org.junit.AfterClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import static org.junit.Assert.*; import static org.mockito.Mockito.*; public class TransportFrameDecoderSuite { + private static final Logger logger = LoggerFactory.getLogger(TransportFrameDecoderSuite.class); private static Random RND = new Random(); @AfterClass @@ -48,6 +51,69 @@ public void testFrameDecoding() throws Exception { verifyAndCloseDecoder(decoder, ctx, data); } + @Test + public void testConsolidationPerf() throws Exception { + long[] testingConsolidateThresholds = new long[] { + ByteUnit.MiB.toBytes(1), + ByteUnit.MiB.toBytes(5), + ByteUnit.MiB.toBytes(10), + ByteUnit.MiB.toBytes(20), + ByteUnit.MiB.toBytes(30), + ByteUnit.MiB.toBytes(50), + ByteUnit.MiB.toBytes(80), + ByteUnit.MiB.toBytes(100), + ByteUnit.MiB.toBytes(300), + ByteUnit.MiB.toBytes(500), + Long.MAX_VALUE }; + for (long threshold : testingConsolidateThresholds) { + TransportFrameDecoder decoder = new TransportFrameDecoder(threshold); + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + List retained = new ArrayList<>(); + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + retained.add(buf); + return null; + }); + + // Testing multiple messages + int numMessages = 3; + long targetBytes = ByteUnit.MiB.toBytes(300); + int pieceBytes = (int) ByteUnit.KiB.toBytes(32); + for (int i = 0; i < numMessages; i++) { + try { + long writtenBytes = 0; + long totalTime = 0; + ByteBuf buf = Unpooled.buffer(8); + buf.writeLong(8 + targetBytes); + decoder.channelRead(ctx, buf); + while (writtenBytes < targetBytes) { + buf = Unpooled.buffer(pieceBytes * 2); + ByteBuf writtenBuf = Unpooled.buffer(pieceBytes).writerIndex(pieceBytes); + buf.writeBytes(writtenBuf); + writtenBuf.release(); + long start = System.currentTimeMillis(); + decoder.channelRead(ctx, buf); + long elapsedTime = System.currentTimeMillis() - start; + totalTime += elapsedTime; + writtenBytes += pieceBytes; + } + logger.info("Writing 300MiB frame buf with consolidation of threshold " + threshold + + " took " + totalTime + " milis"); + } finally { + for (ByteBuf buf : retained) { + release(buf); + } + } + } + long totalBytesGot = 0; + for (ByteBuf buf : retained) { + totalBytesGot += buf.capacity(); + } + assertEquals(numMessages, retained.size()); + assertEquals(targetBytes * numMessages, totalBytesGot); + } + } + @Test public void testInterception() throws Exception { int interceptedReads = 3; @@ -69,7 +135,7 @@ public void testInterception() throws Exception { decoder.channelRead(ctx, len); decoder.channelRead(ctx, dataBuf); verify(interceptor, times(interceptedReads)).handle(any(ByteBuf.class)); - verify(ctx).fireChannelRead(any(ByteBuffer.class)); + verify(ctx).fireChannelRead(any(ByteBuf.class)); assertEquals(0, len.refCnt()); assertEquals(0, dataBuf.refCnt()); } finally { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 098fa7974b87..70dcc8b8b8b6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -29,6 +29,7 @@ import com.codahale.metrics.Metric; import com.codahale.metrics.MetricSet; import com.codahale.metrics.Timer; +import com.codahale.metrics.Counter; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,6 +66,11 @@ public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFi new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); } + @VisibleForTesting + public ExternalShuffleBlockResolver getBlockResolver() { + return blockManager; + } + /** Enables mocking out the StreamManager and BlockManager. */ @VisibleForTesting public ExternalShuffleBlockHandler( @@ -91,7 +97,7 @@ protected void handleMessage( OpenBlocks msg = (OpenBlocks) msgObj; checkAuth(client, msg.appId); long streamId = streamManager.registerStream(client.getClientId(), - new ManagedBufferIterator(msg.appId, msg.execId, msg.blockIds)); + new ManagedBufferIterator(msg.appId, msg.execId, msg.blockIds), client.getChannel()); if (logger.isTraceEnabled()) { logger.trace("Registered streamId {} with {} buffers for client {} from host {}", streamId, @@ -173,7 +179,8 @@ private void checkAuth(TransportClient client, String appId) { /** * A simple class to wrap all shuffle service wrapper metrics */ - private class ShuffleMetrics implements MetricSet { + @VisibleForTesting + public class ShuffleMetrics implements MetricSet { private final Map allMetrics; // Time latency for open block request in ms private final Timer openBlockRequestLatencyMillis = new Timer(); @@ -181,14 +188,20 @@ private class ShuffleMetrics implements MetricSet { private final Timer registerExecutorRequestLatencyMillis = new Timer(); // Block transfer rate in byte per second private final Meter blockTransferRateBytes = new Meter(); + // Number of active connections to the shuffle service + private Counter activeConnections = new Counter(); + // Number of registered connections to the shuffle service + private Counter registeredConnections = new Counter(); - private ShuffleMetrics() { + public ShuffleMetrics() { allMetrics = new HashMap<>(); allMetrics.put("openBlockRequestLatencyMillis", openBlockRequestLatencyMillis); allMetrics.put("registerExecutorRequestLatencyMillis", registerExecutorRequestLatencyMillis); allMetrics.put("blockTransferRateBytes", blockTransferRateBytes); allMetrics.put("registeredExecutorsSize", (Gauge) () -> blockManager.getRegisteredExecutorsSize()); + allMetrics.put("numActiveConnections", activeConnections); + allMetrics.put("numRegisteredConnections", registeredConnections); } @Override @@ -244,4 +257,16 @@ public ManagedBuffer next() { } } + @Override + public void channelActive(TransportClient client) { + metrics.activeConnections.inc(); + super.channelActive(client); + } + + @Override + public void channelInactive(TransportClient client) { + metrics.activeConnections.dec(); + super.channelInactive(client); + } + } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index 02e6eb3a4467..57c1c5e7722c 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -91,6 +91,7 @@ public static void beforeAll() throws IOException { @AfterClass public static void afterAll() { server.close(); + context.close(); } @After @@ -153,13 +154,14 @@ public void testNoSaslClient() throws IOException, InterruptedException { @Test public void testNoSaslServer() { RpcHandler handler = new TestRpcHandler(); - TransportContext context = new TransportContext(conf, handler); - clientFactory = context.createClientFactory( - Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); - try (TransportServer server = context.createServer()) { - clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); - } catch (Exception e) { - assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation")); + try (TransportContext context = new TransportContext(conf, handler)) { + clientFactory = context.createClientFactory( + Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); + try (TransportServer server = context.createServer()) { + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + } catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation")); + } } } @@ -174,18 +176,15 @@ public void testAppIsolation() throws Exception { ExternalShuffleBlockHandler blockHandler = new ExternalShuffleBlockHandler( new OneForOneStreamManager(), blockResolver); TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, secretKeyHolder); - TransportContext blockServerContext = new TransportContext(conf, blockHandler); - TransportServer blockServer = blockServerContext.createServer(Arrays.asList(bootstrap)); - TransportClient client1 = null; - TransportClient client2 = null; - TransportClientFactory clientFactory2 = null; - try { + try ( + TransportContext blockServerContext = new TransportContext(conf, blockHandler); + TransportServer blockServer = blockServerContext.createServer(Arrays.asList(bootstrap)); // Create a client, and make a request to fetch blocks from a different app. - clientFactory = blockServerContext.createClientFactory( + TransportClientFactory clientFactory1 = blockServerContext.createClientFactory( Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); - client1 = clientFactory.createClient(TestUtils.getLocalHost(), - blockServer.getPort()); + TransportClient client1 = clientFactory1.createClient( + TestUtils.getLocalHost(), blockServer.getPort())) { AtomicReference exception = new AtomicReference<>(); @@ -223,41 +222,33 @@ public void onBlockFetchFailure(String blockId, Throwable t) { StreamHandle stream = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response); long streamId = stream.streamId; - // Create a second client, authenticated with a different app ID, and try to read from - // the stream created for the previous app. - clientFactory2 = blockServerContext.createClientFactory( - Arrays.asList(new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); - client2 = clientFactory2.createClient(TestUtils.getLocalHost(), - blockServer.getPort()); - - CountDownLatch chunkReceivedLatch = new CountDownLatch(1); - ChunkReceivedCallback callback = new ChunkReceivedCallback() { - @Override - public void onSuccess(int chunkIndex, ManagedBuffer buffer) { - chunkReceivedLatch.countDown(); - } - @Override - public void onFailure(int chunkIndex, Throwable t) { - exception.set(t); - chunkReceivedLatch.countDown(); - } - }; - - exception.set(null); - client2.fetchChunk(streamId, 0, callback); - chunkReceivedLatch.await(); - checkSecurityException(exception.get()); - } finally { - if (client1 != null) { - client1.close(); - } - if (client2 != null) { - client2.close(); - } - if (clientFactory2 != null) { - clientFactory2.close(); + try ( + // Create a second client, authenticated with a different app ID, and try to read from + // the stream created for the previous app. + TransportClientFactory clientFactory2 = blockServerContext.createClientFactory( + Arrays.asList(new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); + TransportClient client2 = clientFactory2.createClient( + TestUtils.getLocalHost(), blockServer.getPort()) + ) { + CountDownLatch chunkReceivedLatch = new CountDownLatch(1); + ChunkReceivedCallback callback = new ChunkReceivedCallback() { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + chunkReceivedLatch.countDown(); + } + + @Override + public void onFailure(int chunkIndex, Throwable t) { + exception.set(t); + chunkReceivedLatch.countDown(); + } + }; + + exception.set(null); + client2.fetchChunk(streamId, 0, callback); + chunkReceivedLatch.await(); + checkSecurityException(exception.get()); } - blockServer.close(); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 7846b71d5a8b..537c277cd26b 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -27,7 +27,7 @@ import org.mockito.ArgumentCaptor; import static org.junit.Assert.*; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.*; import org.apache.spark.network.buffer.ManagedBuffer; @@ -79,6 +79,8 @@ public void testRegisterExecutor() { @SuppressWarnings("unchecked") @Test public void testOpenShuffleBlocks() { + when(client.getClientId()).thenReturn("app0"); + RpcResponseCallback callback = mock(RpcResponseCallback.class); ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); @@ -101,7 +103,8 @@ public void testOpenShuffleBlocks() { @SuppressWarnings("unchecked") ArgumentCaptor> stream = (ArgumentCaptor>) (ArgumentCaptor) ArgumentCaptor.forClass(Iterator.class); - verify(streamManager, times(1)).registerStream(anyString(), stream.capture()); + verify(streamManager, times(1)).registerStream(anyString(), stream.capture(), + any()); Iterator buffers = stream.getValue(); assertEquals(block0Marker, buffers.next()); assertEquals(block1Marker, buffers.next()); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 526b96b36447..f5b1ec9d46da 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -58,6 +58,7 @@ public class ExternalShuffleIntegrationSuite { static ExternalShuffleBlockHandler handler; static TransportServer server; static TransportConf conf; + static TransportContext transportContext; static byte[][] exec0Blocks = new byte[][] { new byte[123], @@ -87,7 +88,7 @@ public static void beforeAll() throws IOException { conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); handler = new ExternalShuffleBlockHandler(conf, null); - TransportContext transportContext = new TransportContext(conf, handler); + transportContext = new TransportContext(conf, handler); server = transportContext.createServer(); } @@ -95,6 +96,7 @@ public static void beforeAll() throws IOException { public static void afterAll() { dataContext0.cleanup(); server.close(); + transportContext.close(); } @After diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 82caf392b821..67f79021daf5 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -41,14 +41,14 @@ public class ExternalShuffleSecuritySuite { TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); TransportServer server; + TransportContext transportContext; @Before public void beforeEach() throws IOException { - TransportContext context = - new TransportContext(conf, new ExternalShuffleBlockHandler(conf, null)); + transportContext = new TransportContext(conf, new ExternalShuffleBlockHandler(conf, null)); TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, new TestSecretKeyHolder("my-app-id", "secret")); - this.server = context.createServer(Arrays.asList(bootstrap)); + this.server = transportContext.createServer(Arrays.asList(bootstrap)); } @After @@ -57,6 +57,10 @@ public void afterEach() { server.close(); server = null; } + if (transportContext != null) { + transportContext.close(); + transportContext = null; + } } @Test diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index dc947a619bf0..95460637db89 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -28,10 +28,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java index a530e16734db..6f90df5f611a 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java @@ -46,9 +46,9 @@ */ public class RetryingBlockFetcherSuite { - ManagedBuffer block0 = new NioManagedBuffer(ByteBuffer.wrap(new byte[13])); - ManagedBuffer block1 = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - ManagedBuffer block2 = new NioManagedBuffer(ByteBuffer.wrap(new byte[19])); + private final ManagedBuffer block0 = new NioManagedBuffer(ByteBuffer.wrap(new byte[13])); + private final ManagedBuffer block1 = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); + private final ManagedBuffer block2 = new NioManagedBuffer(ByteBuffer.wrap(new byte[19])); @Test public void testNoFailures() throws IOException, InterruptedException { @@ -291,7 +291,7 @@ private static void performInteractions(List> inte } assertNotNull(stub); - stub.when(fetchStarter).createAndStart(any(), anyObject()); + stub.when(fetchStarter).createAndStart(any(), any()); String[] blockIdArray = blockIds.toArray(new String[blockIds.size()]); new RetryingBlockFetcher(conf, fetchStarter, blockIdArray, listener).start(); } diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 55cdc3140aa0..c642f3b4a160 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -35,7 +35,7 @@ provided ${project.build.directory}/scala-${scala.binary.version}/spark-${project.version}-yarn-shuffle.jar - org/spark_project/ + org/sparkproject/ @@ -128,6 +128,50 @@ + + + org.codehaus.mojo + build-helper-maven-plugin + + + regex-property + + regex-property + + + spark.shade.native.packageName + ${spark.shade.packageName} + \. + _ + true + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + unpack + package + + + + + + + + + + + run + + + + diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 72ae1a129523..25592e9873ff 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -113,6 +113,8 @@ public class YarnShuffleService extends AuxiliaryService { // The actual server that serves shuffle files private TransportServer shuffleServer = null; + private TransportContext transportContext = null; + private Configuration _conf = null; // The recovery path used to shuffle service recovery @@ -170,15 +172,6 @@ protected void serviceInit(Configuration conf) throws Exception { TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); - // register metrics on the block handler into the Node Manager's metrics system. - YarnShuffleServiceMetrics serviceMetrics = - new YarnShuffleServiceMetrics(blockHandler.getAllMetrics()); - - MetricsSystemImpl metricsSystem = (MetricsSystemImpl) DefaultMetricsSystem.instance(); - metricsSystem.register( - "sparkShuffleService", "Metrics on the Spark Shuffle Service", serviceMetrics); - logger.info("Registered metrics with Hadoop's DefaultMetricsSystem"); - // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests List bootstraps = Lists.newArrayList(); @@ -193,12 +186,24 @@ protected void serviceInit(Configuration conf) throws Exception { int port = conf.getInt( SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - TransportContext transportContext = new TransportContext(transportConf, blockHandler); + transportContext = new TransportContext(transportConf, blockHandler); shuffleServer = transportContext.createServer(port, bootstraps); // the port should normally be fixed, but for tests its useful to find an open port port = shuffleServer.getPort(); boundPort = port; String authEnabledString = authEnabled ? "enabled" : "not enabled"; + + // register metrics on the block handler into the Node Manager's metrics system. + blockHandler.getAllMetrics().getMetrics().put("numRegisteredConnections", + shuffleServer.getRegisteredConnections()); + YarnShuffleServiceMetrics serviceMetrics = + new YarnShuffleServiceMetrics(blockHandler.getAllMetrics()); + + MetricsSystemImpl metricsSystem = (MetricsSystemImpl) DefaultMetricsSystem.instance(); + metricsSystem.register( + "sparkShuffleService", "Metrics on the Spark Shuffle Service", serviceMetrics); + logger.info("Registered metrics with Hadoop's DefaultMetricsSystem"); + logger.info("Started YARN shuffle service for Spark on port {}. " + "Authentication is {}. Registered executor file is {}", port, authEnabledString, registeredExecutorFile); @@ -315,6 +320,9 @@ protected void serviceStop() { if (shuffleServer != null) { shuffleServer.close(); } + if (transportContext != null) { + transportContext.close(); + } if (blockHandler != null) { blockHandler.close(); } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java index 3e4d479b862b..501237407e9b 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java @@ -107,6 +107,11 @@ public static void collectMetric( throw new IllegalStateException( "Not supported class type of metric[" + name + "] for value " + gaugeValue); } + } else if (metric instanceof Counter) { + Counter c = (Counter) metric; + long counterValue = c.getCount(); + metricsRecordBuilder.addGauge(new ShuffleServiceMetricsInfo(name, "Number of " + + "connections to shuffle service " + name), counterValue); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index 4563efcfcf47..1adf7abfc8a6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -174,11 +174,6 @@ public static float getFloat(Object object, long offset) { } public static void putFloat(Object object, long offset, float value) { - if (Float.isNaN(value)) { - value = Float.NaN; - } else if (value == -0.0f) { - value = 0.0f; - } _UNSAFE.putFloat(object, offset, value); } @@ -187,11 +182,6 @@ public static double getDouble(Object object, long offset) { } public static void putDouble(Object object, long offset, double value) { - if (Double.isNaN(value)) { - value = Double.NaN; - } else if (value == -0.0d) { - value = 0.0d; - } _UNSAFE.putDouble(object, offset, value); } @@ -219,22 +209,33 @@ public static long reallocateMemory(long address, long oldSize, long newSize) { } /** - * Uses internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's - * MaxDirectMemorySize limit (the default limit is too low and we do not want to require users - * to increase it). + * Allocate a DirectByteBuffer, potentially bypassing the JVM's MaxDirectMemorySize limit. */ public static ByteBuffer allocateDirectBuffer(int size) { try { - long memory = allocateMemory(size); - ByteBuffer buffer = (ByteBuffer) DBB_CONSTRUCTOR.newInstance(memory, size); - if (CLEANER_CREATE_METHOD != null) { + if (CLEANER_CREATE_METHOD == null) { + // Can't set a Cleaner (see comments on field), so need to allocate via normal Java APIs try { - DBB_CLEANER_FIELD.set(buffer, - CLEANER_CREATE_METHOD.invoke(null, buffer, (Runnable) () -> freeMemory(memory))); - } catch (IllegalAccessException | InvocationTargetException e) { - throw new IllegalStateException(e); + return ByteBuffer.allocateDirect(size); + } catch (OutOfMemoryError oome) { + // checkstyle.off: RegexpSinglelineJava + throw new OutOfMemoryError("Failed to allocate direct buffer (" + oome.getMessage() + + "); try increasing -XX:MaxDirectMemorySize=... to, for example, your heap size"); + // checkstyle.on: RegexpSinglelineJava } } + // Otherwise, use internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's + // MaxDirectMemorySize limit (the default limit is too low and we do not want to + // require users to increase it). + long memory = allocateMemory(size); + ByteBuffer buffer = (ByteBuffer) DBB_CONSTRUCTOR.newInstance(memory, size); + try { + DBB_CLEANER_FIELD.set(buffer, + CLEANER_CREATE_METHOD.invoke(null, buffer, (Runnable) () -> freeMemory(memory))); + } catch (IllegalAccessException | InvocationTargetException e) { + freeMemory(memory); + throw new IllegalStateException(e); + } return buffer; } catch (Exception e) { throwException(e); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 2cd39bd60c2a..305cc1c5d111 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -23,7 +23,7 @@ /** * An array of long values. Compared with native JVM arrays, this: *
    - *
  • supports using both in-heap and off-heap memory
  • + *
  • supports using both on-heap and off-heap memory
  • *
  • has no bound checking, and thus can crash the JVM process when assert is turned off
  • *
*/ diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java index 74ebc87dc978..897b8a2b7ec5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java @@ -21,7 +21,7 @@ /** * A memory location. Tracked either by a memory address (with off-heap allocation), - * or by an offset from a JVM object (in-heap allocation). + * or by an offset from a JVM object (on-heap allocation). */ public class MemoryLocation { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 621f2c6bf377..e36efa3b0f22 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; +import java.util.Locale; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -66,6 +67,10 @@ private static long toLong(String s) { } } + /** + * Convert a string to CalendarInterval. Return null if the input string is not a valid interval. + * This method is case-sensitive and all characters in the input string should be in lower case. + */ public static CalendarInterval fromString(String s) { if (s == null) { return null; @@ -87,6 +92,26 @@ public static CalendarInterval fromString(String s) { } } + /** + * Convert a string to CalendarInterval. Unlike fromString, this method is case-insensitive and + * will throw IllegalArgumentException when the input string is not a valid interval. + * + * @throws IllegalArgumentException if the string is not a valid internal. + */ + public static CalendarInterval fromCaseInsensitiveString(String s) { + if (s == null || s.trim().isEmpty()) { + throw new IllegalArgumentException("Interval cannot be null or blank."); + } + String sInLowerCase = s.trim().toLowerCase(Locale.ROOT); + String interval = + sInLowerCase.startsWith("interval ") ? sInLowerCase : "interval " + sInLowerCase; + CalendarInterval cal = fromString(interval); + if (cal == null) { + throw new IllegalArgumentException("Invalid interval: " + s); + } + return cal; + } + public static long toLongWithRange(String fieldName, String s, long minValue, long maxValue) throws IllegalArgumentException { long result = 0; @@ -319,6 +344,8 @@ public String toString() { appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond"); rest %= MICROS_PER_MILLI; appendUnit(sb, rest, "microsecond"); + } else if (months == 0) { + sb.append(" 0 microseconds"); } return sb.toString(); diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java index ab34324eb54c..3ad9ac7b4de9 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -157,18 +157,4 @@ public void heapMemoryReuse() { Assert.assertEquals(onheap4.size(), 1024 * 1024 + 7); Assert.assertEquals(obj3, onheap4.getBaseObject()); } - - @Test - // SPARK-26021 - public void writeMinusZeroIsReplacedWithZero() { - byte[] doubleBytes = new byte[Double.BYTES]; - byte[] floatBytes = new byte[Float.BYTES]; - Platform.putDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET, -0.0d); - Platform.putFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET, -0.0f); - double doubleFromPlatform = Platform.getDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET); - float floatFromPlatform = Platform.getFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET); - - Assert.assertEquals(Double.doubleToLongBits(0.0d), Double.doubleToLongBits(doubleFromPlatform)); - Assert.assertEquals(Float.floatToIntBits(0.0f), Float.floatToIntBits(floatFromPlatform)); - } } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 9e69e264ff28..994af8f08244 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -41,6 +41,9 @@ public void equalsTest() { public void toStringTest() { CalendarInterval i; + i = new CalendarInterval(0, 0); + assertEquals("interval 0 microseconds", i.toString()); + i = new CalendarInterval(34, 0); assertEquals("interval 2 years 10 months", i.toString()); @@ -101,6 +104,31 @@ public void fromStringTest() { assertNull(fromString(input)); } + @Test + public void fromCaseInsensitiveStringTest() { + for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { + assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); + } + + for (String input : new String[]{null, "", " "}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("cannot be null or blank")); + } + } + + for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid interval")); + } + } + } + @Test public void fromYearMonthStringTest() { String input; diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index ec1aa187dfb3..e91595dd324b 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -28,8 +28,8 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: log4j.logger.org.apache.spark.repl.Main=WARN # Settings to quiet third party logs that are too verbose -log4j.logger.org.spark_project.jetty=WARN -log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.sparkproject.jetty=WARN +log4j.logger.org.sparkproject.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO log4j.logger.org.apache.parquet=ERROR diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index 4c008a13607c..23407e1f7075 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -64,6 +64,10 @@ # "/metrics/applications/json" endpoints can be sent separately to get # metrics snapshots of the master instance and applications. This # MetricsServlet does not have to be configured. +# 6. The metrics system can also be configured using Spark configuration +# parameters. The relevant parameter names are formed by adding the +# prefix "spark.metrics.conf." to the configuration entries detailed in +# this file (see examples below). ## List of available common sources and their properties. @@ -172,6 +176,14 @@ # Unit of the polling period for the Slf4jSink #*.sink.slf4j.unit=minutes +# Example configuration for Graphite sink +#*.sink.graphite.class=org.apache.spark.metrics.sink.GraphiteSink +#*.sink.graphite.host= +#*.sink.graphite.port= +#*.sink.graphite.period=10 +#*.sink.graphite.unit=seconds +#*.sink.graphite.prefix= + # Enable JvmSource for instance master, worker, driver and executor #master.source.jvm.class=org.apache.spark.metrics.source.JvmSource @@ -179,4 +191,4 @@ #driver.source.jvm.class=org.apache.spark.metrics.source.JvmSource -#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource +#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource \ No newline at end of file diff --git a/core/benchmarks/CoalescedRDDBenchmark-results.txt b/core/benchmarks/CoalescedRDDBenchmark-results.txt new file mode 100644 index 000000000000..dd63b0adea4f --- /dev/null +++ b/core/benchmarks/CoalescedRDDBenchmark-results.txt @@ -0,0 +1,40 @@ +================================================================================================ +Coalesced RDD , large scale +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_201-b09 on Windows 10 10.0 +Intel64 Family 6 Model 63 Stepping 2, GenuineIntel +Coalesced RDD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Coalesce Num Partitions: 100 Num Hosts: 1 346 364 24 0.3 3458.9 1.0X +Coalesce Num Partitions: 100 Num Hosts: 5 258 264 6 0.4 2579.0 1.3X +Coalesce Num Partitions: 100 Num Hosts: 10 242 249 7 0.4 2415.2 1.4X +Coalesce Num Partitions: 100 Num Hosts: 20 237 242 7 0.4 2371.7 1.5X +Coalesce Num Partitions: 100 Num Hosts: 40 230 231 1 0.4 2299.8 1.5X +Coalesce Num Partitions: 100 Num Hosts: 80 222 233 14 0.4 2223.0 1.6X +Coalesce Num Partitions: 500 Num Hosts: 1 659 665 5 0.2 6590.4 0.5X +Coalesce Num Partitions: 500 Num Hosts: 5 340 381 47 0.3 3395.2 1.0X +Coalesce Num Partitions: 500 Num Hosts: 10 279 307 47 0.4 2788.3 1.2X +Coalesce Num Partitions: 500 Num Hosts: 20 259 261 2 0.4 2591.9 1.3X +Coalesce Num Partitions: 500 Num Hosts: 40 241 250 15 0.4 2406.5 1.4X +Coalesce Num Partitions: 500 Num Hosts: 80 235 237 3 0.4 2349.9 1.5X +Coalesce Num Partitions: 1000 Num Hosts: 1 1050 1053 4 0.1 10503.2 0.3X +Coalesce Num Partitions: 1000 Num Hosts: 5 405 407 2 0.2 4049.5 0.9X +Coalesce Num Partitions: 1000 Num Hosts: 10 320 322 2 0.3 3202.7 1.1X +Coalesce Num Partitions: 1000 Num Hosts: 20 276 277 0 0.4 2762.3 1.3X +Coalesce Num Partitions: 1000 Num Hosts: 40 257 260 5 0.4 2571.2 1.3X +Coalesce Num Partitions: 1000 Num Hosts: 80 245 252 13 0.4 2448.9 1.4X +Coalesce Num Partitions: 5000 Num Hosts: 1 3099 3145 55 0.0 30988.6 0.1X +Coalesce Num Partitions: 5000 Num Hosts: 5 1037 1050 20 0.1 10374.4 0.3X +Coalesce Num Partitions: 5000 Num Hosts: 10 626 633 8 0.2 6261.8 0.6X +Coalesce Num Partitions: 5000 Num Hosts: 20 426 431 5 0.2 4258.6 0.8X +Coalesce Num Partitions: 5000 Num Hosts: 40 328 341 22 0.3 3275.4 1.1X +Coalesce Num Partitions: 5000 Num Hosts: 80 272 275 4 0.4 2721.4 1.3X +Coalesce Num Partitions: 10000 Num Hosts: 1 5516 5526 9 0.0 55156.8 0.1X +Coalesce Num Partitions: 10000 Num Hosts: 5 1956 1992 48 0.1 19560.9 0.2X +Coalesce Num Partitions: 10000 Num Hosts: 10 1045 1057 18 0.1 10447.4 0.3X +Coalesce Num Partitions: 10000 Num Hosts: 20 637 658 24 0.2 6373.2 0.5X +Coalesce Num Partitions: 10000 Num Hosts: 40 431 448 15 0.2 4312.9 0.8X +Coalesce Num Partitions: 10000 Num Hosts: 80 326 328 2 0.3 3263.4 1.1X + + diff --git a/core/benchmarks/XORShiftRandomBenchmark-results.txt b/core/benchmarks/XORShiftRandomBenchmark-results.txt new file mode 100644 index 000000000000..1140489e4a7f --- /dev/null +++ b/core/benchmarks/XORShiftRandomBenchmark-results.txt @@ -0,0 +1,44 @@ +================================================================================================ +Pseudo random +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextInt: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +java.util.Random 1362 / 1362 73.4 13.6 1.0X +XORShiftRandom 227 / 227 440.6 2.3 6.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextLong: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +java.util.Random 2732 / 2732 36.6 27.3 1.0X +XORShiftRandom 629 / 629 159.0 6.3 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextDouble: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +java.util.Random 2730 / 2730 36.6 27.3 1.0X +XORShiftRandom 629 / 629 159.0 6.3 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextGaussian: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +java.util.Random 10288 / 10288 9.7 102.9 1.0X +XORShiftRandom 6351 / 6351 15.7 63.5 1.6X + + +================================================================================================ +hash seed +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash seed: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +XORShiftRandom.hashSeed 1193 / 1195 8.4 119.3 1.0X + + diff --git a/core/pom.xml b/core/pom.xml index 36d93212ba9f..9d5702801988 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -33,6 +33,10 @@ Spark Project Core http://spark.apache.org/ + + com.thoughtworks.paranamer + paranamer + org.apache.avro avro @@ -220,6 +224,10 @@ org.scala-lang scala-library + + org.scala-lang + scala-reflect + org.json4s json4s-jackson_${scala.binary.version} @@ -339,7 +347,7 @@ net.razorvine pyrolite - 4.13 + 4.23 net.razorvine @@ -364,6 +372,13 @@ tests test + + org.apache.spark + spark-network-shuffle_${scala.binary.version} + ${project.version} + tests + test + - - ${hive.group} - hive-exec - provided - - - ${hive.group} - hive-metastore - provided - - - org.apache.thrift - libthrift - provided - - - org.apache.thrift - libfb303 - provided - - target/scala-${scala.binary.version}/classes diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 28b646ba3c95..1d9391845be5 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -85,9 +85,9 @@ public class TaskMemoryManager { /** * Similar to an operating system's page table, this array maps page numbers into base object * pointers, allowing us to translate between the hashtable's internal 64-bit address - * representation and the baseObject+offset representation which we use to support both in- and + * representation and the baseObject+offset representation which we use to support both on- and * off-heap addresses. When using an off-heap allocator, every entry in this map will be `null`. - * When using an in-heap allocator, the entries in this map will point to pages' base objects. + * When using an on-heap allocator, the entries in this map will point to pages' base objects. * Entries are added to this map as new data pages are allocated. */ private final MemoryBlock[] pageTable = new MemoryBlock[PAGE_TABLE_SIZE]; @@ -102,7 +102,7 @@ public class TaskMemoryManager { private final long taskAttemptId; /** - * Tracks whether we're in-heap or off-heap. For off-heap, we short-circuit most of these methods + * Tracks whether we're on-heap or off-heap. For off-heap, we short-circuit most of these methods * without doing any masking or lookups. Since this branching should be well-predicted by the JIT, * this extra layer of indirection / abstraction hopefully shouldn't be too expensive. */ diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index fda33cd8293d..32b446785a9f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -34,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.internal.config.package$; import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; @@ -58,9 +59,8 @@ * simultaneously opens separate serializers and file streams for all partitions. As a result, * {@link SortShuffleManager} only selects this write path when *
    - *
  • no Ordering is specified,
  • - *
  • no Aggregator is specified, and
  • - *
  • the number of partitions is less than + *
  • no map-side combine is specified, and
  • + *
  • the number of partitions is less than or equal to * spark.shuffle.sort.bypassMergeThreshold.
  • *
* @@ -105,7 +105,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { SparkConf conf, ShuffleWriteMetricsReporter writeMetrics) { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - this.fileBufferSize = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; + this.fileBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 6ee9d5f0eec3..024756087bf7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -129,7 +129,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { (int) conf.get(package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()); this.writeMetrics = writeMetrics; this.inMemSorter = new ShuffleInMemorySorter( - this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); + this, initialSize, (boolean) conf.get(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT())); this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); @@ -145,6 +145,15 @@ final class ShuffleExternalSorter extends MemoryConsumer { */ private void writeSortedFile(boolean isLastFile) { + // This call performs the actual sort. + final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = + inMemSorter.getSortedIterator(); + + // If there are no sorted records, so we don't need to create an empty spill file. + if (!sortedRecords.hasNext()) { + return; + } + final ShuffleWriteMetricsReporter writeMetricsToUse; if (isLastFile) { @@ -157,10 +166,6 @@ private void writeSortedFile(boolean isLastFile) { writeMetricsToUse = new ShuffleWriteMetrics(); } - // This call performs the actual sort. - final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = - inMemSorter.getSortedIterator(); - // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to // be an API to directly transfer bytes from managed memory to the disk writer, we buffer // data through a byte array. This array does not need to be large enough to hold a single diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 4b0c74341551..9d05f03613ce 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -37,6 +37,7 @@ import org.apache.spark.*; import org.apache.spark.annotation.Private; +import org.apache.spark.internal.config.package$; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.NioBufferedFileInputStream; @@ -55,7 +56,6 @@ import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; -import org.apache.spark.internal.config.package$; @Private public class UnsafeShuffleWriter extends ShuffleWriter { @@ -143,8 +143,8 @@ public UnsafeShuffleWriter( this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); - this.initialSortBufferSize = sparkConf.getInt("spark.shuffle.sort.initialBufferSize", - DEFAULT_INITIAL_SORT_BUFFER_SIZE); + this.initialSortBufferSize = + (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()); this.inputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.outputBufferSizeInBytes = @@ -282,10 +282,10 @@ void forceSorterToSpill() throws IOException { * @return the partition lengths in the merged file. */ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { - final boolean compressionEnabled = sparkConf.getBoolean("spark.shuffle.compress", true); + final boolean compressionEnabled = (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_COMPRESS()); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = - sparkConf.getBoolean("spark.shuffle.unsafe.fastMergeEnabled", true); + (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_UNDAFE_FAST_MERGE_ENABLE()); final boolean fastMergeIsSupported = !compressionEnabled || CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index a4e88598f760..13ca7fb8a318 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -159,11 +159,9 @@ public final class BytesToBytesMap extends MemoryConsumer { */ private final Location loc; - private final boolean enablePerfMetrics; + private long numProbes = 0L; - private long numProbes = 0; - - private long numKeyLookups = 0; + private long numKeyLookups = 0L; private long peakMemoryUsedBytes = 0L; @@ -180,8 +178,7 @@ public BytesToBytesMap( SerializerManager serializerManager, int initialCapacity, double loadFactor, - long pageSizeBytes, - boolean enablePerfMetrics) { + long pageSizeBytes) { super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; @@ -189,7 +186,6 @@ public BytesToBytesMap( this.loadFactor = loadFactor; this.loc = new Location(); this.pageSizeBytes = pageSizeBytes; - this.enablePerfMetrics = enablePerfMetrics; if (initialCapacity <= 0) { throw new IllegalArgumentException("Initial capacity must be greater than 0"); } @@ -209,14 +205,6 @@ public BytesToBytesMap( TaskMemoryManager taskMemoryManager, int initialCapacity, long pageSizeBytes) { - this(taskMemoryManager, initialCapacity, pageSizeBytes, false); - } - - public BytesToBytesMap( - TaskMemoryManager taskMemoryManager, - int initialCapacity, - long pageSizeBytes, - boolean enablePerfMetrics) { this( taskMemoryManager, SparkEnv.get() != null ? SparkEnv.get().blockManager() : null, @@ -224,8 +212,7 @@ public BytesToBytesMap( initialCapacity, // In order to re-use the longArray for sorting, the load factor cannot be larger than 0.5. 0.5, - pageSizeBytes, - enablePerfMetrics); + pageSizeBytes); } /** @@ -268,33 +255,46 @@ private MapIterator(int numRecords, Location loc, boolean destructive) { } private void advanceToNextPage() { - synchronized (this) { - int nextIdx = dataPages.indexOf(currentPage) + 1; - if (destructive && currentPage != null) { - dataPages.remove(currentPage); - freePage(currentPage); - nextIdx --; - } - if (dataPages.size() > nextIdx) { - currentPage = dataPages.get(nextIdx); - pageBaseObject = currentPage.getBaseObject(); - offsetInPage = currentPage.getBaseOffset(); - recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); - offsetInPage += UnsafeAlignedOffset.getUaoSize(); - } else { - currentPage = null; - if (reader != null) { - handleFailedDelete(); + // SPARK-26265: We will first lock this `MapIterator` and then `TaskMemoryManager` when going + // to free a memory page by calling `freePage`. At the same time, it is possibly that another + // memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it + // acquires memory and causes spilling on this `MapIterator`. To avoid deadlock here, we keep + // reference to the page to free and free it after releasing the lock of `MapIterator`. + MemoryBlock pageToFree = null; + + try { + synchronized (this) { + int nextIdx = dataPages.indexOf(currentPage) + 1; + if (destructive && currentPage != null) { + dataPages.remove(currentPage); + pageToFree = currentPage; + nextIdx--; } - try { - Closeables.close(reader, /* swallowIOException = */ false); - reader = spillWriters.getFirst().getReader(serializerManager); - recordsInPage = -1; - } catch (IOException e) { - // Scala iterator does not handle exception - Platform.throwException(e); + if (dataPages.size() > nextIdx) { + currentPage = dataPages.get(nextIdx); + pageBaseObject = currentPage.getBaseObject(); + offsetInPage = currentPage.getBaseOffset(); + recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); + offsetInPage += UnsafeAlignedOffset.getUaoSize(); + } else { + currentPage = null; + if (reader != null) { + handleFailedDelete(); + } + try { + Closeables.close(reader, /* swallowIOException = */ false); + reader = spillWriters.getFirst().getReader(serializerManager); + recordsInPage = -1; + } catch (IOException e) { + // Scala iterator does not handle exception + Platform.throwException(e); + } } } + } finally { + if (pageToFree != null) { + freePage(pageToFree); + } } } @@ -342,52 +342,50 @@ public Location next() { } } - public long spill(long numBytes) throws IOException { - synchronized (this) { - if (!destructive || dataPages.size() == 1) { - return 0L; - } + public synchronized long spill(long numBytes) throws IOException { + if (!destructive || dataPages.size() == 1) { + return 0L; + } - updatePeakMemoryUsed(); + updatePeakMemoryUsed(); - // TODO: use existing ShuffleWriteMetrics - ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); + // TODO: use existing ShuffleWriteMetrics + ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - long released = 0L; - while (dataPages.size() > 0) { - MemoryBlock block = dataPages.getLast(); - // The currentPage is used, cannot be released - if (block == currentPage) { - break; - } + long released = 0L; + while (dataPages.size() > 0) { + MemoryBlock block = dataPages.getLast(); + // The currentPage is used, cannot be released + if (block == currentPage) { + break; + } - Object base = block.getBaseObject(); - long offset = block.getBaseOffset(); - int numRecords = UnsafeAlignedOffset.getSize(base, offset); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - offset += uaoSize; - final UnsafeSorterSpillWriter writer = - new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords); - while (numRecords > 0) { - int length = UnsafeAlignedOffset.getSize(base, offset); - writer.write(base, offset + uaoSize, length, 0); - offset += uaoSize + length + 8; - numRecords--; - } - writer.close(); - spillWriters.add(writer); + Object base = block.getBaseObject(); + long offset = block.getBaseOffset(); + int numRecords = UnsafeAlignedOffset.getSize(base, offset); + int uaoSize = UnsafeAlignedOffset.getUaoSize(); + offset += uaoSize; + final UnsafeSorterSpillWriter writer = + new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords); + while (numRecords > 0) { + int length = UnsafeAlignedOffset.getSize(base, offset); + writer.write(base, offset + uaoSize, length, 0); + offset += uaoSize + length + 8; + numRecords--; + } + writer.close(); + spillWriters.add(writer); - dataPages.removeLast(); - released += block.size(); - freePage(block); + dataPages.removeLast(); + released += block.size(); + freePage(block); - if (released >= numBytes) { - break; - } + if (released >= numBytes) { + break; } - - return released; } + + return released; } @Override @@ -462,15 +460,12 @@ public Location lookup(Object keyBase, long keyOffset, int keyLength, int hash) public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location loc, int hash) { assert(longArray != null); - if (enablePerfMetrics) { - numKeyLookups++; - } + numKeyLookups++; + int pos = hash & mask; int step = 1; while (true) { - if (enablePerfMetrics) { - numProbes++; - } + numProbes++; if (longArray.get(pos * 2) == 0) { // This is a new key. loc.with(pos, hash, false); @@ -717,7 +712,7 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff final long recordOffset = offset; UnsafeAlignedOffset.putSize(base, offset, klen + vlen + uaoSize); UnsafeAlignedOffset.putSize(base, offset + uaoSize, klen); - offset += (2 * uaoSize); + offset += (2L * uaoSize); Platform.copyMemory(kbase, koff, base, offset, klen); offset += klen; Platform.copyMemory(vbase, voff, base, offset, vlen); @@ -785,7 +780,7 @@ private void allocate(int capacity) { assert (capacity >= 0); capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64); assert (capacity <= MAX_CAPACITY); - longArray = allocateArray(capacity * 2); + longArray = allocateArray(capacity * 2L); longArray.zeroOut(); this.growthThreshold = (int) (capacity * loadFactor); @@ -859,10 +854,7 @@ public long getPeakMemoryUsedBytes() { /** * Returns the average number of probes per key lookup. */ - public double getAverageProbesPerLookup() { - if (!enablePerfMetrics) { - throw new IllegalStateException(); - } + public double getAvgHashProbeBucketListIterations() { return (1.0 * numProbes) / numKeyLookups; } diff --git a/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java new file mode 100644 index 000000000000..57d96756c8be --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java @@ -0,0 +1,68 @@ +/* + * 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.util; + +import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Enumeration; + +/** + * A mutable class loader that gives preference to its own URLs over the parent class loader + * when loading classes and resources. + */ +public class ChildFirstURLClassLoader extends MutableURLClassLoader { + + static { + ClassLoader.registerAsParallelCapable(); + } + + private ParentClassLoader parent; + + public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent) { + super(urls, null); + this.parent = new ParentClassLoader(parent); + } + + @Override + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + try { + return super.loadClass(name, resolve); + } catch (ClassNotFoundException cnf) { + return parent.loadClass(name, resolve); + } + } + + @Override + public Enumeration getResources(String name) throws IOException { + ArrayList urls = Collections.list(super.getResources(name)); + urls.addAll(Collections.list(parent.getResources(name))); + return Collections.enumeration(urls); + } + + @Override + public URL getResource(String name) { + URL url = super.getResource(name); + if (url != null) { + return url; + } else { + return parent.getResource(name); + } + } +} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala b/core/src/main/java/org/apache/spark/util/MutableURLClassLoader.java similarity index 64% rename from resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala rename to core/src/main/java/org/apache/spark/util/MutableURLClassLoader.java index 9b8c98cda8da..a7c775db179b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala +++ b/core/src/main/java/org/apache/spark/util/MutableURLClassLoader.java @@ -15,20 +15,26 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.util; -import java.util.concurrent.atomic.AtomicBoolean +import java.net.URL; +import java.net.URLClassLoader; -private[spark] class SimpleExtensionService extends SchedulerExtensionService { +/** + * URL class loader that exposes the `addURL` method in URLClassLoader. + */ +public class MutableURLClassLoader extends URLClassLoader { - /** started flag; set in the `start()` call, stopped in `stop()`. */ - val started = new AtomicBoolean(false) + static { + ClassLoader.registerAsParallelCapable(); + } - override def start(binding: SchedulerExtensionServiceBinding): Unit = { - started.set(true) + public MutableURLClassLoader(URL[] urls, ClassLoader parent) { + super(urls, parent); } - override def stop(): Unit = { - started.set(false) + @Override + public void addURL(URL url) { + super.addURL(url); } } diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/java/org/apache/spark/util/ParentClassLoader.java similarity index 65% rename from core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala rename to core/src/main/java/org/apache/spark/util/ParentClassLoader.java index c9b7493fcdc1..094005c58c78 100644 --- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala +++ b/core/src/main/java/org/apache/spark/util/ParentClassLoader.java @@ -15,23 +15,28 @@ * limitations under the License. */ -package org.apache.spark.util +package org.apache.spark.util; /** * A class loader which makes some protected methods in ClassLoader accessible. */ -private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { +public class ParentClassLoader extends ClassLoader { - override def findClass(name: String): Class[_] = { - super.findClass(name) + static { + ClassLoader.registerAsParallelCapable(); } - override def loadClass(name: String): Class[_] = { - super.loadClass(name) + public ParentClassLoader(ClassLoader parent) { + super(parent); } - override def loadClass(name: String, resolve: Boolean): Class[_] = { - super.loadClass(name, resolve) + @Override + public Class findClass(String name) throws ClassNotFoundException { + return super.findClass(name); } + @Override + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + return super.loadClass(name, resolve); + } } diff --git a/core/src/main/java/org/apache/spark/util/collection/TimSort.java b/core/src/main/java/org/apache/spark/util/collection/TimSort.java index 40b5fb7fe4b4..31428665f963 100644 --- a/core/src/main/java/org/apache/spark/util/collection/TimSort.java +++ b/core/src/main/java/org/apache/spark/util/collection/TimSort.java @@ -409,10 +409,14 @@ private SortState(Buffer a, Comparator c, int len) { * large) stack lengths for smaller arrays. The "magic numbers" in the * computation below must be changed if MIN_MERGE is decreased. See * the MIN_MERGE declaration above for more information. + * The maximum value of 49 allows for an array up to length + * Integer.MAX_VALUE-4, if array is filled by the worst case stack size + * increasing scenario. More explanations are given in section 4 of: + * http://envisage-project.eu/wp-content/uploads/2015/02/sorting.pdf */ int stackLen = (len < 120 ? 5 : len < 1542 ? 10 : - len < 119151 ? 19 : 40); + len < 119151 ? 24 : 49); runBase = new int[stackLen]; runLen = new int[stackLen]; } @@ -439,15 +443,20 @@ private void pushRun(int runBase, int runLen) { * This method is called each time a new run is pushed onto the stack, * so the invariants are guaranteed to hold for i < stackSize upon * entry to the method. + * + * Thanks to Stijn de Gouw, Jurriaan Rot, Frank S. de Boer, + * Richard Bubel and Reiner Hahnle, this is fixed with respect to + * the analysis in "On the Worst-Case Complexity of TimSort" by + * Nicolas Auger, Vincent Jug, Cyril Nicaud, and Carine Pivoteau. */ private void mergeCollapse() { while (stackSize > 1) { int n = stackSize - 2; - if ( (n >= 1 && runLen[n-1] <= runLen[n] + runLen[n+1]) - || (n >= 2 && runLen[n-2] <= runLen[n] + runLen[n-1])) { + if (n > 0 && runLen[n-1] <= runLen[n] + runLen[n+1] || + n > 1 && runLen[n-2] <= runLen[n] + runLen[n-1]) { if (runLen[n - 1] < runLen[n + 1]) n--; - } else if (runLen[n] > runLen[n + 1]) { + } else if (n < 0 || runLen[n] > runLen[n + 1]) { break; // Invariant is established } mergeAt(n); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java index 0910db22af00..bef1bdadb27a 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java @@ -69,6 +69,8 @@ public static final class DoublePrefixComparator { * details see http://stereopsis.com/radix.html. */ public static long computePrefix(double value) { + // normalize -0.0 to 0.0, as they should be equal + value = value == -0.0 ? 0.0 : value; // Java's doubleToLongBits already canonicalizes all NaN values to the smallest possible // positive NaN, so there's nothing special we need to do for NaNs. long bits = Double.doubleToLongBits(value); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 5056652a2420..1b206c11d9a8 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -213,14 +213,12 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { spillWriters.size() > 1 ? " times" : " time"); ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - // We only write out contents of the inMemSorter if it is not empty. - if (inMemSorter.numRecords() > 0) { - final UnsafeSorterSpillWriter spillWriter = - new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, - inMemSorter.numRecords()); - spillWriters.add(spillWriter); - spillIterator(inMemSorter.getSortedIterator(), spillWriter); - } + + final UnsafeSorterSpillWriter spillWriter = + new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, + inMemSorter.numRecords()); + spillWriters.add(spillWriter); + spillIterator(inMemSorter.getSortedIterator(), spillWriter); final long spillSize = freeMemory(); // Note that this is more-or-less going to be a multiple of the page size, so wasted space in @@ -575,19 +573,31 @@ public boolean hasNext() { @Override public void loadNext() throws IOException { - synchronized (this) { - loaded = true; - if (nextUpstream != null) { - // Just consumed the last record from in memory iterator - if (lastPage != null) { - freePage(lastPage); - lastPage = null; + MemoryBlock pageToFree = null; + try { + synchronized (this) { + loaded = true; + if (nextUpstream != null) { + // Just consumed the last record from in memory iterator + if(lastPage != null) { + // Do not free the page here, while we are locking `SpillableIterator`. The `freePage` + // method locks the `TaskMemoryManager`, and it's a bad idea to lock 2 objects in + // sequence. We may hit dead lock if another thread locks `TaskMemoryManager` and + // `SpillableIterator` in sequence, which may happen in + // `TaskMemoryManager.acquireExecutionMemory`. + pageToFree = lastPage; + lastPage = null; + } + upstream = nextUpstream; + nextUpstream = null; } - upstream = nextUpstream; - nextUpstream = null; + numRecords--; + upstream.loadNext(); + } + } finally { + if (pageToFree != null) { + freePage(pageToFree); } - numRecords--; - upstream.loadNext(); } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java index fb179d07edeb..a524c4790407 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java @@ -21,13 +21,13 @@ import com.google.common.io.Closeables; import org.apache.spark.SparkEnv; import org.apache.spark.TaskContext; +import org.apache.spark.internal.config.package$; +import org.apache.spark.internal.config.ConfigEntry; import org.apache.spark.io.NioBufferedFileInputStream; import org.apache.spark.io.ReadAheadInputStream; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.storage.BlockId; import org.apache.spark.unsafe.Platform; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.*; @@ -36,9 +36,7 @@ * of the file format). */ public final class UnsafeSorterSpillReader extends UnsafeSorterIterator implements Closeable { - private static final Logger logger = LoggerFactory.getLogger(UnsafeSorterSpillReader.class); - private static final int DEFAULT_BUFFER_SIZE_BYTES = 1024 * 1024; // 1 MB - private static final int MAX_BUFFER_SIZE_BYTES = 16777216; // 16 mb + public static final int MAX_BUFFER_SIZE_BYTES = 16777216; // 16 mb private InputStream in; private DataInputStream din; @@ -51,7 +49,6 @@ public final class UnsafeSorterSpillReader extends UnsafeSorterIterator implemen private byte[] arr = new byte[1024 * 1024]; private Object baseObject = arr; - private final long baseOffset = Platform.BYTE_ARRAY_OFFSET; private final TaskContext taskContext = TaskContext.get(); public UnsafeSorterSpillReader( @@ -59,28 +56,23 @@ public UnsafeSorterSpillReader( File file, BlockId blockId) throws IOException { assert (file.length() > 0); - long bufferSizeBytes = - SparkEnv.get() == null ? - DEFAULT_BUFFER_SIZE_BYTES: - SparkEnv.get().conf().getSizeAsBytes("spark.unsafe.sorter.spill.reader.buffer.size", - DEFAULT_BUFFER_SIZE_BYTES); - if (bufferSizeBytes > MAX_BUFFER_SIZE_BYTES || bufferSizeBytes < DEFAULT_BUFFER_SIZE_BYTES) { - // fall back to a sane default value - logger.warn("Value of config \"spark.unsafe.sorter.spill.reader.buffer.size\" = {} not in " + - "allowed range [{}, {}). Falling back to default value : {} bytes", bufferSizeBytes, - DEFAULT_BUFFER_SIZE_BYTES, MAX_BUFFER_SIZE_BYTES, DEFAULT_BUFFER_SIZE_BYTES); - bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES; - } + final ConfigEntry bufferSizeConfigEntry = + package$.MODULE$.UNSAFE_SORTER_SPILL_READER_BUFFER_SIZE(); + // This value must be less than or equal to MAX_BUFFER_SIZE_BYTES. Cast to int is always safe. + final int DEFAULT_BUFFER_SIZE_BYTES = + ((Long) bufferSizeConfigEntry.defaultValue().get()).intValue(); + int bufferSizeBytes = SparkEnv.get() == null ? DEFAULT_BUFFER_SIZE_BYTES : + ((Long) SparkEnv.get().conf().get(bufferSizeConfigEntry)).intValue(); - final boolean readAheadEnabled = SparkEnv.get() != null && - SparkEnv.get().conf().getBoolean("spark.unsafe.sorter.spill.read.ahead.enabled", true); + final boolean readAheadEnabled = SparkEnv.get() != null && (boolean)SparkEnv.get().conf().get( + package$.MODULE$.UNSAFE_SORTER_SPILL_READ_AHEAD_ENABLED()); final InputStream bs = - new NioBufferedFileInputStream(file, (int) bufferSizeBytes); + new NioBufferedFileInputStream(file, bufferSizeBytes); try { if (readAheadEnabled) { this.in = new ReadAheadInputStream(serializerManager.wrapStream(blockId, bs), - (int) bufferSizeBytes); + bufferSizeBytes); } else { this.in = serializerManager.wrapStream(blockId, bs); } @@ -132,7 +124,7 @@ public Object getBaseObject() { @Override public long getBaseOffset() { - return baseOffset; + return Platform.BYTE_ARRAY_OFFSET; } @Override diff --git a/core/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider b/core/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider new file mode 100644 index 000000000000..c1f2060cabcf --- /dev/null +++ b/core/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider @@ -0,0 +1,2 @@ +org.apache.spark.deploy.security.HadoopFSDelegationTokenProvider +org.apache.spark.deploy.security.HBaseDelegationTokenProvider diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index 277010015072..eb12848900b5 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -28,8 +28,8 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: log4j.logger.org.apache.spark.repl.Main=WARN # Settings to quiet third party logs that are too verbose -log4j.logger.org.spark_project.jetty=WARN -log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.sparkproject.jetty=WARN +log4j.logger.org.sparkproject.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index 3c8ddddf07b1..3798dc47529a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -83,8 +83,8 @@ $(function() { $(this).parent().find('input[type="checkbox"]').trigger('click'); }); - // Trigger a double click on the span to show full job description. - $(".description-input").dblclick(function() { - $(this).removeClass("description-input").addClass("description-input-full"); + // Show/hide full job description on click event. + $(".description-input").click(function() { + $(this).toggleClass("description-input-full"); }); }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index a48c02ae279b..98d67c91f24b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -114,7 +114,6 @@ $(document).ready(function () { var endPoint = createRESTEndPointForExecutorsPage(appId); $.getJSON(endPoint, function (response, status, jqXHR) { - var summary = []; var allExecCnt = 0; var allRDDBlocks = 0; var allMemoryUsed = 0; @@ -505,7 +504,7 @@ $(document).ready(function () { {data: 'allTotalTasks'}, { data: function (row, type) { - return type === 'display' ? (formatDuration(row.allTotalDuration, type) + ' (' + formatDuration(row.allTotalGCTime, type) + ')') : row.allTotalDuration + return type === 'display' ? (formatDuration(row.allTotalDuration) + ' (' + formatDuration(row.allTotalGCTime) + ')') : row.allTotalDuration }, "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { if (oData.allTotalDuration > 0) { diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index abc2ec0fa653..35e4de9ac664 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -103,12 +103,12 @@ $(document).ready(function() { pageLength: 20 }); - historySummary = $("#history-summary"); - searchString = historySummary["context"]["location"]["search"]; - requestedIncomplete = getParameterByName("showIncomplete", searchString); + var historySummary = $("#history-summary"); + var searchString = historySummary["context"]["location"]["search"]; + var requestedIncomplete = getParameterByName("showIncomplete", searchString); requestedIncomplete = (requestedIncomplete == "true" ? true : false); - appParams = { + var appParams = { limit: appLimit, status: (requestedIncomplete ? "running" : "completed") }; @@ -116,7 +116,7 @@ $(document).ready(function() { $.getJSON(uiRoot + "/api/v1/applications", appParams, function(response,status,jqXHR) { var array = []; var hasMultipleAttempts = false; - for (i in response) { + for (var i in response) { var app = response[i]; if (app["attempts"][0]["completed"] == requestedIncomplete) { continue; // if we want to show for Incomplete, we skip the completed apps; otherwise skip incomplete ones. @@ -127,7 +127,7 @@ $(document).ready(function() { hasMultipleAttempts = true; } var num = app["attempts"].length; - for (j in app["attempts"]) { + for (var j in app["attempts"]) { var attempt = app["attempts"][j]; attempt["startTime"] = formatTimeMillis(attempt["startTimeEpoch"]); attempt["endTime"] = formatTimeMillis(attempt["endTimeEpoch"]); @@ -149,7 +149,7 @@ $(document).ready(function() { "applications": array, "hasMultipleAttempts": hasMultipleAttempts, "showCompletedColumns": !requestedIncomplete, - } + }; $.get(uiRoot + "/static/historypage-template.html", function(template) { var sibling = historySummary.prev(); @@ -157,7 +157,7 @@ $(document).ready(function() { var apps = $(Mustache.render($(template).filter("#history-summary-template").html(),data)); var attemptIdColumnName = 'attemptId'; var startedColumnName = 'started'; - var defaultSortColumn = completedColumnName = 'completed'; + var completedColumnName = 'completed'; var durationColumnName = 'duration'; var conf = { "columns": [ diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js b/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js deleted file mode 100644 index ab28a24729b3..000000000000 --- a/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js +++ /dev/null @@ -1,4 +0,0 @@ -/*! jQuery v1.11.1 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */ -!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k={},l="1.11.1",m=function(a,b){return new m.fn.init(a,b)},n=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,o=/^-ms-/,p=/-([\da-z])/gi,q=function(a,b){return b.toUpperCase()};m.fn=m.prototype={jquery:l,constructor:m,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=m.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return m.each(this,a,b)},map:function(a){return this.pushStack(m.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},m.extend=m.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||m.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(m.isPlainObject(c)||(b=m.isArray(c)))?(b?(b=!1,f=a&&m.isArray(a)?a:[]):f=a&&m.isPlainObject(a)?a:{},g[d]=m.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},m.extend({expando:"jQuery"+(l+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===m.type(a)},isArray:Array.isArray||function(a){return"array"===m.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){return!m.isArray(a)&&a-parseFloat(a)>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==m.type(a)||a.nodeType||m.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(k.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&m.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(o,"ms-").replace(p,q)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=r(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(n,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(r(Object(a))?m.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=r(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),m.isFunction(a)?(c=d.call(arguments,2),e=function(){return a.apply(b||this,c.concat(d.call(arguments)))},e.guid=a.guid=a.guid||m.guid++,e):void 0},now:function(){return+new Date},support:k}),m.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function r(a){var b=a.length,c=m.type(a);return"function"===c||m.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var s=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+-new Date,v=a.document,w=0,x=0,y=gb(),z=gb(),A=gb(),B=function(a,b){return a===b&&(l=!0),0},C="undefined",D=1<<31,E={}.hasOwnProperty,F=[],G=F.pop,H=F.push,I=F.push,J=F.slice,K=F.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},L="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",M="[\\x20\\t\\r\\n\\f]",N="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=N.replace("w","w#"),P="\\["+M+"*("+N+")(?:"+M+"*([*^$|!~]?=)"+M+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+O+"))|)"+M+"*\\]",Q=":("+N+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+P+")*)|.*)\\)|)",R=new RegExp("^"+M+"+|((?:^|[^\\\\])(?:\\\\.)*)"+M+"+$","g"),S=new RegExp("^"+M+"*,"+M+"*"),T=new RegExp("^"+M+"*([>+~]|"+M+")"+M+"*"),U=new RegExp("="+M+"*([^\\]'\"]*?)"+M+"*\\]","g"),V=new RegExp(Q),W=new RegExp("^"+O+"$"),X={ID:new RegExp("^#("+N+")"),CLASS:new RegExp("^\\.("+N+")"),TAG:new RegExp("^("+N.replace("w","w*")+")"),ATTR:new RegExp("^"+P),PSEUDO:new RegExp("^"+Q),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+L+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/^(?:input|select|textarea|button)$/i,Z=/^h\d$/i,$=/^[^{]+\{\s*\[native \w/,_=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ab=/[+~]/,bb=/'|\\/g,cb=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),db=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)};try{I.apply(F=J.call(v.childNodes),v.childNodes),F[v.childNodes.length].nodeType}catch(eb){I={apply:F.length?function(a,b){H.apply(a,J.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fb(a,b,d,e){var f,h,j,k,l,o,r,s,w,x;if((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,d=d||[],!a||"string"!=typeof a)return d;if(1!==(k=b.nodeType)&&9!==k)return[];if(p&&!e){if(f=_.exec(a))if(j=f[1]){if(9===k){if(h=b.getElementById(j),!h||!h.parentNode)return d;if(h.id===j)return d.push(h),d}else if(b.ownerDocument&&(h=b.ownerDocument.getElementById(j))&&t(b,h)&&h.id===j)return d.push(h),d}else{if(f[2])return I.apply(d,b.getElementsByTagName(a)),d;if((j=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return I.apply(d,b.getElementsByClassName(j)),d}if(c.qsa&&(!q||!q.test(a))){if(s=r=u,w=b,x=9===k&&a,1===k&&"object"!==b.nodeName.toLowerCase()){o=g(a),(r=b.getAttribute("id"))?s=r.replace(bb,"\\$&"):b.setAttribute("id",s),s="[id='"+s+"'] ",l=o.length;while(l--)o[l]=s+qb(o[l]);w=ab.test(a)&&ob(b.parentNode)||b,x=o.join(",")}if(x)try{return I.apply(d,w.querySelectorAll(x)),d}catch(y){}finally{r||b.removeAttribute("id")}}}return i(a.replace(R,"$1"),b,d,e)}function gb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function hb(a){return a[u]=!0,a}function ib(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function jb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function kb(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||D)-(~a.sourceIndex||D);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function lb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function mb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function nb(a){return hb(function(b){return b=+b,hb(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function ob(a){return a&&typeof a.getElementsByTagName!==C&&a}c=fb.support={},f=fb.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fb.setDocument=function(a){var b,e=a?a.ownerDocument||a:v,g=e.defaultView;return e!==n&&9===e.nodeType&&e.documentElement?(n=e,o=e.documentElement,p=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){m()},!1):g.attachEvent&&g.attachEvent("onunload",function(){m()})),c.attributes=ib(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ib(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=$.test(e.getElementsByClassName)&&ib(function(a){return a.innerHTML="
",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=ib(function(a){return o.appendChild(a).id=u,!e.getElementsByName||!e.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==C&&p){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){var c=typeof a.getAttributeNode!==C&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==C?b.getElementsByTagName(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return typeof b.getElementsByClassName!==C&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=$.test(e.querySelectorAll))&&(ib(function(a){a.innerHTML="",a.querySelectorAll("[msallowclip^='']").length&&q.push("[*^$]="+M+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+M+"*(?:value|"+L+")"),a.querySelectorAll(":checked").length||q.push(":checked")}),ib(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+M+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=$.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ib(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",Q)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=$.test(o.compareDocumentPosition),t=b||$.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===e||a.ownerDocument===v&&t(v,a)?-1:b===e||b.ownerDocument===v&&t(v,b)?1:k?K.call(k,a)-K.call(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],i=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:k?K.call(k,a)-K.call(k,b):0;if(f===g)return kb(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)i.unshift(c);while(h[d]===i[d])d++;return d?kb(h[d],i[d]):h[d]===v?-1:i[d]===v?1:0},e):n},fb.matches=function(a,b){return fb(a,null,null,b)},fb.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(U,"='$1']"),!(!c.matchesSelector||!p||r&&r.test(b)||q&&q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fb(b,n,null,[a]).length>0},fb.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fb.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&E.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fb.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fb.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=fb.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=fb.selectors={cacheLength:50,createPseudo:hb,match:X,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(cb,db),a[3]=(a[3]||a[4]||a[5]||"").replace(cb,db),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||fb.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&fb.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return X.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&V.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(cb,db).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+M+")"+a+"("+M+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==C&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fb.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){k=q[u]||(q[u]={}),j=k[a]||[],n=j[0]===w&&j[1],m=j[0]===w&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[w,n,m];break}}else if(s&&(j=(b[u]||(b[u]={}))[a])&&j[0]===w)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(s&&((l[u]||(l[u]={}))[a]=[w,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||fb.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?hb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=K.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:hb(function(a){var b=[],c=[],d=h(a.replace(R,"$1"));return d[u]?hb(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:hb(function(a){return function(b){return fb(a,b).length>0}}),contains:hb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:hb(function(a){return W.test(a||"")||fb.error("unsupported lang: "+a),a=a.replace(cb,db).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Z.test(a.nodeName)},input:function(a){return Y.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:nb(function(){return[0]}),last:nb(function(a,b){return[b-1]}),eq:nb(function(a,b,c){return[0>c?c+b:c]}),even:nb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:nb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:nb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:nb(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function rb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(i=b[u]||(b[u]={}),(h=i[d])&&h[0]===w&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function sb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function tb(a,b,c){for(var d=0,e=b.length;e>d;d++)fb(a,b[d],c);return c}function ub(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function vb(a,b,c,d,e,f){return d&&!d[u]&&(d=vb(d)),e&&!e[u]&&(e=vb(e,f)),hb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||tb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ub(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ub(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?K.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ub(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):I.apply(g,r)})}function wb(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=rb(function(a){return a===b},h,!0),l=rb(function(a){return K.call(b,a)>-1},h,!0),m=[function(a,c,d){return!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>i;i++)if(c=d.relative[a[i].type])m=[rb(sb(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return vb(i>1&&sb(m),i>1&&qb(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(R,"$1"),c,e>i&&wb(a.slice(i,e)),f>e&&wb(a=a.slice(e)),f>e&&qb(a))}m.push(c)}return sb(m)}function xb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,m,o,p=0,q="0",r=f&&[],s=[],t=j,u=f||e&&d.find.TAG("*",k),v=w+=null==t?1:Math.random()||.1,x=u.length;for(k&&(j=g!==n&&g);q!==x&&null!=(l=u[q]);q++){if(e&&l){m=0;while(o=a[m++])if(o(l,g,h)){i.push(l);break}k&&(w=v)}c&&((l=!o&&l)&&p--,f&&r.push(l))}if(p+=q,c&&q!==p){m=0;while(o=b[m++])o(r,s,g,h);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=G.call(i));s=ub(s)}I.apply(i,s),k&&!f&&s.length>0&&p+b.length>1&&fb.uniqueSort(i)}return k&&(w=v,j=t),r};return c?hb(f):f}return h=fb.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wb(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xb(e,d)),f.selector=a}return f},i=fb.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(cb,db),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=X.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(cb,db),ab.test(j[0].type)&&ob(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qb(j),!a)return I.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,ab.test(a)&&ob(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ib(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ib(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||jb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ib(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||jb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ib(function(a){return null==a.getAttribute("disabled")})||jb(L,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fb}(a);m.find=s,m.expr=s.selectors,m.expr[":"]=m.expr.pseudos,m.unique=s.uniqueSort,m.text=s.getText,m.isXMLDoc=s.isXML,m.contains=s.contains;var t=m.expr.match.needsContext,u=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,v=/^.[^:#\[\.,]*$/;function w(a,b,c){if(m.isFunction(b))return m.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return m.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(v.test(b))return m.filter(b,a,c);b=m.filter(b,a)}return m.grep(a,function(a){return m.inArray(a,b)>=0!==c})}m.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?m.find.matchesSelector(d,a)?[d]:[]:m.find.matches(a,m.grep(b,function(a){return 1===a.nodeType}))},m.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(m(a).filter(function(){for(b=0;e>b;b++)if(m.contains(d[b],this))return!0}));for(b=0;e>b;b++)m.find(a,d[b],c);return c=this.pushStack(e>1?m.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(w(this,a||[],!1))},not:function(a){return this.pushStack(w(this,a||[],!0))},is:function(a){return!!w(this,"string"==typeof a&&t.test(a)?m(a):a||[],!1).length}});var x,y=a.document,z=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,A=m.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:z.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||x).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof m?b[0]:b,m.merge(this,m.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:y,!0)),u.test(c[1])&&m.isPlainObject(b))for(c in b)m.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=y.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return x.find(a);this.length=1,this[0]=d}return this.context=y,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):m.isFunction(a)?"undefined"!=typeof x.ready?x.ready(a):a(m):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),m.makeArray(a,this))};A.prototype=m.fn,x=m(y);var B=/^(?:parents|prev(?:Until|All))/,C={children:!0,contents:!0,next:!0,prev:!0};m.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!m(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),m.fn.extend({has:function(a){var b,c=m(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(m.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=t.test(a)||"string"!=typeof a?m(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&m.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?m.unique(f):f)},index:function(a){return a?"string"==typeof a?m.inArray(this[0],m(a)):m.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(m.unique(m.merge(this.get(),m(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function D(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}m.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return m.dir(a,"parentNode")},parentsUntil:function(a,b,c){return m.dir(a,"parentNode",c)},next:function(a){return D(a,"nextSibling")},prev:function(a){return D(a,"previousSibling")},nextAll:function(a){return m.dir(a,"nextSibling")},prevAll:function(a){return m.dir(a,"previousSibling")},nextUntil:function(a,b,c){return m.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return m.dir(a,"previousSibling",c)},siblings:function(a){return m.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return m.sibling(a.firstChild)},contents:function(a){return m.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:m.merge([],a.childNodes)}},function(a,b){m.fn[a]=function(c,d){var e=m.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=m.filter(d,e)),this.length>1&&(C[a]||(e=m.unique(e)),B.test(a)&&(e=e.reverse())),this.pushStack(e)}});var E=/\S+/g,F={};function G(a){var b=F[a]={};return m.each(a.match(E)||[],function(a,c){b[c]=!0}),b}m.Callbacks=function(a){a="string"==typeof a?F[a]||G(a):m.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){m.each(b,function(b,c){var d=m.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&m.each(arguments,function(a,c){var d;while((d=m.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?m.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c.slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},m.extend({Deferred:function(a){var b=[["resolve","done",m.Callbacks("once memory"),"resolved"],["reject","fail",m.Callbacks("once memory"),"rejected"],["notify","progress",m.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return m.Deferred(function(c){m.each(b,function(b,f){var g=m.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&m.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?m.extend(a,d):d}},e={};return d.pipe=d.then,m.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&m.isFunction(a.promise)?e:0,g=1===f?a:m.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&m.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var H;m.fn.ready=function(a){return m.ready.promise().done(a),this},m.extend({isReady:!1,readyWait:1,holdReady:function(a){a?m.readyWait++:m.ready(!0)},ready:function(a){if(a===!0?!--m.readyWait:!m.isReady){if(!y.body)return setTimeout(m.ready);m.isReady=!0,a!==!0&&--m.readyWait>0||(H.resolveWith(y,[m]),m.fn.triggerHandler&&(m(y).triggerHandler("ready"),m(y).off("ready")))}}});function I(){y.addEventListener?(y.removeEventListener("DOMContentLoaded",J,!1),a.removeEventListener("load",J,!1)):(y.detachEvent("onreadystatechange",J),a.detachEvent("onload",J))}function J(){(y.addEventListener||"load"===event.type||"complete"===y.readyState)&&(I(),m.ready())}m.ready.promise=function(b){if(!H)if(H=m.Deferred(),"complete"===y.readyState)setTimeout(m.ready);else if(y.addEventListener)y.addEventListener("DOMContentLoaded",J,!1),a.addEventListener("load",J,!1);else{y.attachEvent("onreadystatechange",J),a.attachEvent("onload",J);var c=!1;try{c=null==a.frameElement&&y.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!m.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}I(),m.ready()}}()}return H.promise(b)};var K="undefined",L;for(L in m(k))break;k.ownLast="0"!==L,k.inlineBlockNeedsLayout=!1,m(function(){var a,b,c,d;c=y.getElementsByTagName("body")[0],c&&c.style&&(b=y.createElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),typeof b.style.zoom!==K&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",k.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(d))}),function(){var a=y.createElement("div");if(null==k.deleteExpando){k.deleteExpando=!0;try{delete a.test}catch(b){k.deleteExpando=!1}}a=null}(),m.acceptData=function(a){var b=m.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var M=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,N=/([A-Z])/g;function O(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(N,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:M.test(c)?m.parseJSON(c):c}catch(e){}m.data(a,b,c)}else c=void 0}return c}function P(a){var b;for(b in a)if(("data"!==b||!m.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function Q(a,b,d,e){if(m.acceptData(a)){var f,g,h=m.expando,i=a.nodeType,j=i?m.cache:a,k=i?a[h]:a[h]&&h; -if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||m.guid++:h),j[k]||(j[k]=i?{}:{toJSON:m.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=m.extend(j[k],b):j[k].data=m.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[m.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[m.camelCase(b)])):f=g,f}}function R(a,b,c){if(m.acceptData(a)){var d,e,f=a.nodeType,g=f?m.cache:a,h=f?a[m.expando]:m.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){m.isArray(b)?b=b.concat(m.map(b,m.camelCase)):b in d?b=[b]:(b=m.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!P(d):!m.isEmptyObject(d))return}(c||(delete g[h].data,P(g[h])))&&(f?m.cleanData([a],!0):k.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}m.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?m.cache[a[m.expando]]:a[m.expando],!!a&&!P(a)},data:function(a,b,c){return Q(a,b,c)},removeData:function(a,b){return R(a,b)},_data:function(a,b,c){return Q(a,b,c,!0)},_removeData:function(a,b){return R(a,b,!0)}}),m.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=m.data(f),1===f.nodeType&&!m._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=m.camelCase(d.slice(5)),O(f,d,e[d])));m._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){m.data(this,a)}):arguments.length>1?this.each(function(){m.data(this,a,b)}):f?O(f,a,m.data(f,a)):void 0},removeData:function(a){return this.each(function(){m.removeData(this,a)})}}),m.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=m._data(a,b),c&&(!d||m.isArray(c)?d=m._data(a,b,m.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=m.queue(a,b),d=c.length,e=c.shift(),f=m._queueHooks(a,b),g=function(){m.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return m._data(a,c)||m._data(a,c,{empty:m.Callbacks("once memory").add(function(){m._removeData(a,b+"queue"),m._removeData(a,c)})})}}),m.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},W=/^(?:checkbox|radio)$/i;!function(){var a=y.createElement("input"),b=y.createElement("div"),c=y.createDocumentFragment();if(b.innerHTML="
a",k.leadingWhitespace=3===b.firstChild.nodeType,k.tbody=!b.getElementsByTagName("tbody").length,k.htmlSerialize=!!b.getElementsByTagName("link").length,k.html5Clone="<:nav>"!==y.createElement("nav").cloneNode(!0).outerHTML,a.type="checkbox",a.checked=!0,c.appendChild(a),k.appendChecked=a.checked,b.innerHTML="",k.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,c.appendChild(b),b.innerHTML="",k.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,k.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){k.noCloneEvent=!1}),b.cloneNode(!0).click()),null==k.deleteExpando){k.deleteExpando=!0;try{delete b.test}catch(d){k.deleteExpando=!1}}}(),function(){var b,c,d=y.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(k[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),k[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var X=/^(?:input|select|textarea)$/i,Y=/^key/,Z=/^(?:mouse|pointer|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=/^([^.]*)(?:\.(.+)|)$/;function ab(){return!0}function bb(){return!1}function cb(){try{return y.activeElement}catch(a){}}m.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=m.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof m===K||a&&m.event.triggered===a.type?void 0:m.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(E)||[""],h=b.length;while(h--)f=_.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=m.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=m.event.special[o]||{},l=m.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&m.expr.match.needsContext.test(e),namespace:p.join(".")},i),(n=g[o])||(n=g[o]=[],n.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?n.splice(n.delegateCount++,0,l):n.push(l),m.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m.hasData(a)&&m._data(a);if(r&&(k=r.events)){b=(b||"").match(E)||[""],j=b.length;while(j--)if(h=_.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=m.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,n=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=n.length;while(f--)g=n[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(n.splice(f,1),g.selector&&n.delegateCount--,l.remove&&l.remove.call(a,g));i&&!n.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||m.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)m.event.remove(a,o+b[j],c,d,!0);m.isEmptyObject(k)&&(delete r.handle,m._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,n,o=[d||y],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||y,3!==d.nodeType&&8!==d.nodeType&&!$.test(p+m.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[m.expando]?b:new m.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:m.makeArray(c,[b]),k=m.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!m.isWindow(d)){for(i=k.delegateType||p,$.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocument||y)&&o.push(l.defaultView||l.parentWindow||a)}n=0;while((h=o[n++])&&!b.isPropagationStopped())b.type=n>1?i:k.bindType||p,f=(m._data(h,"events")||{})[b.type]&&m._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&m.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&m.acceptData(d)&&g&&d[p]&&!m.isWindow(d)){l=d[g],l&&(d[g]=null),m.event.triggered=p;try{d[p]()}catch(r){}m.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=m.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(m._data(this,"events")||{})[a.type]||[],k=m.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=m.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace))&&(a.handleObj=e,a.data=e.data,c=((m.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?m(c,this).index(i)>=0:m.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h]","i"),hb=/^\s+/,ib=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,jb=/<([\w:]+)/,kb=/\s*$/g,rb={option:[1,""],legend:[1,"
","
"],area:[1,"",""],param:[1,"",""],thead:[1,"","
"],tr:[2,"","
"],col:[2,"","
"],td:[3,"","
"],_default:k.htmlSerialize?[0,"",""]:[1,"X
","
"]},sb=db(y),tb=sb.appendChild(y.createElement("div"));rb.optgroup=rb.option,rb.tbody=rb.tfoot=rb.colgroup=rb.caption=rb.thead,rb.th=rb.td;function ub(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==K?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==K?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||m.nodeName(d,b)?f.push(d):m.merge(f,ub(d,b));return void 0===b||b&&m.nodeName(a,b)?m.merge([a],f):f}function vb(a){W.test(a.type)&&(a.defaultChecked=a.checked)}function wb(a,b){return m.nodeName(a,"table")&&m.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function xb(a){return a.type=(null!==m.find.attr(a,"type"))+"/"+a.type,a}function yb(a){var b=pb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function zb(a,b){for(var c,d=0;null!=(c=a[d]);d++)m._data(c,"globalEval",!b||m._data(b[d],"globalEval"))}function Ab(a,b){if(1===b.nodeType&&m.hasData(a)){var c,d,e,f=m._data(a),g=m._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)m.event.add(b,c,h[c][d])}g.data&&(g.data=m.extend({},g.data))}}function Bb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!k.noCloneEvent&&b[m.expando]){e=m._data(b);for(d in e.events)m.removeEvent(b,d,e.handle);b.removeAttribute(m.expando)}"script"===c&&b.text!==a.text?(xb(b).text=a.text,yb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),k.html5Clone&&a.innerHTML&&!m.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&W.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}m.extend({clone:function(a,b,c){var d,e,f,g,h,i=m.contains(a.ownerDocument,a);if(k.html5Clone||m.isXMLDoc(a)||!gb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(tb.innerHTML=a.outerHTML,tb.removeChild(f=tb.firstChild)),!(k.noCloneEvent&&k.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||m.isXMLDoc(a)))for(d=ub(f),h=ub(a),g=0;null!=(e=h[g]);++g)d[g]&&Bb(e,d[g]);if(b)if(c)for(h=h||ub(a),d=d||ub(f),g=0;null!=(e=h[g]);g++)Ab(e,d[g]);else Ab(a,f);return d=ub(f,"script"),d.length>0&&zb(d,!i&&ub(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,l,n=a.length,o=db(b),p=[],q=0;n>q;q++)if(f=a[q],f||0===f)if("object"===m.type(f))m.merge(p,f.nodeType?[f]:f);else if(lb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(jb.exec(f)||["",""])[1].toLowerCase(),l=rb[i]||rb._default,h.innerHTML=l[1]+f.replace(ib,"<$1>")+l[2],e=l[0];while(e--)h=h.lastChild;if(!k.leadingWhitespace&&hb.test(f)&&p.push(b.createTextNode(hb.exec(f)[0])),!k.tbody){f="table"!==i||kb.test(f)?""!==l[1]||kb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)m.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}m.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),k.appendChecked||m.grep(ub(p,"input"),vb),q=0;while(f=p[q++])if((!d||-1===m.inArray(f,d))&&(g=m.contains(f.ownerDocument,f),h=ub(o.appendChild(f),"script"),g&&zb(h),c)){e=0;while(f=h[e++])ob.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=m.expando,j=m.cache,l=k.deleteExpando,n=m.event.special;null!=(d=a[h]);h++)if((b||m.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)n[e]?m.event.remove(d,e):m.removeEvent(d,e,g.handle);j[f]&&(delete j[f],l?delete d[i]:typeof d.removeAttribute!==K?d.removeAttribute(i):d[i]=null,c.push(f))}}}),m.fn.extend({text:function(a){return V(this,function(a){return void 0===a?m.text(this):this.empty().append((this[0]&&this[0].ownerDocument||y).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?m.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||m.cleanData(ub(c)),c.parentNode&&(b&&m.contains(c.ownerDocument,c)&&zb(ub(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&m.cleanData(ub(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&m.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return m.clone(this,a,b)})},html:function(a){return V(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(fb,""):void 0;if(!("string"!=typeof a||mb.test(a)||!k.htmlSerialize&&gb.test(a)||!k.leadingWhitespace&&hb.test(a)||rb[(jb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(ib,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(m.cleanData(ub(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,m.cleanData(ub(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,l=this.length,n=this,o=l-1,p=a[0],q=m.isFunction(p);if(q||l>1&&"string"==typeof p&&!k.checkClone&&nb.test(p))return this.each(function(c){var d=n.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(l&&(i=m.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=m.map(ub(i,"script"),xb),f=g.length;l>j;j++)d=i,j!==o&&(d=m.clone(d,!0,!0),f&&m.merge(g,ub(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,m.map(g,yb),j=0;f>j;j++)d=g[j],ob.test(d.type||"")&&!m._data(d,"globalEval")&&m.contains(h,d)&&(d.src?m._evalUrl&&m._evalUrl(d.src):m.globalEval((d.text||d.textContent||d.innerHTML||"").replace(qb,"")));i=c=null}return this}}),m.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){m.fn[a]=function(a){for(var c,d=0,e=[],g=m(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),m(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Cb,Db={};function Eb(b,c){var d,e=m(c.createElement(b)).appendTo(c.body),f=a.getDefaultComputedStyle&&(d=a.getDefaultComputedStyle(e[0]))?d.display:m.css(e[0],"display");return e.detach(),f}function Fb(a){var b=y,c=Db[a];return c||(c=Eb(a,b),"none"!==c&&c||(Cb=(Cb||m("